You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by zh...@apache.org on 2020/09/07 07:08:04 UTC
[bookkeeper] branch master updated: BP-41 BookieId - client side
implementation (#2404)
This is an automated email from the ASF dual-hosted git repository.
zhaijia pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git
The following commit(s) were added to refs/heads/master by this push:
new 057f85e BP-41 BookieId - client side implementation (#2404)
057f85e is described below
commit 057f85ec1b977379938350f12d7ec99ff1162f29
Author: Enrico Olivelli <eo...@gmail.com>
AuthorDate: Mon Sep 7 09:07:56 2020 +0200
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 <eo...@apache.org>
Co-authored-by: Enrico Olivelli <en...@diennea.com>
---
.../apache/bookkeeper/benchmark/BenchBookie.java | 13 +-
.../apache/bookkeeper/benchmark/TestBenchmark.java | 2 +-
.../apache/bookkeeper/common/util/JsonUtil.java | 8 +-
bookkeeper-server/pom.xml | 1 +
.../java/org/apache/bookkeeper/bookie/Bookie.java | 25 +-
.../org/apache/bookkeeper/bookie/BookieShell.java | 4 +-
.../bookkeeper/bookie/BookieStateManager.java | 7 +-
.../java/org/apache/bookkeeper/bookie/Cookie.java | 24 +-
.../bookie/LocalBookieEnsemblePlacementPolicy.java | 34 +-
.../bookie/ScanAndCompareGarbageCollector.java | 10 +-
.../org/apache/bookkeeper/client/BKException.java | 3 +
.../org/apache/bookkeeper/client/BookKeeper.java | 34 +-
.../apache/bookkeeper/client/BookKeeperAdmin.java | 178 ++-
.../apache/bookkeeper/client/BookieInfoReader.java | 44 +-
.../apache/bookkeeper/client/BookieWatcher.java | 25 +-
.../bookkeeper/client/BookieWatcherImpl.java | 63 +-
.../bookkeeper/client/BookiesHealthInfo.java | 6 +-
.../client/DefaultBookieAddressResolver.java | 68 ++
.../client/DefaultEnsemblePlacementPolicy.java | 76 +-
.../bookkeeper/client/DistributionSchedule.java | 6 +-
.../bookkeeper/client/EnsemblePlacementPolicy.java | 38 +-
.../apache/bookkeeper/client/EnsembleUtils.java | 22 +-
.../apache/bookkeeper/client/ForceLedgerOp.java | 8 +-
.../ITopologyAwareEnsemblePlacementPolicy.java | 14 +-
.../apache/bookkeeper/client/LedgerChecker.java | 10 +-
.../apache/bookkeeper/client/LedgerCreateOp.java | 8 +-
.../apache/bookkeeper/client/LedgerFragment.java | 14 +-
.../client/LedgerFragmentReplicator.java | 24 +-
.../org/apache/bookkeeper/client/LedgerHandle.java | 50 +-
.../bookkeeper/client/LedgerMetadataBuilder.java | 8 +-
.../bookkeeper/client/LedgerMetadataImpl.java | 12 +-
.../bookkeeper/client/LedgerMetadataUtils.java | 14 +-
.../org/apache/bookkeeper/client/PendingAddOp.java | 16 +-
.../apache/bookkeeper/client/PendingReadLacOp.java | 6 +-
.../apache/bookkeeper/client/PendingReadOp.java | 46 +-
.../bookkeeper/client/PendingWriteLacOp.java | 8 +-
.../client/RackawareEnsemblePlacementPolicy.java | 41 +-
.../RackawareEnsemblePlacementPolicyImpl.java | 86 +-
.../client/ReadLastConfirmedAndEntryOp.java | 42 +-
.../bookkeeper/client/ReadLastConfirmedOp.java | 6 +-
.../bookkeeper/client/ReadOnlyLedgerHandle.java | 12 +-
.../client/RegionAwareEnsemblePlacementPolicy.java | 68 +-
.../client/RoundRobinDistributionSchedule.java | 12 +-
.../TopologyAwareEnsemblePlacementPolicy.java | 66 +-
.../bookkeeper/client/TryReadLastConfirmedOp.java | 6 +-
.../apache/bookkeeper/client/UpdateLedgerOp.java | 12 +-
.../client/ZoneawareEnsemblePlacementPolicy.java | 35 +-
.../ZoneawareEnsemblePlacementPolicyImpl.java | 114 +-
.../bookkeeper/client/api/LedgerMetadata.java | 6 +-
.../bookkeeper/discover/RegistrationClient.java | 18 +-
.../bookkeeper/discover/RegistrationManager.java | 13 +-
.../bookkeeper/discover/ZKRegistrationClient.java | 113 +-
.../bookkeeper/discover/ZKRegistrationManager.java | 31 +-
.../bookkeeper/meta/LedgerMetadataSerDe.java | 22 +-
.../java/org/apache/bookkeeper/net/BookieId.java | 90 ++
.../java/org/apache/bookkeeper/net/BookieNode.java | 6 +-
.../apache/bookkeeper/net/BookieSocketAddress.java | 51 +
.../bookkeeper/proto/BookieAddressResolver.java | 53 +
.../org/apache/bookkeeper/proto/BookieClient.java | 30 +-
.../apache/bookkeeper/proto/BookieClientImpl.java | 57 +-
.../apache/bookkeeper/proto/BookieNettyServer.java | 5 +-
.../org/apache/bookkeeper/proto/BookieServer.java | 15 +-
.../proto/BookkeeperInternalCallbacks.java | 8 +-
.../proto/DefaultPerChannelBookieClientPool.java | 6 +-
.../bookkeeper/proto/ForceLedgerProcessorV3.java | 4 +-
.../bookkeeper/proto/LocalBookiesRegistry.java | 10 +-
.../bookkeeper/proto/PerChannelBookieClient.java | 246 ++++-
.../proto/PerChannelBookieClientFactory.java | 4 +-
.../proto/ReadLastConfirmedAndEntryContext.java | 8 +-
.../bookkeeper/proto/WriteEntryProcessor.java | 4 +-
.../bookkeeper/proto/WriteEntryProcessorV3.java | 4 +-
.../bookkeeper/proto/WriteLacProcessorV3.java | 4 +-
.../org/apache/bookkeeper/replication/Auditor.java | 61 +-
.../bookkeeper/replication/AuditorElector.java | 8 +-
.../replication/BookieLedgerIndexer.java | 9 +-
.../bookkeeper/replication/ReplicationWorker.java | 10 +-
.../server/http/service/DecommissionService.java | 6 +-
.../server/http/service/ListBookieInfoService.java | 6 +-
.../server/http/service/ListBookiesService.java | 14 +-
.../server/http/service/RecoveryBookieService.java | 9 +-
.../server/http/service/WhoIsAuditorService.java | 9 +-
.../commands/autorecovery/WhoIsAuditorCommand.java | 7 +-
.../cli/commands/bookie/FlipBookieIdCommand.java | 6 +-
.../cli/commands/bookie/ListLedgersCommand.java | 6 +-
.../cli/commands/bookie/ReadLedgerCommand.java | 9 +-
.../bookie/UpdateBookieInLedgerCommand.java | 14 +-
.../cli/commands/bookies/DecommissionCommand.java | 9 +-
.../cli/commands/bookies/EndpointInfoCommand.java | 6 +-
.../tools/cli/commands/bookies/InfoCommand.java | 13 +-
.../cli/commands/bookies/ListBookiesCommand.java | 22 +-
.../tools/cli/commands/bookies/RecoverCommand.java | 33 +-
.../tools/cli/commands/cookie/AdminCommand.java | 6 +-
.../tools/cli/commands/cookie/CookieCommand.java | 5 +-
.../cli/commands/cookie/CreateCookieCommand.java | 3 +-
.../cli/commands/cookie/DeleteCookieCommand.java | 3 +-
.../cli/commands/cookie/GenerateCookieCommand.java | 5 +-
.../cli/commands/cookie/GetCookieCommand.java | 8 +-
.../cli/commands/cookie/UpdateCookieCommand.java | 3 +-
.../tools/cli/helpers/CommandHelpers.java | 18 +-
.../bookkeeper/bookie/AdvertisedAddressTest.java | 9 +-
.../bookie/BookieInitializationTest.java | 26 +-
.../bookkeeper/bookie/BookieJournalForceTest.java | 14 +-
.../bookie/BookieJournalPageCacheFlushTest.java | 10 +-
.../apache/bookkeeper/bookie/BookieShellTest.java | 22 +-
.../bookie/BookieWriteToJournalTest.java | 12 +-
.../bookie/GcOverreplicatedLedgerTest.java | 22 +-
.../apache/bookkeeper/bookie/LedgerCacheTest.java | 4 +-
.../bookkeeper/client/BookKeeperAdminTest.java | 43 +-
.../bookkeeper/client/BookKeeperCloseTest.java | 4 +-
...KeeperDiskSpaceWeightedLedgerPlacementTest.java | 138 +--
.../apache/bookkeeper/client/BookKeeperTest.java | 10 +-
.../bookkeeper/client/BookKeeperTestClient.java | 8 +-
.../bookkeeper/client/BookieDecommissionTest.java | 8 +-
.../bookkeeper/client/BookieRecoveryTest.java | 48 +-
.../bookkeeper/client/BookieWriteLedgerTest.java | 24 +-
.../apache/bookkeeper/client/DeferredSyncTest.java | 6 +-
.../client/GenericEnsemblePlacementPolicyTest.java | 14 +-
.../bookkeeper/client/HandleFailuresTest.java | 11 +-
.../apache/bookkeeper/client/LedgerClose2Test.java | 11 +-
.../apache/bookkeeper/client/LedgerCloseTest.java | 4 +-
.../bookkeeper/client/LedgerMetadataTest.java | 15 +-
.../bookkeeper/client/LedgerRecovery2Test.java | 13 +-
.../bookkeeper/client/LedgerRecoveryTest.java | 14 +-
.../bookkeeper/client/MetadataUpdateLoopTest.java | 66 +-
.../bookkeeper/client/MockBookKeeperTestCase.java | 67 +-
.../bookkeeper/client/MockClientContext.java | 4 +-
.../apache/bookkeeper/client/MockLedgerHandle.java | 8 +-
.../client/ParallelLedgerRecoveryTest.java | 12 +-
.../client/ReadLastConfirmedAndEntryOpTest.java | 20 +-
.../bookkeeper/client/ReadLastConfirmedOpTest.java | 7 +-
.../apache/bookkeeper/client/SlowBookieTest.java | 6 +-
.../client/TestAddEntryQuorumTimeout.java | 8 +-
.../bookkeeper/client/TestBookieHealthCheck.java | 10 +-
.../bookkeeper/client/TestDelayEnsembleChange.java | 20 +-
.../client/TestDisableEnsembleChange.java | 12 +-
.../org/apache/bookkeeper/client/TestFencing.java | 6 +-
.../client/TestGetBookieInfoTimeout.java | 11 +-
.../bookkeeper/client/TestLedgerChecker.java | 44 +-
.../client/TestLedgerFragmentReplication.java | 44 +-
.../apache/bookkeeper/client/TestParallelRead.java | 8 +-
.../TestRackawareEnsemblePlacementPolicy.java | 1159 ++++++++++----------
...ackawareEnsemblePlacementPolicyUsingScript.java | 229 ++--
.../TestRackawarePolicyNotificationUpdates.java | 19 +-
.../bookkeeper/client/TestReadEntryListener.java | 6 +-
.../apache/bookkeeper/client/TestReadTimeout.java | 8 +-
.../TestRegionAwareEnsemblePlacementPolicy.java | 804 +++++++-------
.../bookkeeper/client/TestSpeculativeRead.java | 8 +-
.../bookkeeper/client/TestWatchEnsembleChange.java | 14 +-
.../TestZoneawareEnsemblePlacementPolicy.java | 718 ++++++------
.../bookkeeper/client/UpdateLedgerCmdTest.java | 13 +-
.../bookkeeper/client/UpdateLedgerOpTest.java | 49 +-
.../bookkeeper/discover/BookieServiceInfoTest.java | 9 +-
.../discover/MockRegistrationClient.java | 30 +-
.../discover/TestZkRegistrationClient.java | 90 +-
.../meta/AbstractZkLedgerManagerTest.java | 12 +-
.../org/apache/bookkeeper/meta/GcLedgersTest.java | 9 +-
.../bookkeeper/meta/LedgerManagerIteratorTest.java | 8 +-
.../bookkeeper/meta/TestLedgerMetadataSerDe.java | 20 +-
.../org/apache/bookkeeper/net/BookieIdTest.java | 90 ++
...t.java => ResolvedBookieSocketAddressTest.java} | 2 +-
.../apache/bookkeeper/proto/MockBookieClient.java | 42 +-
.../bookkeeper/proto/TestBackwardCompatCMS42.java | 17 +-
.../proto/TestPerChannelBookieClient.java | 20 +-
.../bookkeeper/replication/AuditorBookieTest.java | 13 +-
.../replication/AuditorLedgerCheckerTest.java | 23 +-
.../AuditorPeriodicBookieCheckTest.java | 7 +-
.../replication/AuditorPeriodicCheckTest.java | 22 +-
.../AuditorPlacementPolicyCheckTest.java | 57 +-
.../replication/AuditorReplicasCheckTest.java | 57 +-
.../replication/AuditorRollingRestartTest.java | 4 +-
.../replication/AutoRecoveryMainTest.java | 10 +-
.../replication/BookieAutoRecoveryTest.java | 40 +-
.../TestAutoRecoveryAlongWithBookieServers.java | 12 +-
.../replication/TestReplicationWorker.java | 52 +-
.../org/apache/bookkeeper/server/TestMain.java | 3 +
.../bookkeeper/server/http/TestHttpService.java | 5 +-
.../server/http/service/ListLedgerServiceTest.java | 7 +-
.../bookkeeper/test/BookKeeperClusterTestCase.java | 80 +-
.../apache/bookkeeper/test/BookieClientTest.java | 20 +-
.../bookkeeper/test/ConcurrentLedgerTest.java | 4 +-
.../bookkeeper/test/LocalBookiesRegistryTest.java | 6 +-
.../apache/bookkeeper/test/ReadOnlyBookieTest.java | 6 +-
.../java/org/apache/bookkeeper/tls/TestTLS.java | 14 +-
.../apache/bookkeeper/util/StaticDNSResolver.java | 5 +-
.../metadata/etcd/EtcdRegistrationClient.java | 37 +-
.../metadata/etcd/EtcdRegistrationManager.java | 19 +-
.../apache/bookkeeper/metadata/etcd/EtcdUtils.java | 13 +-
.../bookkeeper/metadata/etcd/EtcdClusterTest.java | 5 +-
.../bookkeeper/metadata/etcd/EtcdCookieTest.java | 3 +-
.../metadata/etcd/EtcdLedgerManagerTest.java | 14 +-
.../metadata/etcd/EtcdRegistrationTest.java | 55 +-
pom.xml | 1 +
.../grpc/resolver/BKRegistrationNameResolver.java | 9 +-
.../resolver/BKRegistrationNameResolverTest.java | 5 +-
stream/distributedlog/core/pom.xml | 1 +
.../org/apache/bookkeeper/client/LedgerReader.java | 11 +-
.../distributedlog/tools/DistributedLogTool.java | 32 +-
.../stream/server/service/BookieWatchService.java | 4 +-
.../server/service/RegistrationStateService.java | 3 +-
.../impl/cluster/ClusterControllerLeaderImpl.java | 8 +-
.../impl/sc/DefaultStorageContainerController.java | 50 +-
.../impl/sc/StorageContainerController.java | 4 +-
.../cluster/ClusterControllerLeaderImplTest.java | 13 +-
.../sc/DefaultStorageContainerControllerTest.java | 61 +-
.../cluster/BookKeeperClusterTestBase.java | 14 +-
.../integration/cluster/SimpleClusterTest.java | 8 +-
.../bookkeeper/tests/integration/TestCLI.java | 6 +-
.../bookieid/SearchReplaceBookieIdCommand.java | 8 +-
.../autorecovery/WhoIsAuditorCommandTest.java | 11 +-
.../commands/bookie/FlipBookieIdCommandTest.java | 7 +-
.../commands/bookie/ListLedgersCommandTest.java | 10 +-
.../cli/commands/bookie/ReadLedgerCommandTest.java | 16 +-
.../commands/bookies/DecommissionCommandTest.java | 15 +-
.../cli/commands/bookies/InfoCommandTest.java | 9 +-
.../commands/bookies/ListBookiesCommandTest.java | 46 +-
.../cli/commands/bookies/RecoverCommandTest.java | 20 +-
.../cli/commands/cookie/AdminCommandTest.java | 10 +-
.../commands/cookie/CreateCookieCommandTest.java | 11 +-
.../commands/cookie/DeleteCookieCommandTest.java | 11 +-
.../cli/commands/cookie/GetCookieCommandTest.java | 15 +-
.../commands/cookie/UpdateCookieCommandTest.java | 11 +-
221 files changed, 4318 insertions(+), 3469 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 d91b725..a4d2ae9 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.LedgerHandle;
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 @@ public class BenchBookie {
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 @@ public class BenchBookie {
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 class BenchBookie {
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 class BenchBookie {
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 class BenchBookie {
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 class BenchBookie {
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 bee5f90..bb1e37c 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 class TestBenchmark extends BookKeeperClusterTestCase {
@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 afc90a4..90576ae 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 class JsonUtil {
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 class JsonUtil {
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 class JsonUtil {
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 e990b4d..adae9f6 100644
--- a/bookkeeper-server/pom.xml
+++ b/bookkeeper-server/pom.xml
@@ -219,6 +219,7 @@
<artifactId>maven-surefire-plugin</artifactId>
<version>${maven-surefire-plugin.version}</version>
<configuration>
+ <trimStackTrace>false</trimStackTrace>
<properties>
<property>
<name>listener</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 6a2209a..a7960f7 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.LedgerManagerFactory;
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 class Bookie extends BookieCriticalThread {
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 @@ public class Bookie extends BookieCriticalThread {
}
}
- static List<BookieSocketAddress> possibleBookieIds(ServerConfiguration conf)
+ static List<BookieId> 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<BookieSocketAddress> addresses = Lists.newArrayListWithExpectedSize(3);
+ List<BookieId> 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 @@ public class Bookie extends BookieCriticalThread {
.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 @@ public class Bookie extends BookieCriticalThread {
static Versioned<Cookie> readAndVerifyCookieFromRegistrationManager(
Cookie masterCookie, RegistrationManager rm,
- List<BookieSocketAddress> addresses, boolean allowExpansion)
+ List<BookieId> addresses, boolean allowExpansion)
throws BookieException {
Versioned<Cookie> 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 class Bookie extends BookieCriticalThread {
// 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<BookieSocketAddress> possibleBookieIds = possibleBookieIds(conf);
+ List<BookieId> possibleBookieIds = possibleBookieIds(conf);
final Versioned<Cookie> rmCookie = readAndVerifyCookieFromRegistrationManager(
masterCookie, rm, possibleBookieIds, allowExpansion);
@@ -532,6 +533,10 @@ public class Bookie extends BookieCriticalThread {
}
}
+ public static BookieId getBookieId(ServerConfiguration conf) throws UnknownHostException {
+ return getBookieAddress(conf).toBookieId();
+ }
+
/**
* Return the configured address of the bookie.
*/
@@ -1485,7 +1490,7 @@ public class Bookie extends BookieCriticalThread {
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 ad2f234..d17d74d 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.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.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 @@ public class BookieShell implements Tool {
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 5464745..0965124 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.conf.ServerConfiguration;
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<RegistrationManager> rm;
// Expose Stats
@@ -101,7 +102,7 @@ public class BookieStateManager implements StateManager {
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 class BookieStateManager implements StateManager {
StatsLogger statsLogger,
Supplier<RegistrationManager> rm,
List<File> statusDirs,
- Supplier<String> bookieIdSupplier,
+ Supplier<BookieId> bookieIdSupplier,
Supplier<BookieServiceInfo> 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 9a6eadf..b80a789 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.InvalidCookieException;
import org.apache.bookkeeper.bookie.BookieException.UnknownBookieIdException;
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.bookkeeper.discover.RegistrationManager;
-import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.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 class Cookie {
*/
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 class Cookie {
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 class Cookie {
* @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 @@ public class Cookie {
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 @@ public class Cookie {
public static Versioned<Cookie> 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 class Cookie {
* @throws BookieException when fail to read cookie
*/
public static Versioned<Cookie> readFromRegistrationManager(RegistrationManager rm,
- BookieSocketAddress address) throws BookieException {
- Versioned<byte[]> cookieData = rm.readCookie(address.toString());
+ BookieId address) throws BookieException {
+ Versioned<byte[]> 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 a0d2edb..d44e576 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.client.EnsemblePlacementPolicy;
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<DNSToSwitchMapping> 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 class LocalBookieEnsemblePlacementPolicy implements EnsemblePlacementPoli
}
@Override
- public Set<BookieSocketAddress> onClusterChanged(Set<BookieSocketAddress> writableBookies,
- Set<BookieSocketAddress> readOnlyBookies) {
+ public Set<BookieId> onClusterChanged(Set<BookieId> writableBookies,
+ Set<BookieId> readOnlyBookies) {
return Collections.emptySet();
}
@Override
- public PlacementResult<BookieSocketAddress> replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
- java.util.Map<String, byte[]> customMetadata, List<BookieSocketAddress> currentEnsemble,
- BookieSocketAddress bookieToReplace, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<BookieId> replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
+ java.util.Map<String, byte[]> customMetadata, List<BookieId> currentEnsemble,
+ BookieId bookieToReplace, Set<BookieId> 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<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo,
DistributionSchedule.WriteSet writeSet) {
return null;
@@ -102,15 +104,15 @@ public class LocalBookieEnsemblePlacementPolicy implements EnsemblePlacementPoli
@Override
public DistributionSchedule.WriteSet reorderReadLACSequence(
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo,
DistributionSchedule.WriteSet writeSet) {
return null;
}
@Override
- public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize, int writeQuorumSize,
- int ackQuorumSize, java.util.Map<String, byte[]> customMetadata, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<List<BookieId>> newEnsemble(int ensembleSize, int writeQuorumSize,
+ int ackQuorumSize, java.util.Map<String, byte[]> customMetadata, Set<BookieId> excludeBookies)
throws BKNotEnoughBookiesException {
if (ensembleSize > 1) {
throw new IllegalArgumentException("Local ensemble policy can only return 1 bookie");
@@ -120,12 +122,12 @@ public class LocalBookieEnsemblePlacementPolicy implements EnsemblePlacementPoli
}
@Override
- public void updateBookieInfo(Map<BookieSocketAddress, BookieInfo> bookieToFreeSpaceMap) {
+ public void updateBookieInfo(Map<BookieId, BookieInfo> bookieToFreeSpaceMap) {
return;
}
@Override
- public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List<BookieSocketAddress> ensembleList,
+ public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List<BookieId> 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 21f4129..cff0250 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.LedgerRange;
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 class ScanAndCompareGarbageCollector implements GarbageCollector {
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 @@ public class ScanAndCompareGarbageCollector implements GarbageCollector {
if (!metadata.getValue().isClosed()) {
return;
}
- SortedMap<Long, ? extends List<BookieSocketAddress>> ensembles =
+ SortedMap<Long, ? extends List<BookieId>> ensembles =
metadata.getValue().getAllEnsembles();
- for (List<BookieSocketAddress> ensemble : ensembles.values()) {
+ for (List<BookieId> 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 438dd95..393dde9 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 abstract class BKException extends org.apache.bookkeeper.client.api.BKExc
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 0638a87..b0c038e 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.MetadataClientDriver;
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 class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper {
.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 class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper {
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 @@ public class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper {
DNSToSwitchMapping dnsResolver,
HashedWheelTimer timer,
FeatureProvider featureProvider,
- StatsLogger statsLogger)
+ StatsLogger statsLogger,
+ BookieAddressResolver bookieAddressResolver)
throws IOException {
try {
Class<? extends EnsemblePlacementPolicy> 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 class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper {
}
void checkForFaultyBookies() {
- List<BookieSocketAddress> faultyBookies = bookieClient.getFaultyBookies();
- for (BookieSocketAddress faultyBookie : faultyBookies) {
+ List<BookieId> faultyBookies = bookieClient.getFaultyBookies();
+ for (BookieId faultyBookie : faultyBookies) {
if (Math.random() <= bookieQuarantineRatio) {
bookieWatcher.quarantineBookie(faultyBookie);
statsLogger.getCounter(BookKeeperServerStats.BOOKIE_QUARANTINE).inc();
@@ -644,6 +650,10 @@ public class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper {
return bookieWatcher;
}
+ public BookieAddressResolver getBookieAddressResolver() {
+ return bookieWatcher.getBookieAddressResolver();
+ }
+
public OrderedExecutor getMainWorkerPool() {
return mainWorkerPool;
}
@@ -754,7 +764,7 @@ public class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper {
* @throws BKException
* @throws InterruptedException
*/
- public Map<BookieSocketAddress, BookieInfo> getBookieInfo() throws BKException, InterruptedException {
+ public Map<BookieId, BookieInfo> 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 c9cd86f..7dbf4df 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;
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.LedgerManagerFactory;
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 class BookKeeperAdmin implements AutoCloseable {
*
* @return a collection of bookie addresses
*/
- public Collection<BookieSocketAddress> getAvailableBookies()
+ public Collection<BookieId> getAvailableBookies()
throws BKException {
return bkc.bookieWatcher.getBookies();
}
@@ -224,16 +218,20 @@ public class BookKeeperAdmin implements AutoCloseable {
*
* @return a collection of bookie addresses
*/
- public Collection<BookieSocketAddress> getAllBookies()
+ public Collection<BookieId> 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 class BookKeeperAdmin implements AutoCloseable {
* @return a collection of bookie addresses
* @throws BKException if there are issues trying to read the list.
*/
- public Collection<BookieSocketAddress> getReadOnlyBookies() throws BKException {
+ public Collection<BookieId> getReadOnlyBookies() throws BKException {
return bkc.bookieWatcher.getReadOnlyBookies();
}
@@ -479,7 +477,7 @@ public class BookKeeperAdmin implements AutoCloseable {
}
}
- public SortedMap<Long, LedgerMetadata> getLedgersContainBookies(Set<BookieSocketAddress> bookies)
+ public SortedMap<Long, LedgerMetadata> getLedgersContainBookies(Set<BookieId> bookies)
throws InterruptedException, BKException {
final SyncObject sync = new SyncObject();
final AtomicReference<SortedMap<Long, LedgerMetadata>> resultHolder =
@@ -507,7 +505,7 @@ public class BookKeeperAdmin implements AutoCloseable {
return resultHolder.get();
}
- public void asyncGetLedgersContainBookies(final Set<BookieSocketAddress> bookies,
+ public void asyncGetLedgersContainBookies(final Set<BookieId> bookies,
final GenericCallback<SortedMap<Long, LedgerMetadata>> callback) {
final SortedMap<Long, LedgerMetadata> ledgers = new ConcurrentSkipListMap<Long, LedgerMetadata>();
bkc.getLedgerManager().asyncProcessLedgers(new Processor<Long>() {
@@ -524,9 +522,9 @@ public class BookKeeperAdmin implements AutoCloseable {
cb.processResult(BKException.getExceptionCode(exception), null, null);
return;
}
- Set<BookieSocketAddress> bookiesInLedger =
+ Set<BookieId> bookiesInLedger =
LedgerMetadataUtils.getBookiesInThisLedger(metadata.getValue());
- Sets.SetView<BookieSocketAddress> intersection =
+ Sets.SetView<BookieId> intersection =
Sets.intersection(bookiesInLedger, bookies);
if (!intersection.isEmpty()) {
ledgers.put(lid, metadata.getValue());
@@ -557,18 +555,18 @@ public class BookKeeperAdmin implements AutoCloseable {
* 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<BookieSocketAddress> bookiesSrc = Sets.newHashSet(bookieSrc);
+ Set<BookieId> bookiesSrc = Sets.newHashSet(bookieSrc);
recoverBookieData(bookiesSrc);
}
- public void recoverBookieData(final Set<BookieSocketAddress> bookiesSrc)
+ public void recoverBookieData(final Set<BookieId> bookiesSrc)
throws InterruptedException, BKException {
recoverBookieData(bookiesSrc, false, false);
}
- public void recoverBookieData(final Set<BookieSocketAddress> bookiesSrc, boolean dryrun, boolean skipOpenLedgers)
+ public void recoverBookieData(final Set<BookieId> 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 class BookKeeperAdmin implements AutoCloseable {
}
public void recoverBookieData(final long lid,
- final Set<BookieSocketAddress> bookiesSrc,
+ final Set<BookieId> bookiesSrc,
boolean dryrun,
boolean skipOpenLedgers)
throws InterruptedException, BKException {
@@ -644,18 +642,18 @@ public class BookKeeperAdmin implements AutoCloseable {
* @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<BookieSocketAddress> bookiesSrc = Sets.newHashSet(bookieSrc);
+ Set<BookieId> bookiesSrc = Sets.newHashSet(bookieSrc);
asyncRecoverBookieData(bookiesSrc, cb, context);
}
- public void asyncRecoverBookieData(final Set<BookieSocketAddress> bookieSrc,
+ public void asyncRecoverBookieData(final Set<BookieId> bookieSrc,
final RecoverCallback cb, final Object context) {
asyncRecoverBookieData(bookieSrc, false, false, cb, context);
}
- public void asyncRecoverBookieData(final Set<BookieSocketAddress> bookieSrc, boolean dryrun,
+ public void asyncRecoverBookieData(final Set<BookieId> bookieSrc, boolean dryrun,
final boolean skipOpenLedgers, final RecoverCallback cb, final Object context) {
getActiveLedgers(bookieSrc, dryrun, skipOpenLedgers, cb, context);
}
@@ -677,7 +675,7 @@ public class BookKeeperAdmin implements AutoCloseable {
* @param context
* Context for the RecoverCallback to call.
*/
- public void asyncRecoverBookieData(long lid, final Set<BookieSocketAddress> bookieSrc, boolean dryrun,
+ public void asyncRecoverBookieData(long lid, final Set<BookieId> 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 class BookKeeperAdmin implements AutoCloseable {
* @param context
* Context for the RecoverCallback to call.
*/
- private void getActiveLedgers(final Set<BookieSocketAddress> bookiesSrc, final boolean dryrun,
+ private void getActiveLedgers(final Set<BookieId> 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 @@ public class BookKeeperAdmin implements AutoCloseable {
* IterationCallback to invoke once we've recovered the current
* ledger.
*/
- private void recoverLedger(final Set<BookieSocketAddress> bookiesSrc, final long lId, final boolean dryrun,
+ private void recoverLedger(final Set<BookieId> 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 class BookKeeperAdmin implements AutoCloseable {
*/
Map<Long, Long> ledgerFragmentsRange = new HashMap<Long, Long>();
Long curEntryId = null;
- for (Map.Entry<Long, ? extends List<BookieSocketAddress>> entry :
+ for (Map.Entry<Long, ? extends List<BookieId>> entry :
lh.getLedgerMetadata().getAllEnsembles().entrySet()) {
if (curEntryId != null) {
ledgerFragmentsRange.put(curEntryId, entry.getKey() - 1);
@@ -887,9 +885,9 @@ public class BookKeeperAdmin implements AutoCloseable {
*/
for (final Long startEntryId : ledgerFragmentsToRecover) {
Long endEntryId = ledgerFragmentsRange.get(startEntryId);
- List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getAllEnsembles().get(startEntryId);
+ List<BookieId> ensemble = lh.getLedgerMetadata().getAllEnsembles().get(startEntryId);
// Get bookies to replace
- Map<Integer, BookieSocketAddress> targetBookieAddresses;
+ Map<Integer, BookieId> targetBookieAddresses;
try {
targetBookieAddresses = getReplacementBookies(lh, ensemble, bookiesSrc);
} catch (BKException.BKNotEnoughBookiesException e) {
@@ -903,7 +901,7 @@ public class BookKeeperAdmin implements AutoCloseable {
}
if (dryrun) {
- ArrayList<BookieSocketAddress> newEnsemble =
+ ArrayList<BookieId> newEnsemble =
replaceBookiesInEnsemble(ensemble, targetBookieAddresses);
VERBOSE.info(" Fragment [{} - {}] : ", startEntryId, endEntryId);
VERBOSE.info(" old ensemble : {}", formatEnsemble(ensemble, bookiesSrc, '*'));
@@ -935,7 +933,7 @@ public class BookKeeperAdmin implements AutoCloseable {
}, null);
}
- static String formatEnsemble(List<BookieSocketAddress> ensemble, Set<BookieSocketAddress> bookiesSrc,
+ static String formatEnsemble(List<BookieId> ensemble, Set<BookieId> bookiesSrc,
char marker) {
StringBuilder sb = new StringBuilder();
sb.append("[");
@@ -973,19 +971,19 @@ public class BookKeeperAdmin implements AutoCloseable {
private void asyncRecoverLedgerFragment(final LedgerHandle lh,
final LedgerFragment ledgerFragment,
final AsyncCallback.VoidCallback ledgerFragmentMcb,
- final Set<BookieSocketAddress> newBookies,
+ final Set<BookieId> newBookies,
final BiConsumer<Long, Long> onReadEntryFailureCallback) throws InterruptedException {
lfr.replicate(lh, ledgerFragment, ledgerFragmentMcb, newBookies, onReadEntryFailureCallback);
}
- private Map<Integer, BookieSocketAddress> getReplacementBookies(
+ private Map<Integer, BookieId> getReplacementBookies(
LedgerHandle lh,
- List<BookieSocketAddress> ensemble,
- Set<BookieSocketAddress> bookiesToRereplicate)
+ List<BookieId> ensemble,
+ Set<BookieId> bookiesToRereplicate)
throws BKException.BKNotEnoughBookiesException {
Set<Integer> 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 @@ public class BookKeeperAdmin implements AutoCloseable {
lh, ensemble, bookieIndexesToRereplicate, Optional.of(bookiesToRereplicate));
}
- private Map<Integer, BookieSocketAddress> getReplacementBookiesByIndexes(
+ private Map<Integer, BookieId> getReplacementBookiesByIndexes(
LedgerHandle lh,
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
Set<Integer> bookieIndexesToRereplicate,
- Optional<Set<BookieSocketAddress>> excludedBookies)
+ Optional<Set<BookieId>> excludedBookies)
throws BKException.BKNotEnoughBookiesException {
// target bookies to replicate
- Map<Integer, BookieSocketAddress> targetBookieAddresses =
+ Map<Integer, BookieId> targetBookieAddresses =
Maps.newHashMapWithExpectedSize(bookieIndexesToRereplicate.size());
// bookies to exclude for ensemble allocation
- Set<BookieSocketAddress> bookiesToExclude = Sets.newHashSet();
+ Set<BookieId> 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<BookieSocketAddress> replaceBookieResponse =
+ BookieId oldBookie = ensemble.get(bookieIndex);
+ EnsemblePlacementPolicy.PlacementResult<BookieId> replaceBookieResponse =
bkc.getPlacementPolicy().replaceBookie(
lh.getLedgerMetadata().getEnsembleSize(),
lh.getLedgerMetadata().getWriteQuorumSize(),
@@ -1027,7 +1025,7 @@ public class BookKeeperAdmin implements AutoCloseable {
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 @@ public class BookKeeperAdmin implements AutoCloseable {
return targetBookieAddresses;
}
- private ArrayList<BookieSocketAddress> replaceBookiesInEnsemble(
- List<BookieSocketAddress> ensemble,
- Map<Integer, BookieSocketAddress> replacedBookies) {
- ArrayList<BookieSocketAddress> newEnsemble = Lists.newArrayList(ensemble);
- for (Map.Entry<Integer, BookieSocketAddress> entry : replacedBookies.entrySet()) {
+ private ArrayList<BookieId> replaceBookiesInEnsemble(
+ List<BookieId> ensemble,
+ Map<Integer, BookieId> replacedBookies) {
+ ArrayList<BookieId> newEnsemble = Lists.newArrayList(ensemble);
+ for (Map.Entry<Integer, BookieId> entry : replacedBookies.entrySet()) {
newEnsemble.set(entry.getKey(), entry.getValue());
}
return newEnsemble;
@@ -1066,8 +1064,8 @@ public class BookKeeperAdmin implements AutoCloseable {
final LedgerFragment ledgerFragment,
final BiConsumer<Long, Long> onReadEntryFailureCallback)
throws InterruptedException, BKException {
- Optional<Set<BookieSocketAddress>> excludedBookies = Optional.empty();
- Map<Integer, BookieSocketAddress> targetBookieAddresses =
+ Optional<Set<BookieId>> excludedBookies = Optional.empty();
+ Map<Integer, BookieId> targetBookieAddresses =
getReplacementBookiesByIndexes(lh, ledgerFragment.getEnsemble(),
ledgerFragment.getBookiesIndexes(), excludedBookies);
replicateLedgerFragment(lh, ledgerFragment, targetBookieAddresses, onReadEntryFailureCallback);
@@ -1075,7 +1073,7 @@ public class BookKeeperAdmin implements AutoCloseable {
private void replicateLedgerFragment(LedgerHandle lh,
final LedgerFragment ledgerFragment,
- final Map<Integer, BookieSocketAddress> targetBookieAddresses,
+ final Map<Integer, BookieId> targetBookieAddresses,
final BiConsumer<Long, Long> onReadEntryFailureCallback)
throws InterruptedException, BKException {
CompletableFuture<Void> result = new CompletableFuture<>();
@@ -1087,7 +1085,7 @@ public class BookKeeperAdmin implements AutoCloseable {
ledgerFragment.getFirstEntryId(),
getReplacementBookiesMap(ledgerFragment, targetBookieAddresses));
- Set<BookieSocketAddress> targetBookieSet = Sets.newHashSet();
+ Set<BookieId> targetBookieSet = Sets.newHashSet();
targetBookieSet.addAll(targetBookieAddresses.values());
asyncRecoverLedgerFragment(lh, ledgerFragment, cb, targetBookieSet, onReadEntryFailureCallback);
@@ -1098,45 +1096,45 @@ public class BookKeeperAdmin implements AutoCloseable {
}
}
- private static Map<BookieSocketAddress, BookieSocketAddress> getReplacementBookiesMap(
- List<BookieSocketAddress> ensemble,
- Map<Integer, BookieSocketAddress> targetBookieAddresses) {
- Map<BookieSocketAddress, BookieSocketAddress> bookiesMap =
- new HashMap<BookieSocketAddress, BookieSocketAddress>();
- for (Map.Entry<Integer, BookieSocketAddress> entry : targetBookieAddresses.entrySet()) {
- BookieSocketAddress oldBookie = ensemble.get(entry.getKey());
- BookieSocketAddress newBookie = entry.getValue();
+ private static Map<BookieId, BookieId> getReplacementBookiesMap(
+ List<BookieId> ensemble,
+ Map<Integer, BookieId> targetBookieAddresses) {
+ Map<BookieId, BookieId> bookiesMap =
+ new HashMap<BookieId, BookieId>();
+ for (Map.Entry<Integer, BookieId> entry : targetBookieAddresses.entrySet()) {
+ BookieId oldBookie = ensemble.get(entry.getKey());
+ BookieId newBookie = entry.getValue();
bookiesMap.put(oldBookie, newBookie);
}
return bookiesMap;
}
- private static Map<BookieSocketAddress, BookieSocketAddress> getReplacementBookiesMap(
+ private static Map<BookieId, BookieId> getReplacementBookiesMap(
LedgerFragment ledgerFragment,
- Map<Integer, BookieSocketAddress> targetBookieAddresses) {
- Map<BookieSocketAddress, BookieSocketAddress> bookiesMap =
- new HashMap<BookieSocketAddress, BookieSocketAddress>();
+ Map<Integer, BookieId> targetBookieAddresses) {
+ Map<BookieId, BookieId> bookiesMap =
+ new HashMap<BookieId, BookieId>();
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<BookieSocketAddress> bookies) {
+ Set<BookieId> bookies) {
if (lm.getAllEnsembles().size() <= 0) {
return false;
}
Long lastKey = lm.getAllEnsembles().lastKey();
- List<BookieSocketAddress> lastEnsemble = lm.getAllEnsembles().get(lastKey);
+ List<BookieId> lastEnsemble = lm.getAllEnsembles().get(lastKey);
return containBookies(lastEnsemble, bookies);
}
- private static boolean containBookies(List<BookieSocketAddress> ensemble,
- Set<BookieSocketAddress> bookies) {
- for (BookieSocketAddress bookie : ensemble) {
+ private static boolean containBookies(List<BookieId> ensemble,
+ Set<BookieId> bookies) {
+ for (BookieId bookie : ensemble) {
if (bookies.contains(bookie)) {
return true;
}
@@ -1300,7 +1298,7 @@ public class BookKeeperAdmin implements AutoCloseable {
* 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 class BookKeeperAdmin implements AutoCloseable {
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 class BookKeeperAdmin implements AutoCloseable {
* @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 class BookKeeperAdmin implements AutoCloseable {
}
}
- private void waitForLedgersToBeReplicated(Collection<Long> ledgers, BookieSocketAddress thisBookieAddress,
+ private void waitForLedgersToBeReplicated(Collection<Long> 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 @@ public class BookKeeperAdmin implements AutoCloseable {
}
}
- 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 class BookKeeperAdmin implements AutoCloseable {
}
}
- public static boolean areEntriesOfLedgerStoredInTheBookie(long ledgerId, BookieSocketAddress bookieAddress,
+ public static boolean areEntriesOfLedgerStoredInTheBookie(long ledgerId, BookieId bookieAddress,
LedgerMetadata ledgerMetadata) {
- Collection<? extends List<BookieSocketAddress>> ensemblesOfSegments = ledgerMetadata.getAllEnsembles().values();
- Iterator<? extends List<BookieSocketAddress>> ensemblesOfSegmentsIterator = ensemblesOfSegments.iterator();
- List<BookieSocketAddress> ensemble;
+ Collection<? extends List<BookieId>> ensemblesOfSegments = ledgerMetadata.getAllEnsembles().values();
+ Iterator<? extends List<BookieId>> ensemblesOfSegmentsIterator = ensemblesOfSegments.iterator();
+ List<BookieId> ensemble;
int segmentNo = 0;
while (ensemblesOfSegmentsIterator.hasNext()) {
ensemble = ensemblesOfSegmentsIterator.next();
@@ -1595,14 +1593,14 @@ public class BookKeeperAdmin implements AutoCloseable {
}
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<Entry<Long, ? extends List<BookieSocketAddress>>> segments =
+ List<Entry<Long, ? extends List<BookieId>>> segments =
new LinkedList<>(ledgerMetadata.getAllEnsembles().entrySet());
- List<BookieSocketAddress> currentSegmentEnsemble = segments.get(segmentNo).getValue();
+ List<BookieId> currentSegmentEnsemble = segments.get(segmentNo).getValue();
boolean lastSegment = (segmentNo == (segments.size() - 1));
/*
@@ -1691,7 +1689,7 @@ public class BookKeeperAdmin implements AutoCloseable {
* @return <tt>true</tt> if the ledger is adhering to
* EnsemblePlacementPolicy
*/
- public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List<BookieSocketAddress> ensembleBookiesList,
+ public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List<BookieId> ensembleBookiesList,
int writeQuorumSize, int ackQuorumSize) {
return bkc.getPlacementPolicy().isEnsembleAdheringToPlacementPolicy(ensembleBookiesList, writeQuorumSize,
ackQuorumSize);
@@ -1702,12 +1700,12 @@ public class BookKeeperAdmin implements AutoCloseable {
* and returns Future for the result.
*
* @param address
- * BookieSocketAddress of the bookie
+ * BookieId of the bookie
* @param ledgerId
* ledgerId
* @return returns Future
*/
- public CompletableFuture<AvailabilityOfEntriesOfLedger> asyncGetListOfEntriesOfLedger(BookieSocketAddress address,
+ public CompletableFuture<AvailabilityOfEntriesOfLedger> 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 c0d412c..254260a 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.TimeUnit;
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 @@ public class BookieInfoReader {
* Contains the most recently obtained information on the contained bookies.
* When an error happens querying a bookie, the entry is removed.
*/
- private final Map<BookieSocketAddress, BookieInfo> infoMap = new HashMap<>();
+ private final Map<BookieId, BookieInfo> 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<BookieSocketAddress> mostRecentlyReportedBookies = new ArrayList<>();
+ private Collection<BookieId> mostRecentlyReportedBookies = new ArrayList<>();
- public void updateBookies(Collection<BookieSocketAddress> updatedBookieSet) {
+ public void updateBookies(Collection<BookieId> updatedBookieSet) {
if (LOG.isDebugEnabled()) {
LOG.debug(
"updateBookies: current: {}, new: {}",
@@ -115,11 +115,11 @@ public class BookieInfoReader {
}
@SuppressWarnings("unchecked")
- public Collection<BookieSocketAddress> getPartialScanTargets() {
+ public Collection<BookieId> getPartialScanTargets() {
return CollectionUtils.subtract(mostRecentlyReportedBookies, infoMap.keySet());
}
- public Collection<BookieSocketAddress> getFullScanTargets() {
+ public Collection<BookieId> getFullScanTargets() {
return mostRecentlyReportedBookies;
}
@@ -129,7 +129,7 @@ public class BookieInfoReader {
* @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 class BookieInfoReader {
*
* @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 class BookieInfoReader {
* @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<BookieSocketAddress, BookieInfo> getBookieMap() {
+ public Map<BookieId, BookieInfo> getBookieMap() {
return infoMap;
}
}
@@ -246,7 +246,7 @@ public class BookieInfoReader {
LOG.debug("Running periodic BookieInfo scan");
}
try {
- Collection<BookieSocketAddress> updatedBookies = bk.bookieWatcher.getBookies();
+ Collection<BookieId> 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 @@ public class BookieInfoReader {
scheduler.schedule(() -> getReadWriteBookieInfo(), delaySeconds, TimeUnit.SECONDS);
}
- synchronized void availableBookiesChanged(Set<BookieSocketAddress> updatedBookiesList) {
+ synchronized void availableBookiesChanged(Set<BookieId> updatedBookiesList) {
if (LOG.isInfoEnabled()) {
LOG.info("Scheduling bookie info read due to changes in available bookies.");
}
@@ -286,7 +286,7 @@ public class BookieInfoReader {
* @param bookie to lookup
* @return None if absent, free disk space if present
*/
- synchronized Optional<Long> getFreeDiskSpace(BookieSocketAddress bookie) {
+ synchronized Optional<Long> getFreeDiskSpace(BookieId bookie) {
BookieInfo bookieInfo = bookieInfoMap.getInfo(bookie);
if (bookieInfo != null) {
return Optional.of(bookieInfo.getFreeDiskSpace());
@@ -306,7 +306,7 @@ public class BookieInfoReader {
*/
synchronized void getReadWriteBookieInfo() {
State queuedType = instanceState.getAndClearQueuedType();
- Collection<BookieSocketAddress> toScan;
+ Collection<BookieId> toScan;
if (queuedType == State.FULL) {
if (LOG.isDebugEnabled()) {
LOG.debug("Doing full scan");
@@ -335,12 +335,12 @@ public class BookieInfoReader {
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 @@ public class BookieInfoReader {
}
}
- Map<BookieSocketAddress, BookieInfo> getBookieInfo() throws BKException, InterruptedException {
+ Map<BookieId, BookieInfo> getBookieInfo() throws BKException, InterruptedException {
BookieClient bkc = bk.getBookieClient();
final AtomicInteger totalSent = new AtomicInteger();
final AtomicInteger totalCompleted = new AtomicInteger();
- final ConcurrentMap<BookieSocketAddress, BookieInfo> map =
- new ConcurrentHashMap<BookieSocketAddress, BookieInfo>();
+ final ConcurrentMap<BookieId, BookieInfo> map =
+ new ConcurrentHashMap<BookieId, BookieInfo>();
final CountDownLatch latch = new CountDownLatch(1);
long requested = BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE
| BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE;
- Collection<BookieSocketAddress> bookies;
+ Collection<BookieId> 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 b881569..07b1695 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.Map;
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<BookieSocketAddress> getBookies() throws BKException;
- Set<BookieSocketAddress> getAllBookies() throws BKException;
- Set<BookieSocketAddress> getReadOnlyBookies() throws BKException;
+/**
+ * Watch for Bookkeeper cluster status.
+ */
+public interface BookieWatcher {
+ Set<BookieId> getBookies() throws BKException;
+ Set<BookieId> getAllBookies() throws BKException;
+ Set<BookieId> getReadOnlyBookies() throws BKException;
+ BookieAddressResolver getBookieAddressResolver();
/**
* Create an ensemble with given <i>ensembleSize</i> and <i>writeQuorumSize</i>.
@@ -39,7 +44,7 @@ interface BookieWatcher {
* @return list of bookies for new ensemble.
* @throws BKNotEnoughBookiesException
*/
- List<BookieSocketAddress> newEnsemble(int ensembleSize, int writeQuorumSize,
+ List<BookieId> newEnsemble(int ensembleSize, int writeQuorumSize,
int ackQuorumSize, Map<String, byte[]> customMetadata)
throws BKNotEnoughBookiesException;
@@ -52,10 +57,10 @@ interface BookieWatcher {
* @return the bookie to replace.
* @throws BKNotEnoughBookiesException
*/
- BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
+ BookieId replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
Map<String, byte[]> customMetadata,
- List<BookieSocketAddress> existingBookies, int bookieIdx,
- Set<BookieSocketAddress> excludeBookies)
+ List<BookieId> existingBookies, int bookieIdx,
+ Set<BookieId> excludeBookies)
throws BKNotEnoughBookiesException;
@@ -63,5 +68,5 @@ interface BookieWatcher {
* Quarantine <i>bookie</i> 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 f18799c..d68ded7 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.concurrent.FutureUtils;
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<BookieSocketAddress, Boolean> quarantinedBookies;
+ final Cache<BookieId, Boolean> quarantinedBookies;
- private volatile Set<BookieSocketAddress> writableBookies = Collections.emptySet();
- private volatile Set<BookieSocketAddress> readOnlyBookies = Collections.emptySet();
+ private volatile Set<BookieId> writableBookies = Collections.emptySet();
+ private volatile Set<BookieId> 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<BookieSocketAddress, Boolean>() {
+ .removalListener(new RemovalListener<BookieId, Boolean>() {
@Override
- public void onRemoval(RemovalNotification<BookieSocketAddress, Boolean> bookie) {
+ public void onRemoval(RemovalNotification<BookieId, Boolean> bookie) {
log.info("Bookie {} is no longer quarantined", bookie.getKey());
}
@@ -133,7 +138,7 @@ class BookieWatcherImpl implements BookieWatcher {
}
@Override
- public Set<BookieSocketAddress> getBookies() throws BKException {
+ public Set<BookieId> getBookies() throws BKException {
try {
return FutureUtils.result(registrationClient.getWritableBookies(), EXCEPTION_FUNC).getValue();
} catch (BKInterruptedException ie) {
@@ -143,7 +148,7 @@ class BookieWatcherImpl implements BookieWatcher {
}
@Override
- public Set<BookieSocketAddress> getAllBookies() throws BKException {
+ public Set<BookieId> getAllBookies() throws BKException {
try {
return FutureUtils.result(registrationClient.getAllBookies(), EXCEPTION_FUNC).getValue();
} catch (BKInterruptedException ie) {
@@ -153,7 +158,12 @@ class BookieWatcherImpl implements BookieWatcher {
}
@Override
- public Set<BookieSocketAddress> getReadOnlyBookies()
+ public BookieAddressResolver getBookieAddressResolver() {
+ return this.bookieAddressResolver;
+ }
+
+ @Override
+ public Set<BookieId> getReadOnlyBookies()
throws BKException {
try {
return FutureUtils.result(registrationClient.getReadOnlyBookies(), EXCEPTION_FUNC).getValue();
@@ -164,7 +174,7 @@ class BookieWatcherImpl implements BookieWatcher {
}
// this callback is already not executed in zookeeper thread
- private synchronized void processWritableBookiesChanged(Set<BookieSocketAddress> newBookieAddrs) {
+ private synchronized void processWritableBookiesChanged(Set<BookieId> 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 @@ class BookieWatcherImpl implements BookieWatcher {
// }
}
- private synchronized void processReadOnlyBookiesChanged(Set<BookieSocketAddress> readOnlyBookies) {
+ private synchronized void processReadOnlyBookiesChanged(Set<BookieId> readOnlyBookies) {
this.readOnlyBookies = readOnlyBookies;
placementPolicy.onClusterChanged(writableBookies, readOnlyBookies);
}
@@ -196,6 +206,7 @@ class BookieWatcherImpl implements BookieWatcher {
* @throws BKException when failed to read bookies
*/
public void initialBlockingBookieRead() throws BKException {
+
CompletableFuture<?> writable;
CompletableFuture<?> readonly;
synchronized (this) {
@@ -214,7 +225,6 @@ class BookieWatcherImpl implements BookieWatcher {
readonly = initialReadonlyBookiesFuture;
}
}
-
try {
FutureUtils.result(writable, EXCEPTION_FUNC);
} catch (BKInterruptedException ie) {
@@ -232,17 +242,17 @@ class BookieWatcherImpl implements BookieWatcher {
}
@Override
- public List<BookieSocketAddress> newEnsemble(int ensembleSize, int writeQuorumSize,
+ public List<BookieId> newEnsemble(int ensembleSize, int writeQuorumSize,
int ackQuorumSize, Map<String, byte[]> customMetadata)
throws BKNotEnoughBookiesException {
long startTime = MathUtils.nowInNano();
- EnsemblePlacementPolicy.PlacementResult<List<BookieSocketAddress>> newEnsembleResponse;
- List<BookieSocketAddress> socketAddresses;
+ EnsemblePlacementPolicy.PlacementResult<List<BookieId>> newEnsembleResponse;
+ List<BookieId> socketAddresses;
PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy;
try {
- Set<BookieSocketAddress> quarantinedBookiesSet = quarantinedBookies.asMap().keySet();
+ Set<BookieId> quarantinedBookiesSet = quarantinedBookies.asMap().keySet();
newEnsembleResponse = placementPolicy.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize,
- customMetadata, new HashSet<BookieSocketAddress>(quarantinedBookiesSet));
+ customMetadata, new HashSet<BookieId>(quarantinedBookiesSet));
socketAddresses = newEnsembleResponse.getResult();
isEnsembleAdheringToPlacementPolicy = newEnsembleResponse.isAdheringToPolicy();
if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) {
@@ -272,21 +282,21 @@ class BookieWatcherImpl implements BookieWatcher {
}
@Override
- public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
+ public BookieId replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
Map<String, byte[]> customMetadata,
- List<BookieSocketAddress> existingBookies, int bookieIdx,
- Set<BookieSocketAddress> excludeBookies)
+ List<BookieId> existingBookies, int bookieIdx,
+ Set<BookieId> excludeBookies)
throws BKNotEnoughBookiesException {
long startTime = MathUtils.nowInNano();
- BookieSocketAddress addr = existingBookies.get(bookieIdx);
- EnsemblePlacementPolicy.PlacementResult<BookieSocketAddress> replaceBookieResponse;
- BookieSocketAddress socketAddress;
+ BookieId addr = existingBookies.get(bookieIdx);
+ EnsemblePlacementPolicy.PlacementResult<BookieId> replaceBookieResponse;
+ BookieId socketAddress;
PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = PlacementPolicyAdherence.FAIL;
try {
// we exclude the quarantined bookies also first
- Set<BookieSocketAddress> excludedBookiesAndQuarantinedBookies = new HashSet<BookieSocketAddress>(
+ Set<BookieId> excludedBookiesAndQuarantinedBookies = new HashSet<BookieId>(
excludeBookies);
- Set<BookieSocketAddress> quarantinedBookiesSet = quarantinedBookies.asMap().keySet();
+ Set<BookieId> quarantinedBookiesSet = quarantinedBookies.asMap().keySet();
excludedBookiesAndQuarantinedBookies.addAll(quarantinedBookiesSet);
replaceBookieResponse = placementPolicy.replaceBookie(
ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata,
@@ -326,11 +336,12 @@ class BookieWatcherImpl implements BookieWatcher {
* @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 b040440..32a6a91 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 0000000..dab93be
--- /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 8569c71..811766b 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.BookieInfoReader.BookieInfo;
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 @@ import org.slf4j.LoggerFactory;
*/
public class DefaultEnsemblePlacementPolicy implements EnsemblePlacementPolicy {
static final Logger LOG = LoggerFactory.getLogger(DefaultEnsemblePlacementPolicy.class);
- static final Set<BookieSocketAddress> EMPTY_SET = new HashSet<BookieSocketAddress>();
+ static final Set<BookieId> EMPTY_SET = new HashSet<BookieId>();
private boolean isWeighted;
private int maxWeightMultiple;
- private Set<BookieSocketAddress> knownBookies = new HashSet<BookieSocketAddress>();
- private Map<BookieSocketAddress, WeightedObject> bookieInfoMap;
- private WeightedRandomSelection<BookieSocketAddress> weightedSelection;
+ private Set<BookieId> knownBookies = new HashSet<BookieId>();
+ private Map<BookieId, WeightedObject> bookieInfoMap;
+ private WeightedRandomSelection<BookieId> weightedSelection;
private final ReentrantReadWriteLock rwLock;
DefaultEnsemblePlacementPolicy() {
- bookieInfoMap = new HashMap<BookieSocketAddress, WeightedObject>();
+ bookieInfoMap = new HashMap<BookieId, WeightedObject>();
rwLock = new ReentrantReadWriteLock();
}
@Override
- public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize, int quorumSize, int ackQuorumSize,
- Map<String, byte[]> customMetadata, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<List<BookieId>> newEnsemble(int ensembleSize, int quorumSize, int ackQuorumSize,
+ Map<String, byte[]> customMetadata, Set<BookieId> excludeBookies)
throws BKNotEnoughBookiesException {
- ArrayList<BookieSocketAddress> newBookies = new ArrayList<BookieSocketAddress>(ensembleSize);
+ ArrayList<BookieId> newBookies = new ArrayList<BookieId>(ensembleSize);
if (ensembleSize <= 0) {
return PlacementResult.of(newBookies, PlacementPolicyAdherence.FAIL);
}
- List<BookieSocketAddress> allBookies;
+ List<BookieId> allBookies;
rwLock.readLock().lock();
try {
- allBookies = new ArrayList<BookieSocketAddress>(knownBookies);
+ allBookies = new ArrayList<BookieId>(knownBookies);
} finally {
rwLock.readLock().unlock();
}
@@ -88,7 +89,7 @@ public class DefaultEnsemblePlacementPolicy implements EnsemblePlacementPolicy {
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 class DefaultEnsemblePlacementPolicy implements EnsemblePlacementPolicy {
}
} else {
Collections.shuffle(allBookies);
- for (BookieSocketAddress bookie : allBookies) {
+ for (BookieId bookie : allBookies) {
if (excludeBookies.contains(bookie)) {
continue;
}
@@ -120,37 +121,37 @@ public class DefaultEnsemblePlacementPolicy implements EnsemblePlacementPolicy {
}
@Override
- public PlacementResult<BookieSocketAddress> replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
- Map<String, byte[]> customMetadata, List<BookieSocketAddress> currentEnsemble,
- BookieSocketAddress bookieToReplace, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<BookieId> replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
+ Map<String, byte[]> customMetadata, List<BookieId> currentEnsemble,
+ BookieId bookieToReplace, Set<BookieId> excludeBookies)
throws BKNotEnoughBookiesException {
excludeBookies.addAll(currentEnsemble);
- List<BookieSocketAddress> addresses = newEnsemble(1, 1, 1, customMetadata, excludeBookies).getResult();
+ List<BookieId> addresses = newEnsemble(1, 1, 1, customMetadata, excludeBookies).getResult();
- BookieSocketAddress candidateAddr = addresses.get(0);
- List<BookieSocketAddress> newEnsemble = new ArrayList<BookieSocketAddress>(currentEnsemble);
+ BookieId candidateAddr = addresses.get(0);
+ List<BookieId> newEnsemble = new ArrayList<BookieId>(currentEnsemble);
newEnsemble.set(currentEnsemble.indexOf(bookieToReplace), candidateAddr);
return PlacementResult.of(candidateAddr,
isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize));
}
@Override
- public Set<BookieSocketAddress> onClusterChanged(Set<BookieSocketAddress> writableBookies,
- Set<BookieSocketAddress> readOnlyBookies) {
+ public Set<BookieId> onClusterChanged(Set<BookieId> writableBookies,
+ Set<BookieId> readOnlyBookies) {
rwLock.writeLock().lock();
try {
- HashSet<BookieSocketAddress> deadBookies;
- deadBookies = new HashSet<BookieSocketAddress>(knownBookies);
+ HashSet<BookieId> deadBookies;
+ deadBookies = new HashSet<BookieId>(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<BookieSocketAddress> newBookies = CollectionUtils.subtract(writableBookies, knownBookies);
- for (BookieSocketAddress b : newBookies) {
+ Collection<BookieId> 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 class DefaultEnsemblePlacementPolicy implements EnsemblePlacementPolicy {
}
@Override
- public void registerSlowBookie(BookieSocketAddress bookieSocketAddress, long entryId) {
+ public void registerSlowBookie(BookieId bookieSocketAddress, long entryId) {
return;
}
@Override
public DistributionSchedule.WriteSet reorderReadSequence(
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo,
DistributionSchedule.WriteSet writeSet) {
return writeSet;
@@ -179,7 +180,7 @@ public class DefaultEnsemblePlacementPolicy implements EnsemblePlacementPolicy {
@Override
public DistributionSchedule.WriteSet reorderReadLACSequence(
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo,
DistributionSchedule.WriteSet writeSet) {
writeSet.addMissingIndices(ensemble.size());
@@ -188,23 +189,24 @@ public class DefaultEnsemblePlacementPolicy implements EnsemblePlacementPolicy {
@Override
public EnsemblePlacementPolicy initialize(ClientConfiguration conf,
- Optional<DNSToSwitchMapping> optionalDnsResolver,
- HashedWheelTimer timer,
- FeatureProvider featureProvider,
- StatsLogger statsLogger) {
+ Optional<DNSToSwitchMapping> optionalDnsResolver,
+ HashedWheelTimer hashedWheelTimer,
+ FeatureProvider featureProvider,
+ StatsLogger statsLogger,
+ BookieAddressResolver bookieAddressResolver) {
this.isWeighted = conf.getDiskWeightBasedPlacementEnabled();
if (this.isWeighted) {
this.maxWeightMultiple = conf.getBookieMaxWeightMultipleForWeightBasedPlacement();
- this.weightedSelection = new WeightedRandomSelectionImpl<BookieSocketAddress>(this.maxWeightMultiple);
+ this.weightedSelection = new WeightedRandomSelectionImpl<BookieId>(this.maxWeightMultiple);
}
return this;
}
@Override
- public void updateBookieInfo(Map<BookieSocketAddress, BookieInfo> bookieInfoMap) {
+ public void updateBookieInfo(Map<BookieId, BookieInfo> bookieInfoMap) {
rwLock.writeLock().lock();
try {
- for (Map.Entry<BookieSocketAddress, BookieInfo> e : bookieInfoMap.entrySet()) {
+ for (Map.Entry<BookieId, BookieInfo> e : bookieInfoMap.entrySet()) {
this.bookieInfoMap.put(e.getKey(), e.getValue());
}
this.weightedSelection.updateMap(this.bookieInfoMap);
@@ -219,7 +221,7 @@ public class DefaultEnsemblePlacementPolicy implements EnsemblePlacementPolicy {
}
@Override
- public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List<BookieSocketAddress> ensembleList,
+ public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List<BookieId> 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 dc65346..4628dff 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 @@ package org.apache.bookkeeper.client;
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 @@ public interface DistributionSchedule {
* 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<Integer, BookieSocketAddress> getFailedBookies();
+ Map<Integer, BookieId> 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 f60f507..fcd38f2 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.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;
+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 @@ public interface EnsemblePlacementPolicy {
Optional<DNSToSwitchMapping> optionalDnsResolver,
HashedWheelTimer hashedWheelTimer,
FeatureProvider featureProvider,
- StatsLogger statsLogger);
+ StatsLogger statsLogger,
+ BookieAddressResolver bookieAddressResolver);
/**
* Uninitialize the policy.
@@ -239,8 +241,8 @@ public interface EnsemblePlacementPolicy {
* All the bookies in the cluster available for readonly.
* @return the dead bookies during this cluster change.
*/
- Set<BookieSocketAddress> onClusterChanged(Set<BookieSocketAddress> writableBookies,
- Set<BookieSocketAddress> readOnlyBookies);
+ Set<BookieId> onClusterChanged(Set<BookieId> writableBookies,
+ Set<BookieId> readOnlyBookies);
/**
* Choose <i>numBookies</i> bookies for ensemble. If the count is more than the number of available
@@ -267,11 +269,11 @@ public interface EnsemblePlacementPolicy {
* @throws BKNotEnoughBookiesException if not enough bookies available.
* @return a placement result containing list of bookie addresses for the ensemble.
*/
- PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize,
+ PlacementResult<List<BookieId>> newEnsemble(int ensembleSize,
int writeQuorumSize,
int ackQuorumSize,
Map<String, byte[]> customMetadata,
- Set<BookieSocketAddress> excludeBookies)
+ Set<BookieId> excludeBookies)
throws BKNotEnoughBookiesException;
/**
@@ -294,13 +296,13 @@ public interface EnsemblePlacementPolicy {
* @throws BKNotEnoughBookiesException
* @return a placement result containing the new bookie address.
*/
- PlacementResult<BookieSocketAddress> replaceBookie(int ensembleSize,
+ PlacementResult<BookieId> replaceBookie(int ensembleSize,
int writeQuorumSize,
int ackQuorumSize,
Map<String, byte[]> customMetadata,
- List<BookieSocketAddress> currentEnsemble,
- BookieSocketAddress bookieToReplace,
- Set<BookieSocketAddress> excludeBookies)
+ List<BookieId> currentEnsemble,
+ BookieId bookieToReplace,
+ Set<BookieId> excludeBookies)
throws BKNotEnoughBookiesException;
/**
@@ -311,7 +313,7 @@ public interface EnsemblePlacementPolicy {
* @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 <i>writeSet</i>.
@@ -328,7 +330,7 @@ public interface EnsemblePlacementPolicy {
* @since 4.5
*/
DistributionSchedule.WriteSet reorderReadSequence(
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo,
DistributionSchedule.WriteSet writeSet);
@@ -348,7 +350,7 @@ public interface EnsemblePlacementPolicy {
* @since 4.5
*/
DistributionSchedule.WriteSet reorderReadLACSequence(
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo,
DistributionSchedule.WriteSet writeSet);
@@ -359,7 +361,7 @@ public interface EnsemblePlacementPolicy {
* A map that has the bookie to BookieInfo
* @since 4.5
*/
- default void updateBookieInfo(Map<BookieSocketAddress, BookieInfo> bookieInfoMap) {
+ default void updateBookieInfo(Map<BookieId, BookieInfo> bookieInfoMap) {
}
/**
@@ -408,14 +410,14 @@ public interface EnsemblePlacementPolicy {
* 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<BookieSocketAddress> ensembleList,
+ default PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List<BookieId> ensembleList,
int writeQuorumSize, int ackQuorumSize) {
return PlacementPolicyAdherence.FAIL;
}
@@ -426,14 +428,14 @@ public interface EnsemblePlacementPolicy {
* 'minNumRacksPerWriteQuorum' number of racks.
*
* @param ackedBookies
- * list of BookieSocketAddress of bookies that have acknowledged a write.
+ * list of BookieId of bookies that have acknowledged a write.
* @param writeQuorumSize
* writeQuorumSize of the ensemble
* @param ackQuorumSize
* ackQuorumSize of the ensemble
* @return
*/
- default boolean areAckedBookiesAdheringToPlacementPolicy(Set<BookieSocketAddress> ackedBookies,
+ default boolean areAckedBookiesAdheringToPlacementPolicy(Set<BookieId> 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 6995a06..2126d24 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.Map;
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<BookieSocketAddress> replaceBookiesInEnsemble(BookieWatcher bookieWatcher,
+ static List<BookieId> replaceBookiesInEnsemble(BookieWatcher bookieWatcher,
LedgerMetadata metadata,
- List<BookieSocketAddress> oldEnsemble,
- Map<Integer, BookieSocketAddress> failedBookies,
+ List<BookieId> oldEnsemble,
+ Map<Integer, BookieId> failedBookies,
String logContext)
throws BKException.BKNotEnoughBookiesException {
- List<BookieSocketAddress> newEnsemble = new ArrayList<>(oldEnsemble);
+ List<BookieId> newEnsemble = new ArrayList<>(oldEnsemble);
int ensembleSize = metadata.getEnsembleSize();
int writeQ = metadata.getWriteQuorumSize();
int ackQ = metadata.getAckQuorumSize();
Map<String, byte[]> customMetadata = metadata.getCustomMetadata();
- Set<BookieSocketAddress> exclude = new HashSet<>(failedBookies.values());
+ Set<BookieId> exclude = new HashSet<>(failedBookies.values());
int replaced = 0;
- for (Map.Entry<Integer, BookieSocketAddress> entry : failedBookies.entrySet()) {
+ for (Map.Entry<Integer, BookieId> 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 @@ class EnsembleUtils {
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 @@ class EnsembleUtils {
return newEnsemble;
}
- static Set<Integer> diffEnsemble(List<BookieSocketAddress> e1,
- List<BookieSocketAddress> e2) {
+ static Set<Integer> diffEnsemble(List<BookieId> e1,
+ List<BookieId> e2) {
checkArgument(e1.size() == e2.size(), "Ensembles must be of same size");
Set<Integer> 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 2d785ab..23d547c 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 static com.google.common.base.Preconditions.checkState;
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<BookieSocketAddress> currentEnsemble;
+ final List<BookieId> 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<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
CompletableFuture<Void> cb) {
this.lh = lh;
this.bookieClient = bookieClient;
@@ -89,7 +89,7 @@ class ForceLedgerOp extends SafeRunnable implements ForceLedgerCallback {
}
@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 d6197c0..1abc31c 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.Ensemb
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 @@ public interface ITopologyAwareEnsemblePlacementPolicy<T extends Node> extends E
/**
* @return list of addresses representing the ensemble
*/
- List<BookieSocketAddress> toList();
+ List<BookieId> toList();
/**
* Validates if an ensemble is valid.
@@ -93,11 +93,11 @@ public interface ITopologyAwareEnsemblePlacementPolicy<T extends Node> extends E
* @return list of bookies forming the ensemble
* @throws BKException.BKNotEnoughBookiesException
*/
- PlacementResult<List<BookieSocketAddress>> newEnsemble(
+ PlacementResult<List<BookieId>> newEnsemble(
int ensembleSize,
int writeQuorumSize,
int ackQuorumSize,
- Set<BookieSocketAddress> excludeBookies,
+ Set<BookieId> excludeBookies,
Ensemble<T> parentEnsemble,
Predicate<T> parentPredicate)
throws BKException.BKNotEnoughBookiesException;
@@ -176,7 +176,7 @@ public interface ITopologyAwareEnsemblePlacementPolicy<T extends Node> extends E
* @param leftBookies
* bookies that left
*/
- void handleBookiesThatLeft(Set<BookieSocketAddress> leftBookies);
+ void handleBookiesThatLeft(Set<BookieId> leftBookies);
/**
* Handle bookies that joined.
@@ -184,12 +184,12 @@ public interface ITopologyAwareEnsemblePlacementPolicy<T extends Node> extends E
* @param joinedBookies
* bookies that joined.
*/
- void handleBookiesThatJoined(Set<BookieSocketAddress> joinedBookies);
+ void handleBookiesThatJoined(Set<BookieId> joinedBookies);
/**
* Handle rack change for the bookies.
*
* @param bookieAddressList
*/
- void onBookieRackChange(List<BookieSocketAddress> bookieAddressList);
+ void onBookieRackChange(List<BookieId> 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 6cc23cc..a0a9bb2 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.AtomicBoolean;
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 @@ public class LedgerChecker {
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 class LedgerChecker {
final Set<LedgerFragment> fragments = new HashSet<LedgerFragment>();
Long curEntryId = null;
- List<BookieSocketAddress> curEnsemble = null;
- for (Map.Entry<Long, ? extends List<BookieSocketAddress>> e : lh
+ List<BookieId> curEnsemble = null;
+ for (Map.Entry<Long, ? extends List<BookieId>> e : lh
.getLedgerMetadata().getAllEnsembles().entrySet()) {
if (curEntryId != null) {
Set<Integer> bookieIndexes = new HashSet<Integer>();
@@ -386,7 +386,7 @@ public class LedgerChecker {
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 9e9a2c4..557d0c7 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.WriteAdvHandle;
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 @@ class LedgerCreateOp {
// select bookies for first ensemble
try {
- List<BookieSocketAddress> ensemble = bk.getBookieWatcher()
+ List<BookieId> ensemble = bk.getBookieWatcher()
.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata);
metadataBuilder.newEnsembleEntry(0L, ensemble);
} catch (BKNotEnoughBookiesException e) {
@@ -217,10 +217,10 @@ class LedgerCreateOp {
return;
}
- List<BookieSocketAddress> curEns = lh.getLedgerMetadata().getEnsembleAt(0L);
+ List<BookieId> 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 dcf986f..1fb1e50 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.HashSet;
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 @@ import org.apache.bookkeeper.net.BookieSocketAddress;
*/
public class LedgerFragment {
private final Set<Integer> bookieIndexes;
- private final List<BookieSocketAddress> ensemble;
+ private final List<BookieId> 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<Long, ? extends List<BookieSocketAddress>> ensembles = lh
+ SortedMap<Long, ? extends List<BookieId>> ensembles = lh
.getLedgerMetadata().getAllEnsembles();
this.isLedgerClosed = lh.getLedgerMetadata().isClosed()
|| !ensemble.equals(ensembles.get(ensembles.lastKey()));
@@ -106,12 +106,12 @@ public class LedgerFragment {
/**
* Gets the failedBookie address.
*/
- public BookieSocketAddress getAddress(int bookieIndex) {
+ public BookieId getAddress(int bookieIndex) {
return ensemble.get(bookieIndex);
}
- public Set<BookieSocketAddress> getAddresses() {
- Set<BookieSocketAddress> addresses = new HashSet<BookieSocketAddress>();
+ public Set<BookieId> getAddresses() {
+ Set<BookieId> addresses = new HashSet<BookieId>();
for (int bookieIndex : bookieIndexes) {
addresses.add(ensemble.get(bookieIndex));
}
@@ -213,7 +213,7 @@ public class LedgerFragment {
*
* @return the ensemble for the segment which this fragment is a part of
*/
- public List<BookieSocketAddress> getEnsemble() {
+ public List<BookieId> 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 9510a96..b2c1933 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 java.util.stream.Collectors;
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 class LedgerFragmentReplicator {
private void replicateFragmentInternal(final LedgerHandle lh,
final LedgerFragment lf,
final AsyncCallback.VoidCallback ledgerFragmentMcb,
- final Set<BookieSocketAddress> newBookies,
+ final Set<BookieId> newBookies,
final BiConsumer<Long, Long> onReadEntryFailureCallback) throws InterruptedException {
if (!lf.isClosed()) {
LOG.error("Trying to replicate an unclosed fragment;"
@@ -184,7 +184,7 @@ public class LedgerFragmentReplicator {
*/
void replicate(final LedgerHandle lh, final LedgerFragment lf,
final AsyncCallback.VoidCallback ledgerFragmentMcb,
- final Set<BookieSocketAddress> targetBookieAddresses,
+ final Set<BookieId> targetBookieAddresses,
final BiConsumer<Long, Long> onReadEntryFailureCallback)
throws InterruptedException {
Set<LedgerFragment> partionedFragments = splitIntoSubFragments(lh, lf,
@@ -201,7 +201,7 @@ public class LedgerFragmentReplicator {
private void replicateNextBatch(final LedgerHandle lh,
final Iterator<LedgerFragment> fragments,
final AsyncCallback.VoidCallback ledgerFragmentMcb,
- final Set<BookieSocketAddress> targetBookieAddresses,
+ final Set<BookieId> targetBookieAddresses,
final BiConsumer<Long, Long> onReadEntryFailureCallback) {
if (fragments.hasNext()) {
try {
@@ -305,14 +305,14 @@ public class LedgerFragmentReplicator {
private void recoverLedgerFragmentEntry(final Long entryId,
final LedgerHandle lh,
final AsyncCallback.VoidCallback ledgerFragmentEntryMcb,
- final Set<BookieSocketAddress> newBookies,
+ final Set<BookieId> newBookies,
final BiConsumer<Long, Long> 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 class LedgerFragmentReplicator {
.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 @@ public class LedgerFragmentReplicator {
final LedgerHandle lh;
final LedgerManager ledgerManager;
final long fragmentStartId;
- final Map<BookieSocketAddress, BookieSocketAddress> oldBookie2NewBookie;
+ final Map<BookieId, BookieId> oldBookie2NewBookie;
SingleFragmentCallback(AsyncCallback.VoidCallback ledgerFragmentsMcb,
LedgerHandle lh, LedgerManager ledgerManager, long fragmentStartId,
- Map<BookieSocketAddress, BookieSocketAddress> oldBookie2NewBookie) {
+ Map<BookieId, BookieId> oldBookie2NewBookie) {
this.ledgerFragmentsMcb = ledgerFragmentsMcb;
this.lh = lh;
this.ledgerManager = ledgerManager;
@@ -414,7 +414,7 @@ public class LedgerFragmentReplicator {
*/
private static void updateEnsembleInfo(
LedgerManager ledgerManager, AsyncCallback.VoidCallback ensembleUpdatedCb, long fragmentStartId,
- LedgerHandle lh, Map<BookieSocketAddress, BookieSocketAddress> oldBookie2NewBookie) {
+ LedgerHandle lh, Map<BookieId, BookieId> oldBookie2NewBookie) {
MetadataUpdateLoop updateLoop = new MetadataUpdateLoop(
ledgerManager,
@@ -422,12 +422,12 @@ public class LedgerFragmentReplicator {
lh::getVersionedLedgerMetadata,
(metadata) -> {
// returns true if any of old bookies exist in ensemble
- List<BookieSocketAddress> ensemble = metadata.getAllEnsembles().get(fragmentStartId);
+ List<BookieId> ensemble = metadata.getAllEnsembles().get(fragmentStartId);
return oldBookie2NewBookie.keySet().stream().anyMatch(ensemble::contains);
},
(currentMetadata) -> {
// replace all old bookies with new bookies in ensemble
- List<BookieSocketAddress> newEnsemble = currentMetadata.getAllEnsembles().get(fragmentStartId)
+ List<BookieId> 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 f80d0bd..b773d96 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.client.impl.LedgerEntryImpl;
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 @@ public class LedgerHandle implements WriteHandle {
final DigestManager macManager;
final DistributionSchedule distributionSchedule;
final RateLimiter throttler;
- final LoadingCache<BookieSocketAddress, Long> bookieFailureHistory;
+ final LoadingCache<BookieId, Long> bookieFailureHistory;
final BookiesHealthInfo bookiesHealthInfo;
final EnumSet<WriteFlag> writeFlags;
ScheduledFuture<?> timeoutFuture = null;
@VisibleForTesting
- final Map<Integer, BookieSocketAddress> delayedWriteFailedBookies =
- new HashMap<Integer, BookieSocketAddress>();
+ final Map<Integer, BookieId> delayedWriteFailedBookies =
+ new HashMap<Integer, BookieId>();
/**
* Invalid entry id. This value is returned from methods which
@@ -222,21 +222,21 @@ public class LedgerHandle implements WriteHandle {
metadata.getEnsembleSize());
this.bookieFailureHistory = CacheBuilder.newBuilder()
.expireAfterWrite(clientCtx.getConf().bookieFailureHistoryExpirationMSec, TimeUnit.MILLISECONDS)
- .build(new CacheLoader<BookieSocketAddress, Long>() {
+ .build(new CacheLoader<BookieId, Long>() {
@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 class LedgerHandle implements WriteHandle {
* @return count of unique bookies
*/
public synchronized long getNumBookies() {
- Map<Long, ? extends List<BookieSocketAddress>> m = getLedgerMetadata().getAllEnsembles();
- Set<BookieSocketAddress> s = Sets.newHashSet();
- for (List<BookieSocketAddress> aList : m.values()) {
+ Map<Long, ? extends List<BookieId>> m = getLedgerMetadata().getAllEnsembles();
+ Set<BookieId> s = Sets.newHashSet();
+ for (List<BookieId> aList : m.values()) {
s.addAll(aList);
}
return s.size();
@@ -1222,7 +1222,7 @@ public class LedgerHandle implements WriteHandle {
final int requiredWritable = sz - allowedNonWritableCount;
int nonWritableCount = 0;
- List<BookieSocketAddress> currentEnsemble = getCurrentEnsemble();
+ List<BookieId> currentEnsemble = getCurrentEnsemble();
for (int i = 0; i < sz; i++) {
if (!clientCtx.getBookieClient().isWritable(currentEnsemble.get(i), key)) {
nonWritableCount++;
@@ -1830,7 +1830,7 @@ public class LedgerHandle implements WriteHandle {
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 @@ public class LedgerHandle implements WriteHandle {
if (delayedWriteFailedBookies.isEmpty()) {
return;
}
- Map<Integer, BookieSocketAddress> toReplace = new HashMap<>(delayedWriteFailedBookies);
+ Map<Integer, BookieId> toReplace = new HashMap<>(delayedWriteFailedBookies);
delayedWriteFailedBookies.clear();
// Original intent of this change is to do a best-effort ensemble change.
@@ -1852,7 +1852,7 @@ public class LedgerHandle implements WriteHandle {
}
}
- void handleBookieFailure(final Map<Integer, BookieSocketAddress> failedBookies) {
+ void handleBookieFailure(final Map<Integer, BookieId> 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 @@ public class LedgerHandle implements WriteHandle {
boolean triggerLoop = false;
- Map<Integer, BookieSocketAddress> toReplace = null;
- List<BookieSocketAddress> origEnsemble = null;
+ Map<Integer, BookieId> toReplace = null;
+ List<BookieId> origEnsemble = null;
synchronized (metadataLock) {
if (changingEnsemble) {
delayedWriteFailedBookies.putAll(failedBookies);
@@ -1895,7 +1895,7 @@ public class LedgerHandle implements WriteHandle {
}
}
- void ensembleChangeLoop(List<BookieSocketAddress> origEnsemble, Map<Integer, BookieSocketAddress> failedBookies) {
+ void ensembleChangeLoop(List<BookieId> origEnsemble, Map<Integer, BookieId> failedBookies) {
int ensembleChangeId = numEnsembleChanges.incrementAndGet();
String logContext = String.format("[EnsembleChange(ledger:%d, change-id:%010d)]", ledgerId, ensembleChangeId);
@@ -1922,8 +1922,8 @@ public class LedgerHandle implements WriteHandle {
(metadata) -> {
attempts.incrementAndGet();
- List<BookieSocketAddress> currentEnsemble = getCurrentEnsemble();
- List<BookieSocketAddress> newEnsemble = EnsembleUtils.replaceBookiesInEnsemble(
+ List<BookieId> currentEnsemble = getCurrentEnsemble();
+ List<BookieId> newEnsemble = EnsembleUtils.replaceBookiesInEnsemble(
clientCtx.getBookieWatcher(), metadata, currentEnsemble, failedBookies, logContext);
Long lastEnsembleKey = LedgerMetadataUtils.getLastEnsembleKey(metadata);
LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata);
@@ -1964,11 +1964,11 @@ public class LedgerHandle implements WriteHandle {
LOG.debug("{}[attempt:{}] Success updating metadata.", logContext, attempts.get());
}
- List<BookieSocketAddress> newEnsemble = null;
+ List<BookieId> newEnsemble = null;
Set<Integer> replaced = null;
synchronized (metadataLock) {
if (!delayedWriteFailedBookies.isEmpty()) {
- Map<Integer, BookieSocketAddress> toReplace = new HashMap<>(delayedWriteFailedBookies);
+ Map<Integer, BookieId> toReplace = new HashMap<>(delayedWriteFailedBookies);
delayedWriteFailedBookies.clear();
ensembleChangeLoop(origEnsemble, toReplace);
@@ -1987,7 +1987,7 @@ public class LedgerHandle implements WriteHandle {
}, clientCtx.getMainWorkerPool().chooseThread(ledgerId));
}
- void unsetSuccessAndSendWriteRequest(List<BookieSocketAddress> ensemble, final Set<Integer> bookies) {
+ void unsetSuccessAndSendWriteRequest(List<BookieId> ensemble, final Set<Integer> bookies) {
for (PendingAddOp pendingAddOp : pendingAddOps) {
for (Integer bookieIndex: bookies) {
pendingAddOp.unsetSuccessAndSendWriteRequest(ensemble, bookieIndex);
@@ -1995,7 +1995,7 @@ public class LedgerHandle implements WriteHandle {
}
}
- 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 class LedgerHandle implements WriteHandle {
* operations themselves, to avoid adding more dependencies between the classes.
* There are too many already.
*/
- List<BookieSocketAddress> getCurrentEnsemble() {
+ List<BookieId> 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 e4f75ce..a962fe7 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;
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<Long> lastEntryId = Optional.empty();
private Optional<Long> length = Optional.empty();
- private TreeMap<Long, List<BookieSocketAddress>> ensembles = new TreeMap<>();
+ private TreeMap<Long, List<BookieId>> ensembles = new TreeMap<>();
private Optional<DigestType> digestType = Optional.empty();
private Optional<byte[]> password = Optional.empty();
@@ -131,7 +131,7 @@ public class LedgerMetadataBuilder {
return this;
}
- public LedgerMetadataBuilder newEnsembleEntry(long firstEntry, List<BookieSocketAddress> ensemble) {
+ public LedgerMetadataBuilder newEnsembleEntry(long firstEntry, List<BookieId> 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 class LedgerMetadataBuilder {
return this;
}
- public LedgerMetadataBuilder replaceEnsembleEntry(long firstEntry, List<BookieSocketAddress> ensemble) {
+ public LedgerMetadataBuilder replaceEnsembleEntry(long firstEntry, List<BookieId> 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 842244a..81fb4b5 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 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.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<Long, ImmutableList<BookieSocketAddress>> ensembles;
- private final ImmutableList<BookieSocketAddress> currentEnsemble;
+ private final NavigableMap<Long, ImmutableList<BookieId>> ensembles;
+ private final ImmutableList<BookieId> currentEnsemble;
private final boolean hasPassword;
private final DigestType digestType;
@@ -78,7 +78,7 @@ class LedgerMetadataImpl implements LedgerMetadata {
State state,
Optional<Long> lastEntryId,
Optional<Long> length,
- Map<Long, List<BookieSocketAddress>> ensembles,
+ Map<Long, List<BookieId>> ensembles,
Optional<DigestType> digestType,
Optional<byte[]> password,
long ctime,
@@ -136,7 +136,7 @@ class LedgerMetadataImpl implements LedgerMetadata {
}
@Override
- public NavigableMap<Long, ? extends List<BookieSocketAddress>> getAllEnsembles() {
+ public NavigableMap<Long, ? extends List<BookieId>> getAllEnsembles() {
return ensembles;
}
@@ -210,7 +210,7 @@ class LedgerMetadataImpl implements LedgerMetadata {
}
@Override
- public List<BookieSocketAddress> getEnsembleAt(long entryId) {
+ public List<BookieId> 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 07d7340..0c9b6aa 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.List;
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 @@ import org.slf4j.LoggerFactory;
public class LedgerMetadataUtils {
static final Logger LOG = LoggerFactory.getLogger(LedgerMetadataUtils.class);
- static List<BookieSocketAddress> getCurrentEnsemble(LedgerMetadata metadata) {
+ static List<BookieId> getCurrentEnsemble(LedgerMetadata metadata) {
return getLastEnsembleValue(metadata);
}
@@ -46,7 +46,7 @@ public class LedgerMetadataUtils {
* @return the entry id of the next ensemble change (-1 if no further ensemble changes)
*/
static long getNextEnsembleChange(LedgerMetadata metadata, long entryId) {
- SortedMap<Long, ? extends List<BookieSocketAddress>> tailMap = metadata.getAllEnsembles().tailMap(entryId + 1);
+ SortedMap<Long, ? extends List<BookieId>> tailMap = metadata.getAllEnsembles().tailMap(entryId + 1);
if (tailMap.isEmpty()) {
return -1;
@@ -55,15 +55,15 @@ public class LedgerMetadataUtils {
}
}
- static Set<BookieSocketAddress> getBookiesInThisLedger(LedgerMetadata metadata) {
- Set<BookieSocketAddress> bookies = new HashSet<BookieSocketAddress>();
- for (List<BookieSocketAddress> ensemble : metadata.getAllEnsembles().values()) {
+ static Set<BookieId> getBookiesInThisLedger(LedgerMetadata metadata) {
+ Set<BookieId> bookies = new HashSet<BookieId>();
+ for (List<BookieId> ensemble : metadata.getAllEnsembles().values()) {
bookies.addAll(ensemble);
}
return bookies;
}
- static List<BookieSocketAddress> getLastEnsembleValue(LedgerMetadata metadata) {
+ static List<BookieId> 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 cd1a03f..b343d1a 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.RejectedExecutionException;
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<BookieSocketAddress> addEntrySuccessBookies;
+ Set<BookieId> 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<WriteFlag> writeFlags;
boolean allowFailFast = false;
- List<BookieSocketAddress> ensemble;
+ List<BookieId> ensemble;
static PendingAddOp create(LedgerHandle lh, ClientContext clientCtx,
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
ByteBuf payload, EnumSet<WriteFlag> writeFlags,
AddCallbackWithLatency cb, Object ctx) {
PendingAddOp op = RECYCLER.get();
@@ -146,7 +146,7 @@ class PendingAddOp extends SafeRunnable implements WriteCallback {
return this.entryId;
}
- void sendWriteRequest(List<BookieSocketAddress> ensemble, int bookieIndex) {
+ void sendWriteRequest(List<BookieId> ensemble, int bookieIndex) {
int flags = isRecoveryAdd ? FLAG_RECOVERY_ADD | FLAG_HIGH_PRIORITY : FLAG_NONE;
clientCtx.getBookieClient().addEntry(ensemble.get(bookieIndex),
@@ -188,7 +188,7 @@ class PendingAddOp extends SafeRunnable implements WriteCallback {
}
}
- void unsetSuccessAndSendWriteRequest(List<BookieSocketAddress> ensemble, int bookieIndex) {
+ void unsetSuccessAndSendWriteRequest(List<BookieId> ensemble, int bookieIndex) {
// update the ensemble
this.ensemble = ensemble;
@@ -283,7 +283,7 @@ class PendingAddOp extends SafeRunnable implements 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) {
int bookieIndex = (Integer) ctx;
--pendingWriteRequests;
@@ -360,7 +360,7 @@ class PendingAddOp extends SafeRunnable implements WriteCallback {
if (clientCtx.getConf().delayEnsembleChange) {
if (ackSet.failBookieAndCheck(bookieIndex, addr)
|| rc == BKException.Code.WriteOnReadOnlyBookieException) {
- Map<Integer, BookieSocketAddress> failedBookies = ackSet.getFailedBookies();
+ Map<Integer, BookieId> 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 0dad804..2a15adb 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 io.netty.buffer.ByteBuf;
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<BookieSocketAddress> currentEnsemble;
+ final List<BookieId> currentEnsemble;
/*
* Wrapper to get Lac from the request
@@ -63,7 +63,7 @@ class PendingReadLacOp implements ReadLacCallback {
void getLacComplete(int rc, long lac);
}
- PendingReadLacOp(LedgerHandle lh, BookieClient bookieClient, List<BookieSocketAddress> ensemble, LacCallback cb) {
+ PendingReadLacOp(LedgerHandle lh, BookieClient bookieClient, List<BookieId> 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 bfd9a5d..e6c011e 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.api.LedgerMetadata;
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<LedgerEntryRequest> seq;
private final CompletableFuture<LedgerEntries> future;
- private final Set<BookieSocketAddress> heardFromHosts;
+ private final Set<BookieId> heardFromHosts;
private final BitSet heardFromHostsBitSet;
- private final Set<BookieSocketAddress> sentToHosts = new HashSet<BookieSocketAddress>();
+ private final Set<BookieId> sentToHosts = new HashSet<BookieId>();
LedgerHandle lh;
final ClientContext clientCtx;
@@ -87,12 +87,12 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
int firstError = BKException.Code.OK;
int numBookiesMissingEntry = 0;
- final List<BookieSocketAddress> ensemble;
+ final List<BookieId> ensemble;
final DistributionSchedule.WriteSet writeSet;
final LedgerEntryImpl entryImpl;
final long eId;
- LedgerEntryRequest(List<BookieSocketAddress> ensemble, long lId, long eId) {
+ LedgerEntryRequest(List<BookieId> ensemble, long lId, long eId) {
this.entryImpl = LedgerEntryImpl.create(lId, eId);
this.ensemble = ensemble;
this.eId = eId;
@@ -130,7 +130,7 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
* @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 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
* @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 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
* 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 PendingReadOp implements ReadEntryCallback, SafeRunnable {
int numPendings;
- ParallelReadRequest(List<BookieSocketAddress> ensemble, long lId, long eId) {
+ ParallelReadRequest(List<BookieId> ensemble, long lId, long eId) {
super(ensemble, lId, eId);
numPendings = writeSet.size();
}
@@ -288,7 +288,7 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
@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 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
}
@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 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
}
@Override
- BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFromHostsBitSet) {
+ BookieId maybeSendSpeculativeRead(BitSet heardFromHostsBitSet) {
// no speculative read
return null;
}
@@ -330,7 +330,7 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
final BitSet sentReplicas;
final BitSet erroredReplicas;
- SequenceReadRequest(List<BookieSocketAddress> ensemble, long lId, long eId) {
+ SequenceReadRequest(List<BookieId> ensemble, long lId, long eId) {
super(ensemble, lId, eId);
this.sentReplicas = new BitSet(lh.getLedgerMetadata().getWriteQuorumSize());
@@ -362,7 +362,7 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
* @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 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
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 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
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 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
}
@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 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
}
@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 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
// 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 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
void initiate() {
long nextEnsembleChange = startEntryId, i = startEntryId;
this.requestTimeNanos = MathUtils.nowInNano();
- List<BookieSocketAddress> ensemble = null;
+ List<BookieId> ensemble = null;
do {
if (i == nextEnsembleChange) {
ensemble = getLedgerMetadata().getEnsembleAt(i);
@@ -542,11 +542,11 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
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 @@ class PendingReadOp implements ReadEntryCallback, SafeRunnable {
}
}
- 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 e9eb3f2..0b9a7d0 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.BitSet;
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<BookieSocketAddress> currentEnsemble;
+ final List<BookieId> currentEnsemble;
- PendingWriteLacOp(LedgerHandle lh, ClientContext clientCtx, List<BookieSocketAddress> ensemble,
+ PendingWriteLacOp(LedgerHandle lh, ClientContext clientCtx, List<BookieId> ensemble,
AddLacCallback cb, Object ctx) {
this.lh = lh;
this.clientCtx = clientCtx;
@@ -91,7 +91,7 @@ class PendingWriteLacOp implements WriteLacCallback {
}
@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 380f556..626b7cc 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.Map;
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 @@ public class RackawareEnsemblePlacementPolicy extends RackawareEnsemblePlacement
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 class RackawareEnsemblePlacementPolicy extends RackawareEnsemblePlacement
}
@Override
- public Set<BookieSocketAddress> onClusterChanged(Set<BookieSocketAddress> writableBookies,
- Set<BookieSocketAddress> readOnlyBookies) {
- Set<BookieSocketAddress> deadBookies = super.onClusterChanged(writableBookies, readOnlyBookies);
+ public Set<BookieId> onClusterChanged(Set<BookieId> writableBookies,
+ Set<BookieId> readOnlyBookies) {
+ Set<BookieId> deadBookies = super.onClusterChanged(writableBookies, readOnlyBookies);
if (null != slave) {
deadBookies = slave.onClusterChanged(writableBookies, readOnlyBookies);
}
@@ -94,8 +97,8 @@ public class RackawareEnsemblePlacementPolicy extends RackawareEnsemblePlacement
}
@Override
- public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize, int writeQuorumSize,
- int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<List<BookieId>> newEnsemble(int ensembleSize, int writeQuorumSize,
+ int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieId> excludeBookies)
throws BKException.BKNotEnoughBookiesException {
try {
return super.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, excludeBookies);
@@ -109,9 +112,9 @@ public class RackawareEnsemblePlacementPolicy extends RackawareEnsemblePlacement
}
@Override
- public PlacementResult<BookieSocketAddress> replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
- Map<String, byte[]> customMetadata, List<BookieSocketAddress> currentEnsemble,
- BookieSocketAddress bookieToReplace, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<BookieId> replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
+ Map<String, byte[]> customMetadata, List<BookieId> currentEnsemble,
+ BookieId bookieToReplace, Set<BookieId> excludeBookies)
throws BKException.BKNotEnoughBookiesException {
try {
return super.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata,
@@ -128,7 +131,7 @@ public class RackawareEnsemblePlacementPolicy extends RackawareEnsemblePlacement
@Override
public DistributionSchedule.WriteSet reorderReadSequence(
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo,
DistributionSchedule.WriteSet writeSet) {
return super.reorderReadSequence(ensemble, bookiesHealthInfo,
@@ -137,7 +140,7 @@ public class RackawareEnsemblePlacementPolicy extends RackawareEnsemblePlacement
@Override
public DistributionSchedule.WriteSet reorderReadLACSequence(
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo,
DistributionSchedule.WriteSet writeSet) {
return super.reorderReadLACSequence(ensemble, bookiesHealthInfo,
@@ -145,10 +148,10 @@ public class RackawareEnsemblePlacementPolicy extends RackawareEnsemblePlacement
}
@Override
- public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize,
+ public PlacementResult<List<BookieId>> newEnsemble(int ensembleSize,
int writeQuorumSize,
int ackQuorumSize,
- Set<BookieSocketAddress> excludeBookies,
+ Set<BookieId> excludeBookies,
Ensemble<BookieNode> parentEnsemble,
Predicate<BookieNode> parentPredicate)
throws BKException.BKNotEnoughBookiesException {
@@ -234,7 +237,7 @@ public class RackawareEnsemblePlacementPolicy extends RackawareEnsemblePlacement
}
@Override
- public void handleBookiesThatLeft(Set<BookieSocketAddress> leftBookies) {
+ public void handleBookiesThatLeft(Set<BookieId> leftBookies) {
super.handleBookiesThatLeft(leftBookies);
if (null != slave) {
slave.handleBookiesThatLeft(leftBookies);
@@ -242,7 +245,7 @@ public class RackawareEnsemblePlacementPolicy extends RackawareEnsemblePlacement
}
@Override
- public void handleBookiesThatJoined(Set<BookieSocketAddress> joinedBookies) {
+ public void handleBookiesThatJoined(Set<BookieId> 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 13eaddd..5568f0a 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 com.google.common.cache.CacheLoader;
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.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.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.Node;
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<BookieSocketAddress, Long> slowBookies;
+ protected Cache<BookieId, Long> slowBookies;
protected BookieNode localNode;
protected boolean reorderReadsRandom = false;
protected boolean enforceDurability = false;
@@ -159,9 +160,11 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
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 class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
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 class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
Optional<DNSToSwitchMapping> 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 class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
}
slowBookies = CacheBuilder.newBuilder()
.expireAfterWrite(conf.getBookieFailureHistoryExpirationMSec(), TimeUnit.MILLISECONDS)
- .build(new CacheLoader<BookieSocketAddress, Long>() {
+ .build(new CacheLoader<BookieId, Long>() {
@Override
- public Long load(BookieSocketAddress key) throws Exception {
+ public Long load(BookieId key) throws Exception {
return -1L;
}
});
@@ -296,7 +301,8 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
conf.getMinNumRacksPerWriteQuorum(),
conf.getEnforceMinNumRacksPerWriteQuorum(),
conf.getIgnoreLocalNodeInPlacementPolicy(),
- statsLogger);
+ statsLogger,
+ bookieAddressResolver);
}
@Override
@@ -307,16 +313,16 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
/*
* this method should be called in readlock scope of 'rwLock'
*/
- protected Set<BookieSocketAddress> addDefaultRackBookiesIfMinNumRacksIsEnforced(
- Set<BookieSocketAddress> excludeBookies) {
- Set<BookieSocketAddress> comprehensiveExclusionBookiesSet;
+ protected Set<BookieId> addDefaultRackBookiesIfMinNumRacksIsEnforced(
+ Set<BookieId> excludeBookies) {
+ Set<BookieId> comprehensiveExclusionBookiesSet;
if (enforceMinNumRacksPerWriteQuorum) {
- Set<BookieSocketAddress> bookiesInDefaultRack = null;
+ Set<BookieId> bookiesInDefaultRack = null;
Set<Node> defaultRackLeaves = topology.getLeaves(getDefaultRack());
for (Node node : defaultRackLeaves) {
if (node instanceof BookieNode) {
if (bookiesInDefaultRack == null) {
- bookiesInDefaultRack = new HashSet<BookieSocketAddress>(excludeBookies);
+ bookiesInDefaultRack = new HashSet<BookieId>(excludeBookies);
}
bookiesInDefaultRack.add(((BookieNode) node).getAddr());
} else {
@@ -326,7 +332,7 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
if ((bookiesInDefaultRack == null) || bookiesInDefaultRack.isEmpty()) {
comprehensiveExclusionBookiesSet = excludeBookies;
} else {
- comprehensiveExclusionBookiesSet = new HashSet<BookieSocketAddress>(excludeBookies);
+ comprehensiveExclusionBookiesSet = new HashSet<BookieId>(excludeBookies);
comprehensiveExclusionBookiesSet.addAll(bookiesInDefaultRack);
LOG.info("enforceMinNumRacksPerWriteQuorum is enabled, so Excluding bookies of defaultRack: {}",
bookiesInDefaultRack);
@@ -338,14 +344,14 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
}
@Override
- public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize, int writeQuorumSize,
- int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<List<BookieId>> newEnsemble(int ensembleSize, int writeQuorumSize,
+ int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieId> excludeBookies)
throws BKNotEnoughBookiesException {
rwLock.readLock().lock();
try {
- Set<BookieSocketAddress> comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced(
+ Set<BookieId> comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced(
excludeBookies);
- PlacementResult<List<BookieSocketAddress>> newEnsembleResult = newEnsembleInternal(ensembleSize,
+ PlacementResult<List<BookieId>> newEnsembleResult = newEnsembleInternal(ensembleSize,
writeQuorumSize, ackQuorumSize, comprehensiveExclusionBookiesSet, null, null);
return newEnsembleResult;
} finally {
@@ -354,10 +360,10 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
}
@Override
- public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize,
+ public PlacementResult<List<BookieId>> newEnsemble(int ensembleSize,
int writeQuorumSize,
int ackQuorumSize,
- Set<BookieSocketAddress> excludeBookies,
+ Set<BookieId> excludeBookies,
Ensemble<BookieNode> parentEnsemble,
Predicate<BookieNode> parentPredicate)
throws BKNotEnoughBookiesException {
@@ -370,11 +376,11 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
parentPredicate);
}
- protected PlacementResult<List<BookieSocketAddress>> newEnsembleInternal(
+ protected PlacementResult<List<BookieId>> newEnsembleInternal(
int ensembleSize,
int writeQuorumSize,
int ackQuorumSize,
- Set<BookieSocketAddress> excludeBookies,
+ Set<BookieId> excludeBookies,
Ensemble<BookieNode> parentEnsemble,
Predicate<BookieNode> parentPredicate) throws BKNotEnoughBookiesException {
rwLock.readLock().lock();
@@ -400,7 +406,7 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
}
List<BookieNode> bns = selectRandom(ensembleSize, excludeNodes, TruePredicate.INSTANCE,
ensemble);
- ArrayList<BookieSocketAddress> addrs = new ArrayList<BookieSocketAddress>(ensembleSize);
+ ArrayList<BookieId> addrs = new ArrayList<BookieId>(ensembleSize);
for (BookieNode bn : bns) {
addrs.add(bn.getAddr());
}
@@ -422,7 +428,7 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
prevNode = selectFromNetworkLocation(curRack, excludeNodes, ensemble, ensemble,
!enforceMinNumRacksPerWriteQuorum || firstBookieInTheEnsemble);
}
- List<BookieSocketAddress> bookieList = ensemble.toList();
+ List<BookieId> bookieList = ensemble.toList();
if (ensembleSize != bookieList.size()) {
LOG.error("Not enough {} bookies are available to form an ensemble : {}.",
ensembleSize, bookieList);
@@ -437,9 +443,9 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
}
@Override
- public PlacementResult<BookieSocketAddress> replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
- Map<String, byte[]> customMetadata, List<BookieSocketAddress> currentEnsemble,
- BookieSocketAddress bookieToReplace, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<BookieId> replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
+ Map<String, byte[]> customMetadata, List<BookieId> currentEnsemble,
+ BookieId bookieToReplace, Set<BookieId> excludeBookies)
throws BKNotEnoughBookiesException {
rwLock.readLock().lock();
try {
@@ -474,8 +480,8 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
if (LOG.isDebugEnabled()) {
LOG.debug("Bookie {} is chosen to replace bookie {}.", candidate, bn);
}
- BookieSocketAddress candidateAddr = candidate.getAddr();
- List<BookieSocketAddress> newEnsemble = new ArrayList<BookieSocketAddress>(currentEnsemble);
+ BookieId candidateAddr = candidate.getAddr();
+ List<BookieId> newEnsemble = new ArrayList<BookieId>(currentEnsemble);
if (currentEnsemble.isEmpty()) {
/*
* in testing code there are test cases which would pass empty
@@ -773,7 +779,7 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
}
@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 class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
@Override
public DistributionSchedule.WriteSet reorderReadSequence(
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo,
DistributionSchedule.WriteSet writeSet) {
Map<Integer, String> writeSetWithRegion = new HashMap<>();
@@ -824,7 +830,7 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
* @return ordering of bookies to send read to
*/
DistributionSchedule.WriteSet reorderReadSequenceWithRegion(
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
DistributionSchedule.WriteSet writeSet,
Map<Integer, String> writeSetWithRegion,
BookiesHealthInfo bookiesHealthInfo,
@@ -842,7 +848,7 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
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 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
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 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
// this method should be called in readlock scope of 'rwlock'
@Override
- public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List<BookieSocketAddress> ensembleList,
+ public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List<BookieId> ensembleList,
int writeQuorumSize, int ackQuorumSize) {
int ensembleSize = ensembleList.size();
int minNumRacksPerWriteQuorumForThisEnsemble = Math.min(writeQuorumSize, minNumRacksPerWriteQuorum);
HashSet<String> racksInQuorum = new HashSet<String>();
- 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 class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
}
@Override
- public boolean areAckedBookiesAdheringToPlacementPolicy(Set<BookieSocketAddress> ackedBookies,
+ public boolean areAckedBookiesAdheringToPlacementPolicy(Set<BookieId> ackedBookies,
int writeQuorumSize,
int ackQuorumSize) {
HashSet<String> rackCounter = new HashSet<>();
@@ -1046,7 +1052,7 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
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 3dc1a75..871626a 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.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.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<BookieSocketAddress> currentEnsemble;
+ private final List<BookieId> currentEnsemble;
private ScheduledFuture<?> speculativeTask = null;
abstract class ReadLACAndEntryRequest implements AutoCloseable {
@@ -76,12 +76,12 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
int firstError = BKException.Code.OK;
int numMissedEntryReads = 0;
- final List<BookieSocketAddress> ensemble;
+ final List<BookieId> ensemble;
final DistributionSchedule.WriteSet writeSet;
final DistributionSchedule.WriteSet orderedEnsemble;
final LedgerEntryImpl entryImpl;
- ReadLACAndEntryRequest(List<BookieSocketAddress> ensemble, long lId, long eId) {
+ ReadLACAndEntryRequest(List<BookieId> ensemble, long lId, long eId) {
this.entryImpl = LedgerEntryImpl.create(lId, eId);
this.ensemble = ensemble;
this.writeSet = lh.getDistributionSchedule().getEnsembleSet(eId);
@@ -119,7 +119,7 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
* @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 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
* @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 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
* 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 ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
int numPendings;
- ParallelReadRequest(List<BookieSocketAddress> ensemble, long lId, long eId) {
+ ParallelReadRequest(List<BookieId> ensemble, long lId, long eId) {
super(ensemble, lId, eId);
numPendings = orderedEnsemble.size();
}
@@ -255,7 +255,7 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
@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 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
}
@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 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
}
@Override
- BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFromHostsBitSet) {
+ BookieId maybeSendSpeculativeRead(BitSet heardFromHostsBitSet) {
// no speculative read
return null;
}
@@ -297,7 +297,7 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
final BitSet erroredReplicas;
final BitSet emptyResponseReplicas;
- SequenceReadRequest(List<BookieSocketAddress> ensemble, long lId, long eId) {
+ SequenceReadRequest(List<BookieId> ensemble, long lId, long eId) {
super(ensemble, lId, eId);
this.sentReplicas = new BitSet(orderedEnsemble.size());
@@ -335,7 +335,7 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
* @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 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
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 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
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 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
}
@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 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
}
@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 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
// 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 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
ReadLastConfirmedAndEntryOp(LedgerHandle lh,
ClientContext clientCtx,
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
LastConfirmedAndEntryCallback cb,
long prevEntryId,
long timeOutInMillis) {
@@ -506,7 +506,7 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
}
}
- 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 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
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 4e3b127..e5d441a 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 io.netty.buffer.ByteBuf;
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<BookieSocketAddress> currentEnsemble;
+ private final List<BookieId> currentEnsemble;
/**
* Wrapper to get all recovered data from the request.
@@ -62,7 +62,7 @@ class ReadLastConfirmedOp implements ReadEntryCallback {
DistributionSchedule schedule,
DigestManager digestManager,
long ledgerId,
- List<BookieSocketAddress> ensemble,
+ List<BookieId> 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 72930f7..1d21fc9 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.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.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<Long, List<BookieSocketAddress>> newEnsemblesFromRecovery = new TreeMap<>();
+ private final NavigableMap<Long, List<BookieId>> newEnsemblesFromRecovery = new TreeMap<>();
class MetadataUpdater extends SafeRunnable {
@@ -212,7 +212,7 @@ class ReadOnlyLedgerHandle extends LedgerHandle implements LedgerMetadataListene
* on the close.
*/
@Override
- void handleBookieFailure(final Map<Integer, BookieSocketAddress> failedBookies) {
+ void handleBookieFailure(final Map<Integer, BookieId> 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 @@ class ReadOnlyLedgerHandle extends LedgerHandle implements LedgerMetadataListene
long lac = getLastAddConfirmed();
LedgerMetadata metadata = getLedgerMetadata();
- List<BookieSocketAddress> currentEnsemble = getCurrentEnsemble();
+ List<BookieId> currentEnsemble = getCurrentEnsemble();
try {
- List<BookieSocketAddress> newEnsemble = EnsembleUtils.replaceBookiesInEnsemble(
+ List<BookieId> newEnsemble = EnsembleUtils.replaceBookiesInEnsemble(
clientCtx.getBookieWatcher(), metadata, currentEnsemble, failedBookies, logContext);
Set<Integer> replaced = EnsembleUtils.diffEnsemble(currentEnsemble, newEnsemble);
if (!replaced.isEmpty()) {
@@ -339,7 +339,7 @@ class ReadOnlyLedgerHandle extends LedgerHandle implements LedgerMetadataListene
}
@Override
- List<BookieSocketAddress> getCurrentEnsemble() {
+ List<BookieId> 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 16bdcfa..dee4187 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 java.util.concurrent.ConcurrentMap;
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<String, TopologyAwareEnsemblePlacementPolicy> perRegionPlacement;
- protected final ConcurrentMap<BookieSocketAddress, String> address2Region;
+ protected final ConcurrentMap<BookieId, String> 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<String, TopologyAwareEnsemblePlacementPolicy>();
- address2Region = new ConcurrentHashMap<BookieSocketAddress, String>();
+ address2Region = new ConcurrentHashMap<BookieId, String>();
}
- 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 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
}
@Override
- public void handleBookiesThatLeft(Set<BookieSocketAddress> leftBookies) {
+ public void handleBookiesThatLeft(Set<BookieId> leftBookies) {
super.handleBookiesThatLeft(leftBookies);
for (TopologyAwareEnsemblePlacementPolicy policy: perRegionPlacement.values()) {
@@ -119,11 +120,11 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
}
@Override
- public void handleBookiesThatJoined(Set<BookieSocketAddress> joinedBookies) {
- Map<String, Set<BookieSocketAddress>> perRegionClusterChange = new HashMap<String, Set<BookieSocketAddress>>();
+ public void handleBookiesThatJoined(Set<BookieId> joinedBookies) {
+ Map<String, Set<BookieId>> perRegionClusterChange = new HashMap<String, Set<BookieId>>();
// 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 class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
.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<BookieSocketAddress> regionSet = perRegionClusterChange.get(region);
+ Set<BookieId> regionSet = perRegionClusterChange.get(region);
if (null == regionSet) {
- regionSet = new HashSet<BookieSocketAddress>();
+ regionSet = new HashSet<BookieId>();
regionSet.add(addr);
perRegionClusterChange.put(region, regionSet);
} else {
@@ -152,9 +153,9 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
}
for (Map.Entry<String, TopologyAwareEnsemblePlacementPolicy> regionEntry : perRegionPlacement.entrySet()) {
- Set<BookieSocketAddress> regionSet = perRegionClusterChange.get(regionEntry.getKey());
+ Set<BookieId> regionSet = perRegionClusterChange.get(regionEntry.getKey());
if (null == regionSet) {
- regionSet = new HashSet<BookieSocketAddress>();
+ regionSet = new HashSet<BookieId>();
}
regionEntry.getValue().handleBookiesThatJoined(regionSet);
}
@@ -165,8 +166,9 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
Optional<DNSToSwitchMapping> 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 class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
.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 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
@Override
- public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize, int writeQuorumSize,
- int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieSocketAddress> excludedBookies)
+ public PlacementResult<List<BookieId>> newEnsemble(int ensembleSize, int writeQuorumSize,
+ int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieId> excludedBookies)
throws BKException.BKNotEnoughBookiesException {
int effectiveMinRegionsForDurability = disableDurabilityFeature.isAvailable() ? 1 : minRegionsForDurability;
@@ -258,7 +260,7 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
rwLock.readLock().lock();
try {
- Set<BookieSocketAddress> comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced(
+ Set<BookieId> comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced(
excludedBookies);
Set<Node> excludeNodes = convertBookiesToNodes(comprehensiveExclusionBookiesSet);
Set<String> availableRegions = new HashSet<String>();
@@ -280,7 +282,7 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
}
List<BookieNode> bns = selectRandom(ensembleSize, excludeNodes, TruePredicate.INSTANCE,
EnsembleForReplacementWithNoConstraints.INSTANCE);
- ArrayList<BookieSocketAddress> addrs = new ArrayList<BookieSocketAddress>(ensembleSize);
+ ArrayList<BookieId> addrs = new ArrayList<BookieId>(ensembleSize);
for (BookieNode bn : bns) {
addrs.add(bn.getAddr());
}
@@ -352,7 +354,7 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
int newEnsembleSize = currentAllocation.getLeft() + addToEnsembleSize;
int newWriteQuorumSize = currentAllocation.getRight() + addToWriteQuorum;
try {
- List<BookieSocketAddress> allocated = policyWithinRegion
+ List<BookieId> allocated = policyWithinRegion
.newEnsemble(newEnsembleSize, newWriteQuorumSize, newWriteQuorumSize,
comprehensiveExclusionBookiesSet, tempEnsemble, tempEnsemble)
.getResult();
@@ -402,7 +404,7 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
}
} while ((remainingEnsemble > 0) && (remainingEnsemble < remainingEnsembleBeforeIteration));
- List<BookieSocketAddress> bookieList = ensemble.toList();
+ List<BookieId> 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 class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
throw new BKException.BKNotEnoughBookiesException();
}
LOG.info("Bookies allocated successfully {}", ensemble);
- List<BookieSocketAddress> ensembleList = ensemble.toList();
+ List<BookieId> ensembleList = ensemble.toList();
return PlacementResult.of(ensembleList,
isEnsembleAdheringToPlacementPolicy(ensembleList, writeQuorumSize, ackQuorumSize));
} finally {
@@ -424,15 +426,15 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
}
@Override
- public PlacementResult<BookieSocketAddress> replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
- Map<String, byte[]> customMetadata, List<BookieSocketAddress> currentEnsemble,
- BookieSocketAddress bookieToReplace, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<BookieId> replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
+ Map<String, byte[]> customMetadata, List<BookieId> currentEnsemble,
+ BookieId bookieToReplace, Set<BookieId> excludeBookies)
throws BKException.BKNotEnoughBookiesException {
rwLock.readLock().lock();
try {
boolean enforceDurability = enforceDurabilityInReplace && !disableDurabilityFeature.isAvailable();
int effectiveMinRegionsForDurability = enforceDurability ? minRegionsForDurability : 1;
- Set<BookieSocketAddress> comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced(
+ Set<BookieId> comprehensiveExclusionBookiesSet = addDefaultRackBookiesIfMinNumRacksIsEnforced(
excludeBookies);
Set<Node> excludeNodes = convertBookiesToNodes(comprehensiveExclusionBookiesSet);
RRTopologyAwareCoverageEnsemble ensemble = new RRTopologyAwareCoverageEnsemble(ensembleSize,
@@ -448,7 +450,7 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
}
excludeNodes.add(bookieNodeToReplace);
- for (BookieSocketAddress bookieAddress: currentEnsemble) {
+ for (BookieId bookieAddress: currentEnsemble) {
if (bookieAddress.equals(bookieToReplace)) {
continue;
}
@@ -481,8 +483,8 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
if (LOG.isDebugEnabled()) {
LOG.debug("Bookie {} is chosen to replace bookie {}.", candidate, bookieNodeToReplace);
}
- BookieSocketAddress candidateAddr = candidate.getAddr();
- List<BookieSocketAddress> newEnsemble = new ArrayList<BookieSocketAddress>(currentEnsemble);
+ BookieId candidateAddr = candidate.getAddr();
+ List<BookieId> newEnsemble = new ArrayList<BookieId>(currentEnsemble);
if (currentEnsemble.isEmpty()) {
/*
* in testing code there are test cases which would pass empty
@@ -546,7 +548,7 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
@Override
public final DistributionSchedule.WriteSet reorderReadSequence(
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo,
DistributionSchedule.WriteSet writeSet) {
if (UNKNOWN_REGION.equals(myRegion)) {
@@ -564,7 +566,7 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
@Override
public final DistributionSchedule.WriteSet reorderReadLACSequence(
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo,
DistributionSchedule.WriteSet writeSet) {
if (UNKNOWN_REGION.equals(myRegion)) {
@@ -576,7 +578,7 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
}
@Override
- public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List<BookieSocketAddress> ensembleList,
+ public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List<BookieId> 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 85615b8..f092839 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.Arrays;
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 @@ public class RoundRobinDistributionSchedule implements DistributionSchedule {
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<AckSetImpl> recyclerHandle;
private static final Recycler<AckSetImpl> RECYCLER = new Recycler<AckSetImpl>() {
@@ -295,7 +295,7 @@ public class RoundRobinDistributionSchedule implements DistributionSchedule {
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 class RoundRobinDistributionSchedule implements DistributionSchedule {
@Override
public boolean failBookieAndCheck(int bookieIndexHeardFrom,
- BookieSocketAddress address) {
+ BookieId address) {
ackSet.clear(bookieIndexHeardFrom);
failureMap[bookieIndexHeardFrom] = address;
return failed() > (writeQuorumSize - ackQuorumSize);
}
@Override
- public Map<Integer, BookieSocketAddress> getFailedBookies() {
- ImmutableMap.Builder<Integer, BookieSocketAddress> builder = new ImmutableMap.Builder<>();
+ public Map<Integer, BookieId> getFailedBookies() {
+ ImmutableMap.Builder<Integer, BookieId> 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 2e23848..7167c96 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 java.util.function.Supplier;
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.NetUtils;
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<BookieNode> {
static final Logger LOG = LoggerFactory.getLogger(TopologyAwareEnsemblePlacementPolicy.class);
public static final String REPP_DNS_RESOLVER_CLASS = "reppDnsResolverClass";
- protected final Map<BookieSocketAddress, BookieNode> knownBookies = new HashMap<BookieSocketAddress, BookieNode>();
+ protected final Map<BookieId, BookieNode> knownBookies = new HashMap<BookieId, BookieNode>();
protected final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
protected Map<BookieNode, WeightedObject> bookieInfoMap = new HashMap<BookieNode, WeightedObject>();
// Initialize to empty set
- protected ImmutableSet<BookieSocketAddress> readOnlyBookies = ImmutableSet.of();
+ protected ImmutableSet<BookieId> readOnlyBookies = ImmutableSet.of();
boolean isWeighted;
protected WeightedRandomSelection<BookieNode> 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 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
public static final EnsembleForReplacementWithNoConstraints INSTANCE =
new EnsembleForReplacementWithNoConstraints();
- static final List<BookieSocketAddress> EMPTY_LIST = new ArrayList<BookieSocketAddress>(0);
+ static final List<BookieId> EMPTY_LIST = new ArrayList<BookieId>(0);
@Override
public boolean addNode(BookieNode node) {
@@ -101,7 +104,7 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
}
@Override
- public List<BookieSocketAddress> toList() {
+ public List<BookieId> toList() {
return EMPTY_LIST;
}
@@ -458,8 +461,8 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
}
@Override
- public List<BookieSocketAddress> toList() {
- ArrayList<BookieSocketAddress> addresses = new ArrayList<BookieSocketAddress>(ensembleSize);
+ public List<BookieId> toList() {
+ ArrayList<BookieId> addresses = new ArrayList<BookieId>(ensembleSize);
for (BookieNode bn : chosenNodes) {
addresses.add(bn.getAddr());
}
@@ -473,7 +476,7 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
*/
@Override
public boolean validate() {
- HashSet<BookieSocketAddress> addresses = new HashSet<BookieSocketAddress>(ensembleSize);
+ HashSet<BookieId> addresses = new HashSet<BookieId>(ensembleSize);
HashSet<String> racksOrRegions = new HashSet<String>();
for (BookieNode bn : chosenNodes) {
if (addresses.contains(bn.getAddr())) {
@@ -619,7 +622,7 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
@Override
public DistributionSchedule.WriteSet reorderReadSequence(
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo,
DistributionSchedule.WriteSet writeSet) {
return writeSet;
@@ -627,7 +630,7 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
@Override
public DistributionSchedule.WriteSet reorderReadLACSequence(
- List<BookieSocketAddress> ensemble,
+ List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo,
DistributionSchedule.WriteSet writeSet) {
DistributionSchedule.WriteSet retList = reorderReadSequence(
@@ -637,12 +640,12 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
}
@Override
- public Set<BookieSocketAddress> onClusterChanged(Set<BookieSocketAddress> writableBookies,
- Set<BookieSocketAddress> readOnlyBookies) {
+ public Set<BookieId> onClusterChanged(Set<BookieId> writableBookies,
+ Set<BookieId> readOnlyBookies) {
rwLock.writeLock().lock();
try {
- ImmutableSet<BookieSocketAddress> joinedBookies, leftBookies, deadBookies;
- Set<BookieSocketAddress> oldBookieSet = knownBookies.keySet();
+ ImmutableSet<BookieId> joinedBookies, leftBookies, deadBookies;
+ Set<BookieId> 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 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
* this method should be called in writelock scope of 'rwLock'
*/
@Override
- public void handleBookiesThatLeft(Set<BookieSocketAddress> leftBookies) {
- for (BookieSocketAddress addr : leftBookies) {
+ public void handleBookiesThatLeft(Set<BookieId> leftBookies) {
+ for (BookieId addr : leftBookies) {
try {
BookieNode node = knownBookies.remove(addr);
if (null != node) {
@@ -701,9 +704,9 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
* this method should be called in writelock scope of 'rwLock'
*/
@Override
- public void handleBookiesThatJoined(Set<BookieSocketAddress> joinedBookies) {
+ public void handleBookiesThatJoined(Set<BookieId> joinedBookies) {
// node joined
- for (BookieSocketAddress addr : joinedBookies) {
+ for (BookieId addr : joinedBookies) {
try {
BookieNode node = createBookieNode(addr);
topology.add(node);
@@ -729,15 +732,15 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
}
@Override
- public void onBookieRackChange(List<BookieSocketAddress> bookieAddressList) {
+ public void onBookieRackChange(List<BookieId> 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 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
}
@Override
- public void updateBookieInfo(Map<BookieSocketAddress, BookieInfo> bookieInfoMap) {
+ public void updateBookieInfo(Map<BookieId, BookieInfo> bookieInfoMap) {
if (!isWeighted) {
LOG.info("bookieFreeDiskInfo callback called even without weighted placement policy being used.");
return;
@@ -772,17 +775,28 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
}
}
- 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<Node> convertBookiesToNodes(Collection<BookieSocketAddress> excludeBookies) {
+ protected Set<Node> convertBookiesToNodes(Collection<BookieId> excludeBookies) {
Set<Node> nodes = new HashSet<Node>();
- 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 1311d31..ad09521 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 io.netty.buffer.ByteBuf;
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<BookieSocketAddress> currentEnsemble;
+ final List<BookieId> currentEnsemble;
TryReadLastConfirmedOp(LedgerHandle lh, BookieClient bookieClient,
- List<BookieSocketAddress> ensemble, LastConfirmedDataCallback cb, long lac) {
+ List<BookieId> 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 14951fd..1444277 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 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.net.BookieId;
import org.apache.bookkeeper.versioning.Versioned;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -75,7 +75,7 @@ public class UpdateLedgerOp {
* 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 class UpdateLedgerOp {
}
private static LedgerMetadata replaceBookieInEnsembles(LedgerMetadata metadata,
- BookieSocketAddress oldBookieId,
- BookieSocketAddress newBookieId) {
+ BookieId oldBookieId,
+ BookieId newBookieId) {
LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata);
- for (Map.Entry<Long, ? extends List<BookieSocketAddress>> e : metadata.getAllEnsembles().entrySet()) {
- List<BookieSocketAddress> newEnsemble = e.getValue().stream()
+ for (Map.Entry<Long, ? extends List<BookieId>> e : metadata.getAllEnsembles().entrySet()) {
+ List<BookieId> 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 265b0ca..9554b41 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 java.util.Set;
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 class ZoneawareEnsemblePlacementPolicy extends ZoneawareEnsemblePlacement
@Override
public EnsemblePlacementPolicy initialize(ClientConfiguration conf,
- Optional<DNSToSwitchMapping> optionalDnsResolver, HashedWheelTimer timer, FeatureProvider featureProvider,
- StatsLogger statsLogger) {
+ Optional<DNSToSwitchMapping> 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 class ZoneawareEnsemblePlacementPolicy extends ZoneawareEnsemblePlacement
}
@Override
- public Set<BookieSocketAddress> onClusterChanged(Set<BookieSocketAddress> writableBookies,
- Set<BookieSocketAddress> readOnlyBookies) {
- Set<BookieSocketAddress> deadBookies = super.onClusterChanged(writableBookies, readOnlyBookies);
+ public Set<BookieId> onClusterChanged(Set<BookieId> writableBookies,
+ Set<BookieId> readOnlyBookies) {
+ Set<BookieId> deadBookies = super.onClusterChanged(writableBookies, readOnlyBookies);
if (null != slave) {
deadBookies = slave.onClusterChanged(writableBookies, readOnlyBookies);
}
@@ -80,8 +83,8 @@ public class ZoneawareEnsemblePlacementPolicy extends ZoneawareEnsemblePlacement
}
@Override
- public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize, int writeQuorumSize,
- int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<List<BookieId>> newEnsemble(int ensembleSize, int writeQuorumSize,
+ int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieId> excludeBookies)
throws BKException.BKNotEnoughBookiesException {
try {
return super.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, excludeBookies);
@@ -95,9 +98,9 @@ public class ZoneawareEnsemblePlacementPolicy extends ZoneawareEnsemblePlacement
}
@Override
- public PlacementResult<BookieSocketAddress> replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
- Map<String, byte[]> customMetadata, List<BookieSocketAddress> currentEnsemble,
- BookieSocketAddress bookieToReplace, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<BookieId> replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
+ Map<String, byte[]> customMetadata, List<BookieId> currentEnsemble,
+ BookieId bookieToReplace, Set<BookieId> excludeBookies)
throws BKException.BKNotEnoughBookiesException {
try {
return super.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata,
@@ -113,7 +116,7 @@ public class ZoneawareEnsemblePlacementPolicy extends ZoneawareEnsemblePlacement
}
@Override
- public void handleBookiesThatLeft(Set<BookieSocketAddress> leftBookies) {
+ public void handleBookiesThatLeft(Set<BookieId> leftBookies) {
super.handleBookiesThatLeft(leftBookies);
if (null != slave) {
slave.handleBookiesThatLeft(leftBookies);
@@ -121,7 +124,7 @@ public class ZoneawareEnsemblePlacementPolicy extends ZoneawareEnsemblePlacement
}
@Override
- public void handleBookiesThatJoined(Set<BookieSocketAddress> joinedBookies) {
+ public void handleBookiesThatJoined(Set<BookieId> 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 dc15c9b..fcb3c73 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 com.google.common.cache.CacheLoader;
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.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.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.Node;
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<BookieSocketAddress, Long> slowBookies;
+ protected Cache<BookieId, Long> 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<BookieSocketAddress, ZoneAwareNodeLocation> address2NodePlacement;
+ protected final ConcurrentMap<BookieId, ZoneAwareNodeLocation> 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 class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
ZoneawareEnsemblePlacementPolicyImpl() {
super();
- address2NodePlacement = new ConcurrentHashMap<BookieSocketAddress, ZoneAwareNodeLocation>();
+ address2NodePlacement = new ConcurrentHashMap<BookieId, ZoneAwareNodeLocation>();
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 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
@Override
public EnsemblePlacementPolicy initialize(ClientConfiguration conf,
Optional<DNSToSwitchMapping> 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 class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
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 class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
slowBookies = CacheBuilder.newBuilder()
.expireAfterWrite(conf.getBookieFailureHistoryExpirationMSec(), TimeUnit.MILLISECONDS)
- .build(new CacheLoader<BookieSocketAddress, Long>() {
+ .build(new CacheLoader<BookieId, Long>() {
@Override
- public Long load(BookieSocketAddress key) throws Exception {
+ public Long load(BookieId key) throws Exception {
return -1L;
}
});
@@ -287,8 +289,8 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
}
@Override
- public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize, int writeQuorumSize,
- int ackQuorumSize, Set<BookieSocketAddress> excludeBookies,
+ public PlacementResult<List<BookieId>> newEnsemble(int ensembleSize, int writeQuorumSize,
+ int ackQuorumSize, Set<BookieId> excludeBookies,
org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Ensemble<BookieNode> parentEnsemble,
org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Predicate<BookieNode> parentPredicate)
throws BKNotEnoughBookiesException {
@@ -329,8 +331,8 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
}
@Override
- public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize, int writeQuorumSize,
- int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<List<BookieId>> newEnsemble(int ensembleSize, int writeQuorumSize,
+ int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieId> excludeBookies)
throws BKNotEnoughBookiesException {
if (enforceStrictZoneawarePlacement) {
if (ensembleSize % writeQuorumSize != 0) {
@@ -387,7 +389,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
}
}
int desiredNumZonesPerWriteQuorumForThisEnsemble = Math.min(writeQuorumSize, desiredNumZonesPerWriteQuorum);
- List<BookieSocketAddress> newEnsemble = new ArrayList<BookieSocketAddress>(
+ List<BookieId> newEnsemble = new ArrayList<BookieId>(
Collections.nCopies(ensembleSize, null));
rwLock.readLock().lock();
try {
@@ -395,9 +397,9 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
return createNewEnsembleRandomly(newEnsemble, writeQuorumSize, ackQuorumSize, customMetadata,
excludeBookies);
}
- Set<BookieSocketAddress> comprehensiveExclusionBookiesSet = addDefaultFaultDomainBookies(excludeBookies);
+ Set<BookieId> 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 class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
}
@Override
- public PlacementResult<BookieSocketAddress> replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
- Map<String, byte[]> customMetadata, List<BookieSocketAddress> currentEnsemble,
- BookieSocketAddress bookieToReplace, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<BookieId> replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize,
+ Map<String, byte[]> customMetadata, List<BookieId> currentEnsemble,
+ BookieId bookieToReplace, Set<BookieId> excludeBookies)
throws BKNotEnoughBookiesException {
int bookieToReplaceIndex = currentEnsemble.indexOf(bookieToReplace);
int desiredNumZonesPerWriteQuorumForThisEnsemble = (writeQuorumSize < desiredNumZonesPerWriteQuorum)
? writeQuorumSize : desiredNumZonesPerWriteQuorum;
- List<BookieSocketAddress> newEnsemble = new ArrayList<BookieSocketAddress>(currentEnsemble);
+ List<BookieId> newEnsemble = new ArrayList<BookieId>(currentEnsemble);
rwLock.readLock().lock();
try {
if (!enforceStrictZoneawarePlacement) {
return selectBookieRandomly(newEnsemble, bookieToReplace, excludeBookies, writeQuorumSize,
ackQuorumSize);
}
- Set<BookieSocketAddress> comprehensiveExclusionBookiesSet = addDefaultFaultDomainBookies(excludeBookies);
+ Set<BookieId> 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 class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
}
}
- private PlacementResult<List<BookieSocketAddress>> createNewEnsembleRandomly(List<BookieSocketAddress> newEnsemble,
+ private PlacementResult<List<BookieId>> createNewEnsembleRandomly(List<BookieId> newEnsemble,
int writeQuorumSize, int ackQuorumSize, Map<String, byte[]> customMetadata,
- Set<BookieSocketAddress> excludeBookies) throws BKNotEnoughBookiesException {
+ Set<BookieId> excludeBookies) throws BKNotEnoughBookiesException {
int ensembleSize = newEnsemble.size();
Set<BookieNode> bookiesToConsider = getBookiesToConsider(excludeBookies);
if (bookiesToConsider.size() < newEnsemble.size()) {
@@ -455,10 +457,10 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize));
}
- private PlacementResult<BookieSocketAddress> selectBookieRandomly(List<BookieSocketAddress> newEnsemble,
- BookieSocketAddress bookieToReplace, Set<BookieSocketAddress> excludeBookies, int writeQuorumSize,
+ private PlacementResult<BookieId> selectBookieRandomly(List<BookieId> newEnsemble,
+ BookieId bookieToReplace, Set<BookieId> excludeBookies, int writeQuorumSize,
int ackQuorumSize) throws BKNotEnoughBookiesException {
- Set<BookieSocketAddress> bookiesToExcludeIncludingEnsemble = new HashSet<BookieSocketAddress>(excludeBookies);
+ Set<BookieId> bookiesToExcludeIncludingEnsemble = new HashSet<BookieId>(excludeBookies);
bookiesToExcludeIncludingEnsemble.addAll(newEnsemble);
Set<BookieNode> bookiesToConsider = getBookiesToConsider(bookiesToExcludeIncludingEnsemble);
int bookieToReplaceIndex = newEnsemble.indexOf(bookieToReplace);
@@ -467,13 +469,13 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
LOG.error("There is no bookie available to replace a bookie");
throw new BKNotEnoughBookiesException();
}
- BookieSocketAddress candidateAddr = (selectCandidateNode(bookiesToConsider)).getAddr();
+ BookieId candidateAddr = (selectCandidateNode(bookiesToConsider)).getAddr();
newEnsemble.set(bookieToReplaceIndex, candidateAddr);
return PlacementResult.of(candidateAddr,
isEnsembleAdheringToPlacementPolicy(newEnsemble, writeQuorumSize, ackQuorumSize));
}
- private Set<BookieNode> getBookiesToConsider(Set<BookieSocketAddress> excludeBookies) {
+ private Set<BookieNode> getBookiesToConsider(Set<BookieId> excludeBookies) {
Set<Node> leaves = topology.getLeaves(NodeBase.ROOT);
Set<BookieNode> bookiesToConsider = new HashSet<BookieNode>();
BookieNode bookieNode;
@@ -535,11 +537,11 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
* a node randomly. 12) If even after Step10 there are no eligible
* candidates then throw BKNotEnoughBookiesException.
*/
- private BookieSocketAddress setBookieInTheEnsemble(int ensembleSize, int writeQuorumSize,
- List<BookieSocketAddress> currentEnsemble, List<BookieSocketAddress> newEnsemble, int bookieToReplaceIndex,
- int desiredNumZonesPerWriteQuorumForThisEnsemble, Set<BookieSocketAddress> excludeBookies)
+ private BookieId setBookieInTheEnsemble(int ensembleSize, int writeQuorumSize,
+ List<BookieId> currentEnsemble, List<BookieId> newEnsemble, int bookieToReplaceIndex,
+ int desiredNumZonesPerWriteQuorumForThisEnsemble, Set<BookieId> excludeBookies)
throws BKNotEnoughBookiesException {
- BookieSocketAddress bookieToReplace = currentEnsemble.get(bookieToReplaceIndex);
+ BookieId bookieToReplace = currentEnsemble.get(bookieToReplaceIndex);
Set<String> zonesToExclude = null;
Set<BookieNode> bookiesToConsiderAfterExcludingZonesAndUDs = null;
for (int numberOfNeighborsToConsider = (desiredNumZonesPerWriteQuorumForThisEnsemble
@@ -564,7 +566,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
throw new BKNotEnoughBookiesException();
}
- BookieSocketAddress candidateAddr = selectCandidateNode(bookiesToConsiderAfterExcludingZonesAndUDs).getAddr();
+ BookieId candidateAddr = selectCandidateNode(bookiesToConsiderAfterExcludingZonesAndUDs).getAddr();
newEnsemble.set(bookieToReplaceIndex, candidateAddr);
return candidateAddr;
}
@@ -574,8 +576,8 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
* returns a new set, by adding excludedBookies and bookies in
* defaultfaultdomain.
*/
- protected Set<BookieSocketAddress> addDefaultFaultDomainBookies(Set<BookieSocketAddress> excludeBookies) {
- Set<BookieSocketAddress> comprehensiveExclusionBookiesSet = new HashSet<BookieSocketAddress>(excludeBookies);
+ protected Set<BookieId> addDefaultFaultDomainBookies(Set<BookieId> excludeBookies) {
+ Set<BookieId> comprehensiveExclusionBookiesSet = new HashSet<BookieId>(excludeBookies);
Set<Node> defaultFaultDomainLeaves = topology.getLeaves(getDefaultFaultDomain());
for (Node node : defaultFaultDomainLeaves) {
if (node instanceof BookieNode) {
@@ -626,7 +628,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
return excludedZonesString.toString();
}
- private Set<BookieNode> getBookiesToConsider(String excludedZonesString, Set<BookieSocketAddress> excludeBookies) {
+ private Set<BookieNode> getBookiesToConsider(String excludedZonesString, Set<BookieId> excludeBookies) {
Set<BookieNode> bookiesToConsider = new HashSet<BookieNode>();
Set<Node> leaves = topology.getLeaves(excludedZonesString);
for (Node leaf : leaves) {
@@ -652,8 +654,8 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
* upgrade domains.
*/
private Set<BookieNode> getBookiesToConsiderAfterExcludingZonesAndUDs(int ensembleSize, int writeQuorumSize,
- List<BookieSocketAddress> currentEnsemble, int bookieToReplaceIndex,
- Set<BookieSocketAddress> excludeBookies, Set<String> excludeZones) {
+ List<BookieId> currentEnsemble, int bookieToReplaceIndex,
+ Set<BookieId> excludeBookies, Set<String> excludeZones) {
Set<BookieNode> bookiesToConsiderAfterExcludingZonesAndUDs = new HashSet<BookieNode>();
HashMap<String, Set<String>> excludingUDsOfZonesToConsider = new HashMap<String, Set<String>>();
Set<BookieNode> bookiesToConsiderAfterExcludingZones = getBookiesToConsider(
@@ -711,7 +713,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
/*
* Gets the set of zones of neighboring nodes.
*/
- private Set<String> getZonesOfNeighboringNodesInEnsemble(List<BookieSocketAddress> currentEnsemble, int indexOfNode,
+ private Set<String> getZonesOfNeighboringNodesInEnsemble(List<BookieId> currentEnsemble, int indexOfNode,
int numOfNeighboringNodes) {
Set<String> zonesOfNeighboringNodes = new HashSet<String>();
int ensembleSize = currentEnsemble.size();
@@ -720,7 +722,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
continue;
}
int index = (indexOfNode + i + ensembleSize) % ensembleSize;
- BookieSocketAddress addrofNode = currentEnsemble.get(index);
+ BookieId addrofNode = currentEnsemble.get(index);
if (addrofNode == null) {
continue;
}
@@ -735,7 +737,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
* 'indexOfNode', so that writequorums, containing this index, would have
* atleast minNumZonesPerWriteQuorum.
*/
- private Set<String> getZonesToExcludeToMaintainMinZones(List<BookieSocketAddress> currentEnsemble, int indexOfNode,
+ private Set<String> getZonesToExcludeToMaintainMinZones(List<BookieId> currentEnsemble, int indexOfNode,
int writeQuorumSize) {
int ensSize = currentEnsemble.size();
Set<String> zonesToExclude = new HashSet<String>();
@@ -747,7 +749,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
if (indexInEnsemble == indexOfNode) {
continue;
}
- BookieSocketAddress bookieAddr = currentEnsemble.get(indexInEnsemble);
+ BookieId bookieAddr = currentEnsemble.get(indexInEnsemble);
if (bookieAddr == null) {
continue;
}
@@ -774,7 +776,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
* Gets the set of upgradedomains of neighboring nodes (writeQuorumSize)
* which belong to this 'zone'.
*/
- private Set<String> getUpgradeDomainsOfAZoneInNeighboringNodes(List<BookieSocketAddress> currentEnsemble,
+ private Set<String> getUpgradeDomainsOfAZoneInNeighboringNodes(List<BookieId> currentEnsemble,
int indexOfNode, int writeQuorumSize, String zone) {
int ensSize = currentEnsemble.size();
Set<String> upgradeDomainsOfAZoneInNeighboringNodes = new HashSet<String>();
@@ -783,7 +785,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
continue;
}
int indexInEnsemble = (indexOfNode + i + ensSize) % ensSize;
- BookieSocketAddress bookieAddr = currentEnsemble.get(indexInEnsemble);
+ BookieId bookieAddr = currentEnsemble.get(indexInEnsemble);
if (bookieAddr == null) {
continue;
}
@@ -801,7 +803,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
* there are multiple bookies from the 'zone' in a write quorum then they
* will be atleast from minimum of two upgrade domains.
*/
- private Set<String> getUDsToExcludeToMaintainMinUDsInWriteQuorums(List<BookieSocketAddress> currentEnsemble,
+ private Set<String> getUDsToExcludeToMaintainMinUDsInWriteQuorums(List<BookieId> currentEnsemble,
int indexOfNode, int writeQuorumSize, String zone) {
int ensSize = currentEnsemble.size();
Set<String> upgradeDomainsToExclude = new HashSet<String>();
@@ -813,7 +815,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
if (indexInEnsemble == indexOfNode) {
continue;
}
- BookieSocketAddress bookieAddr = currentEnsemble.get(indexInEnsemble);
+ BookieId bookieAddr = currentEnsemble.get(indexInEnsemble);
if (bookieAddr == null) {
continue;
}
@@ -830,18 +832,18 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
}
@Override
- public void registerSlowBookie(BookieSocketAddress bookieSocketAddress, long entryId) {
+ public void registerSlowBookie(BookieId bookieSocketAddress, long entryId) {
// TODO Auto-generated method stub
}
@Override
- public DistributionSchedule.WriteSet reorderReadSequence(List<BookieSocketAddress> ensemble,
+ public DistributionSchedule.WriteSet reorderReadSequence(List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) {
return writeSet;
}
@Override
- public DistributionSchedule.WriteSet reorderReadLACSequence(List<BookieSocketAddress> ensemble,
+ public DistributionSchedule.WriteSet reorderReadLACSequence(List<BookieId> ensemble,
BookiesHealthInfo bookiesHealthInfo, DistributionSchedule.WriteSet writeSet) {
DistributionSchedule.WriteSet retList = reorderReadSequence(ensemble, bookiesHealthInfo, writeSet);
retList.addMissingIndices(ensemble.size());
@@ -857,14 +859,14 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
* domains.
*/
@Override
- public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List<BookieSocketAddress> ensembleList,
+ public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List<BookieId> ensembleList,
int writeQuorumSize, int ackQuorumSize) {
PlacementPolicyAdherence placementPolicyAdherence = PlacementPolicyAdherence.MEETS_STRICT;
rwLock.readLock().lock();
try {
HashMap<String, Set<String>> bookiesLocationInWriteSet = new HashMap<String, Set<String>>();
HashMap<String, Integer> numOfBookiesInZones = new HashMap<String, Integer>();
- BookieSocketAddress bookieNode;
+ BookieId bookieNode;
if (ensembleList.size() % writeQuorumSize != 0) {
placementPolicyAdherence = PlacementPolicyAdherence.FAIL;
if (LOG.isDebugEnabled()) {
@@ -966,7 +968,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
}
@Override
- public boolean areAckedBookiesAdheringToPlacementPolicy(Set<BookieSocketAddress> ackedBookies, int writeQuorumSize,
+ public boolean areAckedBookiesAdheringToPlacementPolicy(Set<BookieId> ackedBookies, int writeQuorumSize,
int ackQuorumSize) {
HashSet<String> zonesOfAckedBookies = new HashSet<>();
int minNumZonesPerWriteQuorumForThisEnsemble = Math.min(writeQuorumSize, minNumZonesPerWriteQuorum);
@@ -974,7 +976,7 @@ public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
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 33e7da3..5c703a7 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.Map;
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<BookieSocketAddress> getEnsembleAt(long entryId);
+ List<BookieId> getEnsembleAt(long entryId);
/**
* Returns all the ensembles of this entry.
*
* @return all the ensembles of this entry.
*/
- NavigableMap<Long, ? extends List<BookieSocketAddress>> getAllEnsembles();
+ NavigableMap<Long, ? extends List<BookieId>> 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 f7823c0..96cc035 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 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.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<Set<BookieSocketAddress>> bookies);
+ void onBookiesChanged(Versioned<Set<BookieId>> bookies);
}
@@ -52,34 +53,35 @@ public interface RegistrationClient extends AutoCloseable {
*
* @return a future represents the list of writable bookies.
*/
- CompletableFuture<Versioned<Set<BookieSocketAddress>>> getWritableBookies();
+ CompletableFuture<Versioned<Set<BookieId>>> getWritableBookies();
/**
* Get the list of all bookies identifiers.
*
* @return a future represents the list of writable bookies.
*/
- CompletableFuture<Versioned<Set<BookieSocketAddress>>> getAllBookies();
+ CompletableFuture<Versioned<Set<BookieId>>> getAllBookies();
/**
* Get the list of readonly bookie identifiers.
*
* @return a future represents the list of readonly bookies.
*/
- CompletableFuture<Versioned<Set<BookieSocketAddress>>> getReadOnlyBookies();
+ CompletableFuture<Versioned<Set<BookieId>>> 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<Versioned<BookieServiceInfo>> getBookieServiceInfo(String bookieId) {
+ default CompletableFuture<Versioned<BookieServiceInfo>> 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 e5b0f6a..5068851 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 @@ package org.apache.bookkeeper.discover;
import org.apache.bookkeeper.bookie.BookieException;
import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate;
import org.apache.bookkeeper.common.annotation.InterfaceStability.Evolving;
+import org.apache.bookkeeper.net.BookieId;
import org.apache.bookkeeper.versioning.Version;
import org.apache.bookkeeper.versioning.Versioned;
@@ -62,7 +63,7 @@ public interface RegistrationManager extends AutoCloseable {
* @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 <i>bookieId</i>.
@@ -71,7 +72,7 @@ public interface RegistrationManager extends AutoCloseable {
* @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 @@ public interface RegistrationManager extends AutoCloseable {
* 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 @@ public interface RegistrationManager extends AutoCloseable {
* @param cookieData cookie data
* @throws BookieException when fail to write cookie
*/
- void writeCookie(String bookieId, Versioned<byte[]> cookieData) throws BookieException;
+ void writeCookie(BookieId bookieId, Versioned<byte[]> cookieData) throws BookieException;
/**
* Read the cookie data, which will be used for verifying the integrity of the bookie environment.
@@ -100,7 +101,7 @@ public interface RegistrationManager extends AutoCloseable {
* @return versioned cookie data
* @throws BookieException when fail to read cookie
*/
- Versioned<byte[]> readCookie(String bookieId) throws BookieException;
+ Versioned<byte[]> readCookie(BookieId bookieId) throws BookieException;
/**
* Remove the cookie data.
@@ -109,7 +110,7 @@ public interface RegistrationManager extends AutoCloseable {
* @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 367e9ac..dc17507 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 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.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.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.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 @@ import org.apache.zookeeper.data.Stat;
/**
* 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<Versioned<Set<BookieSocketAddress>>, Throwable>,
+ BiConsumer<Versioned<Set<BookieId>>, Throwable>,
AutoCloseable {
private final String regPath;
private final Set<RegistrationListener> listeners;
private volatile boolean closed = false;
- private Set<BookieSocketAddress> bookies = null;
+ private Set<BookieId> bookies = null;
private Version version = Version.NEW;
private final CompletableFuture<Void> firstRunFuture;
@@ -127,7 +129,7 @@ public class ZKRegistrationClient implements RegistrationClient {
}
@Override
- public void accept(Versioned<Set<BookieSocketAddress>> bookieSet, Throwable throwable) {
+ public void accept(Versioned<Set<BookieId>> bookieSet, Throwable throwable) {
if (throwable != null) {
if (firstRunFuture.isDone()) {
scheduleWatchTask(ZK_CONNECT_BACKOFF_MS);
@@ -140,9 +142,10 @@ public class ZKRegistrationClient implements RegistrationClient {
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 class ZKRegistrationClient implements RegistrationClient {
private WatchTask watchWritableBookiesTask = null;
@Getter(AccessLevel.PACKAGE)
private WatchTask watchReadOnlyBookiesTask = null;
+ private final ConcurrentHashMap<BookieId, Versioned<BookieServiceInfo>> bookieServiceInfoCache =
+ new ConcurrentHashMap<>();
// registration paths
private final String bookieRegistrationPath;
@@ -204,42 +209,69 @@ public class ZKRegistrationClient implements RegistrationClient {
}
@Override
- public CompletableFuture<Versioned<Set<BookieSocketAddress>>> getWritableBookies() {
+ public CompletableFuture<Versioned<Set<BookieId>>> getWritableBookies() {
return getChildren(bookieRegistrationPath, null);
}
@Override
- public CompletableFuture<Versioned<Set<BookieSocketAddress>>> getAllBookies() {
+ public CompletableFuture<Versioned<Set<BookieId>>> getAllBookies() {
return getChildren(bookieAllRegistrationPath, null);
}
@Override
- public CompletableFuture<Versioned<Set<BookieSocketAddress>>> getReadOnlyBookies() {
+ public CompletableFuture<Versioned<Set<BookieId>>> getReadOnlyBookies() {
return getChildren(bookieReadonlyRegistrationPath, null);
}
@Override
- public CompletableFuture<Versioned<BookieServiceInfo>> getBookieServiceInfo(String bookieId) {
+ public CompletableFuture<Versioned<BookieServiceInfo>> 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<BookieServiceInfo> 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<Versioned<BookieServiceInfo>> readBookieServiceInfo(BookieId bookieId) {
String pathAsWritable = bookieRegistrationPath + "/" + bookieId;
String pathAsReadonly = bookieReadonlyRegistrationPath + "/" + bookieId;
CompletableFuture<Versioned<BookieServiceInfo>> 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<BookieServiceInfo> 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<BookieServiceInfo> 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 class ZKRegistrationClient implements RegistrationClient {
@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 @@ public class ZKRegistrationClient implements RegistrationClient {
return bsi;
}
- private CompletableFuture<Versioned<Set<BookieSocketAddress>>> getChildren(String regPath, Watcher watcher) {
- CompletableFuture<Versioned<Set<BookieSocketAddress>>> 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<Versioned<Set<BookieId>>> getChildren(String regPath, Watcher watcher) {
+ CompletableFuture<Versioned<Set<BookieId>>> 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<BookieSocketAddress> bookies = convertToBookieAddresses(children);
- future.complete(new Versioned<>(bookies, version));
+ Set<BookieId> bookies = convertToBookieAddresses(children);
+ List<CompletableFuture<Versioned<BookieServiceInfo>>> 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<Versioned<BookieServiceInfo>> 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 class ZKRegistrationClient implements RegistrationClient {
}
}
- private static HashSet<BookieSocketAddress> convertToBookieAddresses(List<String> children) {
+ private static HashSet<BookieId> convertToBookieAddresses(List<String> children) {
// Read the bookie addresses into a set for efficient lookup
- HashSet<BookieSocketAddress> newBookieAddrs = Sets.newHashSet();
+ HashSet<BookieId> 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 951bb95..e8eea49 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.LedgerManagerFactory;
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 class ZKRegistrationManager implements RegistrationManager {
* @param bookieId bookie id
* @return
*/
- public String getCookiePath(String bookieId) {
+ public String getCookiePath(BookieId bookieId) {
return this.cookiePath + "/" + bookieId;
}
@@ -217,7 +217,7 @@ public class ZKRegistrationManager implements RegistrationManager {
}
@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 @@ public class ZKRegistrationManager implements RegistrationManager {
}
}
- 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 @@ public class ZKRegistrationManager implements RegistrationManager {
}
@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 @@ public class ZKRegistrationManager implements RegistrationManager {
//
@Override
- public void writeCookie(String bookieId,
+ public void writeCookie(BookieId bookieId,
Versioned<byte[]> cookieData) throws BookieException {
String zkPath = getCookiePath(bookieId);
try {
@@ -365,16 +366,16 @@ public class ZKRegistrationManager implements RegistrationManager {
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<byte[]> readCookie(String bookieId) throws BookieException {
+ public Versioned<byte[]> readCookie(BookieId bookieId) throws BookieException {
String zkPath = getCookiePath(bookieId);
try {
Stat stat = zk.exists(zkPath, false);
@@ -383,19 +384,19 @@ public class ZKRegistrationManager implements RegistrationManager {
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 class ZKRegistrationManager implements RegistrationManager {
null
)) {
if (availableNodeExists) {
- Collection<BookieSocketAddress> rwBookies = FutureUtils
+ Collection<BookieId> 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 class ZKRegistrationManager implements RegistrationManager {
boolean readonlyNodeExists = null != zk.exists(bookieReadonlyRegistrationPath, false);
if (readonlyNodeExists) {
- Collection<BookieSocketAddress> roBookies = FutureUtils
+ Collection<BookieId> 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 class ZKRegistrationManager implements RegistrationManager {
}
@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 aded61d..1e46b7e 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.LedgerMetadataUtils;
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 @@ public class LedgerMetadataSerDe {
}
}
- for (Map.Entry<Long, ? extends List<BookieSocketAddress>> entry : metadata.getAllEnsembles().entrySet()) {
+ for (Map.Entry<Long, ? extends List<BookieId>> 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 @@ public class LedgerMetadataSerDe {
}
}
- for (Map.Entry<Long, ? extends List<BookieSocketAddress>> entry :
+ for (Map.Entry<Long, ? extends List<BookieId>> 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 @@ public class LedgerMetadataSerDe {
writer.append(String.valueOf(metadata.getEnsembleSize())).append(LINE_SPLITTER);
writer.append(String.valueOf(metadata.getLength())).append(LINE_SPLITTER);
- for (Map.Entry<Long, ? extends List<BookieSocketAddress>> entry :
+ for (Map.Entry<Long, ? extends List<BookieId>> 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 @@ public class LedgerMetadataSerDe {
}
for (LedgerMetadataFormat.Segment s : data.getSegmentList()) {
- List<BookieSocketAddress> addrs = new ArrayList<>();
+ List<BookieId> 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 @@ public class LedgerMetadataSerDe {
break;
}
- ArrayList<BookieSocketAddress> addrs = new ArrayList<BookieSocketAddress>();
+ ArrayList<BookieId> addrs = new ArrayList<BookieId>();
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 0000000..d4e6355
--- /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 e7d0ef5..b22ae5f 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 @@ package org.apache.bookkeeper.net;
* 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 22f026a..06da0dc 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 io.netty.channel.local.LocalAddress;
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 class BookieSocketAddress {
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 0000000..f70740b
--- /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 2092a67..ddd7f18 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.List;
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<BookieSocketAddress> getFaultyBookies();
+ List<BookieId> 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 @@ public interface BookieClient {
* @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 @@ public interface BookieClient {
* @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 @@ public interface BookieClient {
* @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<WriteFlag> writeFlags);
@@ -147,7 +147,7 @@ public interface BookieClient {
* 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 @@ public interface BookieClient {
* 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 @@ public interface BookieClient {
* @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 @@ public interface BookieClient {
* @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 @@ public interface BookieClient {
*
* @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 @@ public interface BookieClient {
* and returns Future for the result.
*
* @param address
- * BookieSocketAddress of the bookie
+ * BookieId of the bookie
* @param ledgerId
* ledgerId
* @return returns Future
*/
- CompletableFuture<AvailabilityOfEntriesOfLedger> getListOfEntriesOfLedger(BookieSocketAddress address,
+ CompletableFuture<AvailabilityOfEntriesOfLedger> 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 8fb0f1d..b904c16 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.client.api.WriteFlag;
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<BookieSocketAddress, PerChannelBookieClientPool> channels =
- new ConcurrentHashMap<BookieSocketAddress, PerChannelBookieClientPool>();
+ final ConcurrentHashMap<BookieId, PerChannelBookieClientPool> channels =
+ new ConcurrentHashMap<BookieId, PerChannelBookieClientPool>();
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 class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
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 @@ public class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
}
@Override
- public List<BookieSocketAddress> getFaultyBookies() {
- List<BookieSocketAddress> faultyBookies = Lists.newArrayList();
+ public List<BookieId> getFaultyBookies() {
+ List<BookieId> faultyBookies = Lists.newArrayList();
for (PerChannelBookieClientPool channelPool : channels.values()) {
if (channelPool instanceof DefaultPerChannelBookieClientPool) {
DefaultPerChannelBookieClientPool pool = (DefaultPerChannelBookieClientPool) channelPool;
@@ -164,14 +166,14 @@ public class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
}
@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 class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
}
@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 class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
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 class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
}
@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 class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
}
@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 class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
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 class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
}
@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 class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
}
@Override
- public CompletableFuture<AvailabilityOfEntriesOfLedger> getListOfEntriesOfLedger(BookieSocketAddress address,
+ public CompletableFuture<AvailabilityOfEntriesOfLedger> getListOfEntriesOfLedger(BookieId address,
long ledgerId) {
FutureGetListOfEntriesOfLedger futureResult = new FutureGetListOfEntriesOfLedger(ledgerId);
final PerChannelBookieClientPool client = lookupClient(address);
@@ -383,7 +386,7 @@ public class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
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 @@ public class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
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<WriteFlag> writeFlags) {
ChannelReadyForAddEntryCallback callback = RECYCLER.get();
@@ -456,7 +459,7 @@ public class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
}
@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 class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
}
@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 class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
@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 class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
}
@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 class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
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 class BookieClientImpl implements BookieClient, PerChannelBookieClientFac
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 5b5c288..a48f749 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 @@ class BookieNettyServer {
});
// 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 @@ class BookieNettyServer {
// 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 @@ class BookieNettyServer {
}
}
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 fda7ed7..2864b29 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.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.BookieId;
import org.apache.bookkeeper.net.BookieSocketAddress;
import org.apache.bookkeeper.processor.RequestProcessor;
import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException;
@@ -168,6 +169,11 @@ public class BookieServer {
}
@VisibleForTesting
+ public BookieId getBookieId() throws UnknownHostException {
+ return Bookie.getBookieId(conf);
+ }
+
+ @VisibleForTesting
public Bookie getBookie() {
return bookie;
}
@@ -319,13 +325,14 @@ public class BookieServer {
@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 fdfd379..9904c90 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.BookieInfoReader.BookieInfo;
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 class BookkeeperInternalCallbacks {
* 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 class BookkeeperInternalCallbacks {
* 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 71193de..7305bef 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.AtomicInteger;
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 f889172..55aa97b 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 static com.google.common.base.Preconditions.checkArgument;
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 @@ class ForceLedgerProcessorV3 extends PacketProcessorBaseV3 implements Runnable {
}
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 59b6ed0..b023cf6 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<BookieSocketAddress, Boolean> localBookiesRegistry =
+ private static final ConcurrentHashMap<BookieId, Boolean> 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 94ed865..45650b2 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.Map;
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.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.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 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
}
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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
};
}
- 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 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
if (timedOutOperations > 0) {
LOG.info("Timed-out {} operations to channel {} for {}",
- timedOutOperations, channel, addr);
+ timedOutOperations, channel, bookieId);
}
}
@@ -1014,7 +1039,7 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
}
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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
}
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 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
}
} 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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
@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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
@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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
@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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
@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 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
}
int rc = convertStatus(status, BKException.Code.WriteException);
- writeComplete(rc, ledgerId, entryId, addr, ctx);
+ writeComplete(rc, ledgerId, entryId, bookieId, ctx);
}
}
@@ -2395,7 +2420,7 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
}
@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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
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 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
}
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<? extends Future<? super Void>> listener) {
+ throw new UnsupportedOperationException("Not supported");
+ }
+
+ @Override
+ @SuppressWarnings({"unchecked", "varargs"})
+ public ChannelFuture addListeners(GenericFutureListener<? extends Future<? super Void>>... listeners) {
+ throw new UnsupportedOperationException("Not supported");
+ }
+
+ @Override
+ public ChannelFuture removeListener(GenericFutureListener<? extends Future<? super Void>> listener) {
+ throw new UnsupportedOperationException("Not supported");
+ }
+
+ @Override
+ @SuppressWarnings({"unchecked", "varargs"})
+ public ChannelFuture removeListeners(GenericFutureListener<? extends Future<? super Void>>... 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 48797cd..d4fd575 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 6c3a57f..dea6f6d 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 @@ package org.apache.bookkeeper.proto;
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 @@ import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback
public class ReadLastConfirmedAndEntryContext implements ReadEntryCallbackCtx {
final int bookieIndex;
- final BookieSocketAddress bookie;
+ final BookieId bookie;
long lac = LedgerHandle.INVALID_ENTRY_ID;
Optional<Long> 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 class ReadLastConfirmedAndEntryContext implements ReadEntryCallbackCtx {
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 c8af9f8..a61e0d5 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 java.util.concurrent.TimeUnit;
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 @@ class WriteEntryProcessor extends PacketProcessorBase<ParsedAddRequest> implemen
@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 c8ea067..d8ed70c 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 java.util.concurrent.TimeUnit;
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 @@ class WriteEntryProcessorV3 extends PacketProcessorBaseV3 {
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 691102b..11e030d 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.nio.ByteBuffer;
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 @@ class WriteLacProcessorV3 extends PacketProcessorBaseV3 implements Runnable {
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 db6e331..48f5101 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.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.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 @@ public class Auditor implements AutoCloseable {
private List<String> getAvailableBookies() throws BKException {
// Get the available bookies
- Collection<BookieSocketAddress> availableBkAddresses = admin.getAvailableBookies();
- Collection<BookieSocketAddress> readOnlyBkAddresses = admin.getReadOnlyBookies();
+ Collection<BookieId> availableBkAddresses = admin.getAvailableBookies();
+ Collection<BookieId> readOnlyBkAddresses = admin.getReadOnlyBookies();
availableBkAddresses.addAll(readOnlyBkAddresses);
List<String> availableBookies = new ArrayList<String>();
- for (BookieSocketAddress addr : availableBkAddresses) {
+ for (BookieId addr : availableBkAddresses) {
availableBookies.add(addr.toString());
}
return availableBookies;
@@ -1153,7 +1153,7 @@ public class Auditor implements AutoCloseable {
@Override
public void operationComplete(int rc, Set<LedgerFragment> fragments) {
if (rc == BKException.Code.OK) {
- Set<BookieSocketAddress> bookies = Sets.newHashSet();
+ Set<BookieId> bookies = Sets.newHashSet();
for (LedgerFragment f : fragments) {
bookies.addAll(f.getAddresses());
}
@@ -1162,8 +1162,7 @@ public class Auditor implements AutoCloseable {
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 class Auditor implements AutoCloseable {
if (metadata.isClosed()) {
boolean foundSegmentNotAdheringToPlacementPolicy = false;
boolean foundSegmentSoftlyAdheringToPlacementPolicy = false;
- for (Map.Entry<Long, ? extends List<BookieSocketAddress>> ensemble : metadata
+ for (Map.Entry<Long, ? extends List<BookieId>> ensemble : metadata
.getAllEnsembles().entrySet()) {
long startEntryIdOfSegment = ensemble.getKey();
- List<BookieSocketAddress> ensembleOfSegment = ensemble.getValue();
+ List<BookieId> ensembleOfSegment = ensemble.getValue();
PlacementPolicyAdherence segmentAdheringToPlacementPolicy = admin
.isEnsembleAdheringToPlacementPolicy(ensembleOfSegment, writeQuorumSize,
ackQuorumSize);
@@ -1380,17 +1379,17 @@ public class Auditor implements AutoCloseable {
/*
* segment details, like start entryid of the segment and ensemble List.
*/
- private final Entry<Long, ? extends List<BookieSocketAddress>> segmentEnsemble;
+ private final Entry<Long, ? extends List<BookieId>> 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<Long> unavailableEntriesList;
- private MissingEntriesInfo(long ledgerId, Entry<Long, ? extends List<BookieSocketAddress>> segmentEnsemble,
- BookieSocketAddress bookieMissingEntries, List<Long> unavailableEntriesList) {
+ private MissingEntriesInfo(long ledgerId, Entry<Long, ? extends List<BookieId>> segmentEnsemble,
+ BookieId bookieMissingEntries, List<Long> unavailableEntriesList) {
this.ledgerId = ledgerId;
this.segmentEnsemble = segmentEnsemble;
this.bookieMissingEntries = bookieMissingEntries;
@@ -1401,11 +1400,11 @@ public class Auditor implements AutoCloseable {
return ledgerId;
}
- private Entry<Long, ? extends List<BookieSocketAddress>> getSegmentEnsemble() {
+ private Entry<Long, ? extends List<BookieId>> getSegmentEnsemble() {
return segmentEnsemble;
}
- private BookieSocketAddress getBookieMissingEntries() {
+ private BookieId getBookieMissingEntries() {
return bookieMissingEntries;
}
@@ -1503,7 +1502,7 @@ public class Auditor implements AutoCloseable {
int ensembleSize = metadata.getEnsembleSize();
RoundRobinDistributionSchedule distributionSchedule = new RoundRobinDistributionSchedule(writeQuorumSize,
ackQuorumSize, ensembleSize);
- List<Entry<Long, ? extends List<BookieSocketAddress>>> segments = new LinkedList<>(
+ List<Entry<Long, ? extends List<BookieId>>> segments = new LinkedList<>(
metadata.getAllEnsembles().entrySet());
/*
* since there are multiple segments, MultiCallback should be
@@ -1511,11 +1510,11 @@ public class Auditor implements AutoCloseable {
*/
MultiCallback mcbForThisLedger = new MultiCallback(ensembleSize * segments.size(), mcbForThisLedgerRange,
null, BKException.Code.OK, BKException.Code.ReadException);
- HashMap<BookieSocketAddress, List<BookieExpectedToContainSegmentInfo>> bookiesSegmentInfoMap =
- new HashMap<BookieSocketAddress, List<BookieExpectedToContainSegmentInfo>>();
+ HashMap<BookieId, List<BookieExpectedToContainSegmentInfo>> bookiesSegmentInfoMap =
+ new HashMap<BookieId, List<BookieExpectedToContainSegmentInfo>>();
for (int segmentNum = 0; segmentNum < segments.size(); segmentNum++) {
- final Entry<Long, ? extends List<BookieSocketAddress>> segmentEnsemble = segments.get(segmentNum);
- final List<BookieSocketAddress> ensembleOfSegment = segmentEnsemble.getValue();
+ final Entry<Long, ? extends List<BookieId>> segmentEnsemble = segments.get(segmentNum);
+ final List<BookieId> 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 class Auditor implements AutoCloseable {
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 class Auditor implements AutoCloseable {
lastEntryIdOfSegment, segmentEnsemble, entriesStripedToThisBookie));
}
}
- for (Entry<BookieSocketAddress, List<BookieExpectedToContainSegmentInfo>> bookiesSegmentInfoTuple :
+ for (Entry<BookieId, List<BookieExpectedToContainSegmentInfo>> bookiesSegmentInfoTuple :
bookiesSegmentInfoMap.entrySet()) {
- final BookieSocketAddress bookieInEnsemble = bookiesSegmentInfoTuple.getKey();
+ final BookieId bookieInEnsemble = bookiesSegmentInfoTuple.getKey();
final List<BookieExpectedToContainSegmentInfo> bookieSegmentInfoList = bookiesSegmentInfoTuple
.getValue();
admin.asyncGetListOfEntriesOfLedger(bookieInEnsemble, ledgerInRange)
@@ -1576,11 +1575,11 @@ public class Auditor implements AutoCloseable {
private static class BookieExpectedToContainSegmentInfo {
private final long startEntryIdOfSegment;
private final long lastEntryIdOfSegment;
- private final Entry<Long, ? extends List<BookieSocketAddress>> segmentEnsemble;
+ private final Entry<Long, ? extends List<BookieId>> segmentEnsemble;
private final BitSet entriesOfSegmentStripedToThisBookie;
private BookieExpectedToContainSegmentInfo(long startEntryIdOfSegment, long lastEntryIdOfSegment,
- Entry<Long, ? extends List<BookieSocketAddress>> segmentEnsemble,
+ Entry<Long, ? extends List<BookieId>> segmentEnsemble,
BitSet entriesOfSegmentStripedToThisBookie) {
this.startEntryIdOfSegment = startEntryIdOfSegment;
this.lastEntryIdOfSegment = lastEntryIdOfSegment;
@@ -1596,7 +1595,7 @@ public class Auditor implements AutoCloseable {
return lastEntryIdOfSegment;
}
- public Entry<Long, ? extends List<BookieSocketAddress>> getSegmentEnsemble() {
+ public Entry<Long, ? extends List<BookieId>> getSegmentEnsemble() {
return segmentEnsemble;
}
@@ -1611,14 +1610,14 @@ public class Auditor implements AutoCloseable {
private final int ensembleSize;
private final int writeQuorumSize;
private final int ackQuorumSize;
- private final BookieSocketAddress bookieInEnsemble;
+ private final BookieId bookieInEnsemble;
private final List<BookieExpectedToContainSegmentInfo> bookieExpectedToContainSegmentInfoList;
private final ConcurrentHashMap<Long, MissingEntriesInfoOfLedger> ledgersWithMissingEntries;
private final ConcurrentHashMap<Long, MissingEntriesInfoOfLedger> ledgersWithUnavailableBookies;
private final MultiCallback mcbForThisLedger;
private GetListOfEntriesOfLedgerCallbackForReplicasCheck(long ledgerInRange, int ensembleSize,
- int writeQuorumSize, int ackQuorumSize, BookieSocketAddress bookieInEnsemble,
+ int writeQuorumSize, int ackQuorumSize, BookieId bookieInEnsemble,
List<BookieExpectedToContainSegmentInfo> bookieExpectedToContainSegmentInfoList,
ConcurrentHashMap<Long, MissingEntriesInfoOfLedger> ledgersWithMissingEntries,
ConcurrentHashMap<Long, MissingEntriesInfoOfLedger> ledgersWithUnavailableBookies,
@@ -1688,7 +1687,7 @@ public class Auditor implements AutoCloseable {
final long lastEntryIdOfSegment = bookieExpectedToContainSegmentInfo.getLastEntryIdOfSegment();
final BitSet entriesStripedToThisBookie = bookieExpectedToContainSegmentInfo
.getEntriesOfSegmentStripedToThisBookie();
- final Entry<Long, ? extends List<BookieSocketAddress>> segmentEnsemble =
+ final Entry<Long, ? extends List<BookieId>> segmentEnsemble =
bookieExpectedToContainSegmentInfo.getSegmentEnsemble();
final List<Long> unavailableEntriesList = availabilityOfEntriesOfLedger
.getUnavailableEntries(startEntryIdOfSegment, lastEntryIdOfSegment, entriesStripedToThisBookie);
@@ -1856,7 +1855,7 @@ public class Auditor implements AutoCloseable {
for (int listInd = 0; listInd < missingEntriesInfoList.size(); listInd++) {
MissingEntriesInfo missingEntriesInfo = missingEntriesInfoList.get(listInd);
List<Long> unavailableEntriesList = missingEntriesInfo.getUnavailableEntriesList();
- Entry<Long, ? extends List<BookieSocketAddress>> segmentEnsemble =
+ Entry<Long, ? extends List<BookieId>> segmentEnsemble =
missingEntriesInfo.getSegmentEnsemble();
missingEntries.addAll(unavailableEntriesList);
errMessage.append("In segment starting at " + segmentEnsemble.getKey() + " with ensemble "
@@ -1907,7 +1906,7 @@ public class Auditor implements AutoCloseable {
errMessage.append("Ledger : " + ledgerWithUnavailableBookies + " has following unavailable bookies : ");
for (int listInd = 0; listInd < missingBookiesInfoList.size(); listInd++) {
MissingEntriesInfo missingBookieInfo = missingBookiesInfoList.get(listInd);
- Entry<Long, ? extends List<BookieSocketAddress>> segmentEnsemble =
+ Entry<Long, ? extends List<BookieId>> 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 e644369..f396e5f 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.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.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 @@ public class AuditorElector {
* 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 class AuditorElector {
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 026fd72..f136f28 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 java.util.concurrent.CountDownLatch;
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 class BookieLedgerIndexer {
Processor<Long> ledgerProcessor = new Processor<Long>() {
@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<Long, ? extends List<BookieSocketAddress>> ensemble
+ for (Map.Entry<Long, ? extends List<BookieId>> 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 80cfed3..524d07d 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.conf.ServerConfiguration;
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 @@ public class ReplicationWorker implements Runnable {
return false;
}
- SortedMap<Long, ? extends List<BookieSocketAddress>> ensembles = admin.getLedgerMetadata(lh).getAllEnsembles();
- List<BookieSocketAddress> finalEnsemble = ensembles.get(ensembles.lastKey());
- Collection<BookieSocketAddress> available = admin.getAvailableBookies();
- for (BookieSocketAddress b : finalEnsemble) {
+ SortedMap<Long, ? extends List<BookieId>> ensembles = admin.getLedgerMetadata(lh).getAllEnsembles();
+ List<BookieId> finalEnsemble = ensembles.get(ensembles.lastKey());
+ Collection<BookieId> 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 5295bb7..19cd992 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.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.net.BookieSocketAddress;
+import org.apache.bookkeeper.net.BookieId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -74,9 +74,7 @@ public class DecommissionService implements HttpEndpointService {
HashMap<String, String> 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 b543cec..33a1007 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.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.net.BookieSocketAddress;
+import org.apache.bookkeeper.net.BookieId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -85,7 +85,7 @@ public class ListBookieInfoService implements HttpEndpointService {
clientConf.setDiskWeightBasedPlacementEnabled(true);
BookKeeper bk = new BookKeeper(clientConf);
- Map<BookieSocketAddress, BookieInfoReader.BookieInfo> map = bk.getBookieInfo();
+ Map<BookieId, BookieInfoReader.BookieInfo> map = bk.getBookieInfo();
if (map.size() == 0) {
bk.close();
response.setCode(HttpServer.StatusCode.NOT_FOUND);
@@ -104,7 +104,7 @@ public class ListBookieInfoService implements HttpEndpointService {
*/
LinkedHashMap<String, String> output = Maps.newLinkedHashMapWithExpectedSize(map.size());
Long totalFree = 0L, total = 0L;
- for (Map.Entry<BookieSocketAddress, BookieInfoReader.BookieInfo> infoEntry : map.entrySet()) {
+ for (Map.Entry<BookieId, BookieInfoReader.BookieInfo> 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 f969ff7..7269b2e 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.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.net.BookieId;
import org.apache.bookkeeper.net.BookieSocketAddress;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -59,7 +60,7 @@ public class ListBookiesService implements HttpEndpointService {
HttpServiceResponse response = new HttpServiceResponse();
// GET
if (HttpServer.Method.GET == request.getMethod()) {
- Collection<BookieSocketAddress> bookies = new ArrayList<BookieSocketAddress>();
+ Collection<BookieId> bookies = new ArrayList<BookieId>();
Map<String, String> params = request.getParams();
// default print rw
@@ -79,9 +80,14 @@ public class ListBookiesService implements HttpEndpointService {
// output <bookieSocketAddress: hostname>
Map<String, String> 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 4a0a0a3..a1ca615 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.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.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 class RecoveryBookieService implements HttpEndpointService {
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 9154ccd..8bb7824 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.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.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 class WhoIsAuditorService implements HttpEndpointService {
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 class WhoIsAuditorService implements HttpEndpointService {
}
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 9653db8..3133ede 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 class WhoIsAuditorCommand extends BookieCommand<CliFlags> {
.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 701d562..adcf2fd 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.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.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 @@ public class FlipBookieIdCommand extends BookieCommand<FlipBookieIdCommand.FlipB
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 542216f..664c7c8 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.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.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 @@ public class ListLedgersCommand extends BookieCommand<ListLedgersFlags> {
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 98c1b11..f41ce5e 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.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.net.BookieId;
import org.apache.bookkeeper.proto.BookieClient;
import org.apache.bookkeeper.proto.BookieClientImpl;
import org.apache.bookkeeper.proto.BookieProtocol;
@@ -143,10 +143,10 @@ public class ReadLedgerCommand extends BookieCommand<ReadLedgerCommand.ReadLedge
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 @@ public class ReadLedgerCommand extends BookieCommand<ReadLedgerCommand.ReadLedge
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<Void> 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 88d1b4d..6155027 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.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.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 class UpdateBookieInLedgerCommand extends BookieCommand<UpdateBookieInLed
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 <address>:<port> 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 c7aa537..3466efe 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.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.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 @@ public class DecommissionCommand extends BookieCommand<DecommissionCommand.Decom
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 3ef7e7b..1c098de 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.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.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 @@ public class EndpointInfoCommand extends BookieCommand<EndpointInfoCommand.Endpo
if (bookieId == null || bookieId.isEmpty()) {
throw new IllegalArgumentException("BookieId is required");
}
- BookieSocketAddress address = new BookieSocketAddress(bookieId);
- Collection<BookieSocketAddress> allBookies = admin.getAllBookies();
+ BookieId address = BookieId.parse(bookieId);
+ Collection<BookieId> 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 047c15e..f433b7c 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.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.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 class InfoCommand extends BookieCommand<CliFlags> {
ClientConfiguration clientConf = new ClientConfiguration(conf);
clientConf.setDiskWeightBasedPlacementEnabled(true);
try (BookKeeper bk = new BookKeeper(clientConf)) {
- Map<BookieSocketAddress, BookieInfo> map = bk.getBookieInfo();
+ Map<BookieId, BookieInfo> 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 class InfoCommand extends BookieCommand<CliFlags> {
System.out.println("Free disk space info:");
long totalFree = 0, total = 0;
- for (Map.Entry<BookieSocketAddress, BookieInfo> e : map.entrySet()) {
+ for (Map.Entry<BookieId, BookieInfo> 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 class InfoCommand extends BookieCommand<CliFlags> {
Map<String, BookieInfo> 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 33e1a2d..ee98873 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.Collection;
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 @@ public class ListBookiesCommand extends DiscoveryCommand<Flags> {
boolean hasBookies = false;
if (flags.readwrite) {
- Set<BookieSocketAddress> bookies = result(
+ Set<BookieId> 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<BookieSocketAddress> bookies = result(
+ Set<BookieId> 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<BookieSocketAddress> bookies = result(
+ Set<BookieId> 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 @@ public class ListBookiesCommand extends DiscoveryCommand<Flags> {
}
}
- private static void printBookies(Collection<BookieSocketAddress> bookies) {
- for (BookieSocketAddress b : bookies) {
- System.out.println(getBookieSocketAddrStringRepresentation(b));
+ private static void printBookies(Collection<BookieId> 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 c1a1694..0cce375 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.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.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 @@ public class RecoverCommand extends BookieCommand<RecoverCommand.RecoverFlags> {
// Get bookies list
final String[] bookieStrs = flags.bookieAddress.split(",");
- final Set<BookieSocketAddress> bookieAddrs = new HashSet<>();
+ final Set<BookieId> 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 @@ public class RecoverCommand extends BookieCommand<RecoverCommand.RecoverFlags> {
}
}
- private boolean bkQuery(BookKeeperAdmin bkAdmin, Set<BookieSocketAddress> bookieAddrs)
+ private boolean bkQuery(BookKeeperAdmin bkAdmin, Set<BookieId> bookieAddrs)
throws InterruptedException, BKException {
SortedMap<Long, LedgerMetadata> ledgersContainBookies =
bkAdmin.getLedgersContainBookies(bookieAddrs);
@@ -180,14 +179,14 @@ public class RecoverCommand extends BookieCommand<RecoverCommand.RecoverFlags> {
return true;
}
- private Map<Long, Integer> inspectLedger(LedgerMetadata metadata, Set<BookieSocketAddress> bookiesToInspect) {
+ private Map<Long, Integer> inspectLedger(LedgerMetadata metadata, Set<BookieId> bookiesToInspect) {
Map<Long, Integer> numBookiesToReplacePerEnsemble = new TreeMap<Long, Integer>();
- for (Map.Entry<Long, ? extends List<BookieSocketAddress>> ensemble :
+ for (Map.Entry<Long, ? extends List<BookieId>> ensemble :
metadata.getAllEnsembles().entrySet()) {
- List<BookieSocketAddress> bookieList = ensemble.getValue();
+ List<BookieId> 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 @@ public class RecoverCommand extends BookieCommand<RecoverCommand.RecoverFlags> {
private boolean bkRecoveryLedger(BookKeeperAdmin bkAdmin,
long lid,
- Set<BookieSocketAddress> bookieAddrs,
+ Set<BookieId> bookieAddrs,
boolean dryrun,
boolean skipOpenLedgers,
boolean removeCookies)
@@ -218,12 +217,12 @@ public class RecoverCommand extends BookieCommand<RecoverCommand.RecoverFlags> {
}
private void deleteCookies(ClientConfiguration conf,
- Set<BookieSocketAddress> bookieAddrs) throws BKException {
+ Set<BookieId> 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 @@ public class RecoverCommand extends BookieCommand<RecoverCommand.RecoverFlags> {
}
- private void deleteCookie(RegistrationManager rm, BookieSocketAddress bookieSrc) throws BookieException {
+ private void deleteCookie(RegistrationManager rm, BookieId bookieSrc) throws BookieException {
try {
Versioned<Cookie> cookie = Cookie.readFromRegistrationManager(rm, bookieSrc);
cookie.getValue().deleteFromRegistrationManager(rm, bookieSrc, cookie.getVersion());
@@ -257,7 +256,7 @@ public class RecoverCommand extends BookieCommand<RecoverCommand.RecoverFlags> {
}
private boolean bkRecovery(BookKeeperAdmin bkAdmin,
- Set<BookieSocketAddress> bookieAddrs,
+ Set<BookieId> 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 dce11ce..5cdefd0 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.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.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 @@ public class AdminCommand extends BookieCommand<AdminCommand.AdminFlags> {
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<Cookie> oldCookie = null;
try {
@@ -247,7 +247,7 @@ public class AdminCommand extends BookieCommand<AdminCommand.AdminFlags> {
}
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 fe2c531..d08f1f1 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.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.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 @@ abstract class CookieCommand<CookieFlagsT extends CliFlags>
}
}
- 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 @@ abstract class CookieCommand<CookieFlagsT extends CliFlags>
+ " Bookie id should be in the format of '<hostname>:<port>'");
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 430fed9..46c597d 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 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.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 @@ public class CreateCookieCommand extends CookieCommand<Flags> {
@Override
protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception {
- String bookieId = getBookieId(cmdFlags);
+ BookieId bookieId = getBookieId(cmdFlags);
byte[] data = readCookieDataFromFile(cmdFlags.cookieFile);
Versioned<byte[]> 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 4c42615..73cb4e7 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 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.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 @@ public class DeleteCookieCommand extends CookieCommand<Flags> {
@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 6e3d7bd..3ef8116 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 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.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 @@ public class GenerateCookieCommand extends CookieCommand<Flags> {
@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 @@ public class GenerateCookieCommand extends CookieCommand<Flags> {
}
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 76f5f7c..598127a 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;
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 @@ public class GetCookieCommand extends CookieCommand<Flags> {
@Override
protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception {
- String bookieId = getBookieId(cmdFlags);
+ BookieId bookieId = getBookieId(cmdFlags);
try {
- Versioned<Cookie> cookie = Cookie.readFromRegistrationManager(
- rm, new BookieSocketAddress(bookieId)
- );
+ Versioned<Cookie> 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 77e5f05..4979b6e 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 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.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 @@ public class UpdateCookieCommand extends CookieCommand<Flags> {
@Override
protected void apply(RegistrationManager rm, Flags cmdFlags) throws Exception {
- String bookieId = getBookieId(cmdFlags);
+ BookieId bookieId = getBookieId(cmdFlags);
byte[] data = readCookieDataFromFile(cmdFlags.cookieFile);
Versioned<byte[]> 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 a1dc1ab..de527ad 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 com.google.common.net.InetAddresses;
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 final class CommandHelpers {
}
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 72e59b1..ad522c7 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 java.util.Collection;
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 class AdvertisedAddressTest extends BookKeeperClusterTestCase {
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<BookieSocketAddress> bookies = bka.getAvailableBookies();
+ Collection<BookieId> 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 class AdvertisedAddressTest extends BookKeeperClusterTestCase {
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 f57d268..d627bf7 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.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;
import static org.mockito.Mockito.mock;
@@ -95,7 +94,7 @@ 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.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 class BookieInitializationTest extends BookKeeperClusterTestCase {
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 class BookieInitializationTest extends BookKeeperClusterTestCase {
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 class BookieInitializationTest extends BookKeeperClusterTestCase {
.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 class BookieInitializationTest extends BookKeeperClusterTestCase {
.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 class BookieInitializationTest extends BookKeeperClusterTestCase {
.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 class BookieInitializationTest extends BookKeeperClusterTestCase {
.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 class BookieInitializationTest extends BookKeeperClusterTestCase {
Versioned<byte[]> 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 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
Bookie b = new Bookie(conf);
- final BookieSocketAddress bookieAddress = Bookie.getBookieAddress(conf);
+ final BookieId bookieAddress = Bookie.getBookieId(conf);
// Read cookie from registation manager
Versioned<Cookie> 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 3ac507c..ee55fb2 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.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;
+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 class BookieJournalForceTest {
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 class BookieJournalForceTest {
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 class BookieJournalForceTest {
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 class BookieJournalForceTest {
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 class BookieJournalForceTest {
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 34d80f5..538139d 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.ForceWriteRequest;
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 class BookieJournalPageCacheFlushTest {
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 class BookieJournalPageCacheFlushTest {
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 class BookieJournalPageCacheFlushTest {
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 class BookieJournalPageCacheFlushTest {
// 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 b7e73b9..8ab38c1 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 java.nio.charset.StandardCharsets.UTF_8;
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.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.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 class BookieShellTest {
.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 class BookieShellTest {
@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 @@ public class BookieShellTest {
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 @@ public class BookieShellTest {
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 27a5d28..9420661 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 lombok.extern.slf4j.Slf4j;
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 class BookieWriteToJournalTest {
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 class BookieWriteToJournalTest {
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 class BookieWriteToJournalTest {
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 class BookieWriteToJournalTest {
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 class BookieWriteToJournalTest {
// 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 371971b..d5b454b 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.LedgerManagerFactory;
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 class GcOverreplicatedLedgerTest extends LedgerManagerTestCase {
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 class GcOverreplicatedLedgerTest extends LedgerManagerTestCase {
activeLedgers.put(lh.getId(), true);
LedgerMetadata newLedgerMetadata = ledgerManager.readLedgerMetadata(lh.getId()).get().getValue();
- BookieSocketAddress address = null;
- SortedMap<Long, ? extends List<BookieSocketAddress>> ensembleMap = newLedgerMetadata.getAllEnsembles();
- for (List<BookieSocketAddress> ensemble : ensembleMap.values()) {
+ BookieId address = null;
+ SortedMap<Long, ? extends List<BookieId>> ensembleMap = newLedgerMetadata.getAllEnsembles();
+ for (List<BookieId> ensemble : ensembleMap.values()) {
address = ensemble.get(0);
}
ServerConfiguration bkConf = getBkConf(address);
@@ -150,7 +150,7 @@ public class GcOverreplicatedLedgerTest extends LedgerManagerTestCase {
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 class GcOverreplicatedLedgerTest extends LedgerManagerTestCase {
Assert.assertTrue(activeLedgers.containsKey(lh.getId()));
}
- private BookieSocketAddress getBookieNotInEnsemble(LedgerMetadata ledgerMetadata) throws UnknownHostException {
- List<BookieSocketAddress> allAddresses = Lists.newArrayList();
+ private BookieId getBookieNotInEnsemble(LedgerMetadata ledgerMetadata) throws UnknownHostException {
+ List<BookieId> allAddresses = Lists.newArrayList();
for (BookieServer bk : bs) {
- allAddresses.add(bk.getLocalAddress());
+ allAddresses.add(bk.getBookieId());
}
- SortedMap<Long, ? extends List<BookieSocketAddress>> ensembles = ledgerMetadata.getAllEnsembles();
- for (List<BookieSocketAddress> fragmentEnsembles : ensembles.values()) {
+ SortedMap<Long, ? extends List<BookieId>> ensembles = ledgerMetadata.getAllEnsembles();
+ for (List<BookieId> 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 8258f23..63bd7f7 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.bookie.FileInfoBackingCache.CachedFileInfo;
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 class LedgerCacheTest {
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 15253fb..6d2b972 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.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.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 class BookKeeperAdminTest extends BookKeeperClusterTestCase {
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 class BookKeeperAdminTest extends BookKeeperClusterTestCase {
try (BookKeeperAdmin bkAdmin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString())) {
for (int i = 0; i < bs.size(); i++) {
CompletableFuture<AvailabilityOfEntriesOfLedger> 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 class BookKeeperAdminTest extends BookKeeperClusterTestCase {
try (BookKeeperAdmin bkAdmin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString())) {
for (int i = 0; i < bs.size(); i++) {
CompletableFuture<AvailabilityOfEntriesOfLedger> 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 class BookKeeperAdminTest extends BookKeeperClusterTestCase {
try (BookKeeperAdmin bkAdmin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString())) {
for (int i = 0; i < bs.size(); i++) {
CompletableFuture<AvailabilityOfEntriesOfLedger> 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 class BookKeeperAdminTest extends BookKeeperClusterTestCase {
(zkc.exists(bookieCookiePath, false) != null));
try (BookKeeperAdmin bkAdmin = new BookKeeperAdmin(zkUtil.getZooKeeperConnectString())) {
- Collection<BookieSocketAddress> availableBookies = bkAdmin.getAvailableBookies();
+ Collection<BookieId> 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<BookieSocketAddress> remainingBookies = bkAdmin.getAvailableBookies();
- Assert.assertFalse(remainingBookies.contains(killedBookie.getLocalAddress()));
+ Collection<BookieId> remainingBookies = bkAdmin.getAvailableBookies();
+ Assert.assertFalse(remainingBookies.contains(killedBookie.getBookieId()));
- Collection<BookieSocketAddress> allBookies = bkAdmin.getAllBookies();
+ Collection<BookieId> 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 class BookKeeperAdminTest extends BookKeeperClusterTestCase {
* 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 class BookKeeperAdminTest extends BookKeeperClusterTestCase {
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<BookieSocketAddress> ensembleOfSegment1 = new ArrayList<BookieSocketAddress>();
+ List<BookieId> ensembleOfSegment1 = new ArrayList<BookieId>();
ensembleOfSegment1.add(bookie0);
ensembleOfSegment1.add(bookie1);
ensembleOfSegment1.add(bookie2);
- List<BookieSocketAddress> ensembleOfSegment2 = new ArrayList<BookieSocketAddress>();
+ List<BookieId> ensembleOfSegment2 = new ArrayList<BookieId>();
ensembleOfSegment2.add(bookie3);
ensembleOfSegment2.add(bookie1);
ensembleOfSegment2.add(bookie2);
@@ -681,7 +682,7 @@ public class BookKeeperAdminTest extends BookKeeperClusterTestCase {
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 3bebc21..168d9af 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.AsyncCallback.ReadCallback;
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 class BookKeeperCloseTest extends BookKeeperClusterTestCase {
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 33512ae..a7fe962 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.client.BookKeeper.DigestType;
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 @@ public class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
super(conf);
}
- void blockUntilBookieWeightIs(BookieSocketAddress bookie, Optional<Long> target) throws InterruptedException {
+ void blockUntilBookieWeightIs(BookieId bookie, Optional<Long> target) throws InterruptedException {
long startMsecs = System.currentTimeMillis();
Optional<Long> freeDiskSpace = Optional.empty();
while (System.currentTimeMillis() < (startMsecs + MS_WEIGHT_UPDATE_TIMEOUT)) {
@@ -95,7 +95,7 @@ public class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
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 @@ public class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
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 @@ public class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
replaceBookieWithCustomFreeDiskSpaceBookie(client, 0, multiple * freeDiskSpace);
}
}
- Map<BookieSocketAddress, Integer> m = new HashMap<BookieSocketAddress, Integer>();
+ Map<BookieId, Integer> m = new HashMap<BookieId, Integer>();
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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
// 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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
replaceBookieWithCustomFreeDiskSpaceBookie(client, 0, multiple * freeDiskSpace);
}
}
- Map<BookieSocketAddress, Integer> m = new HashMap<BookieSocketAddress, Integer>();
+ Map<BookieId, Integer> m = new HashMap<BookieId, Integer>();
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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
// 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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
// 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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
replaceBookieWithCustomFreeDiskSpaceBookie(client, 0, multiple * freeDiskSpace);
}
}
- Map<BookieSocketAddress, Integer> m = new HashMap<BookieSocketAddress, Integer>();
+ Map<BookieId, Integer> m = new HashMap<BookieId, Integer>();
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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
// let the last two bookies be down initially
ServerConfiguration conf1 = killBookieAndWaitForZK(numBookies - 1);
ServerConfiguration conf2 = killBookieAndWaitForZK(numBookies - 2);
- Map<BookieSocketAddress, Integer> m = new HashMap<BookieSocketAddress, Integer>();
+ Map<BookieId, Integer> m = new HashMap<BookieId, Integer>();
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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
// 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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
// 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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
client, 0, freeDiskSpace, multiple * freeDiskSpace, useHigherValue);
}
}
- Map<BookieSocketAddress, Integer> m = new HashMap<BookieSocketAddress, Integer>();
+ Map<BookieId, Integer> m = new HashMap<BookieId, Integer>();
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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
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 class BookKeeperDiskSpaceWeightedLedgerPlacementTest extends BookKeeperCl
// 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 a00ec9a..c19e391 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.BookKeeper.DigestType;
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 @@ public class BookKeeperTest extends BookKeeperClusterTestCase {
}
@Override
- public boolean areAckedBookiesAdheringToPlacementPolicy(Set<BookieSocketAddress> ackedBookies,
+ public boolean areAckedBookiesAdheringToPlacementPolicy(Set<BookieId> ackedBookies,
int writeQuorumSize,
int ackQuorumSize) {
conditionFirstInvocationLatch.countDown();
@@ -1027,7 +1027,7 @@ public class BookKeeperTest extends BookKeeperClusterTestCase {
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 class BookKeeperTest extends BookKeeperClusterTestCase {
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 e1d32af..e361294 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 lombok.extern.slf4j.Slf4j;
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 class BookKeeperTestClient extends BookKeeper {
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 class BookKeeperTestClient extends BookKeeper {
* 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 906db8f..1fddabb 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 class BookieDecommissionTest extends BookKeeperClusterTestCase {
* 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 class BookieDecommissionTest extends BookKeeperClusterTestCase {
* 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<UnderreplicatedLedger> ledgersToRereplicate = urLedgerMgr.listLedgersToRereplicate(null);
@@ -101,7 +101,7 @@ public class BookieDecommissionTest extends BookKeeperClusterTestCase {
}
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 class BookieDecommissionTest extends BookKeeperClusterTestCase {
* 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<UnderreplicatedLedger> 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 2fc1d5f..ce66c9f 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.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.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 @@ public class BookieRecoveryTest extends BookKeeperClusterTestCase {
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 class BookieRecoveryTest extends BookKeeperClusterTestCase {
// 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 class BookieRecoveryTest extends BookKeeperClusterTestCase {
// 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 class BookieRecoveryTest extends BookKeeperClusterTestCase {
// 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 class BookieRecoveryTest extends BookKeeperClusterTestCase {
// 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 @@ public class BookieRecoveryTest extends BookKeeperClusterTestCase {
private boolean verifyFullyReplicated(LedgerHandle lh, long untilEntry) throws Exception {
LedgerMetadata md = getLedgerMetadata(lh);
- Map<Long, ? extends List<BookieSocketAddress>> ensembles = md.getAllEnsembles();
+ Map<Long, ? extends List<BookieId>> ensembles = md.getAllEnsembles();
HashMap<Long, Long> ranges = new HashMap<Long, Long>();
ArrayList<Long> keyList = Collections.list(
@@ -516,7 +516,7 @@ public class BookieRecoveryTest extends BookKeeperClusterTestCase {
}
ranges.put(keyList.get(keyList.size() - 1), untilEntry);
- for (Map.Entry<Long, ? extends List<BookieSocketAddress>> e : ensembles.entrySet()) {
+ for (Map.Entry<Long, ? extends List<BookieId>> e : ensembles.entrySet()) {
int quorum = md.getAckQuorumSize();
long startEntryId = e.getKey();
long endEntryId = ranges.get(startEntryId);
@@ -525,7 +525,7 @@ public class BookieRecoveryTest extends BookKeeperClusterTestCase {
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 @@ public class BookieRecoveryTest extends BookKeeperClusterTestCase {
long numDupes = 0;
for (LedgerHandle lh : lhs) {
LedgerMetadata md = getLedgerMetadata(lh);
- for (Map.Entry<Long, ? extends List<BookieSocketAddress>> e : md.getAllEnsembles().entrySet()) {
- HashSet<BookieSocketAddress> set = new HashSet<BookieSocketAddress>();
+ for (Map.Entry<Long, ? extends List<BookieId>> e : md.getAllEnsembles().entrySet()) {
+ HashSet<BookieId> set = new HashSet<BookieId>();
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 class BookieRecoveryTest extends BookKeeperClusterTestCase {
closeLedgers(lhs);
// Shutdown last bookie server in last ensemble
- List<BookieSocketAddress> lastEnsemble = lhs.get(0).getLedgerMetadata().getAllEnsembles()
+ List<BookieId> 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 class BookieRecoveryTest extends BookKeeperClusterTestCase {
writeEntriestoLedgers(numMsgs, 0, lhs);
// Shutdown the first bookie server
- List<BookieSocketAddress> lastEnsemble = lhs.get(0).getLedgerMetadata().getAllEnsembles()
+ List<BookieId> 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 class BookieRecoveryTest extends BookKeeperClusterTestCase {
writeEntriestoLedgers(numMsgs, 0, lhs);
// Shutdown the first bookie server
- List<BookieSocketAddress> lastEnsemble = lhs.get(0).getLedgerMetadata().getAllEnsembles()
+ List<BookieId> 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 class BookieRecoveryTest extends BookKeeperClusterTestCase {
List<LedgerHandle> newLhs = openLedgers(lhs);
for (LedgerHandle newLh : newLhs) {
// first ensemble should contains bookieToKill2 and not contain bookieToKill
- Map.Entry<Long, ? extends List<BookieSocketAddress>> entry =
+ Map.Entry<Long, ? extends List<BookieId>> entry =
newLh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next();
assertFalse(entry.getValue().contains(bookieToKill));
assertTrue(entry.getValue().contains(bookieToKill2));
@@ -725,7 +725,7 @@ public class BookieRecoveryTest extends BookKeeperClusterTestCase {
// 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 class BookieRecoveryTest extends BookKeeperClusterTestCase {
// 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 class BookieRecoveryTest extends BookKeeperClusterTestCase {
}
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 class BookieRecoveryTest extends BookKeeperClusterTestCase {
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 a3fe070..7c1e3da 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.ReadHandle;
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 class BookieWriteLedgerTest extends
startNewBookie();
// Shutdown three bookies in the last ensemble and continue writing
- List<BookieSocketAddress> ensemble = lh.getLedgerMetadata()
+ List<BookieId> ensemble = lh.getLedgerMetadata()
.getAllEnsembles().entrySet().iterator().next().getValue();
killBookie(ensemble.get(0));
killBookie(ensemble.get(1));
@@ -196,7 +196,7 @@ public class BookieWriteLedgerTest extends
CountDownLatch sleepLatch1 = new CountDownLatch(1);
CountDownLatch sleepLatch2 = new CountDownLatch(1);
- List<BookieSocketAddress> ensemble = lh.getLedgerMetadata()
+ List<BookieId> ensemble = lh.getLedgerMetadata()
.getAllEnsembles().entrySet().iterator().next().getValue();
sleepBookie(ensemble.get(0), sleepLatch1);
@@ -293,7 +293,7 @@ public class BookieWriteLedgerTest extends
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 class BookieWriteLedgerTest extends
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 class BookieWriteLedgerTest extends
startNewBookie();
// Shutdown one bookie in the last ensemble and continue writing
- List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next()
+ List<BookieId> ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next()
.getValue();
killBookie(ensemble.get(0));
@@ -529,7 +529,7 @@ public class BookieWriteLedgerTest extends
startNewBookie();
// Shutdown one bookie in the last ensemble and continue writing
- List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next()
+ List<BookieId> ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next()
.getValue();
killBookie(ensemble.get(0));
@@ -879,7 +879,7 @@ public class BookieWriteLedgerTest extends
startNewBookie();
// Shutdown three bookies in the last ensemble and continue writing
- List<BookieSocketAddress> ensemble = lh.getLedgerMetadata()
+ List<BookieId> ensemble = lh.getLedgerMetadata()
.getAllEnsembles().entrySet().iterator().next().getValue();
killBookie(ensemble.get(0));
killBookie(ensemble.get(1));
@@ -952,7 +952,7 @@ public class BookieWriteLedgerTest extends
}
// Start One more bookie and shutdown one from last ensemble before reading
startNewBookie();
- List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next()
+ List<BookieId> ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next()
.getValue();
killBookie(ensemble.get(0));
@@ -1020,7 +1020,7 @@ public class BookieWriteLedgerTest extends
}
CountDownLatch sleepLatch1 = new CountDownLatch(1);
- List<BookieSocketAddress> ensemble;
+ List<BookieId> ensemble;
ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next().getValue();
@@ -1106,7 +1106,7 @@ public class BookieWriteLedgerTest extends
if (j == numEntriesToWrite / 2) {
// Start One more bookie and shutdown one from last ensemble at half-way
startNewBookie();
- List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet()
+ List<BookieId> ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet()
.iterator().next().getValue();
killBookie(ensemble.get(0));
}
@@ -1175,7 +1175,7 @@ public class BookieWriteLedgerTest extends
}
// Start One more bookie and shutdown one from last ensemble before reading
startNewBookie();
- List<BookieSocketAddress> ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next()
+ List<BookieId> 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 996c902..dabf2b6 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 java.util.concurrent.CompletableFuture;
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 class DeferredSyncTest extends MockBookKeeperTestCase {
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 class DeferredSyncTest extends MockBookKeeperTestCase {
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 828a431..646fb32 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.List;
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 class GenericEnsemblePlacementPolicyTest extends BookKeeperClusterTestCas
public static final class CustomEnsemblePlacementPolicy extends DefaultEnsemblePlacementPolicy {
@Override
- public PlacementResult<BookieSocketAddress> replaceBookie(int ensembleSize, int writeQuorumSize,
- int ackQuorumSize, Map<String, byte[]> customMetadata, List<BookieSocketAddress> currentEnsemble,
- BookieSocketAddress bookieToReplace, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<BookieId> replaceBookie(int ensembleSize, int writeQuorumSize,
+ int ackQuorumSize, Map<String, byte[]> customMetadata, List<BookieId> currentEnsemble,
+ BookieId bookieToReplace, Set<BookieId> excludeBookies)
throws BKException.BKNotEnoughBookiesException {
new Exception("replaceBookie " + ensembleSize + "," + customMetadata).printStackTrace();
assertNotNull(customMetadata);
@@ -81,8 +81,8 @@ public class GenericEnsemblePlacementPolicyTest extends BookKeeperClusterTestCas
}
@Override
- public PlacementResult<List<BookieSocketAddress>> newEnsemble(int ensembleSize, int quorumSize,
- int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieSocketAddress> excludeBookies)
+ public PlacementResult<List<BookieId>> newEnsemble(int ensembleSize, int quorumSize,
+ int ackQuorumSize, Map<String, byte[]> customMetadata, Set<BookieId> excludeBookies)
throws BKException.BKNotEnoughBookiesException {
assertNotNull(customMetadata);
customMetadataOnNewEnsembleStack.add(customMetadata);
@@ -144,7 +144,7 @@ public class GenericEnsemblePlacementPolicyTest extends BookKeeperClusterTestCas
try (LedgerHandle lh = bk.createLedger(2, 2, 2, digestType, PASSWORD.getBytes(), customMetadata)) {
lh.addEntry(value);
long lId = lh.getId();
- List<BookieSocketAddress> ensembleAtFirstEntry = lh.getLedgerMetadata().getEnsembleAt(lId);
+ List<BookieId> 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 1b8220c..406fbd5 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 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.BookieId;
import org.apache.bookkeeper.net.BookieSocketAddress;
import org.apache.bookkeeper.versioning.Versioned;
@@ -51,11 +52,11 @@ import org.slf4j.LoggerFactory;
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 612b36e..95a5949 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 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.BookieId;
import org.apache.bookkeeper.net.BookieSocketAddress;
import org.apache.bookkeeper.versioning.Versioned;
@@ -38,11 +39,11 @@ import org.slf4j.LoggerFactory;
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 25f7f1a..5ece1a1 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.AsyncCallback.AddCallback;
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 @@ public class LedgerCloseTest extends BookKeeperClusterTestCase {
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 b993b4f..25a4fd8 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.Collections;
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<BookieSocketAddress> 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<BookieId> 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 class LedgerMetadataTest {
@Test
public void testToString() {
- List<BookieSocketAddress> 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<BookieId> 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 2e17836..4e2ca42 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 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.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<LedgerMetadata> setupLedger(ClientContext clientCtx, long ledgerId,
- List<BookieSocketAddress> bookies) throws Exception {
+ List<BookieId> 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 9a927b6..b89c370 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.AsyncCallback.AddCallback;
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 @@ public class LedgerRecoveryTest extends BookKeeperClusterTestCase {
// 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 class LedgerRecoveryTest extends BookKeeperClusterTestCase {
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 class LedgerRecoveryTest extends BookKeeperClusterTestCase {
// 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 class LedgerRecoveryTest extends BookKeeperClusterTestCase {
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 class LedgerRecoveryTest extends BookKeeperClusterTestCase {
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 e76ee54..1f43cb4 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.DigestType;
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 class MetadataUpdateLoopTest {
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<LedgerMetadata> writtenMetadata = lm.createLedgerMetadata(ledgerId, initMeta).get();
AtomicReference<Versioned<LedgerMetadata>> 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,
... 11085 lines suppressed ...