diff --git a/.gitignore b/.gitignore index 119d7fe7..8dd454b3 100644 --- a/.gitignore +++ b/.gitignore @@ -6,3 +6,5 @@ target/ build/ *.xml *.jar +.idea/ +dist/ 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 92241e6b..03e28411 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 @@ -23,6 +23,8 @@ import java.io.IOException; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.net.BookieIdToAddressMapping; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.zookeeper.KeeperException; @@ -56,7 +58,7 @@ static class LatencyCallback implements WriteCallback { boolean complete; @Override synchronized public void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { if (rc != 0) { LOG.error("Got error " + rc); } @@ -77,7 +79,7 @@ static class ThroughputCallback implements WriteCallback { int count; int waitingCount = Integer.MAX_VALUE; synchronized public void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { if (rc != 0) { LOG.error("Got error " + rc); } @@ -147,7 +149,12 @@ public static void main(String[] args) OrderedSafeExecutor executor = new OrderedSafeExecutor(1); ClientConfiguration conf = new ClientConfiguration(); - BookieClient bc = new BookieClient(conf, channelFactory, executor); + BookieClient bc = new BookieClient(conf, channelFactory, executor, new BookieIdToAddressMapping() { + @Override + public BookieSocketAddress getBookieAddress(BookieId bookieId) { + return new BookieSocketAddress(bookieId.getHostName(), bookieId.getPort(), bookieId.getPort()); + } + }); LatencyCallback lc = new LatencyCallback(); ThroughputCallback tc = new ThroughputCallback(); @@ -161,7 +168,7 @@ public static void main(String[] args) toSend.writeLong(ledger); toSend.writeLong(entry); toSend.writerIndex(toSend.capacity()); - bc.addEntry(new BookieSocketAddress(addr, port), ledger, new byte[20], + bc.addEntry(new BookieId(addr, port), ledger, new byte[20], entry, toSend, tc, null, BookieProtocol.FLAG_NONE); } LOG.info("Waiting for warmup"); @@ -179,7 +186,7 @@ public static void main(String[] args) toSend.writeLong(entry); toSend.writerIndex(toSend.capacity()); lc.resetComplete(); - bc.addEntry(new BookieSocketAddress(addr, port), ledger, new byte[20], + bc.addEntry(new BookieId(addr, port), ledger, new byte[20], entry, toSend, lc, null, BookieProtocol.FLAG_NONE); lc.waitForComplete(); } @@ -199,7 +206,7 @@ public static void main(String[] args) toSend.writeLong(ledger); toSend.writeLong(entry); toSend.writerIndex(toSend.capacity()); - bc.addEntry(new BookieSocketAddress(addr, port), ledger, new byte[20], + bc.addEntry(new BookieId(addr, port), ledger, new byte[20], entry, toSend, tc, null, BookieProtocol.FLAG_NONE); } tc.waitFor(entryCount); 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 d7ce2172..a92eb5f7 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 @@ -25,7 +25,7 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,7 +56,7 @@ public void testThroughputLatency() throws Exception { @Test public void testBookie() throws Exception { - BookieSocketAddress bookie = getBookie(0); + BookieId bookie = getBookie(0); BenchBookie.main(new String[] { "--host", bookie.getHostName(), "--port", String.valueOf(bookie.getPort()), diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index ccea948c..297375ca 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -307,6 +307,14 @@ + + org.apache.maven.plugins + maven-compiler-plugin + + 1.8 + 1.8 + + 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 3cefafb7..f5ff66ba 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 @@ -54,9 +54,11 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.ActiveLedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; +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; +import org.apache.bookkeeper.proto.DataFormats; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.OpStatsLogger; @@ -182,7 +184,7 @@ public long getEntry() { static class NopWriteCallback implements WriteCallback { @Override public void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { if (LOG.isDebugEnabled()) { LOG.debug("Finished writing entry {} @ ledger {} for {} : {}", new Object[] { entryId, ledgerId, addr, rc }); @@ -319,12 +321,25 @@ public static BookieSocketAddress getBookieAddress(ServerConfiguration conf) if (conf.getUseHostNameAsBookieID()) { hostAddress = inetAddr.getAddress().getCanonicalHostName(); } + int readPort; + if (conf.isAlternateBookiePortEnabled()) { + readPort = conf.getAlternateBookiePort(); + } else { + readPort = conf.getBookiePort(); + } BookieSocketAddress addr = - new BookieSocketAddress(hostAddress, conf.getBookiePort()); - if (addr.getSocketAddress().getAddress().isLoopbackAddress() + new BookieSocketAddress(hostAddress, conf.getBookiePort(), readPort); + if ((addr.getReadAddress().getAddress().isLoopbackAddress() + || addr.getWriteAddress().getAddress().isLoopbackAddress()) && !conf.getAllowLoopback()) { + InetSocketAddress loopedAddr; + if (addr.getReadAddress().getAddress().isLoopbackAddress()) { + loopedAddr = addr.getReadAddress(); + } else { + loopedAddr = addr.getWriteAddress(); + } throw new UnknownHostException("Trying to listen on loopback address, " - + addr + " but this is forbidden by default " + + loopedAddr + " but this is forbidden by default " + "(see ServerConfiguration#getAllowLoopback())"); } return addr; @@ -431,7 +446,7 @@ public int getExitCode() { } private String getMyId() throws UnknownHostException { - return Bookie.getBookieAddress(conf).toString(); + return Bookie.getBookieAddress(conf).getBookieId().toString(); } void readJournal() throws IOException, BookieException { @@ -765,7 +780,15 @@ private void doRegisterBookie(final String regPath) throws IOException { try{ if (!checkRegNodeAndWaitExpired(regPath)) { // Create the ZK ephemeral node for this Bookie. - zk.create(regPath, new byte[0], Ids.OPEN_ACL_UNSAFE, + + DataFormats.BookieFormat.Builder builder = DataFormats.BookieFormat.newBuilder() + .setWritePort(conf.getBookiePort()); + if (conf.isAlternateBookiePortEnabled()) { + builder.setReadPort(conf.getAlternateBookiePort()); + } else { + builder.setReadPort(conf.getBookiePort()); + } + zk.create(regPath, builder.build().toByteArray(), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL); LOG.info("Registered myself in ZooKeeper at {}.", regPath); } @@ -1240,7 +1263,7 @@ static class CounterCallback implements WriteCallback { int count; @Override - synchronized public void writeComplete(int rc, long l, long e, BookieSocketAddress addr, Object ctx) { + synchronized public 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 bbb611b5..be32cf0b 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 @@ -41,6 +41,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.util.EntryFormatter; @@ -218,7 +219,7 @@ int runCmd(CommandLine cmdLine) throws Exception { ZooKeeperClient.createConnectedZooKeeperClient(conf.getZkServers(), conf.getZkTimeout()); try { - Cookie.removeCookieForBookie(conf, zkc, address); + Cookie.removeCookieForBookie(conf, zkc, address.getBookieId()); } catch (KeeperException.NoNodeException nne) { // ignore no node exception LOG.warn("No cookie to remove for {} : ", address, nne); @@ -285,7 +286,7 @@ int runCmd(CommandLine cmdLine) throws Exception { // Get bookies list final String[] bookieStrs = args[0].split(","); - final Set bookieAddrs = new HashSet(); + final Set bookieAddrs = new HashSet(); for (String bookieStr : bookieStrs) { final String bookieStrParts[] = bookieStr.split(":"); if (bookieStrParts.length != 2) { @@ -293,7 +294,7 @@ int runCmd(CommandLine cmdLine) throws Exception { + bookieStr); return -1; } - bookieAddrs.add(new BookieSocketAddress(bookieStrParts[0], + bookieAddrs.add(new BookieId(bookieStrParts[0], Integer.parseInt(bookieStrParts[1]))); } @@ -320,7 +321,7 @@ int runCmd(CommandLine cmdLine) throws Exception { } } - private int bkQuery(BookKeeperAdmin bkAdmin, Set bookieAddrs) + private int bkQuery(BookKeeperAdmin bkAdmin, Set bookieAddrs) throws InterruptedException, BKException { SortedMap ledgersContainBookies = bkAdmin.getLedgersContainBookies(bookieAddrs); @@ -340,13 +341,13 @@ private int bkQuery(BookKeeperAdmin bkAdmin, Set bookieAddr return 0; } - private Map inspectLedger(LedgerMetadata metadata, Set bookiesToInspect) { + private Map inspectLedger(LedgerMetadata metadata, Set bookiesToInspect) { Map numBookiesToReplacePerEnsemble = new TreeMap(); - for (Map.Entry> ensemble : metadata.getEnsembles().entrySet()) { - ArrayList bookieList = ensemble.getValue(); + for (Map.Entry> ensemble : metadata.getEnsembles().entrySet()) { + ArrayList 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("*"); @@ -362,24 +363,24 @@ private Map inspectLedger(LedgerMetadata metadata, Set bookieAddrs, + private int bkRecovery(BookKeeperAdmin bkAdmin, long lid, Set bookieAddrs, boolean dryrun, boolean skipOpenLedgers, boolean removeCookies) throws InterruptedException, BKException, KeeperException { bkAdmin.recoverBookieData(lid, bookieAddrs, dryrun, skipOpenLedgers); if (removeCookies) { - for (BookieSocketAddress addr : bookieAddrs) { + for (BookieId addr : bookieAddrs) { Cookie.removeCookieForBookie(bkConf, bkAdmin.getZooKeeper(), addr); } } return 0; } - private int bkRecovery(BookKeeperAdmin bkAdmin, Set bookieAddrs, + private int bkRecovery(BookKeeperAdmin bkAdmin, Set bookieAddrs, boolean dryrun, boolean skipOpenLedgers, boolean removeCookies) throws InterruptedException, BKException, KeeperException { bkAdmin.recoverBookieData(bookieAddrs, dryrun, skipOpenLedgers); if (removeCookies) { - for (BookieSocketAddress addr : bookieAddrs) { + for (BookieId addr : bookieAddrs) { Cookie.removeCookieForBookie(bkConf, bkAdmin.getZooKeeper(), addr); } } 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 12f0aaa0..d1320a11 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 @@ -34,7 +34,7 @@ import java.io.StringReader; import java.net.UnknownHostException; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.KeeperException; @@ -208,7 +208,7 @@ static Cookie generateCookie(ServerConfiguration conf) throws UnknownHostException { Cookie c = new Cookie(); c.layoutVersion = CURRENT_COOKIE_LAYOUT_VERSION; - c.bookieHost = Bookie.getBookieAddress(conf).toString(); + c.bookieHost = Bookie.getBookieAddress(conf).getBookieId().toString(); c.journalDir = conf.getJournalDirName(); StringBuilder b = new StringBuilder(); String[] dirs = conf.getLedgerDirNames(); @@ -253,16 +253,16 @@ public void setInstanceId(String instanceId) { private static String getZkPath(ServerConfiguration conf) throws UnknownHostException { - return getZkPath(conf, Bookie.getBookieAddress(conf)); + return getZkPath(conf, Bookie.getBookieAddress(conf).getBookieId()); } - private static String getZkPath(ServerConfiguration conf, BookieSocketAddress address) { + private static String getZkPath(ServerConfiguration conf, BookieId address) { String bookieCookiePath = conf.getZkLedgersRootPath() + "/" + COOKIE_NODE; return bookieCookiePath + "/" + address; } public static void removeCookieForBookie(ServerConfiguration conf, ZooKeeper zk, - BookieSocketAddress address) + BookieId address) throws KeeperException, InterruptedException { String zkPath = getZkPath(conf, address); zk.delete(zkPath, -1); 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 c7847454..99aa8e55 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 @@ -375,10 +375,9 @@ private BookKeeper(ClientConfiguration conf, ZooKeeper zk, ClientSocketChannelFa .traceTaskWarnTimeMicroSec(conf.getTaskExecutionWarnTimeMicros()) .build(); - - bookieClient = new BookieClient(conf, this.channelFactory, mainWorkerPool, statsLogger, requestTimer, Optional.fromNullable(dnsResolver)); bookieWatcher = new BookieWatcher(conf, scheduler, placementPolicy, this); bookieWatcher.readBookiesBlocking(); + bookieClient = new BookieClient(conf, this.channelFactory, mainWorkerPool, statsLogger, requestTimer, Optional.fromNullable(dnsResolver), bookieWatcher); ledgerManagerFactory = LedgerManagerFactory.newLedgerManagerFactory(conf, this.zk); ledgerManager = TimedLedgerManager.of(new CleanupLedgerManager(ledgerManagerFactory.newLedgerManager()), 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 b13ae98b..fbd2938a 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 @@ -43,7 +43,7 @@ import org.apache.bookkeeper.client.BookKeeper.SyncOpenCallback; import org.apache.bookkeeper.client.LedgerFragmentReplicator.SingleFragmentCallback; 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.proto.BookkeeperInternalCallbacks.MultiCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; @@ -178,16 +178,16 @@ public void close() throws InterruptedException, BKException { * * @return the registered bookie list. */ - public Collection getRegisteredBookies() + public Collection getRegisteredBookies() throws BKException { String cookiePath = bkc.getConf().getZkLedgersRootPath() + "/" + BookKeeperConstants.COOKIE_NODE; try { List children = zk.getChildren(cookiePath, false); - List bookies = new ArrayList(children.size()); + List bookies = new ArrayList(children.size()); for (String child : children) { try { - bookies.add(new BookieSocketAddress(child)); + bookies.add(new BookieId(child)); } catch (IOException ioe) { LOG.error("Error parsing bookie address {} : ", child, ioe); throw new BKException.ZKException(); @@ -311,7 +311,7 @@ public SyncObject() { } } - public SortedMap getLedgersContainBookies(Set bookies) + public SortedMap getLedgersContainBookies(Set bookies) throws InterruptedException, BKException { final SyncObject sync = new SyncObject(); final AtomicReference> resultHolder = @@ -339,7 +339,7 @@ public void operationComplete(int rc, SortedMap result) { return resultHolder.get(); } - public void asyncGetLedgersContainBookies(final Set bookies, + public void asyncGetLedgersContainBookies(final Set bookies, final GenericCallback> callback) { final SortedMap ledgers = new ConcurrentSkipListMap(); bkc.getLedgerManager().asyncProcessLedgers(new Processor() { @@ -356,8 +356,8 @@ public void operationComplete(int rc, LedgerMetadata metadata) { cb.processResult(rc, null, null); return; } - Set bookiesInLedger = metadata.getBookiesInThisLedger(); - Sets.SetView intersection = + Set bookiesInLedger = metadata.getBookiesInThisLedger(); + Sets.SetView intersection = Sets.intersection(bookiesInLedger, bookies); if (!intersection.isEmpty()) { ledgers.put(lid, metadata); @@ -392,19 +392,19 @@ public void processResult(int rc, String path, Object ctx) { * Optional destination bookie that if passed, we will copy all * of the ledger fragments from the source bookie over to it. */ - public void recoverBookieData(final BookieSocketAddress bookieSrc, final BookieSocketAddress bookieDest) + public void recoverBookieData(final BookieId bookieSrc, final BookieId bookieDest) throws InterruptedException, BKException { - Set bookiesSrc = new HashSet(); + Set bookiesSrc = new HashSet(); bookiesSrc.add(bookieSrc); recoverBookieData(bookiesSrc); } - public void recoverBookieData(final Set bookiesSrc) + public void recoverBookieData(final Set bookiesSrc) throws InterruptedException, BKException { recoverBookieData(bookiesSrc, false, false); } - public void recoverBookieData(final Set bookiesSrc, boolean dryrun, boolean skipOpenLedgers) + public void recoverBookieData(final Set bookiesSrc, boolean dryrun, boolean skipOpenLedgers) throws InterruptedException, BKException { SyncObject sync = new SyncObject(); // Call the async method to recover bookie data. @@ -432,7 +432,7 @@ public void recoverComplete(int rc, Object ctx) { } } - public void recoverBookieData(final long lid, final Set bookiesSrc, boolean dryrun, boolean skipOpenLedgers) + public void recoverBookieData(final long lid, final Set bookiesSrc, boolean dryrun, boolean skipOpenLedgers) throws InterruptedException, BKException { SyncObject sync = new SyncObject(); // Call the async method to recover bookie data. @@ -483,19 +483,19 @@ public void recoverComplete(int rc, Object ctx) { * @param context * Context for the RecoverCallback to call. */ - public void asyncRecoverBookieData(final BookieSocketAddress bookieSrc, final BookieSocketAddress bookieDest, + public void asyncRecoverBookieData(final BookieId bookieSrc, final BookieId bookieDest, final RecoverCallback cb, final Object context) { - Set bookiesSrc = new HashSet(); + Set bookiesSrc = new HashSet(); bookiesSrc.add(bookieSrc); asyncRecoverBookieData(bookiesSrc, cb, context); } - public void asyncRecoverBookieData(final Set bookieSrc, + public void asyncRecoverBookieData(final Set bookieSrc, final RecoverCallback cb, final Object context) { asyncRecoverBookieData(bookieSrc, false, false, cb, context); } - public void asyncRecoverBookieData(final Set bookieSrc, boolean dryrun, + public void asyncRecoverBookieData(final Set bookieSrc, boolean dryrun, final boolean skipOpenLedgers, final RecoverCallback cb, final Object context) { getActiveLedgers(bookieSrc, dryrun, skipOpenLedgers, cb, context); } @@ -517,7 +517,7 @@ public void asyncRecoverBookieData(final Set bookieSrc, boo * @param context * Context for the RecoverCallback to call. */ - public void asyncRecoverBookieData(long lid, final Set bookieSrc, boolean dryrun, + public void asyncRecoverBookieData(long lid, final Set bookieSrc, boolean dryrun, boolean skipOpenLedgers, final RecoverCallback callback, final Object context) { AsyncCallback.VoidCallback callbackWrapper = new AsyncCallback.VoidCallback() { @Override @@ -547,7 +547,7 @@ public void processResult(int rc, String path, Object ctx) { * @param context * Context for the RecoverCallback to call. */ - private void getActiveLedgers(final Set bookiesSrc, final boolean dryrun, + private void getActiveLedgers(final Set bookiesSrc, final boolean dryrun, final boolean skipOpenLedgers, final RecoverCallback cb, final Object context) { // Wrapper class around the RecoverCallback so it can be used // as the final VoidCallback to process ledgers @@ -592,7 +592,7 @@ ledgerProcessor, new RecoverCallbackWrapper(cb), * IterationCallback to invoke once we've recovered the current * ledger. */ - private void recoverLedger(final Set bookiesSrc, final long lId, final boolean dryrun, + private void recoverLedger(final Set bookiesSrc, final long lId, final boolean dryrun, final boolean skipOpenLedgers, final AsyncCallback.VoidCallback finalLedgerIterCb) { LOG.debug("Recovering ledger : {}", lId); @@ -623,7 +623,7 @@ public void openComplete(int rc, final LedgerHandle lh, Object ctx) { if (!lm.isClosed() && lm.getEnsembles().size() > 0) { Long lastKey = lm.getEnsembles().lastKey(); - ArrayList lastEnsemble = lm.getEnsembles().get(lastKey); + ArrayList lastEnsemble = lm.getEnsembles().get(lastKey); // the original write has not removed faulty bookie from // current ledger ensemble. to avoid data loss issue in // the case of concurrent updates to the ensemble composition, @@ -692,7 +692,7 @@ public void processResult(int rc, String path, Object ctx) { */ Map ledgerFragmentsRange = new HashMap(); Long curEntryId = null; - for (Map.Entry> entry : lh.getLedgerMetadata().getEnsembles() + for (Map.Entry> entry : lh.getLedgerMetadata().getEnsembles() .entrySet()) { if (curEntryId != null) ledgerFragmentsRange.put(curEntryId, entry.getKey() - 1); @@ -738,9 +738,9 @@ public void processResult(int rc, String path, Object ctx) { */ for (final Long startEntryId : ledgerFragmentsToRecover) { Long endEntryId = ledgerFragmentsRange.get(startEntryId); - ArrayList ensemble = lh.getLedgerMetadata().getEnsembles().get(startEntryId); + ArrayList ensemble = lh.getLedgerMetadata().getEnsembles().get(startEntryId); // Get bookies to replace - Map targetBookieAddresses; + Map targetBookieAddresses; try { targetBookieAddresses = getReplacedBookies(lh, ensemble, bookiesSrc); } catch (BKException.BKNotEnoughBookiesException e) { @@ -754,7 +754,7 @@ public void processResult(int rc, String path, Object ctx) { } if (dryrun) { - ArrayList newEnsemble = + ArrayList newEnsemble = replaceBookiesInEnsemble(ensemble, targetBookieAddresses); System.out.println(" Fragment [" + startEntryId + " - " + endEntryId + " ] : "); System.out.println(" old ensemble : " + formatEnsemble(ensemble, bookiesSrc, '*')); @@ -785,7 +785,7 @@ public void processResult(int rc, String path, Object ctx) { }, null); } - static String formatEnsemble(ArrayList ensemble, Set bookiesSrc, char marker) { + static String formatEnsemble(ArrayList ensemble, Set bookiesSrc, char marker) { StringBuilder sb = new StringBuilder(); sb.append("["); for (int i = 0; i < ensemble.size(); i++) { @@ -822,18 +822,18 @@ static String formatEnsemble(ArrayList ensemble, Set newBookies) throws InterruptedException { + final Set newBookies) throws InterruptedException { lfr.replicate(lh, ledgerFragment, ledgerFragmentMcb, newBookies); } - private Map getReplacedBookies( + private Map getReplacedBookies( LedgerHandle lh, - List ensemble, - Set bookiesToRereplicate) + List ensemble, + Set bookiesToRereplicate) throws BKException.BKNotEnoughBookiesException { Set bookieIndexesToRereplicate = Sets.newHashSet(); for (int bookieIndex = 0; bookieIndex < ensemble.size(); bookieIndex++) { - BookieSocketAddress bookieInEnsemble = ensemble.get(bookieIndex); + BookieId bookieInEnsemble = ensemble.get(bookieIndex); if (bookiesToRereplicate.contains(bookieInEnsemble)) { bookieIndexesToRereplicate.add(bookieIndex); } @@ -842,31 +842,31 @@ private Map getReplacedBookies( lh, ensemble, bookieIndexesToRereplicate, Optional.of(bookiesToRereplicate)); } - private Map getReplacedBookiesByIndexes( + private Map getReplacedBookiesByIndexes( LedgerHandle lh, - List ensemble, + List ensemble, Set bookieIndexesToRereplicate, - Optional> excludedBookies) + Optional> excludedBookies) throws BKException.BKNotEnoughBookiesException { // target bookies to replicate - Map targetBookieAddresses = + Map targetBookieAddresses = Maps.newHashMapWithExpectedSize(bookieIndexesToRereplicate.size()); // bookies to exclude for ensemble allocation - Set bookiesToExclude = Sets.newHashSet(); + Set bookiesToExclude = Sets.newHashSet(); if (excludedBookies.isPresent()) { bookiesToExclude.addAll(excludedBookies.get()); } // excluding bookies that need to be replicated for (Integer bookieIndex : bookieIndexesToRereplicate) { - BookieSocketAddress bookie = ensemble.get(bookieIndex); + BookieId bookie = ensemble.get(bookieIndex); bookiesToExclude.add(bookie); } // allocate bookies for (Integer bookieIndex : bookieIndexesToRereplicate) { - BookieSocketAddress oldBookie = ensemble.get(bookieIndex); - BookieSocketAddress newBookie = + BookieId oldBookie = ensemble.get(bookieIndex); + BookieId newBookie = bkc.getPlacementPolicy().replaceBookie( lh.getLedgerMetadata().getEnsembleSize(), lh.getLedgerMetadata().getWriteQuorumSize(), @@ -881,11 +881,11 @@ private Map getReplacedBookiesByIndexes( return targetBookieAddresses; } - private ArrayList replaceBookiesInEnsemble( - List ensemble, - Map replacedBookies) { - ArrayList newEnsemble = Lists.newArrayList(ensemble); - for (Map.Entry entry : replacedBookies.entrySet()) { + private ArrayList replaceBookiesInEnsemble( + List ensemble, + Map replacedBookies) { + ArrayList newEnsemble = Lists.newArrayList(ensemble); + for (Map.Entry entry : replacedBookies.entrySet()) { newEnsemble.set(entry.getKey(), entry.getValue()); } return newEnsemble; @@ -902,8 +902,8 @@ private ArrayList replaceBookiesInEnsemble( public void replicateLedgerFragment(LedgerHandle lh, final LedgerFragment ledgerFragment) throws InterruptedException, BKException { - Optional> excludedBookies = Optional.absent(); - Map targetBookieAddresses = + Optional> excludedBookies = Optional.absent(); + Map targetBookieAddresses = getReplacedBookiesByIndexes(lh, ledgerFragment.getEnsemble(), ledgerFragment.getBookiesIndexes(), excludedBookies); replicateLedgerFragment(lh, ledgerFragment, targetBookieAddresses); @@ -911,14 +911,14 @@ public void replicateLedgerFragment(LedgerHandle lh, private void replicateLedgerFragment(LedgerHandle lh, final LedgerFragment ledgerFragment, - final Map targetBookieAddresses) + final Map targetBookieAddresses) throws InterruptedException, BKException { SyncCounter syncCounter = new SyncCounter(); ResultCallBack resultCallBack = new ResultCallBack(syncCounter); SingleFragmentCallback cb = new SingleFragmentCallback(resultCallBack, lh, ledgerFragment.getFirstEntryId(), getReplacedBookiesMap(ledgerFragment, targetBookieAddresses)); syncCounter.inc(); - Set targetBookieSet = new HashSet(); + Set targetBookieSet = new HashSet(); targetBookieSet.addAll(targetBookieAddresses.values()); asyncRecoverLedgerFragment(lh, ledgerFragment, cb, targetBookieSet); syncCounter.block(0); @@ -927,34 +927,34 @@ private void replicateLedgerFragment(LedgerHandle lh, } } - private static Map getReplacedBookiesMap( - ArrayList ensemble, - Map targetBookieAddresses) { - Map bookiesMap = - new HashMap(); - for (Map.Entry entry : targetBookieAddresses.entrySet()) { - BookieSocketAddress oldBookie = ensemble.get(entry.getKey()); - BookieSocketAddress newBookie = entry.getValue(); + private static Map getReplacedBookiesMap( + ArrayList ensemble, + Map targetBookieAddresses) { + Map bookiesMap = + new HashMap(); + for (Map.Entry entry : targetBookieAddresses.entrySet()) { + BookieId oldBookie = ensemble.get(entry.getKey()); + BookieId newBookie = entry.getValue(); bookiesMap.put(oldBookie, newBookie); } return bookiesMap; } - private static Map getReplacedBookiesMap( + private static Map getReplacedBookiesMap( LedgerFragment ledgerFragment, - Map targetBookieAddresses) { - Map bookiesMap = - new HashMap(); + Map targetBookieAddresses) { + Map bookiesMap = + new HashMap(); for (Integer bookieIndex : ledgerFragment.getBookiesIndexes()) { - BookieSocketAddress oldBookie = ledgerFragment.getAddress(bookieIndex); - BookieSocketAddress newBookie = targetBookieAddresses.get(bookieIndex); + BookieId oldBookie = ledgerFragment.getAddress(bookieIndex); + BookieId newBookie = targetBookieAddresses.get(bookieIndex); bookiesMap.put(oldBookie, newBookie); } return bookiesMap; } - private static boolean containBookies(ArrayList ensemble, Set bookies) { - for (BookieSocketAddress bookie : ensemble) { + private static boolean containBookies(ArrayList ensemble, Set bookies) { + for (BookieId bookie : ensemble) { if (bookies.contains(bookie)) { return true; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieClusterManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieClusterManager.java index 9c45841c..5679f9cb 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieClusterManager.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieClusterManager.java @@ -21,7 +21,7 @@ package org.apache.bookkeeper.client; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; @@ -74,15 +74,15 @@ long getLastTimestamp() { protected final ServerConfiguration conf; protected final BookKeeper bkc; protected final long staleBookieIntervalInMs; - protected final Map bookieStatuses = - new HashMap(); + protected final Map bookieStatuses = + new HashMap(); protected AtomicBoolean isStarted = new AtomicBoolean(); - protected Set registeredBookies = new HashSet(); - protected Set availableBookies = new HashSet(); - protected Set readOnlyBookies = new HashSet(); - protected Set staleBookies = new HashSet(); - protected Set activeBookies = new HashSet(); - protected Set lostBookies = new HashSet(); + protected Set registeredBookies = new HashSet(); + protected Set availableBookies = new HashSet(); + protected Set readOnlyBookies = new HashSet(); + protected Set staleBookies = new HashSet(); + protected Set activeBookies = new HashSet(); + protected Set lostBookies = new HashSet(); // stats for bookie cluster private StatsLogger statsLogger; @@ -119,17 +119,17 @@ public void start() throws BKException { * So if you keep calling this function, ideally there should be fewer stale bookies fetched each time. * @throws BKException */ - public Set fetchStaleBookies() throws BKException { + public Set fetchStaleBookies() throws BKException { updateBookiesStatuses(this.availableBookies); updateBookiesStatuses(this.readOnlyBookies); - Set staleBookies = new HashSet<>(); - Set activeBookies = new HashSet<>(); + Set staleBookies = new HashSet<>(); + Set activeBookies = new HashSet<>(); long now = MathUtils.now(); synchronized (this) { - Iterator> iter = + Iterator> iter = bookieStatuses.entrySet().iterator(); while (iter.hasNext()) { - Map.Entry entry = iter.next(); + Map.Entry entry = iter.next(); long millisSinceLastSeen = now - entry.getValue().getLastTimestamp(); if (millisSinceLastSeen > staleBookieIntervalInMs) { logger.info("Bookie {} (seen @ {}) become stale for {} ms, remove it.", @@ -158,7 +158,7 @@ public void close() { * @throws BKException */ private void fetchRegisteredBookies() throws BKException { - Collection registeredBookies = this.bkc.bookieWatcher.getRegisteredBookies(); + Collection registeredBookies = this.bkc.bookieWatcher.getRegisteredBookies(); logger.info("Fetch all registered bookies : {}", registeredBookies); updateBookiesStatuses(registeredBookies); updateBookies(this.registeredBookies, registeredBookies); @@ -169,7 +169,7 @@ private void fetchRegisteredBookies() throws BKException { * @throws BKException */ private void fetchAvailableBookies() throws BKException { - Collection availableBookies = this.bkc.bookieWatcher.getAvailableBookies(); + Collection availableBookies = this.bkc.bookieWatcher.getAvailableBookies(); logger.info("Fetch all available bookies: {}", availableBookies); updateBookiesStatuses(availableBookies); updateBookies(this.availableBookies, availableBookies); @@ -180,7 +180,7 @@ private void fetchAvailableBookies() throws BKException { * @throws BKException */ private void fetchReadOnlyBookies() throws BKException { - Collection readOnlyBookies = this.bkc.bookieWatcher.getReadOnlyBookies(); + Collection readOnlyBookies = this.bkc.bookieWatcher.getReadOnlyBookies(); logger.info("Fetch all readonly bookies: {}", readOnlyBookies); updateBookiesStatuses(readOnlyBookies); updateBookies(this.readOnlyBookies, readOnlyBookies); @@ -192,8 +192,8 @@ private void fetchReadOnlyBookies() throws BKException { * * @param bookies bookies to update the lastUpdateTime */ - private synchronized void updateBookiesStatuses(Collection bookies) { - for (BookieSocketAddress bookie : bookies) { + private synchronized void updateBookiesStatuses(Collection bookies) { + for (BookieId bookie : bookies) { UpdateStatus bs = bookieStatuses.get(bookie); if (null == bs) { bs = new UpdateStatus(); @@ -210,38 +210,38 @@ private synchronized void updateBookiesStatuses(Collection * @param oldBookies old bookies to be updated * @param newBookies new bookies */ - private synchronized void updateBookies(Collection oldBookies, - Collection newBookies) { + private synchronized void updateBookies(Collection oldBookies, + Collection newBookies) { oldBookies.clear(); oldBookies.addAll(newBookies); } @Override - public void availableBookiesChanged(Set bookies) { + public void availableBookiesChanged(Set bookies) { updateBookiesStatuses(bookies); updateBookies(this.availableBookies, bookies); } @Override - public void readOnlyBookiesChanged(Set bookies) { + public void readOnlyBookiesChanged(Set bookies) { updateBookiesStatuses(bookies); updateBookies(this.readOnlyBookies, bookies); } - public void lostBookiesChanged(Set bookies) { + public void lostBookiesChanged(Set bookies) { updateBookies(this.lostBookies, bookies); } - public Set getAvailableBookies() { + public Set getAvailableBookies() { return availableBookies; } - public Set getReadOnlyBookies() { + public Set getReadOnlyBookies() { return readOnlyBookies; } - public Set getActiveBookies() { + public Set getActiveBookies() { return activeBookies; } 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 445b25e7..4325be0f 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 @@ -18,21 +18,14 @@ * limitations under the License. */ -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.CopyOnWriteArraySet; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; +import com.google.protobuf.InvalidProtocolBufferException; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.net.BookieIdToAddressMapping; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.DataFormats; import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.bookkeeper.util.SafeRunnable; import org.apache.bookkeeper.util.ZkUtils; @@ -44,10 +37,24 @@ import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.bookkeeper.util.BookKeeperConstants.*; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; /** * This class is responsible for maintaining a consistent view of what bookies @@ -56,11 +63,11 @@ * replacement * */ -class BookieWatcher implements Watcher, ChildrenCallback { +public class BookieWatcher implements Watcher, ChildrenCallback, BookieIdToAddressMapping{ static final Logger logger = LoggerFactory.getLogger(BookieWatcher.class); public static int ZK_CONNECT_BACKOFF_SEC = 1; - private static final Set EMPTY_SET = new HashSet(); + private static final Set EMPTY_SET = new HashSet(); // Bookie registration path in ZK private final String bookieRegistrationPath; @@ -78,6 +85,8 @@ public void safeRun() { } }; private final ReadOnlyBookieWatcher readOnlyBookieWatcher; + private final ConcurrentHashMap addressMapping = + new ConcurrentHashMap(); BookieWatcher(ClientConfiguration conf, ScheduledExecutorService scheduler, @@ -92,6 +101,15 @@ public void safeRun() { readOnlyBookieWatcher = new ReadOnlyBookieWatcher(conf, bk); } + @Override + public BookieSocketAddress getBookieAddress(BookieId bookieId) { + if (addressMapping.containsKey(bookieId)) { + return addressMapping.get(bookieId); + } else { + return bookieId.asBookieSocketAddress(); + } + } + void registerBookiesListener(final BookiesListener listener) { listeners.add(listener); readOnlyBookieWatcher.registerBookiesListener(listener); @@ -107,15 +125,15 @@ void unregisterBookiesListener(final BookiesListener listener) { * * @return the registered bookie list. */ - Collection getRegisteredBookies() throws BKException { + Collection getRegisteredBookies() throws BKException { String cookiePath = bk.getConf().getZkLedgersRootPath() + "/" + BookKeeperConstants.COOKIE_NODE; try { List children = bk.getZkHandle().getChildren(cookiePath, false); - List bookies = new ArrayList(children.size()); + List bookies = new ArrayList(children.size()); for (String child : children) { try { - bookies.add(new BookieSocketAddress(child)); + bookies.add(new BookieId(child)); } catch (IOException ioe) { logger.error("Error parsing bookie address {} : ", child, ioe); throw new BKException.ZKException(); @@ -132,11 +150,11 @@ Collection getRegisteredBookies() throws BKException { } } - Collection getAvailableBookies() throws BKException { + Collection getAvailableBookies() throws BKException { try { List children = bk.getZkHandle().getChildren(this.bookieRegistrationPath, false); children.remove(BookKeeperConstants.READONLY); - return convertToBookieAddresses(children); + return convertToBookieIds(children); } catch (KeeperException ke) { if (ZkUtils.isRecoverableException(ke)) { logger.info("Encountered recoverable zookeeper exception on getting bookie list : code = {}", ke.code()); @@ -152,7 +170,7 @@ Collection getAvailableBookies() throws BKException { } } - Collection getReadOnlyBookies() throws BKException { + Collection getReadOnlyBookies() throws BKException { try { readOnlyBookieWatcher.readROBookiesBlocking(); } catch (KeeperException ke) { @@ -169,7 +187,7 @@ Collection getReadOnlyBookies() throws BKException { logger.error("Interrupted reading bookie list : ", ie); throw new BKException.BKInterruptedException(); } - return new HashSet(readOnlyBookieWatcher.getReadOnlyBookies()); + return new HashSet(readOnlyBookieWatcher.getReadOnlyBookies()); } private void readBookies() { @@ -208,10 +226,34 @@ public void processResult(int rc, String path, Object ctx, List children // available nodes list. children.remove(READONLY); - HashSet newBookieAddrs = convertToBookieAddresses(children); + HashSet newBookieAddrs = convertToBookieIds(children); + for (final BookieId bookieId: newBookieAddrs) { + if (!addressMapping.containsKey(bookieId)) { + bk.getZkHandle().getData(bookieRegistrationPath + "/" + bookieId.toString(), + false, new DataCallback() { + @Override + public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) { + if (data != null && data.length > 0) { + try { + DataFormats.BookieFormat bookieFormat = DataFormats.BookieFormat.parseFrom(data); + BookieSocketAddress socketAddress = new BookieSocketAddress( + bookieId.getHostName(), bookieFormat.getWritePort(), + bookieFormat.getReadPort()); + addressMapping.put(bookieId, socketAddress); + } catch (InvalidProtocolBufferException e) { + logger.warn("Invalid bookie format from zk node"); + addressMapping.put(bookieId, bookieId.asBookieSocketAddress()); + } + } else { + addressMapping.put(bookieId, bookieId.asBookieSocketAddress()); + } + } + }, this); + } + } synchronized (this) { - Set readonlyBookies = readOnlyBookieWatcher.getReadOnlyBookies(); + Set readonlyBookies = readOnlyBookieWatcher.getReadOnlyBookies(); placementPolicy.onClusterChanged(newBookieAddrs, readonlyBookies); } @@ -235,21 +277,19 @@ public void processResult(int rc, String path, Object ctx, List children // } } - private static HashSet convertToBookieAddresses(List children) { + private static HashSet convertToBookieIds(List children) { // Read the bookie addresses into a set for efficient lookup - HashSet newBookieAddrs = new HashSet(); + HashSet newBookieIds = new HashSet(); for (String bookieAddrString : children) { - BookieSocketAddress bookieAddr; try { - bookieAddr = new BookieSocketAddress(bookieAddrString); + newBookieIds.add(new BookieId(bookieAddrString)); } catch (IOException e) { logger.info("Could not parse bookie address: {}, ignoring this bookie : ", - bookieAddrString, e); + bookieAddrString, e); continue; } - newBookieAddrs.add(bookieAddr); } - return newBookieAddrs; + return newBookieIds; } /** @@ -292,23 +332,23 @@ public void processResult(int rc, String path, Object ctx, List children * @return list of bookies for new ensemble. * @throws BKNotEnoughBookiesException */ - public ArrayList newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize) + public ArrayList newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize) throws BKNotEnoughBookiesException { return placementPolicy.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, EMPTY_SET); } /** * Wrapper over the {@link org.apache.bookkeeper.client.EnsemblePlacementPolicy#replaceBookie(int, int, int, - * java.util.Collection, org.apache.bookkeeper.net.BookieSocketAddress, java.util.Set)} method when you just need 1 extra bookie + * java.util.Collection, BookieId, java.util.Set)} method when you just need 1 extra bookie * @param existingBookies * @return replaced bookie * @throws BKNotEnoughBookiesException */ - public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - List existingBookies, - int bookieIdx, Set excludeBookies) + public BookieId replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + List existingBookies, + int bookieIdx, Set excludeBookies) throws BKNotEnoughBookiesException { - BookieSocketAddress addr = existingBookies.get(bookieIdx); + BookieId addr = existingBookies.get(bookieIdx); return placementPolicy.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, existingBookies, addr, excludeBookies); } @@ -319,7 +359,7 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, private static class ReadOnlyBookieWatcher implements Watcher, ChildrenCallback { private final static Logger LOG = LoggerFactory.getLogger(ReadOnlyBookieWatcher.class); - private HashSet readOnlyBookies = new HashSet(); + private HashSet readOnlyBookies = new HashSet(); private final BookKeeper bk; private final String readOnlyBookieRegPath; @@ -398,7 +438,7 @@ public void processResult(int rc, String path, Object ctx, List children return; } - HashSet newReadOnlyBookies = convertToBookieAddresses(children); + HashSet newReadOnlyBookies = convertToBookieIds(children); readOnlyBookies = newReadOnlyBookies; for (BookiesListener listener : listeners) { @@ -407,7 +447,7 @@ public void processResult(int rc, String path, Object ctx, List children } // returns the readonly bookies - public HashSet getReadOnlyBookies() { + public HashSet getReadOnlyBookies() { return readOnlyBookies; } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookiesListener.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookiesListener.java index f8ec598e..f72457f5 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookiesListener.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookiesListener.java @@ -19,7 +19,7 @@ */ -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import java.util.Set; @@ -30,10 +30,10 @@ public interface BookiesListener { /** * Available bookies changed */ - void availableBookiesChanged(Set bookies); + void availableBookiesChanged(Set bookies); /** * ReadOnly bookies changed */ - void readOnlyBookiesChanged(Set bookies); + void readOnlyBookiesChanged(Set bookies); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultEnsemblePlacementPolicy.java index 681e3a57..b96f51c8 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 @@ -30,7 +30,7 @@ import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; 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.stats.AlertStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; @@ -41,23 +41,23 @@ */ public class DefaultEnsemblePlacementPolicy implements EnsemblePlacementPolicy { - static final Set EMPTY_SET = new HashSet(); + static final Set EMPTY_SET = new HashSet(); - private Set knownBookies = new HashSet(); + private Set knownBookies = new HashSet(); @Override - public ArrayList newEnsemble(int ensembleSize, int quorumSize, int ackQuorumSize, - Set excludeBookies) throws BKNotEnoughBookiesException { - ArrayList newBookies = new ArrayList(ensembleSize); + public ArrayList newEnsemble(int ensembleSize, int quorumSize, int ackQuorumSize, + Set excludeBookies) throws BKNotEnoughBookiesException { + ArrayList newBookies = new ArrayList(ensembleSize); if (ensembleSize <= 0) { return newBookies; } - List allBookies; + List allBookies; synchronized (this) { - allBookies = new ArrayList(knownBookies); + allBookies = new ArrayList(knownBookies); } Collections.shuffle(allBookies); - for (BookieSocketAddress bookie : allBookies) { + for (BookieId bookie : allBookies) { if (excludeBookies.contains(bookie)) { continue; } @@ -71,19 +71,19 @@ public ArrayList newEnsemble(int ensembleSize, int quorumSi } @Override - public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Collection currentEnsemble, - BookieSocketAddress bookieToReplace, - Set excludeBookies) throws BKNotEnoughBookiesException { + public BookieId replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Collection currentEnsemble, + BookieId bookieToReplace, + Set excludeBookies) throws BKNotEnoughBookiesException { excludeBookies.addAll(currentEnsemble); - ArrayList addresses = newEnsemble(1, 1, 1, excludeBookies); + ArrayList addresses = newEnsemble(1, 1, 1, excludeBookies); return addresses.get(0); } @Override - public synchronized Set onClusterChanged(Set writableBookies, - Set readOnlyBookies) { - HashSet deadBookies; - deadBookies = new HashSet(knownBookies); + public synchronized Set onClusterChanged(Set writableBookies, + Set readOnlyBookies) { + HashSet deadBookies; + deadBookies = new HashSet(knownBookies); deadBookies.removeAll(writableBookies); // readonly bookies should not be treated as dead bookies deadBookies.removeAll(readOnlyBookies); @@ -92,12 +92,12 @@ public synchronized Set onClusterChanged(Set reorderReadSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { + public List reorderReadSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { return writeSet; } @Override - public List reorderReadLACSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { + public List reorderReadLACSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { List retList = new ArrayList(writeSet); if (retList.size() < ensemble.size()) { for (int i = 0; i < ensemble.size(); i++) { 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 c0d78e9c..e80d9040 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DistributionSchedule.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DistributionSchedule.java @@ -17,7 +17,7 @@ */ package org.apache.bookkeeper.client; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import java.util.List; import java.util.Map; @@ -61,14 +61,14 @@ public interface AckSet { * bookie address * @return true if ack quorum is broken, false otherwise. */ - public boolean failBookieAndCheck(int bookieIndexHeardFrom, BookieSocketAddress address); + public boolean failBookieAndCheck(int bookieIndexHeardFrom, BookieId address); /** * Return the list of bookies that already failed. * * @return the list of bookies that already failed. */ - public Map getFailedBookies(); + public Map getFailedBookies(); /** * Invalidate a previous bookie response. diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsemblePlacementPolicy.java index 0f11d888..792c70fe 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 @@ -28,7 +28,7 @@ import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; 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.stats.AlertStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; @@ -72,8 +72,8 @@ public EnsemblePlacementPolicy initialize(ClientConfiguration conf, * All the bookies in the cluster available for readonly. * @return the dead bookies during this cluster change. */ - public Set onClusterChanged(Set writableBookies, - Set readOnlyBookies); + public Set onClusterChanged(Set writableBookies, + Set readOnlyBookies); /** * Choose numBookies bookies for ensemble. If the count is more than the number of available @@ -88,8 +88,8 @@ public Set onClusterChanged(Set writab * @return list of bookies chosen as targets. * @throws BKNotEnoughBookiesException if not enough bookies available. */ - public ArrayList newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Set excludeBookies) throws BKNotEnoughBookiesException; + public ArrayList newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Set excludeBookies) throws BKNotEnoughBookiesException; /** * Choose a new bookie to replace bookieToReplace. If no bookie available in the cluster, @@ -102,9 +102,9 @@ public ArrayList newEnsemble(int ensembleSize, int writeQuo * @return the bookie chosen as target. * @throws BKNotEnoughBookiesException */ - public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Collection currentEnsemble, BookieSocketAddress bookieToReplace, - Set excludeBookies) throws BKNotEnoughBookiesException; + public BookieId replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Collection currentEnsemble, BookieId bookieToReplace, + Set excludeBookies) throws BKNotEnoughBookiesException; /** * Reorder the read sequence of a given write quorum writeSet. @@ -117,8 +117,8 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, * Observed failures on the bookies * @return read sequence of bookies */ - public List reorderReadSequence(ArrayList ensemble, - List writeSet, Map bookieFailureHistory); + public List reorderReadSequence(ArrayList ensemble, + List writeSet, Map bookieFailureHistory); /** @@ -132,6 +132,6 @@ public List reorderReadSequence(ArrayList ensemble * Observed failures on the bookies * @return read sequence of bookies */ - public List reorderReadLACSequence(ArrayList ensemble, - List writeSet, Map bookieFailureHistory); + public List reorderReadLACSequence(ArrayList ensemble, + List writeSet, Map bookieFailureHistory); } 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 d65a7da8..18224ec6 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 @@ -1,6 +1,6 @@ package org.apache.bookkeeper.client; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.Node; import java.util.ArrayList; @@ -36,7 +36,7 @@ public static interface Ensemble { /** * @return list of addresses representing the ensemble */ - public ArrayList toList(); + public ArrayList toList(); /** * Validates if an ensemble is valid @@ -63,11 +63,11 @@ public static interface Ensemble { * @return list of bookies forming the ensemble * @throws BKException.BKNotEnoughBookiesException */ - ArrayList newEnsemble( + ArrayList newEnsemble( int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Set excludeBookies, + Set excludeBookies, Ensemble parentEnsemble, Predicate parentPredicate) throws BKException.BKNotEnoughBookiesException; @@ -98,7 +98,7 @@ T selectFromNetworkLocation(String networkLoc, * @param leftBookies * bookies that left */ - void handleBookiesThatLeft(Set leftBookies); + void handleBookiesThatLeft(Set leftBookies); /** * Handle bookies that joined @@ -106,5 +106,5 @@ T selectFromNetworkLocation(String networkLoc, * @param joinedBookies * bookies that joined. */ - void handleBookiesThatJoined(Set joinedBookies); + void handleBookiesThatJoined(Set joinedBookies); } 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 6eebb442..9ff09cae 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 @@ -28,7 +28,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; @@ -194,7 +194,7 @@ private void verifyLedgerFragment(LedgerFragment fragment, long firstStored = fragment.getFirstStoredEntryId(bookieIndex); long lastStored = fragment.getLastStoredEntryId(bookieIndex); - BookieSocketAddress bookie = fragment.getAddress(bookieIndex); + BookieId bookie = fragment.getAddress(bookieIndex); if (null == bookie) { throw new InvalidFragmentException(); } @@ -289,8 +289,8 @@ public void checkLedger(LedgerHandle lh, final Set fragments = new HashSet(); Long curEntryId = null; - ArrayList curEnsemble = null; - for (Map.Entry> e : lh + ArrayList curEnsemble = null; + for (Map.Entry> e : lh .getLedgerMetadata().getEnsembles().entrySet()) { if (curEntryId != null) { Set bookieIndexes = new HashSet(); @@ -351,7 +351,7 @@ public void operationComplete(int rc, Boolean result) { }); for (int bi : lh.getDistributionSchedule().getWriteSet(entryToRead)) { - BookieSocketAddress addr = curEnsemble.get(bi); + BookieId addr = curEnsemble.get(bi); bookieClient.readEntry(addr, lh.getId(), entryToRead, eecb, null); } @@ -396,11 +396,11 @@ private void checkFragments(Set fragments, private void handleClosedEmptyFragment(LedgerFragment fragment, GenericCallback cb){ Set badBookies = new HashSet<>(); - Set available = this.bcm.getAvailableBookies(); - Set readOnly = this.bcm.getReadOnlyBookies(); + Set available = this.bcm.getAvailableBookies(); + Set readOnly = this.bcm.getReadOnlyBookies(); Set bookiesToCheck = fragment.getBookiesIndexes(); for (Integer bookieIndex : bookiesToCheck) { - BookieSocketAddress bookieAddress = fragment.getAddress(bookieIndex); + BookieId bookieAddress = fragment.getAddress(bookieIndex); boolean isBadBookie = !(available.contains(bookieAddress) || readOnly.contains(bookieAddress)); if (isBadBookie) { 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 57134d92..6134d0dd 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 @@ -27,7 +27,7 @@ import org.apache.bookkeeper.client.AsyncCallback.CreateCallback; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.OpStatsLogger; @@ -102,7 +102,7 @@ public void initiate() { * Adding bookies to ledger handle */ - ArrayList ensemble; + ArrayList ensemble; try { ensemble = bk.bookieWatcher .newEnsemble(metadata.getEnsembleSize(), metadata.getWriteQuorumSize(), metadata.getAckQuorumSize()); 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 df98b7f3..3854d01f 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 @@ -19,7 +19,7 @@ */ package org.apache.bookkeeper.client; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import java.util.ArrayList; import java.util.HashSet; @@ -35,7 +35,7 @@ */ public class LedgerFragment { private final Set bookieIndexes; - private final List ensemble; + private final List ensemble; private final long firstEntryId; private final long lastKnownEntryId; private final long ledgerId; @@ -52,7 +52,7 @@ public class LedgerFragment { this.bookieIndexes = bookieIndexes; this.ensemble = lh.getLedgerMetadata().getEnsemble(firstEntryId); this.schedule = lh.getDistributionSchedule(); - SortedMap> ensembles = lh + SortedMap> ensembles = lh .getLedgerMetadata().getEnsembles(); this.isLedgerClosed = lh.getLedgerMetadata().isClosed() || !ensemble.equals(ensembles.get(ensembles.lastKey())); @@ -108,12 +108,12 @@ long getLastKnownEntryId() { /** * Gets the failedBookie address */ - public BookieSocketAddress getAddress(int bookieIndex) { + public BookieId getAddress(int bookieIndex) { return ensemble.get(bookieIndex); } - public Set getAddresses() { - Set addresses = new HashSet(); + public Set getAddresses() { + Set addresses = new HashSet(); for (int bookieIndex : bookieIndexes) { addresses.add(ensemble.get(bookieIndex)); } @@ -207,7 +207,7 @@ public Long getLastStoredEntryId(int bookieIndex) { * * @return the ensemble for the segment which this fragment is a part of */ - public List getEnsemble() { + public List getEnsemble() { return this.ensemble; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java index ffabb58f..f51a7783 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 @@ -31,7 +31,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.bookkeeper.client.AsyncCallback.ReadCallback; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.MultiCallback; @@ -61,7 +61,7 @@ public LedgerFragmentReplicator(BookKeeper bkc) { private void replicateFragmentInternal(final LedgerHandle lh, final LedgerFragment lf, final AsyncCallback.VoidCallback ledgerFragmentMcb, - final Set newBookies) throws InterruptedException { + final Set newBookies) throws InterruptedException { if (!lf.isClosed()) { LOG.error("Trying to replicate an unclosed fragment;" + " This is not safe {}", lf); @@ -134,7 +134,7 @@ private void replicateFragmentInternal(final LedgerHandle lh, */ void replicate(final LedgerHandle lh, final LedgerFragment lf, final AsyncCallback.VoidCallback ledgerFragmentMcb, - final Set targetBookieAddresses) + final Set targetBookieAddresses) throws InterruptedException { Set partionedFragments = splitIntoSubFragments(lh, lf, bkc.getConf().getRereplicationEntryBatchSize()); @@ -148,7 +148,7 @@ void replicate(final LedgerHandle lh, final LedgerFragment lf, private void replicateNextBatch(final LedgerHandle lh, final Iterator fragments, final AsyncCallback.VoidCallback ledgerFragmentMcb, - final Set targetBookieAddresses) { + final Set targetBookieAddresses) { if (fragments.hasNext()) { try { replicateFragmentInternal(lh, fragments.next(), @@ -239,12 +239,12 @@ static Set splitIntoSubFragments(LedgerHandle lh, private void recoverLedgerFragmentEntry(final Long entryId, final LedgerHandle lh, final AsyncCallback.VoidCallback ledgerFragmentEntryMcb, - final Set newBookies) throws InterruptedException { + final Set newBookies) throws InterruptedException { final AtomicInteger numCompleted = new AtomicInteger(0); final AtomicBoolean completed = new AtomicBoolean(false); final WriteCallback multiWriteCallback = new WriteCallback() { @Override - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + 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: {}", new Object[] { ledgerId, entryId, addr, BKException.create(rc) }); @@ -284,7 +284,7 @@ public void readComplete(int rc, LedgerHandle lh, */ LedgerEntry entry = seq.nextElement(); byte[] data = entry.getEntry(); - for (BookieSocketAddress newBookie : newBookies) { + for (BookieId newBookie : newBookies) { ChannelBuffer toSend = lh.getDigestManager() .computeDigestAndPackageForSending(entryId, lh.getLastAddConfirmed(), entry.getLength(), @@ -307,11 +307,11 @@ static class SingleFragmentCallback implements AsyncCallback.VoidCallback { final AsyncCallback.VoidCallback ledgerFragmentsMcb; final LedgerHandle lh; final long fragmentStartId; - final Map oldBookie2NewBookie; + final Map oldBookie2NewBookie; SingleFragmentCallback(AsyncCallback.VoidCallback ledgerFragmentsMcb, LedgerHandle lh, long fragmentStartId, - Map oldBookie2NewBookie) { + Map oldBookie2NewBookie) { this.ledgerFragmentsMcb = ledgerFragmentsMcb; this.lh = lh; this.fragmentStartId = fragmentStartId; @@ -333,14 +333,14 @@ public void processResult(int rc, String path, Object ctx) { /** Updates the ensemble with newBookie and notify the ensembleUpdatedCb */ private static void updateEnsembleInfo( AsyncCallback.VoidCallback ensembleUpdatedCb, long fragmentStartId, - LedgerHandle lh, Map oldBookie2NewBookie) { + LedgerHandle lh, Map oldBookie2NewBookie) { /* * Update the ledger metadata's ensemble info to point to the new * bookie. */ - ArrayList ensemble = lh.getLedgerMetadata() + ArrayList ensemble = lh.getLedgerMetadata() .getEnsembles().get(fragmentStartId); - for (Map.Entry entry : oldBookie2NewBookie.entrySet()) { + for (Map.Entry entry : oldBookie2NewBookie.entrySet()) { int deadBookieIndex = ensemble.indexOf(entry.getKey()); // update ensemble info might happen after re-read ledger metadata, so the ensemble might already // change. if ensemble is already changed, skip replacing the bookie doesn't exist. @@ -363,11 +363,11 @@ private static class UpdateEnsembleCb implements GenericCallback { final AsyncCallback.VoidCallback ensembleUpdatedCb; final LedgerHandle lh; final long fragmentStartId; - final Map oldBookie2NewBookie; + final Map oldBookie2NewBookie; public UpdateEnsembleCb(AsyncCallback.VoidCallback ledgerFragmentsMcb, long fragmentStartId, LedgerHandle lh, - Map oldBookie2NewBookie) { + Map oldBookie2NewBookie) { this.ensembleUpdatedCb = ledgerFragmentsMcb; this.lh = lh; this.fragmentStartId = fragmentStartId; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java index f44e8af1..f1215afa 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java @@ -42,7 +42,7 @@ import org.apache.bookkeeper.client.AsyncCallback.ReadCallback; import org.apache.bookkeeper.client.AsyncCallback.ReadLastConfirmedCallback; import org.apache.bookkeeper.client.BookKeeper.DigestType; -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.BookkeeperInternalCallbacks.GenericCallback; @@ -81,7 +81,7 @@ public class LedgerHandle { final DistributionSchedule distributionSchedule; final AtomicInteger refCount; final RateLimiter throttler; - final LoadingCache bookieFailureHistory; + final LoadingCache bookieFailureHistory; /** * Invalid entry id. This value is returned from methods which @@ -127,8 +127,8 @@ public class LedgerHandle { this.distributionSchedule = new RoundRobinDistributionSchedule( metadata.getWriteQuorumSize(), metadata.getAckQuorumSize(), metadata.getEnsembleSize()); this.bookieFailureHistory = CacheBuilder.newBuilder(). - expireAfterWrite(bk.getConf().getBookieFailureHistoryExpirationMSec(), TimeUnit.MILLISECONDS).build(new CacheLoader() { - public Long load(BookieSocketAddress key) { + expireAfterWrite(bk.getConf().getBookieFailureHistoryExpirationMSec(), TimeUnit.MILLISECONDS).build(new CacheLoader() { + public Long load(BookieId key) { return -1L; } }); @@ -990,17 +990,17 @@ void sendAddSuccessCallbacks() { numSubmittedPerCallbackStatsLogger.registerSuccessfulEvent(numSuccesses); } - EnsembleInfo replaceBookieInMetadata(final Map failedBookies, + EnsembleInfo replaceBookieInMetadata(final Map failedBookies, int ensembleChangeIdx) throws BKException.BKNotEnoughBookiesException { - final ArrayList newEnsemble = new ArrayList(); + final ArrayList newEnsemble = new ArrayList(); final long newEnsembleStartEntry = getLastAddConfirmed() + 1; final HashSet replacedBookies = new HashSet(); synchronized (metadata) { newEnsemble.addAll(metadata.currentEnsemble); - for (Map.Entry entry : failedBookies.entrySet()) { + for (Map.Entry entry : failedBookies.entrySet()) { int idx = entry.getKey(); - BookieSocketAddress addr = entry.getValue(); + BookieId addr = entry.getValue(); LOG.info("[EnsembleChange-L{}-{}] : replacing bookie: {} index: {}", new Object[] { getId(), ensembleChangeIdx, addr, idx }); if (!newEnsemble.get(idx).equals(addr)) { @@ -1012,13 +1012,13 @@ EnsembleInfo replaceBookieInMetadata(final Map fai continue; } try { - BookieSocketAddress newBookie = bk.bookieWatcher.replaceBookie( + BookieId newBookie = bk.bookieWatcher.replaceBookie( metadata.getEnsembleSize(), metadata.getWriteQuorumSize(), metadata.getAckQuorumSize(), newEnsemble, idx, - new HashSet(failedBookies.values())); + new HashSet(failedBookies.values())); newEnsemble.set(idx, newBookie); replacedBookies.add(idx); } catch (BKException.BKNotEnoughBookiesException e) { @@ -1039,7 +1039,7 @@ EnsembleInfo replaceBookieInMetadata(final Map fai return new EnsembleInfo(newEnsemble, failedBookies, replacedBookies); } - void handleBookieFailure(final Map failedBookies) { + void handleBookieFailure(final Map failedBookies) { int curBlockAddCompletions = blockAddCompletions.incrementAndGet(); if (bk.disableEnsembleChangeFeature.isAvailable()) { @@ -1071,12 +1071,12 @@ void handleBookieFailure(final Map failedBookies) // Contains newly reformed ensemble, bookieIndex, failedBookieAddress static final class EnsembleInfo { - private final ArrayList newEnsemble; - private final Map failedBookies; + private final ArrayList newEnsemble; + private final Map failedBookies; final Set replacedBookies; - public EnsembleInfo(ArrayList newEnsemble, - Map failedBookies, + public EnsembleInfo(ArrayList newEnsemble, + Map failedBookies, Set replacedBookies) { this.newEnsemble = newEnsemble; this.failedBookies = failedBookies; @@ -1273,8 +1273,8 @@ private boolean resolveConflict(LedgerMetadata newMeta) { private boolean areFailedBookiesReplaced(LedgerMetadata newMeta, EnsembleInfo ensembleInfo) { boolean replaced = true; for (Integer replacedBookieIdx : ensembleInfo.replacedBookies) { - BookieSocketAddress failedBookieAddr = ensembleInfo.failedBookies.get(replacedBookieIdx); - BookieSocketAddress replacedBookieAddr = newMeta.currentEnsemble.get(replacedBookieIdx); + BookieId failedBookieAddr = ensembleInfo.failedBookies.get(replacedBookieIdx); + BookieId replacedBookieAddr = newMeta.currentEnsemble.get(replacedBookieIdx); replaced &= !Objects.equal(replacedBookieAddr, failedBookieAddr); } return replaced; @@ -1321,7 +1321,7 @@ void rereadMetadata(final GenericCallback cb) { bk.getLedgerManager().readLedgerMetadata(ledgerId, cb); } - void registerOperationFailureOnBookie(BookieSocketAddress bookie, long entryId) { + void registerOperationFailureOnBookie(BookieId bookie, long entryId) { if (bk.getConf().getEnableBookieFailureTracking()) { bookieFailureHistory.put(bookie, entryId); } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java index 5a866260..ba38f930 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java @@ -31,7 +31,7 @@ import java.util.SortedMap; import java.util.TreeMap; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat; import org.apache.bookkeeper.versioning.Version; import org.slf4j.Logger; @@ -72,8 +72,8 @@ public class LedgerMetadata { private long lastEntryId; private LedgerMetadataFormat.State state; - private SortedMap> ensembles = new TreeMap>(); - ArrayList currentEnsemble; + private SortedMap> ensembles = new TreeMap>(); + ArrayList currentEnsemble; volatile Version version = Version.NEW; private boolean hasPassword = false; @@ -118,9 +118,9 @@ public LedgerMetadata(int ensembleSize, int writeQuorumSize, int ackQuorumSize, this.password = new byte[other.password.length]; System.arraycopy(other.password, 0, this.password, 0, other.password.length); // copy the ensembles - for (Entry> entry : other.ensembles.entrySet()) { + for (Entry> entry : other.ensembles.entrySet()) { long startEntryId = entry.getKey(); - ArrayList newEnsemble = new ArrayList(entry.getValue()); + ArrayList newEnsemble = new ArrayList(entry.getValue()); this.addEnsemble(startEntryId, newEnsemble); } } @@ -137,11 +137,11 @@ private LedgerMetadata() { * @return SortedMap of Ledger Fragments and the corresponding * bookie ensembles that store the entries. */ - public SortedMap> getEnsembles() { + public SortedMap> getEnsembles() { return ensembles; } - void setEnsembles(SortedMap> ensembles) { + void setEnsembles(SortedMap> ensembles) { this.ensembles = ensembles; } @@ -216,14 +216,14 @@ void close(long entryId) { state = LedgerMetadataFormat.State.CLOSED; } - void addEnsemble(long startEntryId, ArrayList ensemble) { + void addEnsemble(long startEntryId, ArrayList ensemble) { assert ensembles.isEmpty() || startEntryId >= ensembles.lastKey(); ensembles.put(startEntryId, ensemble); currentEnsemble = ensemble; } - ArrayList getEnsemble(long entryId) { + ArrayList getEnsemble(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()); @@ -237,7 +237,7 @@ ArrayList getEnsemble(long entryId) { * @return */ long getNextEnsembleChange(long entryId) { - SortedMap> tailMap = ensembles.tailMap(entryId + 1); + SortedMap> tailMap = ensembles.tailMap(entryId + 1); if (tailMap.isEmpty()) { return -1; @@ -264,10 +264,10 @@ public byte[] serialize() { builder.setDigestType(digestType).setPassword(ByteString.copyFrom(password)); } - for (Map.Entry> entry : ensembles.entrySet()) { + for (Map.Entry> entry : ensembles.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()); @@ -285,9 +285,9 @@ private byte[] serializeVersion1() { s.append(VERSION_KEY).append(tSplitter).append(metadataFormatVersion).append(lSplitter); s.append(writeQuorumSize).append(lSplitter).append(ensembleSize).append(lSplitter).append(length); - for (Map.Entry> entry : ensembles.entrySet()) { + for (Map.Entry> entry : ensembles.entrySet()) { s.append(lSplitter).append(entry.getKey()); - for (BookieSocketAddress addr : entry.getValue()) { + for (BookieId addr : entry.getValue()) { s.append(tSplitter); s.append(addr.toString()); } @@ -378,9 +378,9 @@ public static LedgerMetadata parseConfig(byte[] bytes, Version version) throws I } for (LedgerMetadataFormat.Segment s : data.getSegmentList()) { - ArrayList addrs = new ArrayList(); + ArrayList addrs = new ArrayList(); for (String member : s.getEnsembleMemberList()) { - addrs.add(new BookieSocketAddress(member)); + addrs.add(new BookieId(member)); } lc.addEnsemble(s.getFirstEntryId(), addrs); } @@ -411,9 +411,9 @@ static LedgerMetadata parseVersion1Config(LedgerMetadata lc, lc.state = LedgerMetadataFormat.State.OPEN; } - ArrayList addrs = new ArrayList(); + ArrayList addrs = new ArrayList(); for (int j = 1; j < parts.length; j++) { - addrs.add(new BookieSocketAddress(parts[j])); + addrs.add(new BookieId(parts[j])); } lc.addEnsemble(new Long(parts[0]), addrs); line = reader.readLine(); @@ -513,7 +513,7 @@ public String toString() { return sb.toString(); } - void mergeEnsembles(SortedMap> newEnsembles) { + void mergeEnsembles(SortedMap> newEnsembles) { // allow new metadata to be one ensemble less than current metadata // since ensemble change might kick in when recovery changed metadata int diff = ensembles.size() - newEnsembles.size(); @@ -521,21 +521,21 @@ void mergeEnsembles(SortedMap> newEnsembles return; } int i = 0; - for (Entry> entry : newEnsembles.entrySet()) { + for (Entry> entry : newEnsembles.entrySet()) { ++i; if (ensembles.size() != i) { // we should use last ensemble from current metadata // not the new metadata read from zookeeper long key = entry.getKey(); - ArrayList ensemble = entry.getValue(); + ArrayList ensemble = entry.getValue(); ensembles.put(key, ensemble); } } } - Set getBookiesInThisLedger() { - Set bookies = new HashSet(); - for (ArrayList ensemble : ensembles.values()) { + Set getBookiesInThisLedger() { + Set bookies = new HashSet(); + for (ArrayList ensemble : ensembles.values()) { bookies.addAll(ensemble); } return bookies; 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 f11fa93c..d432d6a5 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 @@ -21,7 +21,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.bookkeeper.client.AsyncCallback.AddCallback; -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.util.MathUtils; @@ -163,7 +163,7 @@ void initiate(ChannelBuffer toSend, int entryLength) { } @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; if (!lh.metadata.currentEnsemble.get(bookieIndex).equals(addr)) { @@ -222,7 +222,7 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddre default: if (delayEnsembleChange) { if (ackSet.failBookieAndCheck(bookieIndex, addr) || rc == BKException.Code.WriteOnReadOnlyBookieException) { - Map failedBookies = ackSet.getFailedBookies(); + Map failedBookies = ackSet.getFailedBookies(); LOG.warn("Failed to write entry ({}, {}) to bookies {}, handling failures.", new Object[] { 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/PendingReadOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java index 88de1a06..8375a81c 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java @@ -35,7 +35,7 @@ import org.apache.bookkeeper.client.AsyncCallback.ReadCallback; import org.apache.bookkeeper.client.BKException.BKDigestMatchException; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallbackCtx; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; import org.apache.bookkeeper.util.MathUtils; @@ -79,10 +79,10 @@ abstract class LedgerEntryRequest extends LedgerEntry implements SpeculativeRequ int firstError = BKException.Code.OK; int numMissedEntryReads = 0; - final ArrayList ensemble; + final ArrayList ensemble; final List writeSet; - LedgerEntryRequest(ArrayList ensemble, long lId, long eId) { + LedgerEntryRequest(ArrayList ensemble, long lId, long eId) { super(lId, eId); this.ensemble = ensemble; @@ -107,7 +107,7 @@ abstract class LedgerEntryRequest extends LedgerEntry implements SpeculativeRequ * @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 ChannelBuffer buffer) { + boolean complete(int bookieIndex, BookieId host, final ChannelBuffer buffer) { ChannelBufferInputStream is; try { is = lh.macManager.verifyDigestAndReturnData(entryId, buffer); @@ -160,7 +160,7 @@ boolean fail(int rc) { * @param rc * read result code */ - synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { + synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, String errMsg, int rc) { if (BKException.Code.OK == firstError || BKException.Code.NoSuchEntryException == firstError || BKException.Code.NoSuchLedgerExistsException == firstError) { @@ -192,7 +192,7 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress * the set of hosts that we already received responses. * @return host we sent to if we sent. null otherwise. */ - abstract BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFromHostsBitSet); + abstract BookieId maybeSendSpeculativeRead(BitSet heardFromHostsBitSet); /** * Whether the read request completed. @@ -245,7 +245,7 @@ class ParallelReadRequest extends LedgerEntryRequest { int numPendings; - ParallelReadRequest(ArrayList ensemble, long lId, long eId) { + ParallelReadRequest(ArrayList ensemble, long lId, long eId) { super(ensemble, lId, eId); numPendings = writeSet.size(); } @@ -253,7 +253,7 @@ class ParallelReadRequest extends LedgerEntryRequest { @Override void read() { for (int bookieIndex : writeSet) { - BookieSocketAddress to = ensemble.get(bookieIndex); + BookieId to = ensemble.get(bookieIndex); try { sendReadTo(bookieIndex, to, this); } catch (InterruptedException ie) { @@ -266,7 +266,7 @@ void read() { } @Override - synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { + synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, String errMsg, int rc) { super.logErrorAndReattemptRead(bookieIndex, host, errMsg, rc); --numPendings; // if received all responses or this entry doesn't meet quorum write, complete the request. @@ -281,7 +281,7 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress } @Override - BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFromHostsBitSet) { + BookieId maybeSendSpeculativeRead(BitSet heardFromHostsBitSet) { // no speculative read return null; } @@ -294,7 +294,7 @@ class SequenceReadRequest extends LedgerEntryRequest { final BitSet sentReplicas; final BitSet erroredReplicas; - SequenceReadRequest(ArrayList ensemble, long lId, long eId) { + SequenceReadRequest(ArrayList ensemble, long lId, long eId) { super(ensemble, lId, eId); this.sentReplicas = new BitSet(lh.getLedgerMetadata().getWriteQuorumSize()); @@ -330,7 +330,7 @@ private boolean readsOutstanding() { * @return host we sent to if we sent. null otherwise. */ @Override - synchronized BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFrom) { + synchronized BookieId maybeSendSpeculativeRead(BitSet heardFrom) { if (nextReplicaIndexToReadFrom >= getLedgerMetadata().getWriteQuorumSize()) { return null; } @@ -352,7 +352,7 @@ void read() { sendNextRead(); } - synchronized BookieSocketAddress sendNextRead() { + synchronized BookieId sendNextRead() { if (nextReplicaIndexToReadFrom >= getLedgerMetadata().getWriteQuorumSize()) { // we are done, the read has failed from all replicas, just fail the // read @@ -375,7 +375,7 @@ synchronized BookieSocketAddress sendNextRead() { nextReplicaIndexToReadFrom++; try { - BookieSocketAddress to = ensemble.get(bookieIndex); + BookieId to = ensemble.get(bookieIndex); sendReadTo(bookieIndex, to, this); sentReplicas.set(replica); return to; @@ -388,7 +388,7 @@ synchronized BookieSocketAddress sendNextRead() { } @Override - synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { + synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, String errMsg, int rc) { super.logErrorAndReattemptRead(bookieIndex, host, errMsg, rc); int replica = getReplicaIndex(bookieIndex); @@ -404,7 +404,7 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress } @Override - boolean complete(int bookieIndex, BookieSocketAddress host, ChannelBuffer buffer) { + boolean complete(int bookieIndex, BookieId host, ChannelBuffer buffer) { boolean completed = super.complete(bookieIndex, host, buffer); if (completed) { lh.getStatsLogger().getOpStatsLogger(BookKeeperClientStats.SPECULATIVES_PER_READ) @@ -456,7 +456,7 @@ PendingReadOp enablePiggybackLAC(boolean enabled) { public void initiate() { long nextEnsembleChange = startEntryId, i = startEntryId; this.requestTimeNanos = MathUtils.nowInNano(); - ArrayList ensemble = null; + ArrayList ensemble = null; do { if (i == nextEnsembleChange) { @@ -483,11 +483,11 @@ public void initiate() { 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; @@ -504,7 +504,7 @@ public long getLastAddConfirmed() { } } - void sendReadTo(int bookieIndex, BookieSocketAddress to, LedgerEntryRequest entry) throws InterruptedException { + void sendReadTo(int bookieIndex, BookieId to, LedgerEntryRequest entry) throws InterruptedException { lh.throttler.acquire(); lh.bk.bookieClient.readEntry(to, lh.ledgerId, entry.entryId, 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 17c8f8c1..651ca9f0 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 @@ -6,8 +6,7 @@ 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.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.Node; import org.apache.bookkeeper.stats.AlertStatsLogger; @@ -54,8 +53,8 @@ public void uninitalize() { } @Override - public Set onClusterChanged(Set writableBookies, Set readOnlyBookies) { - Set deadBookies = super.onClusterChanged(writableBookies, readOnlyBookies); + public Set onClusterChanged(Set writableBookies, Set readOnlyBookies) { + Set deadBookies = super.onClusterChanged(writableBookies, readOnlyBookies); if (null != slave) { deadBookies = slave.onClusterChanged(writableBookies, readOnlyBookies); } @@ -63,11 +62,11 @@ public Set onClusterChanged(Set writab } @Override - public ArrayList newEnsemble( + public ArrayList newEnsemble( int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Set excludeBookies) + Set excludeBookies) throws BKException.BKNotEnoughBookiesException { try { return super.newEnsemble(ensembleSize, writeQuorumSize, ackQuorumSize, excludeBookies); @@ -81,13 +80,13 @@ public ArrayList newEnsemble( } @Override - public BookieSocketAddress replaceBookie( + public BookieId replaceBookie( int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Collection currentEnsemble, - BookieSocketAddress bookieToReplace, - Set excludeBookies) + Collection currentEnsemble, + BookieId bookieToReplace, + Set excludeBookies) throws BKException.BKNotEnoughBookiesException { try { return super.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize, @@ -103,26 +102,26 @@ public BookieSocketAddress replaceBookie( } @Override - public List reorderReadSequence(ArrayList ensemble, + public List reorderReadSequence(ArrayList ensemble, List writeSet, - Map bookieFailureHistory) { + Map bookieFailureHistory) { return super.reorderReadSequence(ensemble, writeSet, bookieFailureHistory); } @Override - public List reorderReadLACSequence(ArrayList ensemble, + public List reorderReadLACSequence(ArrayList ensemble, List writeSet, - Map bookieFailureHistory) { + Map bookieFailureHistory) { return super.reorderReadLACSequence(ensemble, writeSet, bookieFailureHistory); } @Override - public ArrayList newEnsemble(int ensembleSize, - int writeQuorumSize, - int ackQuorumSize, - Set excludeBookies, - Ensemble parentEnsemble, - Predicate parentPredicate) + public ArrayList newEnsemble(int ensembleSize, + int writeQuorumSize, + int ackQuorumSize, + Set excludeBookies, + Ensemble parentEnsemble, + Predicate parentPredicate) throws BKException.BKNotEnoughBookiesException { try { return super.newEnsemble( @@ -161,7 +160,7 @@ public BookieNode selectFromNetworkLocation( } @Override - public void handleBookiesThatLeft(Set leftBookies) { + public void handleBookiesThatLeft(Set leftBookies) { super.handleBookiesThatLeft(leftBookies); if (null != slave) { slave.handleBookiesThatLeft(leftBookies); @@ -169,7 +168,7 @@ public void handleBookiesThatLeft(Set leftBookies) { } @Override - public void handleBookiesThatJoined(Set joinedBookies) { + public void handleBookiesThatJoined(Set joinedBookies) { super.handleBookiesThatJoined(joinedBookies); if (null != slave) { slave.handleBookiesThatJoined(joinedBookies); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java index bc6ce44e..45f888d6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java @@ -18,6 +18,7 @@ package org.apache.bookkeeper.client; import java.net.InetAddress; +import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Collection; @@ -33,8 +34,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.Configurable; import org.apache.bookkeeper.feature.FeatureProvider; -import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.net.CachedDNSToSwitchMapping; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetUtils; import org.apache.bookkeeper.net.NetworkTopology; @@ -90,10 +90,10 @@ public void reloadCachedMappings() { protected NetworkTopology topology; protected DNSToSwitchMapping dnsResolver; protected HashedWheelTimer timer; - protected final Map knownBookies; + protected final Map knownBookies; protected BookieNode localNode; protected final ReentrantReadWriteLock rwLock; - protected ImmutableSet readOnlyBookies = null; + protected ImmutableSet readOnlyBookies = null; protected boolean reorderReadsRandom = false; protected boolean enforceDurability = false; protected int stabilizePeriodSeconds = 0; @@ -107,12 +107,12 @@ public void reloadCachedMappings() { RackawareEnsemblePlacementPolicyImpl(boolean enforceDurability) { this.enforceDurability = enforceDurability; topology = new NetworkTopologyImpl(); - knownBookies = new HashMap(); + knownBookies = new HashMap(); rwLock = new ReentrantReadWriteLock(); } - protected BookieNode createBookieNode(BookieSocketAddress addr) { + protected BookieNode createBookieNode(BookieId addr) { return new BookieNode(addr, resolveNetworkLocation(addr)); } @@ -144,7 +144,7 @@ protected RackawareEnsemblePlacementPolicyImpl initialize(DNSToSwitchMapping dns BookieNode bn; try { - bn = createBookieNode(new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), 0)); + bn = createBookieNode(new BookieId(InetAddress.getLocalHost().getHostAddress(), 0)); } catch (UnknownHostException e) { LOG.error("Failed to get local host address : ", e); bn = null; @@ -192,17 +192,17 @@ public void uninitalize() { // do nothing } - protected String resolveNetworkLocation(BookieSocketAddress addr) { - return NetUtils.resolveNetworkLocation(dnsResolver, addr.getSocketAddress()); + protected String resolveNetworkLocation(BookieId addr) { + return NetUtils.resolveNetworkLocation(dnsResolver, new InetSocketAddress(addr.getHostName(), addr.getPort())); } @Override - public Set onClusterChanged(Set writableBookies, - Set readOnlyBookies) { + public Set onClusterChanged(Set writableBookies, + Set readOnlyBookies) { rwLock.writeLock().lock(); try { - ImmutableSet joinedBookies, leftBookies, deadBookies; - Set oldBookieSet = knownBookies.keySet(); + ImmutableSet joinedBookies, leftBookies, deadBookies; + Set oldBookieSet = knownBookies.keySet(); // left bookies : bookies in known bookies, but not in new writable bookie cluster. leftBookies = Sets.difference(oldBookieSet, writableBookies).immutableCopy(); // joined bookies : bookies in new writable bookie cluster, but not in known bookies @@ -228,8 +228,8 @@ public Set onClusterChanged(Set writab } @Override - public void handleBookiesThatLeft(Set leftBookies) { - for (BookieSocketAddress addr : leftBookies) { + public void handleBookiesThatLeft(Set leftBookies) { + for (BookieId addr : leftBookies) { BookieNode node = knownBookies.remove(addr); if(null != node) { topology.remove(node); @@ -241,9 +241,9 @@ public void handleBookiesThatLeft(Set leftBookies) { } @Override - public void handleBookiesThatJoined(Set joinedBookies) { + public void handleBookiesThatJoined(Set joinedBookies) { // node joined - for (BookieSocketAddress addr : joinedBookies) { + for (BookieId addr : joinedBookies) { BookieNode node = createBookieNode(addr); topology.add(node); knownBookies.put(addr, node); @@ -253,9 +253,9 @@ public void handleBookiesThatJoined(Set joinedBookies) { } } - protected Set convertBookiesToNodes(Set excludeBookies) { + protected Set convertBookiesToNodes(Set excludeBookies) { Set nodes = new HashSet(); - for (BookieSocketAddress addr : excludeBookies) { + for (BookieId addr : excludeBookies) { BookieNode bn = knownBookies.get(addr); if (null == bn) { bn = createBookieNode(addr); @@ -266,17 +266,17 @@ protected Set convertBookiesToNodes(Set excludeBookie } @Override - public ArrayList newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Set excludeBookies) + public ArrayList newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Set excludeBookies) throws BKNotEnoughBookiesException { return newEnsembleInternal(ensembleSize, writeQuorumSize, excludeBookies, null, null); } - protected ArrayList newEnsembleInternal(int ensembleSize, - int writeQuorumSize, - Set excludeBookies, - Ensemble parentEnsemble, - Predicate parentPredicate) + protected ArrayList newEnsembleInternal(int ensembleSize, + int writeQuorumSize, + Set excludeBookies, + Ensemble parentEnsemble, + Predicate parentPredicate) throws BKNotEnoughBookiesException { return newEnsembleInternal( ensembleSize, @@ -288,12 +288,12 @@ protected ArrayList newEnsembleInternal(int ensembleSize, } @Override - public ArrayList newEnsemble(int ensembleSize, - int writeQuorumSize, - int ackQuorumSize, - Set excludeBookies, - Ensemble parentEnsemble, - Predicate parentPredicate) + public ArrayList newEnsemble(int ensembleSize, + int writeQuorumSize, + int ackQuorumSize, + Set excludeBookies, + Ensemble parentEnsemble, + Predicate parentPredicate) throws BKNotEnoughBookiesException { return newEnsembleInternal( ensembleSize, @@ -304,11 +304,11 @@ public ArrayList newEnsemble(int ensembleSize, parentPredicate); } - protected ArrayList newEnsembleInternal( + protected ArrayList newEnsembleInternal( int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Set excludeBookies, + Set excludeBookies, Ensemble parentEnsemble, Predicate parentPredicate) throws BKNotEnoughBookiesException { rwLock.readLock().lock(); @@ -329,7 +329,7 @@ protected ArrayList newEnsembleInternal( if (numRacks < 2) { List bns = selectRandom(ensembleSize, excludeNodes, TruePredicate.instance, ensemble); - ArrayList addrs = new ArrayList(ensembleSize); + ArrayList addrs = new ArrayList(ensembleSize); for (BookieNode bn : bns) { addrs.add(bn.getAddr()); } @@ -350,7 +350,7 @@ protected ArrayList newEnsembleInternal( } prevNode = selectFromNetworkLocation(curRack, excludeNodes, ensemble, ensemble); } - ArrayList bookieList = ensemble.toList(); + ArrayList bookieList = ensemble.toList(); if (ensembleSize != bookieList.size()) { LOG.error("Not enough {} bookies are available to form an ensemble : {}.", ensembleSize, bookieList); @@ -363,10 +363,10 @@ protected ArrayList newEnsembleInternal( } @Override - public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Collection currentEnsemble, - BookieSocketAddress bookieToReplace, - Set excludeBookies) + public BookieId replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Collection currentEnsemble, + BookieId bookieToReplace, + Set excludeBookies) throws BKNotEnoughBookiesException { rwLock.readLock().lock(); try { @@ -516,14 +516,14 @@ protected List selectRandomInternal(List bookiesToSelect @Override - public List reorderReadSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { + public List reorderReadSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { int ensembleSize = ensemble.size(); List finalList = new ArrayList(writeSet.size()); List observedFailuresList = new ArrayList(writeSet.size()); List readOnlyList = new ArrayList(writeSet.size()); List unAvailableList = new ArrayList(writeSet.size()); for (Integer idx : writeSet) { - BookieSocketAddress address = ensemble.get(idx); + BookieId address = ensemble.get(idx); Long lastFailedEntryOnBookie = bookieFailureHistory.get(address); if (null == knownBookies.get(address)) { // there isn't too much differences between readonly bookies from unavailable bookies. since there 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 dcf1dee3..5b88e079 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 @@ -11,7 +11,7 @@ import com.google.common.base.Optional; import com.google.common.util.concurrent.ListenableFuture; -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.util.MathUtils; @@ -52,11 +52,11 @@ abstract class ReadLACAndEntryRequest extends LedgerEntry { int firstError = BKException.Code.OK; int numMissedEntryReads = 0; - final ArrayList ensemble; + final ArrayList ensemble; final List writeSet; final List orderedEnsemble; - ReadLACAndEntryRequest(ArrayList ensemble, long lId, long eId) { + ReadLACAndEntryRequest(ArrayList ensemble, long lId, long eId) { super(lId, eId); this.ensemble = ensemble; @@ -86,7 +86,7 @@ synchronized int getFirstError() { * @return return true if we managed to complete the entry; * otherwise return false if the read entry is not complete or it is already completed before */ - boolean complete(int bookieIndex, BookieSocketAddress host, final ChannelBuffer buffer, long entryId) { + boolean complete(int bookieIndex, BookieId host, final ChannelBuffer buffer, long entryId) { ChannelBufferInputStream is; try { is = lh.macManager.verifyDigestAndReturnData(entryId, buffer); @@ -155,7 +155,7 @@ synchronized private void translateAndSetFirstError(int rc) { * @param rc * read result code */ - synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { + synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, String errMsg, int rc) { translateAndSetFirstError(rc); if (BKException.Code.NoSuchEntryException == rc || @@ -183,7 +183,7 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress * the set of hosts that we already received responses. * @return host we sent to if we sent. null otherwise. */ - abstract BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFromHostsBitSet); + abstract BookieId maybeSendSpeculativeRead(BitSet heardFromHostsBitSet); /** * Whether the read request completed. @@ -213,7 +213,7 @@ class ParallelReadRequest extends ReadLACAndEntryRequest { int numPendings; - ParallelReadRequest(ArrayList ensemble, long lId, long eId) { + ParallelReadRequest(ArrayList ensemble, long lId, long eId) { super(ensemble, lId, eId); numPendings = orderedEnsemble.size(); } @@ -221,7 +221,7 @@ class ParallelReadRequest extends ReadLACAndEntryRequest { @Override void read() { for (int bookieIndex : orderedEnsemble) { - BookieSocketAddress to = ensemble.get(bookieIndex); + BookieId to = ensemble.get(bookieIndex); try { sendReadTo(bookieIndex, to, this); } catch (InterruptedException ie) { @@ -234,7 +234,7 @@ void read() { } @Override - synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { + synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, String errMsg, int rc) { super.logErrorAndReattemptRead(bookieIndex, host, errMsg, rc); --numPendings; // if received all responses or this entry doesn't meet quorum write, complete the request. @@ -249,7 +249,7 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress } @Override - BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFromHostsBitSet) { + BookieId maybeSendSpeculativeRead(BitSet heardFromHostsBitSet) { // no speculative read return null; } @@ -263,7 +263,7 @@ class SequenceReadRequest extends ReadLACAndEntryRequest { final BitSet erroredReplicas; final BitSet emptyResponseReplicas; - SequenceReadRequest(ArrayList ensemble, long lId, long eId) { + SequenceReadRequest(ArrayList ensemble, long lId, long eId) { super(ensemble, lId, eId); this.sentReplicas = new BitSet(orderedEnsemble.size()); @@ -300,7 +300,7 @@ private boolean readsOutstanding() { * @return host we sent to if we sent. null otherwise. */ @Override - synchronized BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFrom) { + synchronized BookieId maybeSendSpeculativeRead(BitSet heardFrom) { if (nextReplicaIndexToReadFrom >= getLedgerMetadata().getEnsembleSize()) { return null; } @@ -322,7 +322,7 @@ void read() { sendNextRead(); } - synchronized BookieSocketAddress sendNextRead() { + synchronized BookieId sendNextRead() { if (nextReplicaIndexToReadFrom >= getLedgerMetadata().getEnsembleSize()) { // we are done, the read has failed from all replicas, just fail the // read @@ -343,7 +343,7 @@ synchronized BookieSocketAddress sendNextRead() { nextReplicaIndexToReadFrom++; try { - BookieSocketAddress to = ensemble.get(bookieIndex); + BookieId to = ensemble.get(bookieIndex); sendReadTo(bookieIndex, to, this); sentReplicas.set(replica); return to; @@ -356,7 +356,7 @@ synchronized BookieSocketAddress sendNextRead() { } @Override - synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { + synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, String errMsg, int rc) { super.logErrorAndReattemptRead(bookieIndex, host, errMsg, rc); int replica = getReplicaIndex(bookieIndex); @@ -377,7 +377,7 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress } @Override - boolean complete(int bookieIndex, BookieSocketAddress host, ChannelBuffer buffer, long entryId) { + boolean complete(int bookieIndex, BookieId host, ChannelBuffer buffer, long entryId) { boolean completed = super.complete(bookieIndex, host, buffer, entryId); if (completed) { lh.getStatsLogger().getOpStatsLogger(BookKeeperClientStats.SPECULATIVES_PER_READ_LAC) @@ -462,7 +462,7 @@ public void initiate() { } } - void sendReadTo(int bookieIndex, BookieSocketAddress to, ReadLACAndEntryRequest entry) throws InterruptedException { + void sendReadTo(int bookieIndex, BookieId to, ReadLACAndEntryRequest entry) throws InterruptedException { if (LOG.isDebugEnabled()) { LOG.debug("Calling Read LAC and Entry with {} and long polling interval {} on Bookie {} - Parallel {}", new Object[] { prevEntryId, timeOutInMillis, to, parallelRead }); @@ -486,11 +486,11 @@ interface LastConfirmedAndEntryCallback { public static class ReadLastConfirmedAndEntryContext implements BookkeeperInternalCallbacks.ReadEntryCallbackCtx { final int bookieIndex; - final BookieSocketAddress bookie; + final BookieId bookie; long lac = LedgerHandle.INVALID_ENTRY_ID; Optional lacUpdateTimestamp = Optional.absent(); - ReadLastConfirmedAndEntryContext(int bookieIndex, BookieSocketAddress bookie) { + ReadLastConfirmedAndEntryContext(int bookieIndex, BookieId bookie) { this.bookieIndex = bookieIndex; this.bookie = bookie; } @@ -544,7 +544,7 @@ public void readEntryComplete(int rc, long ledgerId, long entryId, ChannelBuffer new Object[] { getClass().getName(), ledgerId, entryId, rc }); } ReadLastConfirmedAndEntryContext rCtx = (ReadLastConfirmedAndEntryContext) ctx; - BookieSocketAddress bookie = rCtx.bookie; + BookieId bookie = rCtx.bookie; numResponsesPending--; if (BKException.Code.OK == rc) { if (LOG.isTraceEnabled()) { 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 66e0a6ba..e1d7482a 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 @@ -23,7 +23,7 @@ import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.AsyncCallback.CloseCallback; import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.LedgerMetadataListener; import org.apache.bookkeeper.util.SafeRunnable; import org.apache.bookkeeper.versioning.Version; @@ -120,7 +120,7 @@ public void asyncAddEntry(final byte[] data, final int offset, final int length, } @Override - void handleBookieFailure(final Map failedBookies) { + void handleBookieFailure(final Map failedBookies) { blockAddCompletions.incrementAndGet(); synchronized (metadata) { try { 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 65bae3c0..1022472d 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 @@ -31,11 +31,10 @@ import com.google.common.base.Optional; -import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.Feature; 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.net.NetworkTopology; import org.apache.bookkeeper.net.Node; @@ -66,7 +65,7 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme static final int REMOTE_NODE_IN_REORDER_SEQUENCE = 2; protected final Map perRegionPlacement; - protected final ConcurrentMap address2Region; + protected final ConcurrentMap address2Region; protected FeatureProvider featureProvider; protected String disallowBookiePlacementInRegionFeatureName; protected String myRegion = null; @@ -78,10 +77,10 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme RegionAwareEnsemblePlacementPolicy() { super(); perRegionPlacement = new HashMap(); - address2Region = new ConcurrentHashMap(); + address2Region = new ConcurrentHashMap(); } - protected String getRegion(BookieSocketAddress addr) { + protected String getRegion(BookieId addr) { String region = address2Region.get(addr); if (null == region) { String networkLocation = resolveNetworkLocation(addr); @@ -108,7 +107,7 @@ protected String getLocalRegion(BookieNode node) { } @Override - public void handleBookiesThatLeft(Set leftBookies) { + public void handleBookiesThatLeft(Set leftBookies) { super.handleBookiesThatLeft(leftBookies); for(TopologyAwareEnsemblePlacementPolicy policy: perRegionPlacement.values()) { @@ -117,11 +116,11 @@ public void handleBookiesThatLeft(Set leftBookies) { } @Override - public void handleBookiesThatJoined(Set joinedBookies) { - Map> perRegionClusterChange = new HashMap>(); + public void handleBookiesThatJoined(Set joinedBookies) { + Map> perRegionClusterChange = new HashMap>(); // node joined - for (BookieSocketAddress addr : joinedBookies) { + for (BookieId addr : joinedBookies) { BookieNode node = createBookieNode(addr); topology.add(node); knownBookies.put(addr, node); @@ -131,9 +130,9 @@ public void handleBookiesThatJoined(Set joinedBookies) { .initialize(dnsResolver, timer, this.reorderReadsRandom, this.stabilizePeriodSeconds, statsLogger, alertStatsLogger)); } - Set regionSet = perRegionClusterChange.get(region); + Set regionSet = perRegionClusterChange.get(region); if (null == regionSet) { - regionSet = new HashSet(); + regionSet = new HashSet(); regionSet.add(addr); perRegionClusterChange.put(region, regionSet); } else { @@ -146,9 +145,9 @@ public void handleBookiesThatJoined(Set joinedBookies) { } for(String region: perRegionPlacement.keySet()) { - Set regionSet = perRegionClusterChange.get(region); + Set regionSet = perRegionClusterChange.get(region); if (null == regionSet) { - regionSet = new HashSet(); + regionSet = new HashSet(); } perRegionPlacement.get(region).handleBookiesThatJoined(regionSet); } @@ -215,8 +214,8 @@ protected List selectRandomFromRegions(Set availableRegions, @Override - public ArrayList newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, - Set excludeBookies) throws BKException.BKNotEnoughBookiesException { + public ArrayList newEnsemble(int ensembleSize, int writeQuorumSize, int ackQuorumSize, + Set excludeBookies) throws BKException.BKNotEnoughBookiesException { int effectiveMinRegionsForDurability = disableDurabilityFeature.isAvailable() ? 1 : minRegionsForDurability; @@ -260,7 +259,7 @@ public ArrayList newEnsemble(int ensembleSize, int writeQuo } List bns = selectRandom(ensembleSize, excludeNodes, TruePredicate.instance, EnsembleForReplacementWithNoConstraints.instance); - ArrayList addrs = new ArrayList(ensembleSize); + ArrayList addrs = new ArrayList(ensembleSize); for (BookieNode bn : bns) { addrs.add(bn.getAddr()); } @@ -331,7 +330,7 @@ public ArrayList newEnsemble(int ensembleSize, int writeQuo int newEnsembleSize = currentAllocation.getLeft() + addToEnsembleSize; int newWriteQuorumSize = currentAllocation.getRight() + addToWriteQuorum; try { - List allocated = policyWithinRegion.newEnsemble(newEnsembleSize, newWriteQuorumSize, newWriteQuorumSize, excludeBookies, tempEnsemble, tempEnsemble); + List allocated = policyWithinRegion.newEnsemble(newEnsembleSize, newWriteQuorumSize, newWriteQuorumSize, excludeBookies, tempEnsemble, tempEnsemble); ensemble = tempEnsemble; remainingEnsemble -= addToEnsembleSize; remainingWriteQuorum -= addToWriteQuorum; @@ -370,6 +369,7 @@ public ArrayList newEnsemble(int ensembleSize, int writeQuo new Object[]{currentAllocation.getLeft(), region, ensemble}); } } + numRemainingRegions -= 1; } if (regionsReachedMaxAllocation.containsAll(regionsWiseAllocation.keySet())) { @@ -377,7 +377,7 @@ public ArrayList newEnsemble(int ensembleSize, int writeQuo } } while ((remainingEnsemble > 0) && (remainingEnsemble < remainingEnsembleBeforeIteration)); - ArrayList bookieList = ensemble.toList(); + ArrayList bookieList = ensemble.toList(); if (ensembleSize != bookieList.size()) { LOG.error("Not enough {} bookies are available to form an ensemble : {}.", ensembleSize, bookieList); @@ -397,8 +397,8 @@ public ArrayList newEnsemble(int ensembleSize, int writeQuo } @Override - public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Collection currentEnsemble, BookieSocketAddress bookieToReplace, - Set excludeBookies) throws BKException.BKNotEnoughBookiesException { + public BookieId replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuorumSize, Collection currentEnsemble, BookieId bookieToReplace, + Set excludeBookies) throws BKException.BKNotEnoughBookiesException { rwLock.readLock().lock(); try { boolean enforceDurability = enforceDurabilityInReplace && !disableDurabilityFeature.isAvailable(); @@ -418,7 +418,7 @@ public BookieSocketAddress replaceBookie(int ensembleSize, int writeQuorumSize, } excludeNodes.add(bookieNodeToReplace); - for(BookieSocketAddress bookieAddress: currentEnsemble) { + for(BookieId bookieAddress: currentEnsemble) { if (bookieAddress.equals(bookieToReplace)) { continue; } @@ -501,7 +501,7 @@ protected BookieNode replaceFromRack(BookieNode bookieNodeToReplace, } @Override - public final List reorderReadSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { + public final List reorderReadSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { if (UNKNOWN_REGION.equals(myRegion)) { return super.reorderReadSequence(ensemble, writeSet, bookieFailureHistory); } else { @@ -514,7 +514,7 @@ public final List reorderReadSequence(ArrayList en List readOnlyList = new ArrayList(writeSet.size()); List unAvailableList = new ArrayList(writeSet.size()); for (Integer idx : writeSet) { - BookieSocketAddress address = ensemble.get(idx); + BookieId address = ensemble.get(idx); String region = getRegion(address); Long lastFailedEntryOnBookie = bookieFailureHistory.get(address); if (null == knownBookies.get(address)) { @@ -589,7 +589,7 @@ public final List reorderReadSequence(ArrayList en } @Override - public final List reorderReadLACSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { + public final List reorderReadLACSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { if (UNKNOWN_REGION.equals(myRegion)) { return super.reorderReadLACSequence(ensemble, writeSet, bookieFailureHistory); } 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 4eaf37ba..37fac61b 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 @@ -18,7 +18,7 @@ package org.apache.bookkeeper.client; import com.google.common.collect.ImmutableMap; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import java.util.HashMap; import java.util.List; @@ -57,8 +57,8 @@ public List getWriteSet(long entryId) { @Override public AckSet getAckSet() { final HashSet ackSet = new HashSet(); - final HashMap failureMap = - new HashMap(); + final HashMap failureMap = + new HashMap(); return new AckSet() { public boolean completeBookieAndCheck(int bookieIndexHeardFrom) { failureMap.remove(bookieIndexHeardFrom); @@ -67,14 +67,14 @@ public boolean completeBookieAndCheck(int bookieIndexHeardFrom) { } @Override - public boolean failBookieAndCheck(int bookieIndexHeardFrom, BookieSocketAddress address) { + public boolean failBookieAndCheck(int bookieIndexHeardFrom, BookieId address) { ackSet.remove(bookieIndexHeardFrom); failureMap.put(bookieIndexHeardFrom, address); return failureMap.size() > (writeQuorumSize - ackQuorumSize); } @Override - public Map getFailedBookies() { + public Map getFailedBookies() { return ImmutableMap.copyOf(failureMap); } 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 4558bdf0..f4e2019b 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 @@ -10,7 +10,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.net.NodeBase; import org.apache.bookkeeper.stats.AlertStatsLogger; @@ -34,7 +34,7 @@ public boolean apply(BookieNode candidate, Ensemble chosenNodes) { protected static class EnsembleForReplacementWithNoConstraints implements Ensemble { public static final EnsembleForReplacementWithNoConstraints instance = new EnsembleForReplacementWithNoConstraints(); - static final ArrayList EMPTY_LIST = new ArrayList(0); + static final ArrayList EMPTY_LIST = new ArrayList(0); @Override public boolean addNode(BookieNode node) { @@ -43,7 +43,7 @@ public boolean addNode(BookieNode node) { } @Override - public ArrayList toList() { + public ArrayList toList() { return EMPTY_LIST; } @@ -61,14 +61,14 @@ public boolean validate() { protected static class BookieNode extends NodeBase { - private final BookieSocketAddress addr; // identifier of a bookie node. + private final BookieId addr; // identifier of a bookie node. - BookieNode(BookieSocketAddress addr, String networkLoc) { + BookieNode(BookieId addr, String networkLoc) { super(addr.toString(), networkLoc); this.addr = addr; } - public BookieSocketAddress getAddr() { + public BookieId getAddr() { return addr; } @@ -404,8 +404,8 @@ public boolean addNode(BookieNode node) { } @Override - public ArrayList toList() { - ArrayList addresses = new ArrayList(ensembleSize); + public ArrayList toList() { + ArrayList addresses = new ArrayList(ensembleSize); for (BookieNode bn : chosenNodes) { addresses.add(bn.getAddr()); } @@ -419,7 +419,7 @@ public ArrayList toList() { */ @Override public boolean validate() { - HashSet addresses = new HashSet(ensembleSize); + HashSet addresses = new HashSet(ensembleSize); HashSet racksOrRegions = new HashSet(); for (BookieNode bn : chosenNodes) { if (addresses.contains(bn.getAddr())) { @@ -440,12 +440,12 @@ public String toString() { } @Override - public List reorderReadSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { + public List reorderReadSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { return writeSet; } @Override - public List reorderReadLACSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { + public List reorderReadLACSequence(ArrayList ensemble, List writeSet, Map bookieFailureHistory) { List retList = new ArrayList(reorderReadSequence(ensemble, writeSet, bookieFailureHistory)); if (retList.size() < ensemble.size()) { for (int i = 0; i < ensemble.size(); i++) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java index eb0b9582..1b7099b1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java @@ -76,6 +76,8 @@ public class ServerConfiguration extends AbstractConfiguration { protected final static String JOURNAL_FORMAT_VERSION_TO_WRITE = "journalFormatVersionToWrite"; // Bookie Parameters protected final static String BOOKIE_PORT = "bookiePort"; + protected final static String ALTERNATE_BOOKIE_PORT_ENABLED = "alternateBookiePortEnabled"; + protected final static String ALTERNATE_BOOKIE_PORT = "alternateBookiePort"; protected final static String JOURNAL_DIR = "journalDirectory"; protected final static String LEDGER_DIRS = "ledgerDirectories"; protected final static String INDEX_DIRS = "indexDirectories"; @@ -547,6 +549,48 @@ public ServerConfiguration setBookiePort(int port) { return this; } + /** + * Check if an alternate bookie port is enabled and listening. + * Useful for splitting read and write traffic. + * + * @return alternate bookie port status + */ + public boolean isAlternateBookiePortEnabled() { + return this.getBoolean(ALTERNATE_BOOKIE_PORT_ENABLED, false); + } + + /** + * Get alternate bookie port that bookie server listen on + * + * @return alternate bookie port + */ + public int getAlternateBookiePort() { + return this.getInt(ALTERNATE_BOOKIE_PORT); + } + + /** + * Enable or disable alternate bookie port + * + * @param enabled + * @return server configuration + */ + public ServerConfiguration setAlternateBookiePortEnabled(boolean enabled) { + this.setProperty(ALTERNATE_BOOKIE_PORT_ENABLED, enabled); + return this; + } + + /** + * Set new alternate bookie port that bookie server listen on + * + * @param port + * Port to listen on + * @return server configuration + */ + public ServerConfiguration setAlternateBookiePort(int port) { + this.setProperty(ALTERNATE_BOOKIE_PORT, port); + return this; + } + /** * Get dir name to store journal files * 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 00000000..cad2bd04 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieId.java @@ -0,0 +1,102 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.net; + +import java.net.InetSocketAddress; +import java.net.UnknownHostException; + +import static org.apache.bookkeeper.util.BookKeeperConstants.COLON; + +/** + * This is a data wrapper class that is an InetSocketAddress, it would use the hostname + * provided in constructors directly. + *

+ * The string representation of a BookieId is : : + */ +public class BookieId { + + // Member fields that make up this class. + private final String hostname; + private final int port; + + private final InetSocketAddress socketAddress; + private final BookieSocketAddress bookieSocketAddress; + + // Constructor that takes in both a port. + public BookieId(String hostname, int port) { + this.hostname = hostname; + this.port = port; + socketAddress = new InetSocketAddress(hostname, port); + bookieSocketAddress = new BookieSocketAddress(hostname, port, port); + } + + // Constructor from a String "serialized" version of this class. + public BookieId(String addr) throws UnknownHostException { + String[] parts = addr.split(COLON); + if (parts.length < 2) { + throw new UnknownHostException(addr); + } + this.hostname = parts[0]; + try { + this.port = Integer.parseInt(parts[1]); + } catch (NumberFormatException nfe) { + throw new UnknownHostException(addr); + } + socketAddress = new InetSocketAddress(hostname, port); + bookieSocketAddress = new BookieSocketAddress(hostname, port, port); + } + + // Public getters + public String getHostName() { + return hostname; + } + + public int getPort() { + return port; + } + + public BookieSocketAddress asBookieSocketAddress() { + return bookieSocketAddress; + } + + // Return the String "serialized" version of this object. + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(hostname).append(COLON).append(port); + return sb.toString(); + } + + // Implement an equals method comparing two HedwigSocketAddress objects. + @Override + public boolean equals(Object obj) { + if (!(obj instanceof BookieId)) + return false; + BookieId that = (BookieId) obj; + return this.hostname.equals(that.hostname) && (this.port == that.port); + } + + @Override + public int hashCode() { + return this.hostname.hashCode() + 13 * this.port; + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieIdToAddressMapping.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieIdToAddressMapping.java new file mode 100644 index 00000000..ca22be11 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/net/BookieIdToAddressMapping.java @@ -0,0 +1,5 @@ +package org.apache.bookkeeper.net; + +public interface BookieIdToAddressMapping { + public BookieSocketAddress getBookieAddress(BookieId bookieId); +} 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 3dcb1d30..909aacc1 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 @@ -1,5 +1,6 @@ +package org.apache.bookkeeper.net; + /** - * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -8,93 +9,42 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. + * http://www.apache.org/licenses/LICENSE-2.0 * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.net.InetSocketAddress; -import java.net.UnknownHostException; - -import static org.apache.bookkeeper.util.BookKeeperConstants.COLON; -/** - * This is a data wrapper class that is an InetSocketAddress, it would use the hostname - * provided in constructors directly. - *

- * The string representation of a BookieSocketAddress is : : - */ public class BookieSocketAddress { - // Member fields that make up this class. - private final String hostname; - private final int port; - - private final InetSocketAddress socketAddress; - - // Constructor that takes in both a port. - public BookieSocketAddress(String hostname, int port) { - this.hostname = hostname; - this.port = port; - socketAddress = new InetSocketAddress(hostname, port); - } - - // Constructor from a String "serialized" version of this class. - public BookieSocketAddress(String addr) throws UnknownHostException { - String[] parts = addr.split(COLON); - if (parts.length < 2) { - throw new UnknownHostException(addr); - } - this.hostname = parts[0]; - try { - this.port = Integer.parseInt(parts[1]); - } catch (NumberFormatException nfe) { - throw new UnknownHostException(addr); - } - socketAddress = new InetSocketAddress(hostname, port); - } - - // Public getters - public String getHostName() { - return hostname; - } + private final String hostname; + private final int writePort; + private final int readPort; - public int getPort() { - return port; - } + public BookieSocketAddress(String hostname, int writePort, int readPort) { + this.hostname = hostname; + this.writePort = writePort; + this.readPort = readPort; + } - // Method to return an InetSocketAddress for the regular port. - public InetSocketAddress getSocketAddress() { - return socketAddress; - } + public InetSocketAddress getWriteAddress () { + return new InetSocketAddress(hostname, writePort); + } - // Return the String "serialized" version of this object. - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append(hostname).append(COLON).append(port); - return sb.toString(); - } + public InetSocketAddress getReadAddress() { + return new InetSocketAddress(hostname, readPort); + } - // Implement an equals method comparing two HedwigSocketAddress objects. - @Override - public boolean equals(Object obj) { - if (!(obj instanceof BookieSocketAddress)) - return false; - BookieSocketAddress that = (BookieSocketAddress) obj; - return this.hostname.equals(that.hostname) && (this.port == that.port); - } + public BookieId getBookieId() { + return new BookieId(hostname, writePort); + } - @Override - public int hashCode() { - return this.hostname.hashCode() + 13 * this.port; - } + public String toString() { + throw new RuntimeException("don't use toString on bookieSocketAddress");} } 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 5a5bb11f..4ccf35f7 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,10 +25,13 @@ import com.google.common.base.Optional; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookieWatcher; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieIdToAddressMapping; +import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetUtils; -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.ReadEntryCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; @@ -47,9 +50,11 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.net.InetSocketAddress; import java.util.HashSet; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; @@ -64,8 +69,8 @@ public class BookieClient implements PerChannelBookieClientFactory { final OrderedSafeExecutor executor; final ClientSocketChannelFactory channelFactory; - final ConcurrentHashMap channels = - new ConcurrentHashMap(); + final ConcurrentHashMap channels = + new ConcurrentHashMap(); final HashedWheelTimer requestTimer; private final ClientConfiguration conf; private volatile boolean closed; @@ -76,17 +81,21 @@ public class BookieClient implements PerChannelBookieClientFactory { // instantiated us private final boolean ownTimer; private final Optional dnsResolver; + private final BookieIdToAddressMapping bookieAddressMapping; - public BookieClient(ClientConfiguration conf, ClientSocketChannelFactory channelFactory, OrderedSafeExecutor executor) { - this(conf, channelFactory, executor, NullStatsLogger.INSTANCE, null, Optional.absent()); + public BookieClient(ClientConfiguration conf, ClientSocketChannelFactory channelFactory, OrderedSafeExecutor executor, + BookieIdToAddressMapping bookieAddressMapping) { + this(conf, channelFactory, executor, NullStatsLogger.INSTANCE, null, Optional.absent(), + bookieAddressMapping); } public BookieClient(ClientConfiguration conf, ClientSocketChannelFactory channelFactory, OrderedSafeExecutor executor, - StatsLogger statsLogger, HashedWheelTimer requestTimer, Optional dnsResolver) { + StatsLogger statsLogger, HashedWheelTimer requestTimer, Optional dnsResolver, + BookieIdToAddressMapping bookieAddressMapping) { if (null == channelFactory) { throw new NullPointerException(); } - + this.bookieAddressMapping = bookieAddressMapping; this.conf = conf; this.channelFactory = channelFactory; this.executor = executor; @@ -120,20 +129,20 @@ private int getRc(int rc) { } @Override - public PerChannelBookieClient create(BookieSocketAddress address) { + public PerChannelBookieClient create(BookieId bookieId, InetSocketAddress socketAddress) { Optional networkLocation = Optional.absent(); try { if (dnsResolver.isPresent()) { - networkLocation = Optional.of(NetUtils.resolveNetworkLocation(dnsResolver.get(), address.getSocketAddress())); + networkLocation = Optional.of(NetUtils.resolveNetworkLocation(dnsResolver.get(), socketAddress)); } } catch (Exception exc) { LOG.info("Failed to resolve the network location", exc); } - return new PerChannelBookieClient(conf, executor, channelFactory, address, + return new PerChannelBookieClient(conf, executor, channelFactory, socketAddress, bookieId, requestTimer, statsLogger, networkLocation); } - private PerChannelBookieClientPool lookupClient(BookieSocketAddress addr, Object key) { + private PerChannelBookieClientPool lookupClient(BookieId bookieId, InetSocketAddress addr, Object key) { PerChannelBookieClientPool clientPool = channels.get(addr); if (null == clientPool) { closeLock.readLock().lock(); @@ -142,7 +151,7 @@ private PerChannelBookieClientPool lookupClient(BookieSocketAddress addr, Object return null; } PerChannelBookieClientPool newClientPool = - new DefaultPerChannelBookieClientPool(this, addr, numConnectionsPerBookie); + new DefaultPerChannelBookieClientPool(this, bookieId, addr, numConnectionsPerBookie); PerChannelBookieClientPool oldClientPool = channels.putIfAbsent(addr, newClientPool); if (null == oldClientPool) { clientPool = newClientPool; @@ -159,12 +168,12 @@ private PerChannelBookieClientPool lookupClient(BookieSocketAddress addr, Object return clientPool; } - public void closeClients(final Set addrs) { + public void closeClients(final Set addrs) { final HashSet clients = new HashSet(); closeLock.readLock().lock(); try { - for (BookieSocketAddress a : addrs) { + for (BookieId a : addrs) { PerChannelBookieClientPool c = channels.get(a); if (c != null) { clients.add(c); @@ -185,7 +194,7 @@ public void closeClients(final Set addrs) { private void completeWrite(final int rc, final long ledgerId, final long entryId, - final BookieSocketAddress addr, + final BookieId addr, final WriteCallback cb, final Object ctx) { try { @@ -204,7 +213,7 @@ public String toString() { } } - public void addEntry(final BookieSocketAddress addr, + public void addEntry(final BookieId bookieId, final long ledgerId, final byte[] masterKey, final long entryId, @@ -214,10 +223,12 @@ public void addEntry(final BookieSocketAddress addr, final int options) { closeLock.readLock().lock(); try { - final PerChannelBookieClientPool client = lookupClient(addr, entryId); + final BookieSocketAddress bookieSocketAddress = bookieAddressMapping.getBookieAddress(bookieId); + final PerChannelBookieClientPool client = lookupClient(bookieId, + bookieSocketAddress.getWriteAddress(), entryId); if (client == null) { completeWrite(BKException.Code.BookieHandleNotAvailableException, - ledgerId, entryId, addr, cb, ctx); + ledgerId, entryId, bookieId, cb, ctx); return; } @@ -225,7 +236,7 @@ public void addEntry(final BookieSocketAddress addr, @Override public void operationComplete(final int rc, PerChannelBookieClient pcbc) { if (rc != BKException.Code.OK) { - completeWrite(rc, ledgerId, entryId, addr, cb, ctx); + completeWrite(rc, ledgerId, entryId, bookieId, cb, ctx); return; } pcbc.addEntry(ledgerId, masterKey, entryId, toSend, cb, ctx, options); @@ -255,7 +266,7 @@ public void safeRun() { } } - public void readEntryAndFenceLedger(final BookieSocketAddress addr, + public void readEntryAndFenceLedger(final BookieId bookieId, final long ledgerId, final byte[] masterKey, final long entryId, @@ -263,7 +274,10 @@ public void readEntryAndFenceLedger(final BookieSocketAddress addr, final Object ctx) { closeLock.readLock().lock(); try { - final PerChannelBookieClientPool client = lookupClient(addr, entryId); + final BookieSocketAddress bookieSocketAddress = bookieAddressMapping.getBookieAddress(bookieId); + /** Use write address to avoid read rate limiting since high priority */ + final PerChannelBookieClientPool client = lookupClient(bookieSocketAddress.getBookieId(), + bookieSocketAddress.getWriteAddress(), entryId); if (client == null) { completeRead(BKException.Code.BookieHandleNotAvailableException, ledgerId, entryId, null, cb, ctx); @@ -285,14 +299,16 @@ public void operationComplete(final int rc, PerChannelBookieClient pcbc) { } } - public void readEntry(final BookieSocketAddress addr, + public void readEntry(final BookieId bookieId, final long ledgerId, final long entryId, final ReadEntryCallback cb, final Object ctx) { closeLock.readLock().lock(); try { - final PerChannelBookieClientPool client = lookupClient(addr, entryId); + final BookieSocketAddress bookieSocketAddress = bookieAddressMapping.getBookieAddress(bookieId); + final PerChannelBookieClientPool client = lookupClient(bookieSocketAddress.getBookieId(), + bookieSocketAddress.getReadAddress(), entryId); if (client == null) { completeRead(BKException.Code.BookieHandleNotAvailableException, ledgerId, entryId, null, cb, ctx); @@ -315,7 +331,7 @@ public void operationComplete(final int rc, PerChannelBookieClient pcbc) { } } - public void readEntryWaitForLACUpdate(final BookieSocketAddress addr, + public void readEntryWaitForLACUpdate(final BookieId bookieId, final long ledgerId, final long entryId, final long previousLAC, @@ -325,7 +341,9 @@ public void readEntryWaitForLACUpdate(final BookieSocketAddress addr, final Object ctx) { closeLock.readLock().lock(); try { - final PerChannelBookieClientPool client = lookupClient(addr, entryId); + final BookieSocketAddress bookieSocketAddress = bookieAddressMapping.getBookieAddress(bookieId); + final PerChannelBookieClientPool client = lookupClient(bookieSocketAddress.getBookieId(), + bookieSocketAddress.getReadAddress(), entryId); if (client == null) { completeRead(BKException.Code.BookieHandleNotAvailableException, ledgerId, entryId, null, cb, ctx); @@ -410,7 +428,7 @@ public static void main(String[] args) throws NumberFormatException, IOException } WriteCallback cb = new WriteCallback() { - 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) { @@ -424,12 +442,19 @@ public void writeComplete(int rc, long ledger, long entry, BookieSocketAddress a ClientSocketChannelFactory channelFactory = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors .newCachedThreadPool()); OrderedSafeExecutor executor = new OrderedSafeExecutor(1); - BookieClient bc = new BookieClient(new ClientConfiguration(), channelFactory, executor); - BookieSocketAddress addr = new BookieSocketAddress(args[0], Integer.parseInt(args[1])); + final BookieSocketAddress addr = new BookieSocketAddress(args[0], Integer.parseInt(args[1]), Integer.parseInt(args[1])); + BookieClient bc = new BookieClient(new ClientConfiguration(), channelFactory, executor, + new BookieIdToAddressMapping() { + @Override + public BookieSocketAddress getBookieAddress(BookieId bookieId) { + return new BookieSocketAddress(bookieId.getHostName(), bookieId.getPort(), bookieId.getPort()); + } + }); + for (int i = 0; i < 100000; i++) { counter.inc(); - bc.addEntry(addr, ledger, new byte[0], i, ChannelBuffers.wrappedBuffer(hello), cb, counter, 0); + bc.addEntry(addr.getBookieId(), ledger, new byte[0], i, ChannelBuffers.wrappedBuffer(hello), cb, counter, 0); } counter.wait(0); System.out.println("Total = " + counter.total()); 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 a94f8321..6c624f58 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 @@ -161,7 +161,7 @@ public ChannelPipeline getPipeline() throws Exception { pipeline.addLast("bookieProtoDecoder", new BookieProtoEncoding.RequestDecoder()); pipeline.addLast("bookieProtoEncoder", new BookieProtoEncoding.ResponseEncoder()); - SimpleChannelHandler requestHandler = isRunning.get() ? new BookieRequestHandler(conf, processor, allChannels) + SimpleChannelHandler requestHandler = isRunning.get() ? new BookieRequestHandler(processor, allChannels) : new RejectRequestHandler(); pipeline.addLast("bookieRequestHandler", requestHandler); return pipeline; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java index 9e7d89f6..f0a767b6 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieRequestHandler.java @@ -41,7 +41,7 @@ class BookieRequestHandler extends SimpleChannelHandler { private final RequestProcessor processor; private final ChannelGroup allChannels; - BookieRequestHandler(ServerConfiguration conf, RequestProcessor processor, ChannelGroup allChannels) { + BookieRequestHandler(RequestProcessor processor, ChannelGroup allChannels) { this.processor = processor; this.allChannels = allChannels; } 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 5804f0c5..257479e1 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java @@ -32,6 +32,7 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.bookie.ExitCode; import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.replication.AutoRecoveryMain; import org.apache.bookkeeper.replication.ReplicationException; @@ -45,6 +46,7 @@ import org.apache.commons.cli.Options; import org.apache.commons.cli.ParseException; import org.apache.commons.configuration.ConfigurationException; +import org.apache.commons.configuration.ConfigurationUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; @@ -64,6 +66,7 @@ public class BookieServer { final ServerConfiguration conf; final BookieNettyServer nettyServer; + BookieNettyServer alternateNettyServer; private volatile boolean running = false; Bookie bookie; DeathWatcher deathWatcher; @@ -98,6 +101,12 @@ public BookieServer(ServerConfiguration conf, StatsLogger statsLogger) // this.bookie = newBookie(conf); this.nettyServer = new BookieNettyServer(this.conf, this.bookie, statsLogger.scope(SERVER_SCOPE)); + if (conf.isAlternateBookiePortEnabled()) { + ServerConfiguration alternatePortConfig = new ServerConfiguration(); + ConfigurationUtils.copy(this.conf, alternatePortConfig); + alternatePortConfig.setBookiePort(conf.getAlternateBookiePort()); + alternateNettyServer = new BookieNettyServer(alternatePortConfig, this.bookie, statsLogger.scope(SERVER_SCOPE)); + } this.bookie.initialize(); isAutoRecoveryDaemonEnabled = conf.isAutoRecoveryDaemonEnabled(); @@ -136,6 +145,9 @@ public void start() throws IOException { } this.nettyServer.start(); + if (this.conf.isAlternateBookiePortEnabled()) { + alternateNettyServer.start(); + } running = true; deathWatcher = new DeathWatcher(conf); @@ -148,10 +160,16 @@ public BookieSocketAddress getLocalAddress() { return Bookie.getBookieAddress(conf); } catch (UnknownHostException uhe) { InetSocketAddress localAddress = nettyServer.getLocalAddress(); - return new BookieSocketAddress(localAddress.getHostName(), localAddress.getPort()); + int altPort = alternateNettyServer.getLocalAddress().getPort(); + return new BookieSocketAddress(localAddress.getHostName(), localAddress.getPort(), altPort); } } + @VisibleForTesting + public BookieId getId() { + return getLocalAddress().getBookieId(); + } + @VisibleForTesting public Bookie getBookie() { return bookie; @@ -181,10 +199,17 @@ public synchronized void shutdown() { running = false; // shutdown netty after bookie is shutdown to prevent netty exception this.nettyServer.shutdown(); + if (alternateNettyServer != null) { + alternateNettyServer.shutdown(); + } } public boolean isRunning() { - return bookie.isRunning() && nettyServer.isRunning() && running; + if (this.conf.isAlternateBookiePortEnabled()) { + return bookie.isRunning() && nettyServer.isRunning() && alternateNettyServer.isRunning() && running; + } else { + return bookie.isRunning() && nettyServer.isRunning() && running; + } } /** 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 745b6891..ad6063b7 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 @@ -29,7 +29,7 @@ import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.LedgerMetadata; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.stats.OpStatsLogger; import org.apache.bookkeeper.util.MathUtils; import org.apache.zookeeper.AsyncCallback; @@ -68,7 +68,7 @@ public interface LedgerMetadataListener { } 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); } public interface GenericCallback { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/DataFormats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/DataFormats.java index 65d5444d..13760149 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/DataFormats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/DataFormats.java @@ -3949,6 +3949,401 @@ void setBookieId(com.google.protobuf.ByteString value) { // @@protoc_insertion_point(class_scope:AuditorVoteFormat) } + public interface BookieFormatOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional int32 readPort = 1; + boolean hasReadPort(); + int getReadPort(); + + // optional int32 writePort = 2; + boolean hasWritePort(); + int getWritePort(); + } + public static final class BookieFormat extends + com.google.protobuf.GeneratedMessage + implements BookieFormatOrBuilder { + // Use BookieFormat.newBuilder() to construct. + private BookieFormat(Builder builder) { + super(builder); + } + private BookieFormat(boolean noInit) {} + + private static final BookieFormat defaultInstance; + public static BookieFormat getDefaultInstance() { + return defaultInstance; + } + + public BookieFormat getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.bookkeeper.proto.DataFormats.internal_static_BookieFormat_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.bookkeeper.proto.DataFormats.internal_static_BookieFormat_fieldAccessorTable; + } + + private int bitField0_; + // optional int32 readPort = 1; + public static final int READPORT_FIELD_NUMBER = 1; + private int readPort_; + public boolean hasReadPort() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public int getReadPort() { + return readPort_; + } + + // optional int32 writePort = 2; + public static final int WRITEPORT_FIELD_NUMBER = 2; + private int writePort_; + public boolean hasWritePort() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getWritePort() { + return writePort_; + } + + private void initFields() { + readPort_ = 0; + writePort_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeInt32(1, readPort_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeInt32(2, writePort_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(1, readPort_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeInt32Size(2, writePort_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + public static org.apache.bookkeeper.proto.DataFormats.BookieFormat parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.bookkeeper.proto.DataFormats.BookieFormat parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.bookkeeper.proto.DataFormats.BookieFormat parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.bookkeeper.proto.DataFormats.BookieFormat parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.bookkeeper.proto.DataFormats.BookieFormat parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.bookkeeper.proto.DataFormats.BookieFormat parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.bookkeeper.proto.DataFormats.BookieFormat parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.bookkeeper.proto.DataFormats.BookieFormat parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.bookkeeper.proto.DataFormats.BookieFormat parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.bookkeeper.proto.DataFormats.BookieFormat parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.bookkeeper.proto.DataFormats.BookieFormat prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.bookkeeper.proto.DataFormats.BookieFormatOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.bookkeeper.proto.DataFormats.internal_static_BookieFormat_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.bookkeeper.proto.DataFormats.internal_static_BookieFormat_fieldAccessorTable; + } + + // Construct using org.apache.bookkeeper.proto.DataFormats.BookieFormat.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + readPort_ = 0; + bitField0_ = (bitField0_ & ~0x00000001); + writePort_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.bookkeeper.proto.DataFormats.BookieFormat.getDescriptor(); + } + + public org.apache.bookkeeper.proto.DataFormats.BookieFormat getDefaultInstanceForType() { + return org.apache.bookkeeper.proto.DataFormats.BookieFormat.getDefaultInstance(); + } + + public org.apache.bookkeeper.proto.DataFormats.BookieFormat build() { + org.apache.bookkeeper.proto.DataFormats.BookieFormat result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.bookkeeper.proto.DataFormats.BookieFormat buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.bookkeeper.proto.DataFormats.BookieFormat result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.bookkeeper.proto.DataFormats.BookieFormat buildPartial() { + org.apache.bookkeeper.proto.DataFormats.BookieFormat result = new org.apache.bookkeeper.proto.DataFormats.BookieFormat(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.readPort_ = readPort_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.writePort_ = writePort_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.bookkeeper.proto.DataFormats.BookieFormat) { + return mergeFrom((org.apache.bookkeeper.proto.DataFormats.BookieFormat)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.bookkeeper.proto.DataFormats.BookieFormat other) { + if (other == org.apache.bookkeeper.proto.DataFormats.BookieFormat.getDefaultInstance()) return this; + if (other.hasReadPort()) { + setReadPort(other.getReadPort()); + } + if (other.hasWritePort()) { + setWritePort(other.getWritePort()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + readPort_ = input.readInt32(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + writePort_ = input.readInt32(); + break; + } + } + } + } + + private int bitField0_; + + // optional int32 readPort = 1; + private int readPort_ ; + public boolean hasReadPort() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public int getReadPort() { + return readPort_; + } + public Builder setReadPort(int value) { + bitField0_ |= 0x00000001; + readPort_ = value; + onChanged(); + return this; + } + public Builder clearReadPort() { + bitField0_ = (bitField0_ & ~0x00000001); + readPort_ = 0; + onChanged(); + return this; + } + + // optional int32 writePort = 2; + private int writePort_ ; + public boolean hasWritePort() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getWritePort() { + return writePort_; + } + public Builder setWritePort(int value) { + bitField0_ |= 0x00000002; + writePort_ = value; + onChanged(); + return this; + } + public Builder clearWritePort() { + bitField0_ = (bitField0_ & ~0x00000002); + writePort_ = 0; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:BookieFormat) + } + + static { + defaultInstance = new BookieFormat(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:BookieFormat) + } + private static com.google.protobuf.Descriptors.Descriptor internal_static_LedgerMetadataFormat_descriptor; private static @@ -3984,6 +4379,11 @@ void setBookieId(com.google.protobuf.ByteString value) { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_AuditorVoteFormat_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_BookieFormat_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_BookieFormat_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -4011,8 +4411,10 @@ void setBookieId(com.google.protobuf.ByteString value) { "t\022\022\n\nbookieHost\030\001 \002(\t\022\022\n\njournalDir\030\002 \002(" + "\t\022\022\n\nledgerDirs\030\003 \002(\t\022\022\n\ninstanceId\030\004 \001(" + "\t\"\"\n\016LockDataFormat\022\020\n\010bookieId\030\001 \001(\t\"%\n" + - "\021AuditorVoteFormat\022\020\n\010bookieId\030\001 \001(\tB\037\n\033" + - "org.apache.bookkeeper.protoH\001" + "\021AuditorVoteFormat\022\020\n\010bookieId\030\001 \001(\t\"3\n\014" + + "BookieFormat\022\020\n\010readPort\030\001 \001(\005\022\021\n\twriteP", + "ort\030\002 \001(\005B\037\n\033org.apache.bookkeeper.proto" + + "H\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -4075,6 +4477,14 @@ public com.google.protobuf.ExtensionRegistry assignDescriptors( new java.lang.String[] { "BookieId", }, org.apache.bookkeeper.proto.DataFormats.AuditorVoteFormat.class, org.apache.bookkeeper.proto.DataFormats.AuditorVoteFormat.Builder.class); + internal_static_BookieFormat_descriptor = + getDescriptor().getMessageTypes().get(6); + internal_static_BookieFormat_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_BookieFormat_descriptor, + new java.lang.String[] { "ReadPort", "WritePort", }, + org.apache.bookkeeper.proto.DataFormats.BookieFormat.class, + org.apache.bookkeeper.proto.DataFormats.BookieFormat.Builder.class); return null; } }; 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 920515ba..3334f969 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 @@ -21,12 +21,13 @@ package org.apache.bookkeeper.proto; import com.google.common.base.Preconditions; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.util.MathUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.InetSocketAddress; import java.util.concurrent.atomic.AtomicInteger; /** @@ -39,19 +40,20 @@ class DefaultPerChannelBookieClientPool implements PerChannelBookieClientPool, static final Logger logger = LoggerFactory.getLogger(DefaultPerChannelBookieClientPool.class); final PerChannelBookieClientFactory factory; - final BookieSocketAddress address; + final BookieId bookieId; final PerChannelBookieClient[] clients; final AtomicInteger counter = new AtomicInteger(0); DefaultPerChannelBookieClientPool(PerChannelBookieClientFactory factory, - BookieSocketAddress address, + BookieId bookieId, + InetSocketAddress socketAddress, int coreSize) { Preconditions.checkArgument(coreSize > 0); this.factory = factory; - this.address = address; + this.bookieId = bookieId; this.clients = new PerChannelBookieClient[coreSize]; for (int i = 0; i < coreSize; i++) { - this.clients[i] = factory.create(address); + this.clients[i] = factory.create(bookieId, socketAddress); } } 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 040692ef..c954bb50 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 @@ -23,7 +23,7 @@ import org.apache.bookkeeper.client.BookKeeperClientStats; import org.apache.bookkeeper.client.ReadLastConfirmedAndEntryOp; 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.proto.BookkeeperInternalCallbacks.ReadEntryCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallbackCtx; @@ -61,6 +61,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.net.InetSocketAddress; import java.nio.channels.ClosedChannelException; import java.util.ArrayDeque; import java.util.Queue; @@ -94,7 +95,8 @@ public class PerChannelBookieClient extends SimpleChannelHandler implements Chan */ private final ConcurrentMap completionObjects = new ConcurrentHashMap(); - final BookieSocketAddress addr; + final InetSocketAddress addr; + final BookieId bookieId; final ClientSocketChannelFactory channelFactory; final OrderedSafeExecutor executor; final HashedWheelTimer requestTimer; @@ -112,15 +114,18 @@ public enum ConnectionState { private final ClientConfiguration conf; public PerChannelBookieClient(OrderedSafeExecutor executor, ClientSocketChannelFactory channelFactory, - BookieSocketAddress addr) { - this(new ClientConfiguration(), executor, channelFactory, addr, null, NullStatsLogger.INSTANCE, Optional.absent()); + InetSocketAddress addr, BookieId bookieId) { + this(new ClientConfiguration(), executor, channelFactory, addr, bookieId, + null, NullStatsLogger.INSTANCE, Optional.absent()); } public PerChannelBookieClient(ClientConfiguration conf, OrderedSafeExecutor executor, - ClientSocketChannelFactory channelFactory, BookieSocketAddress addr, - HashedWheelTimer requestTimer, StatsLogger parentStatsLogger, Optional networkLocation) { + ClientSocketChannelFactory channelFactory, InetSocketAddress addr, + BookieId bookieId, HashedWheelTimer requestTimer, + StatsLogger parentStatsLogger, Optional networkLocation) { this.conf = conf; this.addr = addr; + this.bookieId = bookieId; this.executor = executor; this.channelFactory = channelFactory; this.state = ConnectionState.DISCONNECTED; @@ -173,7 +178,7 @@ private void connect() { final long connectStartNanos = MathUtils.nowInNano(); - ChannelFuture future = bootstrap.connect(addr.getSocketAddress()); + ChannelFuture future = bootstrap.connect(addr); future.addListener(new ChannelFutureListener() { @Override @@ -674,7 +679,7 @@ public void safeRun() { LOG.debug("Could not write request for adding entry: {} ledger-id: {} bookie: {}", new Object[] { ac.entryId, ac.ledgerId, bAddress }); } - ac.cb.writeComplete(rc, ac.ledgerId, ac.entryId, addr, ac.ctx); + ac.cb.writeComplete(rc, ac.ledgerId, ac.entryId, bookieId, ac.ctx); LOG.debug("Invoked callback method: {}", ac.entryId); } @@ -869,7 +874,7 @@ void handleAddResponse(AddResponse response, CompletionValue completionValue) { } rcToRet = BKException.Code.WriteException; } - ac.cb.writeComplete(rcToRet, ledgerId, entryId, addr, ac.ctx); + ac.cb.writeComplete(rcToRet, ledgerId, entryId, bookieId, ac.ctx); } void handleReadResponse(ReadResponse response, CompletionValue completionValue) { @@ -984,7 +989,7 @@ public AddCompletion(final StatsLogger statsLogger, final WriteCallback original super(BookKeeperClientStats.CHANNEL_ADD_ENTRY, originalCtx, ledgerId, entryId, timeout); this.cb = 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) { cancelTimeout(); if (rc != BKException.Code.OK) { statsLogger.getOpStatsLogger(BookKeeperClientStats.CHANNEL_ADD_ENTRY) @@ -1113,7 +1118,7 @@ private long getTxnId() { } static StatsLogger getPerChannelBookieClientStatsLogger( - String scope, ClientConfiguration conf, BookieSocketAddress addr, + String scope, ClientConfiguration conf, InetSocketAddress addr, StatsLogger parentStatsLogger, Optional networkLocation) { StatsLogger underlyingLogger = parentStatsLogger.scope(BookKeeperClientStats.CHANNEL_SCOPE); if (!"".equals(scope)) { 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 bd45e920..b72eedeb 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,9 @@ */ package org.apache.bookkeeper.proto; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; + +import java.net.InetSocketAddress; /** * Factory to create {@link org.apache.bookkeeper.proto.PerChannelBookieClient}. @@ -33,5 +35,5 @@ interface PerChannelBookieClientFactory { * * @return the client connected to address. */ - PerChannelBookieClient create(BookieSocketAddress address); + PerChannelBookieClient create(BookieId bookieId, InetSocketAddress inetSocketAddress); } 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 33306cae..3c48084d 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 @@ -21,7 +21,7 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieProtocol.AddRequest; import org.apache.bookkeeper.proto.BookieProtocol.Request; import org.apache.bookkeeper.stats.StatsLogger; @@ -67,7 +67,7 @@ public void safeRun() { 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 (rc == BookieProtocol.EOK) { statsLogger.getOpStatsLogger(ADD_ENTRY).registerSuccessfulEvent(MathUtils.elapsedMicroSec(startTimeNanos)); } else { 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 c5c52cf7..e00e5574 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 @@ -8,9 +8,8 @@ import org.apache.bookkeeper.bookie.Bookie; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.client.BKException.BKDigestMatchException; -import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.DigestManager; -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; @@ -62,7 +61,7 @@ private AddResponse getAddResponse() { BookkeeperInternalCallbacks.WriteCallback wcb = new BookkeeperInternalCallbacks.WriteCallback() { @Override public void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { if (rc == BookieProtocol.EOK) { statsLogger.getOpStatsLogger(ADD_ENTRY) .registerSuccessfulEvent(MathUtils.elapsedMicroSec(startTimeNanos)); 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 51279559..4fdb3bf7 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 @@ -48,7 +48,7 @@ import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerManager; 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.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.replication.ReplicationException.BKAuditException; @@ -91,7 +91,7 @@ public class Auditor { private final ScheduledExecutorService bookieCheckerExecutor; private final ScheduledExecutorService urLedgerCheckerExecutor; private final String bookieIdentifier; - private Map> ledgerDetails; + private Map> ledgerDetails; private Set underreplicatedLedgers; // auditor stats @@ -297,7 +297,7 @@ private void auditBookies() } // find failed bookies - Set lostBookies = findLostBookies(ledgerDetails.keySet()); + Set lostBookies = findLostBookies(ledgerDetails.keySet()); // reset the counter this.published_underreplicated_ledgers.clear(); // publish suspected ledgers if any @@ -309,28 +309,28 @@ private void auditBookies() } } - public Set findLostBookies(Set bookiesFromLedgers) throws BKException { - Set lostBookies = new HashSet(); - Set staleBookies = bcm.fetchStaleBookies(); - Set activeBookies = bcm.getActiveBookies(); + public Set findLostBookies(Set bookiesFromLedgers) throws BKException { + Set lostBookies = new HashSet(); + Set staleBookies = bcm.fetchStaleBookies(); + Set activeBookies = bcm.getActiveBookies(); lostBookies.addAll(staleBookies); lostBookies.addAll(Sets.difference(bookiesFromLedgers, activeBookies)); bcm.lostBookiesChanged(lostBookies); return lostBookies; } - private Map> generateBookie2LedgersIndex() + private Map> generateBookie2LedgersIndex() throws BKAuditException { return bookieLedgerIndexer.getBookieToLedgerIndex(); } - private void handleLostBookies(Collection lostBookies, - Map> ledgerDetails) throws BKAuditException, + private void handleLostBookies(Collection lostBookies, + Map> ledgerDetails) throws BKAuditException, InterruptedException { LOG.info("Following are the failed bookies: " + lostBookies + " and searching its ledgers for re-replication"); - for (BookieSocketAddress bookieIP : lostBookies) { + for (BookieId bookieIP : lostBookies) { // identify all the ledgers in bookieIP and publishing these ledgers // as under-replicated. publishSuspectedLedgers(bookieIP.toString(), ledgerDetails.get(bookieIP)); @@ -376,12 +376,12 @@ private class ProcessLostFragmentsCb implements GenericCallback fragments) { try { if (rc == BKException.Code.OK) { - Set bookies = Sets.newHashSet(); + Set bookies = Sets.newHashSet(); for (LedgerFragment f : fragments) { bookies.addAll(f.getAddresses()); } // TODO: publish ledger with failed bookies to reduce zookeeper accesses - for (BookieSocketAddress bookie : bookies) { + for (BookieId bookie : bookies) { publishSuspectedLedgers(bookie.toString(), Sets.newHashSet(lh.getId())); } 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 de9d3b2d..2fd5af25 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 @@ -26,9 +26,8 @@ import java.io.Serializable; import java.io.IOException; -import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.client.BookieClusterManager; -import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.DataFormats.AuditorVoteFormat; import com.google.common.annotations.VisibleForTesting; @@ -342,7 +341,7 @@ Auditor getAuditor() { * Query zookeeper for the currently elected auditor * @return the bookie id of the current auditor */ - public static BookieSocketAddress getCurrentAuditor(ServerConfiguration conf, ZooKeeper zk) + public static BookieId getCurrentAuditor(ServerConfiguration conf, ZooKeeper zk) throws KeeperException, InterruptedException, IOException { String electionRoot = conf.getZkLedgersRootPath() + '/' + BookKeeperConstants.UNDER_REPLICATION_NODE + '/' + ELECTION_ZNODE; @@ -359,7 +358,7 @@ public static BookieSocketAddress getCurrentAuditor(ServerConfiguration conf, Zo TextFormat.merge(new String(data, UTF_8), builder); AuditorVoteFormat v = builder.build(); String[] parts = v.getBookieId().split(":"); - return new BookieSocketAddress(parts[0], Integer.valueOf(parts[1])); + return new BookieId(parts[0], Integer.valueOf(parts[1])); } /** diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java index eb4a1551..e387f4dd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AutoRecoveryMain.java @@ -120,7 +120,7 @@ public void process(WatchedEvent event) { return; } auditorElector = new AuditorElector( - Bookie.getBookieAddress(conf).toString(), conf, + Bookie.getBookieAddress(conf).getBookieId().toString(), conf, zk, bcm, statsLogger.scope(AUDITOR_SCOPE)); replicationWorker = new ReplicationWorker(zk, conf, bcm, statsLogger.scope(REPLICATION_WORKER_SCOPE)); deathWatcher = new AutoRecoveryDeathWatcher(this); 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 e26b9fb7..2911991d 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 @@ -30,7 +30,7 @@ import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.LedgerMetadata; import org.apache.bookkeeper.meta.LedgerManager; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor; import org.apache.bookkeeper.replication.ReplicationException.BKAuditException; @@ -59,11 +59,11 @@ public BookieLedgerIndexer(LedgerManager ledgerManager) { * @throws BKAuditException * exception while getting bookie-ledgers */ - public Map> getBookieToLedgerIndex() + public Map> getBookieToLedgerIndex() throws BKAuditException { // bookie vs ledgers map - final ConcurrentHashMap> bookie2ledgersMap - = new ConcurrentHashMap>(); + final ConcurrentHashMap> bookie2ledgersMap + = new ConcurrentHashMap>(); final CountDownLatch ledgerCollectorLatch = new CountDownLatch(1); LOG.info("Generating bookie to ledger index ..."); @@ -81,9 +81,9 @@ public void operationComplete(final int rc, LedgerMetadata ledgerMetadata) { int rcToReturn = rc; if (rc == BKException.Code.OK) { - for (Map.Entry> ensemble : ledgerMetadata + for (Map.Entry> ensemble : ledgerMetadata .getEnsembles().entrySet()) { - for (BookieSocketAddress bookie : ensemble + for (BookieId bookie : ensemble .getValue()) { putLedger(bookie2ledgersMap, bookie, @@ -136,8 +136,8 @@ public void processResult(int rc, String s, Object obj) { return bookie2ledgersMap; } - private void putLedger(ConcurrentHashMap> bookie2ledgersMap, - BookieSocketAddress bookie, long ledgerId) { + private void putLedger(ConcurrentHashMap> bookie2ledgersMap, + BookieId bookie, long ledgerId) { Set ledgers = bookie2ledgersMap.get(bookie); // creates an empty list and add to bookie for keeping its ledgers if (ledgers == null) { 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 81f7f351..a5f20b83 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 @@ -49,7 +49,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; @@ -323,11 +323,11 @@ private boolean isLastSegmentOpenAndMissingBookies(LedgerHandle lh) throws BKExc return false; } - SortedMap> ensembles + SortedMap> ensembles = admin.getLedgerMetadata(lh).getEnsembles(); - ArrayList finalEnsemble = ensembles.get(ensembles.lastKey()); - Collection available = bcm.getAvailableBookies(); - for (BookieSocketAddress b : finalEnsemble) { + ArrayList finalEnsemble = ensembles.get(ensembles.lastKey()); + Collection available = bcm.getAvailableBookies(); + for (BookieId b : finalEnsemble) { if (!available.contains(b)) { return true; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/BookKeeperTools.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/BookKeeperTools.java index 511d702c..0c9ac69f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/BookKeeperTools.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/BookKeeperTools.java @@ -25,7 +25,7 @@ import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.client.BKException; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; @@ -67,9 +67,9 @@ public static void main(String[] args) System.err.println("BookieSrc inputted has invalid name format (host:port expected): " + args[1]); return; } - final BookieSocketAddress bookieSrc = new BookieSocketAddress(bookieSrcString[0], Integer + final BookieId bookieSrc = new BookieId(bookieSrcString[0], Integer .parseInt(bookieSrcString[1])); - BookieSocketAddress bookieDest = null; + BookieId bookieDest = null; if (args.length < 3) { String bookieDestString[] = args[2].split(":"); if (bookieDestString.length < 2) { @@ -77,7 +77,7 @@ public static void main(String[] args) + args[2]); return; } - bookieDest = new BookieSocketAddress(bookieDestString[0], Integer.parseInt(bookieDestString[1])); + bookieDest = new BookieId(bookieDestString[0], Integer.parseInt(bookieDestString[1])); } // Create the BookKeeperTools instance and perform the bookie recovery diff --git a/bookkeeper-server/src/main/proto/DataFormats.proto b/bookkeeper-server/src/main/proto/DataFormats.proto index 67623ac6..8b775ce3 100644 --- a/bookkeeper-server/src/main/proto/DataFormats.proto +++ b/bookkeeper-server/src/main/proto/DataFormats.proto @@ -81,4 +81,9 @@ message LockDataFormat { */ message AuditorVoteFormat { optional string bookieId = 1; +} + +message BookieFormat { + optional int32 writePort = 1; + optional int32 readPort = 2; } \ No newline at end of file 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 3a51d66a..d465ea0c 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 @@ -29,7 +29,7 @@ import org.apache.bookkeeper.client.BKException.BKClientClosedException; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; @@ -556,10 +556,10 @@ public void testBookKeeperAdmin() throws Exception { LedgerHandle lh3 = createLedgerWithEntries(bk, 100); lh3.close(); - BookieSocketAddress bookieToKill = getBookie(0); + BookieId bookieToKill = getBookie(0); killBookie(bookieToKill); startNewBookie(); - BookieSocketAddress newBookie = getBookie(2); + BookieId newBookie = getBookie(2); CheckerCb checkercb = new CheckerCb(); LedgerChecker lc = new LedgerChecker(bk); 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 066f0429..a548a643 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 @@ -35,7 +35,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; @@ -250,7 +250,7 @@ void metadataConflictWithRecovery(BookKeeper bkc) throws Exception { for (int i = 0; i < numEntries; i++) { lh.addEntry(data); } - BookieSocketAddress bookieToKill = lh.getLedgerMetadata().getEnsemble(numEntries - 1).get(1); + BookieId bookieToKill = lh.getLedgerMetadata().getEnsemble(numEntries - 1).get(1); killBookie(bookieToKill); startNewBookie(); for (int i = 0; i < numEntries; i++) { @@ -289,16 +289,16 @@ private void testBookieRecoveryWholeEnsemble(boolean testOnLastEnsemble) throws int numMsgs = 10; writeEntriestoLedgers(numMsgs, 0, lhs); - Set bookiesSrc = new HashSet(); + Set bookiesSrc = new HashSet(); for (ServerConfiguration conf : bsConfs) { int port = conf.getBookiePort(); - BookieSocketAddress bookie = - new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), port); + BookieId bookie = + new BookieId(InetAddress.getLocalHost().getHostAddress(), port); bookiesSrc.add(bookie); } Set confsKilled = new HashSet(); if (!testOnLastEnsemble) { - for (BookieSocketAddress bookie : bookiesSrc) { + for (BookieId bookie : bookiesSrc) { LOG.info("Killed bookie {}", bookie); confsKilled.add(killBookie(bookie)); } @@ -368,8 +368,8 @@ public void testAsyncBookieRecoveryToSpecificBookie() throws Exception { writeEntriestoLedgers(numMsgs, 10, lhs); // Call the async recover bookie method. - BookieSocketAddress bookieSrc = new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), initialPort); - BookieSocketAddress bookieDest = new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), newBookiePort); + BookieId bookieSrc = new BookieId(InetAddress.getLocalHost().getHostAddress(), initialPort); + BookieId bookieDest = new BookieId(InetAddress.getLocalHost().getHostAddress(), newBookiePort); LOG.info("Now recover the data on the killed bookie (" + bookieSrc + ") and replicate it to the new one (" + bookieDest + ")"); // Initiate the sync object @@ -384,7 +384,7 @@ public void testAsyncBookieRecoveryToSpecificBookie() throws Exception { assertTrue(bookieRecoverCb.success); } - Set bookiesSrc = new HashSet(); + Set bookiesSrc = new HashSet(); bookiesSrc.add(bookieSrc); // Verify the recovered ledger metadata verifyLedgerMetadata(lhs, bookiesSrc); @@ -425,13 +425,13 @@ public void testSyncBookieRecoveryToSpecificBookie() throws Exception { writeEntriestoLedgers(numMsgs, 10, lhs); // Call the sync recover bookie method. - BookieSocketAddress bookieSrc = new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), initialPort); - BookieSocketAddress bookieDest = new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), newBookiePort); + BookieId bookieSrc = new BookieId(InetAddress.getLocalHost().getHostAddress(), initialPort); + BookieId bookieDest = new BookieId(InetAddress.getLocalHost().getHostAddress(), newBookiePort); LOG.info("Now recover the data on the killed bookie (" + bookieSrc + ") and replicate it to the new one (" + bookieDest + ")"); bkAdmin.recoverBookieData(bookieSrc, bookieDest); - Set bookiesSrc = new HashSet(); + Set bookiesSrc = new HashSet(); bookiesSrc.add(bookieSrc); // Verify the recovered ledger metadata verifyLedgerMetadata(lhs, bookiesSrc); @@ -491,13 +491,13 @@ private void testBookieRecoveryToRandomBookies(boolean async, int numBookiesToKi // Shutdown the first bookie server LOG.info("Finished writing all ledger entries so shutdown {} bookies.", numBookiesToKill); - Set bookiesSrc = new HashSet(); + Set bookiesSrc = new HashSet(); for (int i = 0; i < numBookiesToKill; i++) { int portToKill = bsConfs.get(0).getBookiePort(); bs.get(0).shutdown(); bs.remove(0); - BookieSocketAddress bookieToKill = - new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), portToKill); + BookieId bookieToKill = + new BookieId(InetAddress.getLocalHost().getHostAddress(), portToKill); bookiesSrc.add(bookieToKill); } @@ -547,7 +547,7 @@ private static class ReplicationVerificationCallback implements ReadEntryCallbac @Override public void readEntryComplete(int rc, long ledgerId, long entryId, ChannelBuffer buffer, Object ctx) { if (LOG.isDebugEnabled()) { - BookieSocketAddress addr = (BookieSocketAddress)ctx; + BookieId addr = (BookieId)ctx; LOG.debug("Got " + rc + " for ledger " + ledgerId + " entry " + entryId + " from " + ctx); } if (rc == BKException.Code.OK) { @@ -566,21 +566,21 @@ long await() throws InterruptedException { } } - private void verifyLedgerMetadata(List lhs, Set bookiesReplaced) throws Exception { + private void verifyLedgerMetadata(List lhs, Set bookiesReplaced) throws Exception { for (LedgerHandle lh : lhs) { verifyLedgerMetadata(lh, bookiesReplaced); } } - private void verifyLedgerMetadata(LedgerHandle lh, Set bookiesReplaced) throws Exception { + private void verifyLedgerMetadata(LedgerHandle lh, Set bookiesReplaced) throws Exception { LedgerMetadata md = getLedgerMetadata(lh); boolean containReplacedBookies = false; - for (Map.Entry> e : md.getEnsembles().entrySet()) { - Set uniqueBookies = new HashSet(); + for (Map.Entry> e : md.getEnsembles().entrySet()) { + Set uniqueBookies = new HashSet(); uniqueBookies.addAll(e.getValue()); assertEquals("Duplicated bookies found in " + e.getValue(), e.getValue().size(), uniqueBookies.size()); - for (BookieSocketAddress addr : e.getValue()) { + for (BookieId addr : e.getValue()) { if (bookiesReplaced.contains(addr)) { containReplacedBookies = true; } @@ -596,7 +596,7 @@ private void verifyLedgerMetadata(LedgerHandle lh, Set book private boolean verifyFullyReplicated(LedgerHandle lh, long untilEntry) throws Exception { LedgerMetadata md = getLedgerMetadata(lh); - Map> ensembles = md.getEnsembles(); + Map> ensembles = md.getEnsembles(); HashMap ranges = new HashMap(); ArrayList keyList = Collections.list( @@ -607,7 +607,7 @@ private boolean verifyFullyReplicated(LedgerHandle lh, long untilEntry) throws E } ranges.put(keyList.get(keyList.size()-1), untilEntry); - for (Map.Entry> e : ensembles.entrySet()) { + for (Map.Entry> e : ensembles.entrySet()) { int quorum = md.getAckQuorumSize(); long startEntryId = e.getKey(); long endEntryId = ranges.get(startEntryId); @@ -616,7 +616,7 @@ private boolean verifyFullyReplicated(LedgerHandle lh, long untilEntry) throws E ReplicationVerificationCallback cb = new ReplicationVerificationCallback(numRequests); for (long i = startEntryId; i < endEntryId; i++) { - for (BookieSocketAddress addr : e.getValue()) { + for (BookieId addr : e.getValue()) { bkc.bookieClient.readEntry(addr, lh.getId(), i, cb, addr); } } @@ -675,11 +675,11 @@ private boolean findDupesInEnsembles(List lhs) throws Exception { long numDupes = 0; for (LedgerHandle lh : lhs) { LedgerMetadata md = getLedgerMetadata(lh); - for (Map.Entry> e : md.getEnsembles().entrySet()) { - HashSet set = new HashSet(); + for (Map.Entry> e : md.getEnsembles().entrySet()) { + HashSet set = new HashSet(); long fragment = e.getKey(); - for (BookieSocketAddress addr : e.getValue()) { + for (BookieId addr : e.getValue()) { if (set.contains(addr)) { LOG.error("Dupe " + addr + " found in ensemble for fragment " + fragment + " of ledger " + lh.getId() + " : " + e.getValue()); @@ -708,15 +708,15 @@ public void testBookieRecoveryOnClosedLedgers() throws Exception { closeLedgers(lhs); // Shutdown last bookie server in last ensemble - ArrayList lastEnsemble = lhs.get(0).getLedgerMetadata().getEnsembles() + ArrayList lastEnsemble = lhs.get(0).getLedgerMetadata().getEnsembles() .entrySet().iterator().next().getValue(); - BookieSocketAddress bookieToKill = lastEnsemble.get(lastEnsemble.size() - 1); + BookieId bookieToKill = lastEnsemble.get(lastEnsemble.size() - 1); killBookie(bookieToKill); // start a new bookie startNewBookie(); - BookieSocketAddress bookieDest = null; + BookieId bookieDest = null; LOG.info("Now recover the data on the killed bookie (" + bookieToKill + ") and replicate it to a random available one"); @@ -742,15 +742,15 @@ public void testBookieRecoveryOnOpenedLedgers() throws Exception { writeEntriestoLedgers(numMsgs, 0, lhs); // Shutdown the first bookie server - ArrayList lastEnsemble = lhs.get(0).getLedgerMetadata().getEnsembles() + ArrayList lastEnsemble = lhs.get(0).getLedgerMetadata().getEnsembles() .entrySet().iterator().next().getValue(); - BookieSocketAddress bookieToKill = lastEnsemble.get(lastEnsemble.size() - 1); + BookieId bookieToKill = lastEnsemble.get(lastEnsemble.size() - 1); killBookie(bookieToKill); // start a new bookie startNewBookie(); - BookieSocketAddress bookieDest = null; + BookieId bookieDest = null; LOG.info("Now recover the data on the killed bookie (" + bookieToKill + ") and replicate it to a random available one"); @@ -779,13 +779,13 @@ public void testBookieRecoveryOnInRecoveryLedger() throws Exception { writeEntriestoLedgers(numMsgs, 0, lhs); // Shutdown the first bookie server - ArrayList lastEnsemble = lhs.get(0).getLedgerMetadata().getEnsembles() + ArrayList lastEnsemble = lhs.get(0).getLedgerMetadata().getEnsembles() .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 @@ -822,7 +822,7 @@ public void testBookieRecoveryOnInRecoveryLedger() throws Exception { List newLhs = openLedgers(lhs); for (LedgerHandle newLh : newLhs) { // first ensemble should contains bookieToKill2 and not contain bookieToKill - Map.Entry> entry = + Map.Entry> entry = newLh.getLedgerMetadata().getEnsembles().entrySet().iterator().next(); assertFalse(entry.getValue().contains(bookieToKill)); assertTrue(entry.getValue().contains(bookieToKill2)); @@ -847,8 +847,8 @@ public void testAsyncBookieRecoveryToRandomBookiesNotEnoughBookies() throws Exce bs.remove(0); // Call the async recover bookie method. - BookieSocketAddress bookieSrc = new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), initialPort); - BookieSocketAddress bookieDest = null; + BookieId bookieSrc = new BookieId(InetAddress.getLocalHost().getHostAddress(), initialPort); + BookieId bookieDest = null; LOG.info("Now recover the data on the killed bookie (" + bookieSrc + ") and replicate it to a random available one"); // Initiate the sync object @@ -876,7 +876,7 @@ public void testSyncBookieRecoveryToRandomBookiesCheckForDupes() throws Exceptio // Shutdown the first bookie server LOG.info("Finished writing all ledger entries so shutdown one of the bookies."); int removeIndex = r.nextInt(bs.size()); - BookieSocketAddress bookieSrc = bs.get(removeIndex).getLocalAddress(); + BookieId bookieSrc = bs.get(removeIndex).getId(); bs.get(removeIndex).shutdown(); bs.remove(removeIndex); @@ -919,7 +919,7 @@ public void recoverWithoutPasswordInConf() throws Exception { } lh.close(); - BookieSocketAddress bookieSrc = bs.get(0).getLocalAddress(); + BookieId bookieSrc = bs.get(0).getId(); bs.get(0).shutdown(); bs.remove(0); startNewBookie(); @@ -945,7 +945,7 @@ public void recoverWithoutPasswordInConf() throws Exception { assertTrue("Should be back to fully replication", verifyFullyReplicated(lh, 100)); lh.close(); - bookieSrc = bs.get(0).getLocalAddress(); + bookieSrc = bs.get(0).getId(); bs.get(0).shutdown(); bs.remove(0); startNewBookie(); @@ -972,113 +972,113 @@ public void recoverWithoutPasswordInConf() throws Exception { * Test that when we try to recover a ledger which doesn't have * the password stored in the configuration, we don't succeed */ - @Test - public void ensurePasswordUsedForOldLedgers() throws Exception { - // stop all bookies - // and wipe the ledger layout so we can use an old client - zkUtil.getZooKeeperClient().delete("/ledgers/LAYOUT", -1); - - byte[] passwdCorrect = "AAAAAA".getBytes(); - byte[] passwdBad = "BBBBBB".getBytes(); - DigestType digestCorrect = digestType; - DigestType digestBad = digestCorrect == DigestType.MAC ? DigestType.CRC32 : DigestType.MAC; - - org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper.DigestType digestCorrect410 - = org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper.DigestType.valueOf(digestType.toString()); - - org.apache.bk_v4_1_0.bookkeeper.conf.ClientConfiguration c - = new org.apache.bk_v4_1_0.bookkeeper.conf.ClientConfiguration(); - c.setZkServers(zkUtil.getZooKeeperConnectString()) - .setLedgerManagerType( - ledgerManagerFactory.equals("org.apache.bookkeeper.meta.FlatLedgerManagerFactory") ? - "flat" : "hierarchical"); - - // create client to set up layout, close it, restart bookies, and open a new client. - // the new client is necessary to ensure that it has all the restarted bookies in the - // its available bookie list - org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper bkc41 - = new org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper(c); - bkc41.close(); - restartBookies(); - bkc41 = new org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper(c); - - org.apache.bk_v4_1_0.bookkeeper.client.LedgerHandle lh41 - = bkc41.createLedger(3, 2, digestCorrect410, passwdCorrect); - long ledgerId = lh41.getId(); - for (int i = 0; i < 100; i++) { - lh41.addEntry("foobar".getBytes()); - } - lh41.close(); - bkc41.close(); - - // Startup a new bookie server - int newBookiePort = startNewBookie(); - int removeIndex = 0; - BookieSocketAddress bookieSrc = bs.get(removeIndex).getLocalAddress(); - bs.get(removeIndex).shutdown(); - bs.remove(removeIndex); - - // Check that entries are missing - LedgerHandle lh = bkc.openLedgerNoRecovery(ledgerId, digestCorrect, passwdCorrect); - assertFalse("Should be entries missing", verifyFullyReplicated(lh, 100)); - lh.close(); - - // Try to recover with bad password in conf - // if the digest type is MAC - // for CRC32, the password is only checked - // when adding new entries, which recovery will - // never do - ClientConfiguration adminConf; - BookKeeperAdmin bka; - if (digestCorrect == DigestType.MAC) { - adminConf = new ClientConfiguration(); - adminConf.setZkServers(zkUtil.getZooKeeperConnectString()); - adminConf.setLedgerManagerFactoryClassName(ledgerManagerFactory); - adminConf.setBookieRecoveryDigestType(digestCorrect); - adminConf.setBookieRecoveryPasswd(passwdBad); - - bka = new BookKeeperAdmin(adminConf); - try { - bka.recoverBookieData(bookieSrc, null); - fail("Shouldn't be able to recover with wrong password"); - } catch (BKException bke) { - // correct behaviour - } finally { - bka.close(); - } - } - - // Try to recover with bad digest in conf - adminConf = new ClientConfiguration(); - adminConf.setZkServers(zkUtil.getZooKeeperConnectString()); - adminConf.setLedgerManagerFactoryClassName(ledgerManagerFactory); - adminConf.setBookieRecoveryDigestType(digestBad); - adminConf.setBookieRecoveryPasswd(passwdCorrect); - - bka = new BookKeeperAdmin(adminConf); - try { - bka.recoverBookieData(bookieSrc, null); - fail("Shouldn't be able to recover with wrong digest"); - } catch (BKException bke) { - // correct behaviour - } finally { - bka.close(); - } - - // Check that entries are still missing - lh = bkc.openLedgerNoRecovery(ledgerId, digestCorrect, passwdCorrect); - assertFalse("Should be entries missing", verifyFullyReplicated(lh, 100)); - lh.close(); - - adminConf.setBookieRecoveryDigestType(digestCorrect); - adminConf.setBookieRecoveryPasswd(passwdCorrect); - - bka = new BookKeeperAdmin(adminConf); - bka.recoverBookieData(bookieSrc, null); - bka.close(); - - lh = bkc.openLedgerNoRecovery(ledgerId, digestCorrect, passwdCorrect); - assertTrue("Should have recovered everything", verifyFullyReplicated(lh, 100)); - lh.close(); - } +// @Test +// public void ensurePasswordUsedForOldLedgers() throws Exception { +// // stop all bookies +// // and wipe the ledger layout so we can use an old client +// zkUtil.getZooKeeperClient().delete("/ledgers/LAYOUT", -1); +// +// byte[] passwdCorrect = "AAAAAA".getBytes(); +// byte[] passwdBad = "BBBBBB".getBytes(); +// DigestType digestCorrect = digestType; +// DigestType digestBad = digestCorrect == DigestType.MAC ? DigestType.CRC32 : DigestType.MAC; +// +// org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper.DigestType digestCorrect410 +// = org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper.DigestType.valueOf(digestType.toString()); +// +// org.apache.bk_v4_1_0.bookkeeper.conf.ClientConfiguration c +// = new org.apache.bk_v4_1_0.bookkeeper.conf.ClientConfiguration(); +// c.setZkServers(zkUtil.getZooKeeperConnectString()) +// .setLedgerManagerType( +// ledgerManagerFactory.equals("org.apache.bookkeeper.meta.FlatLedgerManagerFactory") ? +// "flat" : "hierarchical"); +// +// // create client to set up layout, close it, restart bookies, and open a new client. +// // the new client is necessary to ensure that it has all the restarted bookies in the +// // its available bookie list +// org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper bkc41 +// = new org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper(c); +// bkc41.close(); +// restartBookies(); +// bkc41 = new org.apache.bk_v4_1_0.bookkeeper.client.BookKeeper(c); +// +// org.apache.bk_v4_1_0.bookkeeper.client.LedgerHandle lh41 +// = bkc41.createLedger(3, 2, digestCorrect410, passwdCorrect); +// long ledgerId = lh41.getId(); +// for (int i = 0; i < 100; i++) { +// lh41.addEntry("foobar".getBytes()); +// } +// lh41.close(); +// bkc41.close(); +// +// // Startup a new bookie server +// int newBookiePort = startNewBookie(); +// int removeIndex = 0; +// BookieId bookieSrc = bs.get(removeIndex).getLocalAddress().getBookieId(); +// bs.get(removeIndex).shutdown(); +// bs.remove(removeIndex); +// +// // Check that entries are missing +// LedgerHandle lh = bkc.openLedgerNoRecovery(ledgerId, digestCorrect, passwdCorrect); +// assertFalse("Should be entries missing", verifyFullyReplicated(lh, 100)); +// lh.close(); +// +// // Try to recover with bad password in conf +// // if the digest type is MAC +// // for CRC32, the password is only checked +// // when adding new entries, which recovery will +// // never do +// ClientConfiguration adminConf; +// BookKeeperAdmin bka; +// if (digestCorrect == DigestType.MAC) { +// adminConf = new ClientConfiguration(); +// adminConf.setZkServers(zkUtil.getZooKeeperConnectString()); +// adminConf.setLedgerManagerFactoryClassName(ledgerManagerFactory); +// adminConf.setBookieRecoveryDigestType(digestCorrect); +// adminConf.setBookieRecoveryPasswd(passwdBad); +// +// bka = new BookKeeperAdmin(adminConf); +// try { +// bka.recoverBookieData(bookieSrc, null); +// fail("Shouldn't be able to recover with wrong password"); +// } catch (BKException bke) { +// // correct behaviour +// } finally { +// bka.close(); +// } +// } +// +// // Try to recover with bad digest in conf +// adminConf = new ClientConfiguration(); +// adminConf.setZkServers(zkUtil.getZooKeeperConnectString()); +// adminConf.setLedgerManagerFactoryClassName(ledgerManagerFactory); +// adminConf.setBookieRecoveryDigestType(digestBad); +// adminConf.setBookieRecoveryPasswd(passwdCorrect); +// +// bka = new BookKeeperAdmin(adminConf); +// try { +// bka.recoverBookieData(bookieSrc, null); +// fail("Shouldn't be able to recover with wrong digest"); +// } catch (BKException bke) { +// // correct behaviour +// } finally { +// bka.close(); +// } +// +// // Check that entries are still missing +// lh = bkc.openLedgerNoRecovery(ledgerId, digestCorrect, passwdCorrect); +// assertFalse("Should be entries missing", verifyFullyReplicated(lh, 100)); +// lh.close(); +// +// adminConf.setBookieRecoveryDigestType(digestCorrect); +// adminConf.setBookieRecoveryPasswd(passwdCorrect); +// +// bka = new BookKeeperAdmin(adminConf); +// bka.recoverBookieData(bookieSrc, null); +// bka.close(); +// +// lh = bkc.openLedgerNoRecovery(ledgerId, digestCorrect, passwdCorrect); +// assertTrue("Should have recovered everything", verifyFullyReplicated(lh, 100)); +// lh.close(); +// } } 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 4e954afb..78fabea5 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 @@ -27,7 +27,7 @@ import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.MultiLedgerManagerMultiDigestTestCase; import org.junit.Before; import org.junit.Test; @@ -107,7 +107,7 @@ public void testWithMultipleBookieFailuresInLastEnsemble() throws Exception { startNewBookie(); // Shutdown three bookies in the last ensemble and continue writing - ArrayList ensemble = lh.getLedgerMetadata() + ArrayList ensemble = lh.getLedgerMetadata() .getEnsembles().entrySet().iterator().next().getValue(); killBookie(ensemble.get(0)); killBookie(ensemble.get(1)); @@ -156,7 +156,7 @@ public void testAsyncWritesWithMultipleFailuresInLastEnsemble() startNewBookie(); // Shutdown three bookies in the last ensemble and continue writing - ArrayList ensemble = lh.getLedgerMetadata() + ArrayList ensemble = lh.getLedgerMetadata() .getEnsembles().entrySet().iterator().next().getValue(); killBookie(ensemble.get(0)); killBookie(ensemble.get(1)); 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 a65ece40..29224575 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 @@ -34,7 +34,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; @@ -140,7 +140,7 @@ private void verifyMetadataConsistency(int numEntries, LedgerHandle lh) final CountDownLatch recoverDoneLatch = new CountDownLatch(1); final CountDownLatch failedLatch = new CountDownLatch(1); // kill first bookie to replace with a unauthorize bookie - BookieSocketAddress bookie = lh.getLedgerMetadata().currentEnsemble.get(0); + BookieId bookie = lh.getLedgerMetadata().currentEnsemble.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/LedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java index 0f2ffb11..3a7998ca 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 @@ -34,7 +34,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; @@ -184,7 +184,7 @@ private void ledgerRecoveryWithSlowBookie(int ensembleSize, int writeQuorumSize, // kill first bookie server to start a fake one to simulate a slow bookie // and failed to add entry on crash // until write succeed - BookieSocketAddress host = beforelh.getLedgerMetadata().currentEnsemble.get(slowBookieIdx); + BookieId host = beforelh.getLedgerMetadata().currentEnsemble.get(slowBookieIdx); ServerConfiguration conf = killBookie(host); Bookie fakeBookie = new Bookie(conf) { @@ -258,7 +258,7 @@ public void recoveryAddEntry(ByteBuffer entry, WriteCallback cb, Object ctx, byt bs.add(startBookie(conf, deadBookie1)); // kill first bookie server - BookieSocketAddress bookie1 = lhbefore.getLedgerMetadata().currentEnsemble.get(0); + BookieId bookie1 = lhbefore.getLedgerMetadata().currentEnsemble.get(0); ServerConfiguration conf1 = killBookie(bookie1); // Try to recover and fence the ledger after killing one bookie in the @@ -273,7 +273,7 @@ public void recoveryAddEntry(ByteBuffer entry, WriteCallback cb, Object ctx, byt // restart the first server, kill the second bsConfs.add(conf1); bs.add(startBookie(conf1)); - BookieSocketAddress bookie2 = lhbefore.getLedgerMetadata().currentEnsemble.get(1); + BookieId bookie2 = lhbefore.getLedgerMetadata().currentEnsemble.get(1); ServerConfiguration conf2 = killBookie(bookie2); // using async, because this could trigger an assertion @@ -339,7 +339,7 @@ public void recoveryAddEntry(ByteBuffer entry, WriteCallback cb, Object ctx, byt bs.add(startBookie(conf, deadBookie1)); // kill first bookie server - BookieSocketAddress bookie1 = lhbefore.getLedgerMetadata().currentEnsemble.get(0); + BookieId bookie1 = lhbefore.getLedgerMetadata().currentEnsemble.get(0); ServerConfiguration conf1 = killBookie(bookie1); // Try to recover and fence the ledger after killing one bookie in the @@ -391,9 +391,9 @@ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) { fail("Failed to add " + numEntries + " to ledger handle " + lh.getId()); } // kill first 2 bookies to replace bookies - BookieSocketAddress bookie1 = lh.getLedgerMetadata().currentEnsemble.get(0); + BookieId bookie1 = lh.getLedgerMetadata().currentEnsemble.get(0); ServerConfiguration conf1 = killBookie(bookie1); - BookieSocketAddress bookie2 = lh.getLedgerMetadata().currentEnsemble.get(1); + BookieId bookie2 = lh.getLedgerMetadata().currentEnsemble.get(1); ServerConfiguration conf2 = killBookie(bookie2); // replace these two bookies @@ -569,7 +569,7 @@ public void testRecoveryOnBookieHandleNotAvailable() throws Exception { lh.bk.bookieClient.addEntry(lh.metadata.currentEnsemble.get(bid), lh.getId(), lh.ledgerKey, entryId, toSend, 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) { addSuccess.set(BKException.Code.OK == rc); addLatch.countDown(); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java index bcf7a1ee..dd320dfd 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java @@ -25,7 +25,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; @@ -357,7 +357,7 @@ public void testRecoveryOnEntryGap() throws Exception { lh.bk.bookieClient.addEntry(lh.metadata.currentEnsemble.get(0), lh.getId(), lh.ledgerKey, entryId, toSend, 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) { addSuccess.set(BKException.Code.OK == rc); addLatch.countDown(); } @@ -411,12 +411,12 @@ static final class WriteCallbackEntry { private final int rc; private final long ledgerId; private final long entryId; - private final BookieSocketAddress addr; + private final BookieId addr; private final Object ctx; WriteCallbackEntry(WriteCallback cb, int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { this.cb = cb; this.rc = rc; this.ledgerId = ledgerId; @@ -448,7 +448,7 @@ public void addEntry(ByteBuffer entry, final WriteCallback cb, Object ctx, byte[ super.addEntry(entry, new WriteCallback() { @Override public void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { if (delayAddResponse.get()) { delayQueue.add(new WriteCallbackEntry(cb, rc, ledgerId, entryId, addr, ctx)); } else { @@ -510,7 +510,7 @@ public void testRecoveryWhenClosingLedgerHandle() throws Exception { LOG.info("Create ledger {}", lh0.getId()); // 0) place the bookie with a fake bookie - BookieSocketAddress address = lh0.getLedgerMetadata().currentEnsemble.get(0); + BookieId address = lh0.getLedgerMetadata().currentEnsemble.get(0); ServerConfiguration conf = killBookie(address); conf.setSortedLedgerStorageEnabled(false); DelayResponseBookie fakeBookie = new DelayResponseBookie(conf); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java index 221288ce..175f5b64 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/SlowBookieTest.java @@ -29,7 +29,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; @@ -61,7 +61,7 @@ public void testSlowBookie() throws Exception { } final CountDownLatch b0latch = new CountDownLatch(1); final CountDownLatch b1latch = new CountDownLatch(1); - List curEns = lh.getLedgerMetadata().currentEnsemble; + List curEns = lh.getLedgerMetadata().currentEnsemble; try { sleepBookie(curEns.get(0), b0latch); for (int i = 0; i < 10; i++) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestAddEntryQuorumTimeout.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestAddEntryQuorumTimeout.java index 52212b4d..aebddf7e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestAddEntryQuorumTimeout.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestAddEntryQuorumTimeout.java @@ -23,7 +23,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.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Before; import org.junit.Test; @@ -81,7 +81,7 @@ LedgerHandle getTestLedger() throws Exception { @Test(timeout = 60000) public void testBasicTimeout() throws Exception { LedgerHandle lh = getTestLedger(); - List curEns = lh.getLedgerMetadata().currentEnsemble; + List curEns = lh.getLedgerMetadata().currentEnsemble; byte[] data = "foobar".getBytes(); lh.addEntry(data); sleepBookie(curEns.get(0), 5).await(); @@ -95,7 +95,7 @@ public void testBasicTimeout() throws Exception { @Test(timeout = 60000) public void testTimeoutWithPendingOps() throws Exception { LedgerHandle lh = getTestLedger(); - List curEns = lh.getLedgerMetadata().currentEnsemble; + List curEns = lh.getLedgerMetadata().currentEnsemble; byte[] data = "foobar".getBytes(); SyncObj syncObj1 = new SyncObj(); @@ -122,7 +122,7 @@ public void testTimeoutWithPendingOps() throws Exception { @Test(timeout = 60000) public void testLedgerClosedAfterTimeout() throws Exception { LedgerHandle lh = getTestLedger(); - List curEns = lh.getLedgerMetadata().currentEnsemble; + List curEns = lh.getLedgerMetadata().currentEnsemble; byte[] data = "foobar".getBytes(); CountDownLatch b0latch = sleepBookie(curEns.get(0), 5); try { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java index a04aea31..edde917b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDelayEnsembleChange.java @@ -23,7 +23,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.jboss.netty.buffer.ChannelBuffer; @@ -90,9 +90,9 @@ private void verifyEntries(LedgerHandle lh, long startEntry, long untilEntry, LedgerMetadata md = lh.getLedgerMetadata(); for (long eid = startEntry; eid < untilEntry; eid++) { - ArrayList addresses = md.getEnsemble(eid); + ArrayList addresses = md.getEnsemble(eid); VerificationCallback callback = new VerificationCallback(addresses.size()); - for (BookieSocketAddress addr : addresses) { + for (BookieId addr : addresses) { bkc.bookieClient.readEntry(addr, lh.getId(), eid, callback, addr); } callback.latch.await(); @@ -107,9 +107,9 @@ private void verifyEntriesRange(LedgerHandle lh, long startEntry, long untilEntr LedgerMetadata md = lh.getLedgerMetadata(); for (long eid = startEntry; eid < untilEntry; eid++) { - ArrayList addresses = md.getEnsemble(eid); + ArrayList addresses = md.getEnsemble(eid); VerificationCallback callback = new VerificationCallback(addresses.size()); - for (BookieSocketAddress addr : addresses) { + for (BookieId addr : addresses) { bkc.bookieClient.readEntry(addr, lh.getId(), eid, callback, addr); } callback.latch.await(); @@ -213,8 +213,8 @@ public void testChangeEnsembleIfBrokenAckQuorum() throws Exception { assertEquals("There should be ensemble change if ack quorum couldn't be formed.", 2, lh.getLedgerMetadata().getEnsembles().size()); - ArrayList firstFragment = lh.getLedgerMetadata().getEnsemble(0); - ArrayList secondFragment = lh.getLedgerMetadata().getEnsemble(3 * numEntries); + ArrayList firstFragment = lh.getLedgerMetadata().getEnsemble(0); + ArrayList secondFragment = lh.getLedgerMetadata().getEnsemble(3 * numEntries); assertFalse(firstFragment.get(0).equals(secondFragment.get(0))); assertFalse(firstFragment.get(1).equals(secondFragment.get(1))); assertFalse(firstFragment.get(2).equals(secondFragment.get(2))); @@ -385,8 +385,8 @@ public void testChangeEnsembleSecondBookieReadOnly() throws Exception { lh.addEntry(data); } - BookieSocketAddress failedBookie = lh.getLedgerMetadata().currentEnsemble.get(0); - BookieSocketAddress readOnlyBookie = lh.getLedgerMetadata().currentEnsemble.get(1); + BookieId failedBookie = lh.getLedgerMetadata().currentEnsemble.get(0); + BookieId readOnlyBookie = lh.getLedgerMetadata().currentEnsemble.get(1); ServerConfiguration conf0 = killBookie(failedBookie); for (int i = 0; i < numEntries; i++) { @@ -424,7 +424,7 @@ public void testMarkBookieFailures() throws Exception { int numEntries = 10; lh.addEntry(data); - BookieSocketAddress failedBookie = lh.getLedgerMetadata().currentEnsemble.get(0); + BookieId failedBookie = lh.getLedgerMetadata().currentEnsemble.get(0); ServerConfiguration conf0 = killBookie(failedBookie); for (int i = 0; i < numEntries; i++) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java index ec737599..a9a7dfea 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDisableEnsembleChange.java @@ -5,7 +5,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.feature.SettableFeature; import org.apache.bookkeeper.feature.SettableFeatureProvider; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; import org.slf4j.Logger; @@ -65,8 +65,8 @@ void disableEnsembleChangeTest(boolean startNewBookie) throws Exception { final byte[] entry = "test-disable-ensemble-change".getBytes(UTF_8); assertEquals(1, lh.getLedgerMetadata().getEnsembles().size()); - ArrayList ensembleBeforeFailure = - new ArrayList(lh.getLedgerMetadata().getEnsembles().entrySet().iterator().next().getValue()); + ArrayList ensembleBeforeFailure = + new ArrayList(lh.getLedgerMetadata().getEnsembles().entrySet().iterator().next().getValue()); final RateLimiter rateLimiter = RateLimiter.create(10); @@ -97,10 +97,10 @@ public void run() { // check the ensemble after failure assertEquals("No new ensemble should be added when disable ensemble change.", 1, lh.getLedgerMetadata().getEnsembles().size()); - ArrayList ensembleAfterFailure = - new ArrayList(lh.getLedgerMetadata().getEnsembles().entrySet().iterator().next().getValue()); - assertArrayEquals(ensembleBeforeFailure.toArray(new BookieSocketAddress[ensembleBeforeFailure.size()]), - ensembleAfterFailure.toArray(new BookieSocketAddress[ensembleAfterFailure.size()])); + ArrayList ensembleAfterFailure = + new ArrayList(lh.getLedgerMetadata().getEnsembles().entrySet().iterator().next().getValue()); + assertArrayEquals(ensembleBeforeFailure.toArray(new BookieId[ensembleBeforeFailure.size()]), + ensembleAfterFailure.toArray(new BookieId[ensembleAfterFailure.size()])); // enable ensemble change disableEnsembleChangeFeature.set(false); @@ -219,7 +219,7 @@ public void testRetrySlowBookie() throws Exception { lh.addEntry(entry); } - List curEns = lh.getLedgerMetadata().currentEnsemble; + List curEns = lh.getLedgerMetadata().currentEnsemble; final CountDownLatch wakeupLatch = new CountDownLatch(1); final CountDownLatch suspendLatch = new CountDownLatch(1); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java index af028eb6..dbdfb930 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestFencing.java @@ -27,7 +27,7 @@ import java.util.concurrent.CountDownLatch; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BaseTestCase; import org.slf4j.Logger; @@ -261,7 +261,7 @@ public void testFencingInteractionWithBookieRecovery() throws Exception { writelh.addEntry(tmp.getBytes()); } - BookieSocketAddress bookieToKill + BookieId bookieToKill = writelh.getLedgerMetadata().getEnsemble(numEntries).get(0); killBookie(bookieToKill); @@ -314,7 +314,7 @@ public void testFencingInteractionWithBookieRecovery2() throws Exception { LedgerHandle readlh = bkc.openLedger(writelh.getId(), digestType, "testPasswd".getBytes()); // should be fenced by now - BookieSocketAddress bookieToKill + BookieId bookieToKill = writelh.getLedgerMetadata().getEnsemble(numEntries).get(0); killBookie(bookieToKill); admin.recoverBookieData(bookieToKill, null); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java index 2dc4a499..d62e6a32 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerChecker.java @@ -24,7 +24,7 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; @@ -74,7 +74,7 @@ public void testChecker() throws Exception { for (int i = 0; i < 10; i++) { lh.addEntry(TEST_LEDGER_ENTRY_DATA); } - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getEnsembles() + BookieId replicaToKill = lh.getLedgerMetadata().getEnsembles() .get(0L).get(0); LOG.info("Killing {}", replicaToKill); killBookie(replicaToKill); @@ -97,7 +97,7 @@ public void testChecker() throws Exception { assertEquals("Fragment should be missing from first replica", lf.getAddress(0), replicaToKill); - BookieSocketAddress replicaToKill2 = lh.getLedgerMetadata() + BookieId replicaToKill2 = lh.getLedgerMetadata() .getEnsembles().get(0L).get(1); LOG.info("Killing {}", replicaToKill2); killBookie(replicaToKill2); @@ -143,9 +143,9 @@ public void testShouldNotGetTheFragmentIfThereIsNoMissedEntry() // Entry should have added in first 2 Bookies. // Kill the 3rd BK from ensemble. - ArrayList firstEnsemble = lh.getLedgerMetadata() + ArrayList firstEnsemble = lh.getLedgerMetadata() .getEnsembles().get(0L); - BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get(2); + BookieId lastBookieFromEnsemble = firstEnsemble.get(2); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); killBookie(lastBookieFromEnsemble); @@ -186,13 +186,13 @@ public void testShouldGetTwoFrgamentsIfTwoBookiesFailedInSameEnsemble() startNewBookie(); lh.addEntry(TEST_LEDGER_ENTRY_DATA); - ArrayList firstEnsemble = lh.getLedgerMetadata() + ArrayList firstEnsemble = lh.getLedgerMetadata() .getEnsembles().get(0L); - BookieSocketAddress firstBookieFromEnsemble = firstEnsemble.get(0); + BookieId firstBookieFromEnsemble = firstEnsemble.get(0); killBookie(firstEnsemble, firstBookieFromEnsemble); - BookieSocketAddress secondBookieFromEnsemble = firstEnsemble.get(1); + BookieId secondBookieFromEnsemble = firstEnsemble.get(1); killBookie(firstEnsemble, secondBookieFromEnsemble); lh.addEntry(TEST_LEDGER_ENTRY_DATA); Set result = getUnderReplicatedFragments(lh); @@ -219,9 +219,9 @@ public void testShouldNotGetAnyFragmentIfNoLedgerPresent() LedgerHandle lh = bkc.createLedger(3, 2, BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); - ArrayList firstEnsemble = lh.getLedgerMetadata() + ArrayList firstEnsemble = lh.getLedgerMetadata() .getEnsembles().get(0L); - BookieSocketAddress firstBookieFromEnsemble = firstEnsemble.get(0); + BookieId firstBookieFromEnsemble = firstEnsemble.get(0); killBookie(firstBookieFromEnsemble); startNewBookie(); lh.addEntry(TEST_LEDGER_ENTRY_DATA); @@ -251,9 +251,9 @@ public void testShouldGetFailedEnsembleNumberOfFgmntsIfEnsembleBookiesFailedOnNe } // Kill all three bookies - ArrayList firstEnsemble = lh.getLedgerMetadata() + ArrayList firstEnsemble = lh.getLedgerMetadata() .getEnsembles().get(0L); - for (BookieSocketAddress bkAddr : firstEnsemble) { + for (BookieId bkAddr : firstEnsemble) { killBookie(firstEnsemble, bkAddr); } @@ -295,7 +295,7 @@ public void testShouldNotGetAnyFragmentWithEmptyLedger() throws Exception { public void testShouldGet2FragmentsWithEmptyLedgerButBookiesDead() throws Exception { LedgerHandle lh = bkc.createLedger(3, 2, BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); - for (BookieSocketAddress b : lh.getLedgerMetadata().getEnsembles().get(0L)) { + for (BookieId b : lh.getLedgerMetadata().getEnsembles().get(0L)) { killBookie(b); } Set result = getUnderReplicatedFragments(lh); @@ -314,9 +314,9 @@ public void testShouldGetOneFragmentWithSingleEntryOpenedLedger() throws Excepti LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); lh.addEntry(TEST_LEDGER_ENTRY_DATA); - ArrayList firstEnsemble = lh.getLedgerMetadata() + ArrayList firstEnsemble = lh.getLedgerMetadata() .getEnsembles().get(0L); - BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get(0); + BookieId lastBookieFromEnsemble = firstEnsemble.get(0); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); killBookie(lastBookieFromEnsemble); @@ -348,9 +348,9 @@ public void testSingleEntryAfterEnsembleChange() throws Exception { for (int i = 0; i < 10; i++) { lh.addEntry(TEST_LEDGER_ENTRY_DATA); } - ArrayList firstEnsemble = lh.getLedgerMetadata() + ArrayList firstEnsemble = lh.getLedgerMetadata() .getEnsembles().get(0L); - BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get( + BookieId lastBookieFromEnsemble = firstEnsemble.get( lh.getDistributionSchedule().getWriteSet(lh.getLastAddPushed()).get(0)); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); @@ -392,11 +392,11 @@ public void testSingleEntryAfterEnsembleChange() throws Exception { public void testClosedEmptyLedger() throws Exception { LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); - ArrayList firstEnsemble = lh.getLedgerMetadata() + ArrayList firstEnsemble = lh.getLedgerMetadata() .getEnsembles().get(0L); lh.close(); - BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get(0); + BookieId lastBookieFromEnsemble = firstEnsemble.get(0); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); killBookie(lastBookieFromEnsemble); @@ -419,13 +419,13 @@ public void testClosedEmptyLedger() throws Exception { public void testClosedSingleEntryLedger() throws Exception { LedgerHandle lh = bkc.createLedger(3, 2, BookKeeper.DigestType.CRC32, TEST_LEDGER_PASSWORD); - ArrayList firstEnsemble = lh.getLedgerMetadata() + ArrayList firstEnsemble = lh.getLedgerMetadata() .getEnsembles().get(0L); lh.addEntry(TEST_LEDGER_ENTRY_DATA); lh.close(); // kill bookie 2 - BookieSocketAddress lastBookieFromEnsemble = firstEnsemble.get(2); + BookieId lastBookieFromEnsemble = firstEnsemble.get(2); LOG.info("Killing " + lastBookieFromEnsemble + " from ensemble=" + firstEnsemble); killBookie(lastBookieFromEnsemble); @@ -487,8 +487,8 @@ private Set getUnderReplicatedFragments(LedgerHandle lh) return cb.waitAndGetResult(); } - private void killBookie(ArrayList firstEnsemble, - BookieSocketAddress ensemble) throws InterruptedException { + private void killBookie(ArrayList firstEnsemble, + BookieId ensemble) throws InterruptedException { LOG.info("Killing " + ensemble + " from ensemble=" + firstEnsemble); killBookie(ensemble); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java index 43e65ea5..bedcb401 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java @@ -29,7 +29,7 @@ import java.util.concurrent.CountDownLatch; import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; @@ -82,7 +82,7 @@ public void testReplicateLFShouldCopyFailedBookieFragmentsToTargetBookie() for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getEnsembles() + BookieId replicaToKill = lh.getLedgerMetadata().getEnsembles() .get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); @@ -93,7 +93,7 @@ public void testReplicateLFShouldCopyFailedBookieFragmentsToTargetBookie() lh.addEntry(data); } - BookieSocketAddress newBkAddr = new BookieSocketAddress(InetAddress + BookieId newBkAddr = new BookieId(InetAddress .getLocalHost().getHostAddress(), startNewBookie); LOG.info("New Bookie addr :" + newBkAddr); Set result = getFragmentsToReplicate(lh); @@ -107,13 +107,13 @@ public void testReplicateLFShouldCopyFailedBookieFragmentsToTargetBookie() } // Killing all bookies except newly replicated bookie - SortedMap> allBookiesBeforeReplication = lh + SortedMap> allBookiesBeforeReplication = lh .getLedgerMetadata().getEnsembles(); - Set>> entrySet = allBookiesBeforeReplication + Set>> entrySet = allBookiesBeforeReplication .entrySet(); - for (Entry> entry : entrySet) { - ArrayList bookies = entry.getValue(); - for (BookieSocketAddress bookie : bookies) { + for (Entry> entry : entrySet) { + ArrayList bookies = entry.getValue(); + for (BookieId bookie : bookies) { if (newBkAddr.equals(bookie)) { continue; } @@ -139,7 +139,7 @@ public void testReplicateLFFailsOnlyOnLastUnClosedFragments() for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getEnsembles() + BookieId replicaToKill = lh.getLedgerMetadata().getEnsembles() .get(0L).get(0); startNewBookie(); @@ -151,14 +151,14 @@ public void testReplicateLFFailsOnlyOnLastUnClosedFragments() lh.addEntry(data); } - BookieSocketAddress replicaToKill2 = lh.getLedgerMetadata() + BookieId replicaToKill2 = lh.getLedgerMetadata() .getEnsembles().get(0L).get(1); int startNewBookie2 = startNewBookie(); LOG.info("Killing Bookie", replicaToKill2); killBookie(replicaToKill2); - BookieSocketAddress newBkAddr = new BookieSocketAddress(InetAddress + BookieId newBkAddr = new BookieId(InetAddress .getLocalHost().getHostAddress(), startNewBookie2); LOG.info("New Bookie addr :" + newBkAddr); Set result = getFragmentsToReplicate(lh); @@ -199,7 +199,7 @@ public void testReplicateLFShouldReturnFalseIfTheReplicationFails() } // Kill the first Bookie - BookieSocketAddress replicaToKill = lh.getLedgerMetadata().getEnsembles() + BookieId replicaToKill = lh.getLedgerMetadata().getEnsembles() .get(0L).get(0); killBookie(replicaToKill); LOG.info("Killed Bookie =" + replicaToKill); @@ -235,7 +235,7 @@ public void testSplitIntoSubFragmentsWithDifferentFragmentBoundaries() LedgerMetadata metadata = new LedgerMetadata(3, 3, 3, TEST_DIGEST_TYPE, TEST_PSSWD) { @Override - ArrayList getEnsemble(long entryId) { + ArrayList getEnsemble(long entryId) { return null; } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java index df18a65f..fce79458 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestParallelRead.java @@ -23,7 +23,7 @@ import org.apache.bookkeeper.client.AsyncCallback.ReadCallback; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.conf.ClientConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; import org.slf4j.Logger; @@ -173,7 +173,7 @@ public void testFailParallelReadMissingEntryImmediately() throws Exception { LedgerHandle lh = bkc.openLedger(id, digestType, passwd); - ArrayList ensemble = + ArrayList ensemble = lh.getLedgerMetadata().getEnsemble(10); CountDownLatch latch1 = new CountDownLatch(1); CountDownLatch latch2 = new CountDownLatch(1); @@ -208,7 +208,7 @@ public void testParallelReadWithFailedBookies() throws Exception { LedgerHandle lh = bkc.openLedger(id, digestType, passwd); - ArrayList ensemble = + ArrayList ensemble = lh.getLedgerMetadata().getEnsemble(5); // kill two bookies killBookie(ensemble.get(0)); @@ -249,7 +249,7 @@ public void testParallelReadFailureWithFailedBookies() throws Exception { LedgerHandle lh = bkc.openLedger(id, digestType, passwd); - ArrayList ensemble = + ArrayList ensemble = lh.getLedgerMetadata().getEnsemble(5); // kill two bookies killBookie(ensemble.get(0)); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java index cd23b9a0..2c932115 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java @@ -34,7 +34,7 @@ import junit.framework.TestCase; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.conf.ClientConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.util.StaticDNSResolver; @@ -48,10 +48,10 @@ public class TestRackawareEnsemblePlacementPolicy extends TestCase { static final Logger LOG = LoggerFactory.getLogger(TestRackawareEnsemblePlacementPolicy.class); RackawareEnsemblePlacementPolicy repp; - final ArrayList ensemble = new ArrayList(); + final ArrayList ensemble = new ArrayList(); final List writeSet = new ArrayList(); ClientConfiguration conf = new ClientConfiguration(); - BookieSocketAddress addr1, addr2, addr3, addr4; + BookieId addr1, addr2, addr3, addr4; HashedWheelTimer timer; @Override @@ -63,10 +63,10 @@ protected void setUp() throws Exception { StaticDNSResolver.addNodeToRack("localhost", NetworkTopology.DEFAULT_RACK); LOG.info("Set up static DNS Resolver."); conf.setProperty(REPP_DNS_RESOLVER_CLASS, StaticDNSResolver.class.getName()); - addr1 = new BookieSocketAddress("127.0.0.2", 3181); - addr2 = new BookieSocketAddress("127.0.0.3", 3181); - addr3 = new BookieSocketAddress("127.0.0.4", 3181); - addr4 = new BookieSocketAddress("127.0.0.5", 3181); + addr1 = new BookieId("127.0.0.2", 3181); + addr2 = new BookieId("127.0.0.3", 3181); + addr3 = new BookieId("127.0.0.4", 3181); + addr4 = new BookieId("127.0.0.5", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), NetworkTopology.DEFAULT_REGION + "/rack1"); StaticDNSResolver.addNodeToRack(addr2.getHostName(), NetworkTopology.DEFAULT_RACK); @@ -110,16 +110,16 @@ public void testNodeDown() throws Exception { repp = new RackawareEnsemblePlacementPolicy(); repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); addrs.remove(addr1); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); List expectedSet = new ArrayList(); expectedSet.add(1); expectedSet.add(2); @@ -139,18 +139,18 @@ public void testNodeReadOnly() throws Exception { repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); addrs.remove(addr1); - Set ro = new HashSet(); + Set ro = new HashSet(); ro.add(addr1); repp.onClusterChanged(addrs, ro); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); List expectedSet = new ArrayList(); expectedSet.add(1); expectedSet.add(2); @@ -170,17 +170,17 @@ public void testTwoNodesDown() throws Exception { repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); addrs.remove(addr1); addrs.remove(addr2); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); List expectedSet = new ArrayList(); expectedSet.add(2); expectedSet.add(3); @@ -200,18 +200,18 @@ public void testNodeDownAndReadOnly() throws Exception { repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); addrs.remove(addr1); addrs.remove(addr2); - Set roAddrs = new HashSet(); + Set roAddrs = new HashSet(); roAddrs.add(addr2); repp.onClusterChanged(addrs, roAddrs); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); List expectedSet = new ArrayList(); expectedSet.add(2); expectedSet.add(3); @@ -223,49 +223,49 @@ public void testNodeDownAndReadOnly() throws Exception { @Test(timeout = 60000) public void testReplaceBookieWithEnoughBookiesInSameRack() throws Exception { - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), NetworkTopology.DEFAULT_RACK); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/default-region/r2"); StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/default-region/r2"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/default-region/r3"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, new HashSet(), addr2, new HashSet()); + BookieId replacedBookie = repp.replaceBookie(1, 1, 1, new HashSet(), addr2, new HashSet()); assertEquals(addr3, replacedBookie); } @Test(timeout = 60000) public void testReplaceBookieWithEnoughBookiesInDifferentRack() throws Exception { - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), NetworkTopology.DEFAULT_RACK); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/default-region/r2"); StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/default-region/r3"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/default-region/r4"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Set excludedAddrs = new HashSet(); + Set excludedAddrs = new HashSet(); excludedAddrs.add(addr1); - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, new HashSet(), addr2, excludedAddrs); + BookieId replacedBookie = repp.replaceBookie(1, 1, 1, new HashSet(), addr2, excludedAddrs); assertFalse(addr1.equals(replacedBookie)); assertTrue(addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); @@ -273,29 +273,29 @@ public void testReplaceBookieWithEnoughBookiesInDifferentRack() throws Exception @Test(timeout = 60000) public void testReplaceBookieWithNotEnoughBookies() throws Exception { - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), NetworkTopology.DEFAULT_RACK); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/default-region/r2"); StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/default-region/r3"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/default-region/r4"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Set excludedAddrs = new HashSet(); + Set excludedAddrs = new HashSet(); excludedAddrs.add(addr1); excludedAddrs.add(addr3); excludedAddrs.add(addr4); try { - repp.replaceBookie(1, 1, 1, new HashSet(), addr2, excludedAddrs); + repp.replaceBookie(1, 1, 1, new HashSet(), addr2, excludedAddrs); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { // should throw not enou @@ -304,21 +304,21 @@ public void testReplaceBookieWithNotEnoughBookies() throws Exception { @Test(timeout = 60000) public void testNewEnsembleWithSingleRack() throws Exception { - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.6", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.7", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.8", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.9", 3181); + BookieId addr1 = new BookieId("127.0.0.6", 3181); + BookieId addr2 = new BookieId("127.0.0.7", 3181); + BookieId addr3 = new BookieId("127.0.0.8", 3181); + BookieId addr4 = new BookieId("127.0.0.9", 3181); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); try { - ArrayList ensemble = repp.newEnsemble(3, 2, 2, new HashSet()); + ArrayList ensemble = repp.newEnsemble(3, 2, 2, new HashSet()); assertEquals(0, getNumCoveredWriteQuorums(ensemble, 2)); - ArrayList ensemble2 = repp.newEnsemble(4, 2, 2, new HashSet()); + ArrayList ensemble2 = repp.newEnsemble(4, 2, 2, new HashSet()); assertEquals(0, getNumCoveredWriteQuorums(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -327,27 +327,27 @@ public void testNewEnsembleWithSingleRack() throws Exception { @Test(timeout = 60000) public void testNewEnsembleWithMultipleRacks() throws Exception { - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.1", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.4", 3181); + BookieId addr1 = new BookieId("127.0.0.1", 3181); + BookieId addr2 = new BookieId("127.0.0.2", 3181); + BookieId addr3 = new BookieId("127.0.0.3", 3181); + BookieId addr4 = new BookieId("127.0.0.4", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), NetworkTopology.DEFAULT_RACK); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/default-region/r2"); StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/default-region/r2"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/default-region/r2"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); try { - ArrayList ensemble = repp.newEnsemble(3, 2, 2, new HashSet()); + ArrayList ensemble = repp.newEnsemble(3, 2, 2, new HashSet()); int numCovered = getNumCoveredWriteQuorums(ensemble, 2); assertTrue(numCovered >= 1 && numCovered < 3); - ArrayList ensemble2 = repp.newEnsemble(4, 2, 2, new HashSet()); + ArrayList ensemble2 = repp.newEnsemble(4, 2, 2, new HashSet()); numCovered = getNumCoveredWriteQuorums(ensemble2, 2); assertTrue(numCovered >= 1 && numCovered < 3); } catch (BKNotEnoughBookiesException bnebe) { @@ -357,14 +357,14 @@ public void testNewEnsembleWithMultipleRacks() throws Exception { @Test(timeout = 60000) public void testNewEnsembleWithEnoughRacks() throws Exception { - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); - BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); - BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); - BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); - BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); + BookieId addr5 = new BookieId("127.0.0.6", 3181); + BookieId addr6 = new BookieId("127.0.0.7", 3181); + BookieId addr7 = new BookieId("127.0.0.8", 3181); + BookieId addr8 = new BookieId("127.0.0.9", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), NetworkTopology.DEFAULT_RACK); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/default-region/r2"); @@ -375,7 +375,7 @@ public void testNewEnsembleWithEnoughRacks() throws Exception { StaticDNSResolver.addNodeToRack(addr7.getHostName(), "/default-region/r3"); StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/default-region/r4"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); @@ -384,11 +384,11 @@ public void testNewEnsembleWithEnoughRacks() throws Exception { addrs.add(addr6); addrs.add(addr7); addrs.add(addr8); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); try { - ArrayList ensemble1 = repp.newEnsemble(3, 2, 2, new HashSet()); + ArrayList ensemble1 = repp.newEnsemble(3, 2, 2, new HashSet()); assertEquals(3, getNumCoveredWriteQuorums(ensemble1, 2)); - ArrayList ensemble2 = repp.newEnsemble(4, 2, 2, new HashSet()); + ArrayList ensemble2 = repp.newEnsemble(4, 2, 2, new HashSet()); assertEquals(4, getNumCoveredWriteQuorums(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -400,27 +400,27 @@ public void testNewEnsembleWithEnoughRacks() throws Exception { */ @Test(timeout = 60000) public void testRemoveBookieFromCluster() { - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), NetworkTopology.DEFAULT_RACK); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/default-region/r2"); StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/default-region/r2"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/default-region/r3"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); addrs.remove(addr1); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); } - private int getNumCoveredWriteQuorums(ArrayList ensemble, int writeQuorumSize) + private int getNumCoveredWriteQuorums(ArrayList ensemble, int writeQuorumSize) throws Exception { int ensembleSize = ensemble.size(); int numCoveredWriteQuorums = 0; @@ -428,7 +428,7 @@ private int getNumCoveredWriteQuorums(ArrayList ensemble, i Set racks = new HashSet(); for (int j = 0; j < writeQuorumSize; j++) { int bookieIdx = (i + j) % ensembleSize; - BookieSocketAddress addr = ensemble.get(bookieIdx); + BookieId addr = ensemble.get(bookieIdx); racks.add(StaticDNSResolver.getRack(addr.getHostName())); } numCoveredWriteQuorums += (racks.size() > 1 ? 1 : 0); @@ -444,14 +444,14 @@ public void testNodeWithFailures() throws Exception { repp = new RackawareEnsemblePlacementPolicy(); repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); - HashMap bookieFailures = new HashMap(); + HashMap bookieFailures = new HashMap(); bookieFailures.put(addr1, 20L); bookieFailures.put(addr2, 22L); @@ -475,26 +475,26 @@ public void testPlacementOnStabilizeNetworkTopology() throws Exception { confLocal.setNetworkTopologyStabilizePeriodSeconds(99999); repp.initialize(confLocal, Optional.absent(), timer, DISABLE_ALL, null, null); - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); // addr4 left addrs.remove(addr4); - Set deadBookies = repp.onClusterChanged(addrs, new HashSet()); + Set deadBookies = repp.onClusterChanged(addrs, new HashSet()); assertTrue(deadBookies.isEmpty()); // we will never use addr4 even it is in the stabilized network topology for (int i = 0 ; i < 5; i++) { - ArrayList ensemble = - repp.newEnsemble(3, 3, 3, new HashSet()); + ArrayList ensemble = + repp.newEnsemble(3, 3, 3, new HashSet()); assertFalse(ensemble.contains(addr4)); } // we could still use addr4 for urgent allocation if it is just bookie flapping - ArrayList ensemble = repp.newEnsemble(4, 4, 4, new HashSet()); + ArrayList ensemble = repp.newEnsemble(4, 4, 4, new HashSet()); assertTrue(ensemble.contains(addr4)); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java index fbcf586d..f74ea276 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadEntryListener.java @@ -21,7 +21,7 @@ package org.apache.bookkeeper.client; import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; @@ -220,7 +220,7 @@ private void readWithFailedBookiesTest(boolean parallelRead) throws Exception { LedgerHandle lh = bkc.openLedger(id, digestType, passwd); - ArrayList ensemble = + ArrayList ensemble = lh.getLedgerMetadata().getEnsemble(5); // kill two bookies killBookie(ensemble.get(0)); @@ -260,7 +260,7 @@ private void readFailureWithFailedBookiesTest(boolean parallelRead) throws Excep LedgerHandle lh = bkc.openLedger(id, digestType, passwd); - ArrayList ensemble = + ArrayList ensemble = lh.getLedgerMetadata().getEnsemble(5); // kill bookies killBookie(ensemble.get(0)); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java index 0ef9032d..d9ef3a63 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadTimeout.java @@ -26,7 +26,7 @@ import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import java.util.HashSet; import java.util.Set; @@ -71,12 +71,12 @@ public void testReadTimeout() throws Exception { writelh.addEntry(tmp.getBytes()); } - Set beforeSet = new HashSet(); - for (BookieSocketAddress addr : writelh.getLedgerMetadata().getEnsemble(numEntries)) { + Set beforeSet = new HashSet(); + for (BookieId addr : writelh.getLedgerMetadata().getEnsemble(numEntries)) { beforeSet.add(addr); } - final BookieSocketAddress bookieToSleep + final BookieId bookieToSleep = writelh.getLedgerMetadata().getEnsemble(numEntries).get(0); int sleeptime = baseClientConf.getReadTimeout()*3; CountDownLatch latch = sleepBookie(bookieToSleep, sleeptime); @@ -92,8 +92,8 @@ public void addComplete(int rc, LedgerHandle lh, Thread.sleep((baseClientConf.getReadTimeout()*3)*1000); Assert.assertTrue("Write request did not finish", completed.get()); - Set afterSet = new HashSet(); - for (BookieSocketAddress addr : writelh.getLedgerMetadata().getEnsemble(numEntries+1)) { + Set afterSet = new HashSet(); + for (BookieId addr : writelh.getLedgerMetadata().getEnsemble(numEntries+1)) { afterSet.add(addr); } beforeSet.removeAll(afterSet); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java index e5c23f36..4ae5b502 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java @@ -29,11 +29,10 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.conf.ClientConfiguration; -import org.apache.bookkeeper.feature.Feature; import org.apache.bookkeeper.feature.FeatureProvider; import org.apache.bookkeeper.feature.SettableFeature; import org.apache.bookkeeper.feature.SettableFeatureProvider; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.DNSToSwitchMapping; import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.util.BookKeeperConstants; @@ -55,9 +54,9 @@ public class TestRegionAwareEnsemblePlacementPolicy extends TestCase { RegionAwareEnsemblePlacementPolicy repp; final ClientConfiguration conf = new ClientConfiguration(); - final ArrayList ensemble = new ArrayList(); + final ArrayList ensemble = new ArrayList(); final List writeSet = new ArrayList(); - BookieSocketAddress addr1, addr2, addr3, addr4; + BookieId addr1, addr2, addr3, addr4; HashedWheelTimer timer; static void updateMyRack(String rack) throws Exception { @@ -75,10 +74,10 @@ protected void setUp() throws Exception { LOG.info("Set up static DNS Resolver."); conf.setProperty(REPP_DNS_RESOLVER_CLASS, StaticDNSResolver.class.getName()); - addr1 = new BookieSocketAddress("127.0.0.2", 3181); - addr2 = new BookieSocketAddress("127.0.0.3", 3181); - addr3 = new BookieSocketAddress("127.0.0.4", 3181); - addr4 = new BookieSocketAddress("127.0.0.5", 3181); + addr1 = new BookieId("127.0.0.2", 3181); + addr2 = new BookieId("127.0.0.3", 3181); + addr3 = new BookieId("127.0.0.4", 3181); + addr4 = new BookieId("127.0.0.5", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/r1/rack1"); StaticDNSResolver.addNodeToRack(addr2.getHostName(), NetworkTopology.DEFAULT_RACK); @@ -115,7 +114,7 @@ public void testNotReorderReadIfInDefaultRack() throws Exception { repp = new RegionAwareEnsemblePlacementPolicy(); repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); - List reorderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + List reorderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); assertFalse(reorderSet == writeSet); assertEquals(writeSet, reorderSet); } @@ -128,14 +127,14 @@ public void testNodeInSameRegion() throws Exception { repp = new RegionAwareEnsemblePlacementPolicy(); repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); List expectedSet = new ArrayList(); expectedSet.add(0); expectedSet.add(3); @@ -154,7 +153,7 @@ public void testNodeNotInSameRegions() throws Exception { repp = new RegionAwareEnsemblePlacementPolicy(); repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); LOG.info("reorder set : {}", reoderSet); assertFalse(reoderSet == writeSet); assertEquals(writeSet, reoderSet); @@ -169,16 +168,16 @@ public void testNodeDown() throws Exception { repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); addrs.remove(addr1); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); List expectedSet = new ArrayList(); expectedSet.add(3); expectedSet.add(1); @@ -198,18 +197,18 @@ public void testNodeReadOnly() throws Exception { repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); addrs.remove(addr1); - Set ro = new HashSet(); + Set ro = new HashSet(); ro.add(addr1); repp.onClusterChanged(addrs, ro); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); List expectedSet = new ArrayList(); expectedSet.add(3); expectedSet.add(1); @@ -229,17 +228,17 @@ public void testTwoNodesDown() throws Exception { repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); addrs.remove(addr1); addrs.remove(addr2); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); - List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + List reoderSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); List expectedSet = new ArrayList(); expectedSet.add(3); expectedSet.add(2); @@ -252,49 +251,49 @@ public void testTwoNodesDown() throws Exception { @Test(timeout = 60000) public void testReplaceBookieWithEnoughBookiesInSameRegion() throws Exception { - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), NetworkTopology.DEFAULT_RACK); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region1/r1"); StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/region1/r2"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/default-region/r3"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, new HashSet(), addr2, new HashSet()); + BookieId replacedBookie = repp.replaceBookie(1, 1, 1, new HashSet(), addr2, new HashSet()); assertEquals(addr3, replacedBookie); } @Test(timeout = 60000) public void testReplaceBookieWithEnoughBookiesInDifferentRegion() throws Exception { - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), NetworkTopology.DEFAULT_RACK); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region1/r2"); StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/region2/r3"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/region3/r4"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Set excludedAddrs = new HashSet(); + Set excludedAddrs = new HashSet(); excludedAddrs.add(addr1); - BookieSocketAddress replacedBookie = repp.replaceBookie(1, 1, 1, new HashSet(), addr2, excludedAddrs); + BookieId replacedBookie = repp.replaceBookie(1, 1, 1, new HashSet(), addr2, excludedAddrs); assertFalse(addr1.equals(replacedBookie)); assertTrue(addr3.equals(replacedBookie) || addr4.equals(replacedBookie)); @@ -302,24 +301,24 @@ public void testReplaceBookieWithEnoughBookiesInDifferentRegion() throws Excepti @Test(timeout = 60000) public void testNewEnsembleBookieWithNotEnoughBookies() throws Exception { - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), NetworkTopology.DEFAULT_RACK); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region2/r2"); StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/region3/r3"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/region4/r4"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); try { - ArrayList list = repp.newEnsemble(5, 5, 3, new HashSet()); + ArrayList list = repp.newEnsemble(5, 5, 3, new HashSet()); LOG.info("Ensemble : {}", list); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { @@ -329,29 +328,29 @@ public void testNewEnsembleBookieWithNotEnoughBookies() throws Exception { @Test(timeout = 60000) public void testReplaceBookieWithNotEnoughBookies() throws Exception { - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), NetworkTopology.DEFAULT_RACK); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region2/r2"); StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/region3/r3"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/region4/r4"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); // replace node under r2 - Set excludedAddrs = new HashSet(); + Set excludedAddrs = new HashSet(); excludedAddrs.add(addr1); excludedAddrs.add(addr3); excludedAddrs.add(addr4); try { - repp.replaceBookie(1, 1, 1, new HashSet(), addr2, excludedAddrs); + repp.replaceBookie(1, 1, 1, new HashSet(), addr2, excludedAddrs); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { // should throw not enou @@ -363,26 +362,26 @@ public void testNewEnsembleWithSingleRegion() throws Exception { repp.uninitalize(); repp = new RegionAwareEnsemblePlacementPolicy(); repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/region1/r2"); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region1/r2"); StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/region1/r2"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/region1/r2"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); try { - ArrayList ensemble = repp.newEnsemble(3, 2, 2, new HashSet()); + ArrayList ensemble = repp.newEnsemble(3, 2, 2, new HashSet()); assertEquals(0, getNumCoveredRegionsInWriteQuorum(ensemble, 2)); - ArrayList ensemble2 = repp.newEnsemble(4, 2, 2, new HashSet()); + ArrayList ensemble2 = repp.newEnsemble(4, 2, 2, new HashSet()); assertEquals(0, getNumCoveredRegionsInWriteQuorum(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -394,24 +393,24 @@ public void testNewEnsembleWithMultipleRegions() throws Exception { repp.uninitalize(); repp = new RegionAwareEnsemblePlacementPolicy(); repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), NetworkTopology.DEFAULT_RACK); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region1/r2"); StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/region1/r2"); StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/region1/r2"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); addrs.add(addr4); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); try { - ArrayList ensemble = repp.newEnsemble(3, 2, 2, new HashSet()); + ArrayList ensemble = repp.newEnsemble(3, 2, 2, new HashSet()); int numCovered = getNumCoveredRegionsInWriteQuorum(ensemble, 2); assertTrue(numCovered >= 1); assertTrue(numCovered < 3); @@ -419,7 +418,7 @@ public void testNewEnsembleWithMultipleRegions() throws Exception { fail("Should not get not enough bookies exception even there is only one rack."); } try { - ArrayList ensemble2 = repp.newEnsemble(4, 2, 2, new HashSet()); + ArrayList ensemble2 = repp.newEnsemble(4, 2, 2, new HashSet()); int numCovered = getNumCoveredRegionsInWriteQuorum(ensemble2, 2); assertTrue(numCovered >= 1 && numCovered < 3); } catch (BKNotEnoughBookiesException bnebe) { @@ -429,14 +428,14 @@ public void testNewEnsembleWithMultipleRegions() throws Exception { @Test(timeout = 60000) public void testNewEnsembleWithEnoughRegions() throws Exception { - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); - BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); - BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); - BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); - BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); + BookieId addr5 = new BookieId("127.0.0.6", 3181); + BookieId addr6 = new BookieId("127.0.0.7", 3181); + BookieId addr7 = new BookieId("127.0.0.8", 3181); + BookieId addr8 = new BookieId("127.0.0.9", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/default-region/default-rack1"); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region1/r2"); @@ -447,7 +446,7 @@ public void testNewEnsembleWithEnoughRegions() throws Exception { StaticDNSResolver.addNodeToRack(addr7.getHostName(), "/region2/r13"); StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/region3/r14"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); @@ -456,11 +455,11 @@ public void testNewEnsembleWithEnoughRegions() throws Exception { addrs.add(addr6); addrs.add(addr7); addrs.add(addr8); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); try { - ArrayList ensemble1 = repp.newEnsemble(3, 2, 2, new HashSet()); + ArrayList ensemble1 = repp.newEnsemble(3, 2, 2, new HashSet()); assertEquals(3, getNumCoveredRegionsInWriteQuorum(ensemble1, 2)); - ArrayList ensemble2 = repp.newEnsemble(4, 2, 2, new HashSet()); + ArrayList ensemble2 = repp.newEnsemble(4, 2, 2, new HashSet()); assertEquals(4, getNumCoveredRegionsInWriteQuorum(ensemble2, 2)); } catch (BKNotEnoughBookiesException bnebe) { fail("Should not get not enough bookies exception even there is only one rack."); @@ -472,16 +471,16 @@ public void testNewEnsembleWithThreeRegions() throws Exception { repp.uninitalize(); repp = new RegionAwareEnsemblePlacementPolicy(); repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); - BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); - BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); - BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); - BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); - BookieSocketAddress addr9 = new BookieSocketAddress("127.0.0.10", 3181); - BookieSocketAddress addr10 = new BookieSocketAddress("127.0.0.11", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); + BookieId addr5 = new BookieId("127.0.0.6", 3181); + BookieId addr6 = new BookieId("127.0.0.7", 3181); + BookieId addr7 = new BookieId("127.0.0.8", 3181); + BookieId addr8 = new BookieId("127.0.0.9", 3181); + BookieId addr9 = new BookieId("127.0.0.10", 3181); + BookieId addr10 = new BookieId("127.0.0.11", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/region2/r1"); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region1/r2"); @@ -494,7 +493,7 @@ public void testNewEnsembleWithThreeRegions() throws Exception { StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/region2/r23"); StaticDNSResolver.addNodeToRack(addr10.getHostName(), "/region1/r24"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); @@ -505,24 +504,24 @@ public void testNewEnsembleWithThreeRegions() throws Exception { addrs.add(addr8); addrs.add(addr9); addrs.add(addr10); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); try { - ArrayList ensemble = repp.newEnsemble(6, 6, 4, new HashSet()); + ArrayList ensemble = repp.newEnsemble(6, 6, 4, new HashSet()); assert(ensemble.contains(addr4)); assert(ensemble.contains(addr8)); assert(ensemble.size() == 6); assertEquals(3, getNumRegionsInEnsemble(ensemble)); - ensemble = repp.newEnsemble(7, 7, 4, new HashSet()); + ensemble = repp.newEnsemble(7, 7, 4, new HashSet()); assert(ensemble.contains(addr4)); assert(ensemble.contains(addr8)); assert(ensemble.size() == 7); assertEquals(3, getNumRegionsInEnsemble(ensemble)); - ensemble = repp.newEnsemble(8, 8, 5, new HashSet()); + ensemble = repp.newEnsemble(8, 8, 5, new HashSet()); assert(ensemble.contains(addr4)); assert(ensemble.contains(addr8)); assert(ensemble.size() == 8); assertEquals(3, getNumRegionsInEnsemble(ensemble)); - ensemble = repp.newEnsemble(9, 9, 5, new HashSet()); + ensemble = repp.newEnsemble(9, 9, 5, new HashSet()); assert(ensemble.contains(addr4)); assert(ensemble.contains(addr8)); assert(ensemble.size() == 9); @@ -539,16 +538,16 @@ public void testNewEnsembleWithThreeRegionsWithDisable() throws Exception { repp = new RegionAwareEnsemblePlacementPolicy(); conf.setProperty(REPP_DISALLOW_BOOKIE_PLACEMENT_IN_REGION_FEATURE_NAME, "disallowBookies"); repp.initialize(conf, Optional.absent(), timer, featureProvider, null, null); - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); - BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); - BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); - BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); - BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); - BookieSocketAddress addr9 = new BookieSocketAddress("127.0.0.10", 3181); - BookieSocketAddress addr10 = new BookieSocketAddress("127.0.0.11", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); + BookieId addr5 = new BookieId("127.0.0.6", 3181); + BookieId addr6 = new BookieId("127.0.0.7", 3181); + BookieId addr7 = new BookieId("127.0.0.8", 3181); + BookieId addr8 = new BookieId("127.0.0.9", 3181); + BookieId addr9 = new BookieId("127.0.0.10", 3181); + BookieId addr10 = new BookieId("127.0.0.11", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/region2/r1"); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region1/r2"); @@ -561,7 +560,7 @@ public void testNewEnsembleWithThreeRegionsWithDisable() throws Exception { StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/region2/r23"); StaticDNSResolver.addNodeToRack(addr10.getHostName(), "/region1/r24"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); @@ -572,10 +571,10 @@ public void testNewEnsembleWithThreeRegionsWithDisable() throws Exception { addrs.add(addr8); addrs.add(addr9); addrs.add(addr10); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); try { ((SettableFeature) featureProvider.scope("region1").getFeature("disallowBookies")).set(true); - ArrayList ensemble = repp.newEnsemble(6, 6, 4, new HashSet()); + ArrayList ensemble = repp.newEnsemble(6, 6, 4, new HashSet()); assertEquals(2, getNumRegionsInEnsemble(ensemble)); assert(ensemble.contains(addr1)); assert(ensemble.contains(addr3)); @@ -589,14 +588,14 @@ public void testNewEnsembleWithThreeRegionsWithDisable() throws Exception { } try { ((SettableFeature) featureProvider.scope("region2").getFeature("disallowBookies")).set(true); - ArrayList ensemble = repp.newEnsemble(6, 6, 4, new HashSet()); + ArrayList ensemble = repp.newEnsemble(6, 6, 4, new HashSet()); fail("Should get not enough bookies exception even there is only one region with insufficient bookies."); } catch (BKNotEnoughBookiesException bnebe) { // Expected } try { ((SettableFeature) featureProvider.scope("region2").getFeature("disallowBookies")).set(false); - ArrayList ensemble = repp.newEnsemble(6, 6, 4, new HashSet()); + ArrayList ensemble = repp.newEnsemble(6, 6, 4, new HashSet()); assert(ensemble.contains(addr1)); assert(ensemble.contains(addr3)); assert(ensemble.contains(addr4)); @@ -618,21 +617,21 @@ public void testNewEnsembleWithFiveRegions() throws Exception { conf.setProperty(REPP_REGIONS_TO_WRITE, "region1;region2;region3;region4;region5"); conf.setProperty(REPP_MINIMUM_REGIONS_FOR_DURABILITY, 5); repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); - BookieSocketAddress addr1 = new BookieSocketAddress("127.1.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.1.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.1.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.1.0.5", 3181); - BookieSocketAddress addr5 = new BookieSocketAddress("127.1.0.6", 3181); - BookieSocketAddress addr6 = new BookieSocketAddress("127.1.0.7", 3181); - BookieSocketAddress addr7 = new BookieSocketAddress("127.1.0.8", 3181); - BookieSocketAddress addr8 = new BookieSocketAddress("127.1.0.9", 3181); - BookieSocketAddress addr9 = new BookieSocketAddress("127.1.0.10", 3181); - BookieSocketAddress addr10 = new BookieSocketAddress("127.1.0.11", 3181); - BookieSocketAddress addr11 = new BookieSocketAddress("127.1.0.12", 3181); - BookieSocketAddress addr12 = new BookieSocketAddress("127.1.0.13", 3181); - BookieSocketAddress addr13 = new BookieSocketAddress("127.1.0.14", 3181); - BookieSocketAddress addr14 = new BookieSocketAddress("127.1.0.15", 3181); - BookieSocketAddress addr15 = new BookieSocketAddress("127.1.0.16", 3181); + BookieId addr1 = new BookieId("127.1.0.2", 3181); + BookieId addr2 = new BookieId("127.1.0.3", 3181); + BookieId addr3 = new BookieId("127.1.0.4", 3181); + BookieId addr4 = new BookieId("127.1.0.5", 3181); + BookieId addr5 = new BookieId("127.1.0.6", 3181); + BookieId addr6 = new BookieId("127.1.0.7", 3181); + BookieId addr7 = new BookieId("127.1.0.8", 3181); + BookieId addr8 = new BookieId("127.1.0.9", 3181); + BookieId addr9 = new BookieId("127.1.0.10", 3181); + BookieId addr10 = new BookieId("127.1.0.11", 3181); + BookieId addr11 = new BookieId("127.1.0.12", 3181); + BookieId addr12 = new BookieId("127.1.0.13", 3181); + BookieId addr13 = new BookieId("127.1.0.14", 3181); + BookieId addr14 = new BookieId("127.1.0.15", 3181); + BookieId addr15 = new BookieId("127.1.0.16", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/region1/r1"); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region1/r2"); @@ -650,7 +649,7 @@ public void testNewEnsembleWithFiveRegions() throws Exception { StaticDNSResolver.addNodeToRack(addr14.getHostName(), "/region5/r34"); StaticDNSResolver.addNodeToRack(addr15.getHostName(), "/region5/r35"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); @@ -666,10 +665,10 @@ public void testNewEnsembleWithFiveRegions() throws Exception { addrs.add(addr13); addrs.add(addr14); addrs.add(addr15); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); try { - ArrayList ensemble = repp.newEnsemble(10, 10, 10, new HashSet()); + ArrayList ensemble = repp.newEnsemble(10, 10, 10, new HashSet()); assert(ensemble.size() == 10); assertEquals(5, getNumRegionsInEnsemble(ensemble)); } catch (BKNotEnoughBookiesException bnebe) { @@ -678,9 +677,9 @@ public void testNewEnsembleWithFiveRegions() throws Exception { } try{ - Set excludedAddrs = new HashSet(); + Set excludedAddrs = new HashSet(); excludedAddrs.add(addr10); - ArrayList ensemble = repp.newEnsemble(10, 10, 10, excludedAddrs); + ArrayList ensemble = repp.newEnsemble(10, 10, 10, excludedAddrs); assert(ensemble.contains(addr11) && ensemble.contains(addr12)); assert(ensemble.size() == 10); assertEquals(5, getNumRegionsInEnsemble(ensemble)); @@ -723,15 +722,15 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole conf.setProperty(REPP_DISALLOW_BOOKIE_PLACEMENT_IN_REGION_FEATURE_NAME, "disallowBookies"); repp.initialize(conf, Optional.absent(), timer, featureProvider, null, null); - BookieSocketAddress addr1 = new BookieSocketAddress("127.1.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.1.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.1.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.1.0.5", 3181); - BookieSocketAddress addr5 = new BookieSocketAddress("127.1.0.6", 3181); - BookieSocketAddress addr6 = new BookieSocketAddress("127.1.0.7", 3181); - BookieSocketAddress addr7 = new BookieSocketAddress("127.1.0.8", 3181); - BookieSocketAddress addr8 = new BookieSocketAddress("127.1.0.9", 3181); - BookieSocketAddress addr9 = new BookieSocketAddress("127.1.0.10", 3181); + BookieId addr1 = new BookieId("127.1.0.2", 3181); + BookieId addr2 = new BookieId("127.1.0.3", 3181); + BookieId addr3 = new BookieId("127.1.0.4", 3181); + BookieId addr4 = new BookieId("127.1.0.5", 3181); + BookieId addr5 = new BookieId("127.1.0.6", 3181); + BookieId addr6 = new BookieId("127.1.0.7", 3181); + BookieId addr7 = new BookieId("127.1.0.8", 3181); + BookieId addr8 = new BookieId("127.1.0.9", 3181); + BookieId addr9 = new BookieId("127.1.0.10", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/region1/r1"); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region1/r2"); @@ -744,7 +743,7 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/region3/r23"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); @@ -754,7 +753,7 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole addrs.add(addr7); addrs.add(addr8); addrs.add(addr9); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); SettableFeature disableDurabilityFeature = (SettableFeature) featureProvider.getFeature( @@ -769,9 +768,9 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole ackQuorum = 5; } - ArrayList ensemble; + ArrayList ensemble; try { - ensemble = repp.newEnsemble(6, 6, ackQuorum, new HashSet()); + ensemble = repp.newEnsemble(6, 6, ackQuorum, new HashSet()); assert(ensemble.size() == 6); assertEquals(3, getNumRegionsInEnsemble(ensemble)); } catch (BKNotEnoughBookiesException bnebe) { @@ -782,17 +781,17 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole if (disableOneRegion) { ((SettableFeature) featureProvider.scope("region2").getFeature("disallowBookies")).set(true); - Set region2Bookies = new HashSet(); + Set region2Bookies = new HashSet(); region2Bookies.add(addr4); region2Bookies.add(addr5); region2Bookies.add(addr6); - Set region1And3Bookies = new HashSet(addrs); + Set region1And3Bookies = new HashSet(addrs); region1And3Bookies.removeAll(region2Bookies); - Set excludedAddrs = new HashSet(); - for(BookieSocketAddress addr: region2Bookies) { + Set excludedAddrs = new HashSet(); + for(BookieId addr: region2Bookies) { if (ensemble.contains(addr)) { - BookieSocketAddress replacedBookie = repp.replaceBookie(6, 6, ackQuorum, ensemble, addr, excludedAddrs); + BookieId replacedBookie = repp.replaceBookie(6, 6, ackQuorum, ensemble, addr, excludedAddrs); ensemble.remove(addr); ensemble.add(replacedBookie); } @@ -800,8 +799,8 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole assertEquals(2, getNumRegionsInEnsemble(ensemble)); assertTrue(ensemble.containsAll(region1And3Bookies)); } else { - BookieSocketAddress bookieToReplace; - BookieSocketAddress replacedBookieExpected; + BookieId bookieToReplace; + BookieId replacedBookieExpected; if (ensemble.contains(addr4)) { bookieToReplace = addr4; if (ensemble.contains(addr5)) { @@ -813,10 +812,10 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole replacedBookieExpected = addr4; bookieToReplace = addr5; } - Set excludedAddrs = new HashSet(); + Set excludedAddrs = new HashSet(); try { - BookieSocketAddress replacedBookie = repp.replaceBookie(6, 6, ackQuorum, ensemble, bookieToReplace, excludedAddrs); + BookieId replacedBookie = repp.replaceBookie(6, 6, ackQuorum, ensemble, bookieToReplace, excludedAddrs); assert (replacedBookie.equals(replacedBookieExpected)); assertEquals(3, getNumRegionsInEnsemble(ensemble)); } catch (BKNotEnoughBookiesException bnebe) { @@ -825,7 +824,7 @@ public void testEnsembleWithThreeRegionsReplaceInternal(int minDurability, boole excludedAddrs.add(replacedBookieExpected); try { - BookieSocketAddress replacedBookie = repp.replaceBookie(6, 6, ackQuorum, ensemble, bookieToReplace, excludedAddrs); + BookieId replacedBookie = repp.replaceBookie(6, 6, ackQuorum, ensemble, bookieToReplace, excludedAddrs); if (minDurability > 1 && !disableDurabilityFeature.isAvailable()) { fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } @@ -860,15 +859,15 @@ public void testEnsembleDurabilityDisabledInternal(int minDurability, boolean di } repp.initialize(conf, Optional.absent(), timer, featureProvider, null, null); - BookieSocketAddress addr1 = new BookieSocketAddress("127.1.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.1.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.1.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.1.0.5", 3181); - BookieSocketAddress addr5 = new BookieSocketAddress("127.1.0.6", 3181); - BookieSocketAddress addr6 = new BookieSocketAddress("127.1.0.7", 3181); - BookieSocketAddress addr7 = new BookieSocketAddress("127.1.0.8", 3181); - BookieSocketAddress addr8 = new BookieSocketAddress("127.1.0.9", 3181); - BookieSocketAddress addr9 = new BookieSocketAddress("127.1.0.10", 3181); + BookieId addr1 = new BookieId("127.1.0.2", 3181); + BookieId addr2 = new BookieId("127.1.0.3", 3181); + BookieId addr3 = new BookieId("127.1.0.4", 3181); + BookieId addr4 = new BookieId("127.1.0.5", 3181); + BookieId addr5 = new BookieId("127.1.0.6", 3181); + BookieId addr6 = new BookieId("127.1.0.7", 3181); + BookieId addr7 = new BookieId("127.1.0.8", 3181); + BookieId addr8 = new BookieId("127.1.0.9", 3181); + BookieId addr9 = new BookieId("127.1.0.10", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/region1/r1"); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region1/r2"); @@ -881,7 +880,7 @@ public void testEnsembleDurabilityDisabledInternal(int minDurability, boolean di StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/region1/r23"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); @@ -891,16 +890,16 @@ public void testEnsembleDurabilityDisabledInternal(int minDurability, boolean di addrs.add(addr7); addrs.add(addr8); addrs.add(addr9); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); if (disableDurability) { ((SettableFeature) featureProvider.getFeature(BookKeeperConstants.FEATURE_REPP_DISABLE_DURABILITY_ENFORCEMENT)) .set(true); } - ArrayList ensemble; + ArrayList ensemble; try { - ensemble = repp.newEnsemble(6, 6, 4, new HashSet()); + ensemble = repp.newEnsemble(6, 6, 4, new HashSet()); assert(ensemble.size() == 6); } catch (BKNotEnoughBookiesException bnebe) { LOG.error("BKNotEnoughBookiesException", bnebe); @@ -908,7 +907,7 @@ public void testEnsembleDurabilityDisabledInternal(int minDurability, boolean di throw bnebe; } - Set excludedAddrs = new HashSet(); + Set excludedAddrs = new HashSet(); try{ repp.replaceBookie(6, 6, 4, ensemble, addr4, excludedAddrs); @@ -925,16 +924,16 @@ public void testNewEnsembleFailWithFiveRegions() throws Exception { conf.setProperty(REPP_MINIMUM_REGIONS_FOR_DURABILITY, 5); conf.setProperty(REPP_ENABLE_VALIDATION, false); repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); - BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); - BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); - BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); - BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); - BookieSocketAddress addr9 = new BookieSocketAddress("127.0.0.10", 3181); - BookieSocketAddress addr10 = new BookieSocketAddress("127.0.0.11", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); + BookieId addr5 = new BookieId("127.0.0.6", 3181); + BookieId addr6 = new BookieId("127.0.0.7", 3181); + BookieId addr7 = new BookieId("127.0.0.8", 3181); + BookieId addr8 = new BookieId("127.0.0.9", 3181); + BookieId addr9 = new BookieId("127.0.0.10", 3181); + BookieId addr10 = new BookieId("127.0.0.11", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/region1/r1"); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region1/r2"); @@ -947,7 +946,7 @@ public void testNewEnsembleFailWithFiveRegions() throws Exception { StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/region5/r23"); StaticDNSResolver.addNodeToRack(addr10.getHostName(), "/region5/r24"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); @@ -958,13 +957,13 @@ public void testNewEnsembleFailWithFiveRegions() throws Exception { addrs.add(addr8); addrs.add(addr9); addrs.add(addr10); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); - Set excludedAddrs = new HashSet(); + Set excludedAddrs = new HashSet(); excludedAddrs.add(addr10); excludedAddrs.add(addr9); try { - ArrayList list = repp.newEnsemble(5, 5, 5, excludedAddrs); + ArrayList list = repp.newEnsemble(5, 5, 5, excludedAddrs); LOG.info("Ensemble : {}", list); fail("Should throw BKNotEnoughBookiesException when there is not enough bookies"); } catch (BKNotEnoughBookiesException bnebe) { @@ -979,15 +978,15 @@ private void prepareNetworkTopologyForReorderTests(String myRegion) throws Excep repp = new RegionAwareEnsemblePlacementPolicy(); repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); - BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181); - BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181); - BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181); - BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181); - BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); - BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); - BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); - BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); - BookieSocketAddress addr9 = new BookieSocketAddress("127.0.0.10", 3181); + BookieId addr1 = new BookieId("127.0.0.2", 3181); + BookieId addr2 = new BookieId("127.0.0.3", 3181); + BookieId addr3 = new BookieId("127.0.0.4", 3181); + BookieId addr4 = new BookieId("127.0.0.5", 3181); + BookieId addr5 = new BookieId("127.0.0.6", 3181); + BookieId addr6 = new BookieId("127.0.0.7", 3181); + BookieId addr7 = new BookieId("127.0.0.8", 3181); + BookieId addr8 = new BookieId("127.0.0.9", 3181); + BookieId addr9 = new BookieId("127.0.0.10", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/region1/r1"); StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region1/r2"); @@ -999,7 +998,7 @@ private void prepareNetworkTopologyForReorderTests(String myRegion) throws Excep StaticDNSResolver.addNodeToRack(addr8.getHostName(), "/region3/r2"); StaticDNSResolver.addNodeToRack(addr9.getHostName(), "/region3/r3"); // Update cluster - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); @@ -1009,7 +1008,7 @@ private void prepareNetworkTopologyForReorderTests(String myRegion) throws Excep addrs.add(addr7); addrs.add(addr8); addrs.add(addr9); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); } @Test(timeout = 60000) @@ -1025,7 +1024,7 @@ public void testBasicReorderReadLACSequenceWithLocalRegion() throws Exception { private void basicReorderReadSequenceWithLocalRegionTest(String myRegion, boolean isReadLAC) throws Exception { prepareNetworkTopologyForReorderTests(myRegion); - ArrayList ensemble = repp.newEnsemble(9, 9, 5, new HashSet()); + ArrayList ensemble = repp.newEnsemble(9, 9, 5, new HashSet()); assertEquals(9, getNumCoveredRegionsInWriteQuorum(ensemble, 9)); DistributionSchedule ds = new RoundRobinDistributionSchedule(9, 9, 9); @@ -1037,9 +1036,9 @@ private void basicReorderReadSequenceWithLocalRegionTest(String myRegion, boolea List writeSet = ds.getWriteSet(i); List readSet; if (isReadLAC) { - readSet = repp.reorderReadLACSequence(ensemble, writeSet, new HashMap()); + readSet = repp.reorderReadLACSequence(ensemble, writeSet, new HashMap()); } else { - readSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + readSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); } LOG.info("Reorder {} => {}.", writeSet, readSet); @@ -1047,17 +1046,17 @@ private void basicReorderReadSequenceWithLocalRegionTest(String myRegion, boolea // first few nodes less than REMOTE_NODE_IN_REORDER_SEQUENCE should be local region int k = 0; for (; k < RegionAwareEnsemblePlacementPolicy.REMOTE_NODE_IN_REORDER_SEQUENCE; k++) { - BookieSocketAddress address = ensemble.get(readSet.get(k)); + BookieId address = ensemble.get(readSet.get(k)); assertEquals(myRegion, StaticDNSResolver.getRegion(address.getHostName())); } - BookieSocketAddress remoteAddress = ensemble.get(readSet.get(k)); + BookieId remoteAddress = ensemble.get(readSet.get(k)); assertFalse(myRegion.equals(StaticDNSResolver.getRegion(remoteAddress.getHostName()))); k++; - BookieSocketAddress localAddress = ensemble.get(readSet.get(k)); + BookieId localAddress = ensemble.get(readSet.get(k)); assertEquals(myRegion, StaticDNSResolver.getRegion(localAddress.getHostName())); k++; for (; k < ensembleSize; k++) { - BookieSocketAddress address = ensemble.get(readSet.get(k)); + BookieId address = ensemble.get(readSet.get(k)); assertFalse(myRegion.equals(StaticDNSResolver.getRegion(address.getHostName()))); } } @@ -1076,7 +1075,7 @@ public void testBasicReorderReadLACSequenceWithRemoteRegion() throws Exception { private void basicReorderReadSequenceWithRemoteRegionTest(String myRegion, boolean isReadLAC) throws Exception { prepareNetworkTopologyForReorderTests(myRegion); - ArrayList ensemble = repp.newEnsemble(9, 9, 5, new HashSet()); + ArrayList ensemble = repp.newEnsemble(9, 9, 5, new HashSet()); assertEquals(9, getNumCoveredRegionsInWriteQuorum(ensemble, 9)); DistributionSchedule ds = new RoundRobinDistributionSchedule(9, 9, 9); @@ -1089,9 +1088,9 @@ private void basicReorderReadSequenceWithRemoteRegionTest(String myRegion, boole List readSet; if (isReadLAC) { - readSet = repp.reorderReadLACSequence(ensemble, writeSet, new HashMap()); + readSet = repp.reorderReadLACSequence(ensemble, writeSet, new HashMap()); } else { - readSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + readSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); } assertEquals(writeSet, readSet); @@ -1108,9 +1107,9 @@ public void testReorderReadLACSequenceWithUnavailableOrReadOnlyBookies() throws reorderReadSequenceWithUnavailableOrReadOnlyBookiesTest(true); } - static Set getBookiesForRegion(ArrayList ensemble, String region) { - Set regionBookies = new HashSet(); - for (BookieSocketAddress address : ensemble) { + static Set getBookiesForRegion(ArrayList ensemble, String region) { + Set regionBookies = new HashSet(); + for (BookieId address : ensemble) { String r = StaticDNSResolver.getRegion(address.getHostName()); if (r.equals(region)) { regionBookies.add(address); @@ -1119,7 +1118,7 @@ static Set getBookiesForRegion(ArrayList ensemble, + static void appendBookieIndexByRegion(ArrayList ensemble, List writeSet, String region, List finalSet) { @@ -1139,15 +1138,15 @@ private void reorderReadSequenceWithUnavailableOrReadOnlyBookiesTest(boolean isR prepareNetworkTopologyForReorderTests(myRegion); - ArrayList ensemble = repp.newEnsemble(9, 9, 5, new HashSet()); + ArrayList ensemble = repp.newEnsemble(9, 9, 5, new HashSet()); assertEquals(9, getNumCoveredRegionsInWriteQuorum(ensemble, 9)); DistributionSchedule ds = new RoundRobinDistributionSchedule(9, 9, 9); LOG.info("My region is {}, ensemble : {}", repp.myRegion, ensemble); - Set readOnlyBookies = getBookiesForRegion(ensemble, readOnlyRegion); - Set writeBookies = getBookiesForRegion(ensemble, writeRegion); + Set readOnlyBookies = getBookiesForRegion(ensemble, readOnlyRegion); + Set writeBookies = getBookiesForRegion(ensemble, writeRegion); repp.onClusterChanged(writeBookies, readOnlyBookies); @@ -1158,9 +1157,9 @@ private void reorderReadSequenceWithUnavailableOrReadOnlyBookiesTest(boolean isR List writeSet = ds.getWriteSet(i); List readSet; if (isReadLAC) { - readSet = repp.reorderReadLACSequence(ensemble, writeSet, new HashMap()); + readSet = repp.reorderReadLACSequence(ensemble, writeSet, new HashMap()); } else { - readSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); + readSet = repp.reorderReadSequence(ensemble, writeSet, new HashMap()); } LOG.info("Reorder {} => {}.", writeSet, readSet); @@ -1177,15 +1176,15 @@ private void reorderReadSequenceWithUnavailableOrReadOnlyBookiesTest(boolean isR } } - private int getNumRegionsInEnsemble(ArrayList ensemble) { + private int getNumRegionsInEnsemble(ArrayList ensemble) { Set regions = new HashSet(); - for(BookieSocketAddress addr: ensemble) { + for(BookieId addr: ensemble) { regions.add(StaticDNSResolver.getRegion(addr.getHostName())); } return regions.size(); } - private int getNumCoveredRegionsInWriteQuorum(ArrayList ensemble, int writeQuorumSize) + private int getNumCoveredRegionsInWriteQuorum(ArrayList ensemble, int writeQuorumSize) throws Exception { int ensembleSize = ensemble.size(); int numCoveredWriteQuorums = 0; @@ -1193,7 +1192,7 @@ private int getNumCoveredRegionsInWriteQuorum(ArrayList ens Set regions = new HashSet(); for (int j = 0; j < writeQuorumSize; j++) { int bookieIdx = (i + j) % ensembleSize; - BookieSocketAddress addr = ensemble.get(bookieIdx); + BookieId addr = ensemble.get(bookieIdx); regions.add(StaticDNSResolver.getRegion(addr.getHostName())); } numCoveredWriteQuorums += (regions.size() > 1 ? 1 : 0); @@ -1209,10 +1208,10 @@ public void testNodeWithFailures() throws Exception { repp = new RegionAwareEnsemblePlacementPolicy(); repp.initialize(conf, Optional.absent(), timer, DISABLE_ALL, null, null); - BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181); - BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181); - BookieSocketAddress addr7 = new BookieSocketAddress("127.0.0.8", 3181); - BookieSocketAddress addr8 = new BookieSocketAddress("127.0.0.9", 3181); + BookieId addr5 = new BookieId("127.0.0.6", 3181); + BookieId addr6 = new BookieId("127.0.0.7", 3181); + BookieId addr7 = new BookieId("127.0.0.8", 3181); + BookieId addr8 = new BookieId("127.0.0.9", 3181); // update dns mapping StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/r2/rack1"); StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/r2/rack2"); @@ -1229,7 +1228,7 @@ public void testNodeWithFailures() throws Exception { writeSet.add(i); } - Set addrs = new HashSet(); + Set addrs = new HashSet(); addrs.add(addr1); addrs.add(addr2); addrs.add(addr3); @@ -1238,9 +1237,9 @@ public void testNodeWithFailures() throws Exception { addrs.add(addr6); addrs.add(addr7); addrs.add(addr8); - repp.onClusterChanged(addrs, new HashSet()); + repp.onClusterChanged(addrs, new HashSet()); - HashMap bookieFailures = new HashMap(); + HashMap bookieFailures = new HashMap(); bookieFailures.put(addr1, 20L); bookieFailures.put(addr2, 22L); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSeparatePorts.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSeparatePorts.java new file mode 100644 index 00000000..01177f40 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSeparatePorts.java @@ -0,0 +1,55 @@ +package org.apache.bookkeeper.client; + +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.test.PortManager; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Enumeration; + +public class TestSeparatePorts extends BookKeeperClusterTestCase { + private ServerConfiguration conf; + public TestSeparatePorts() { + super(1); + } + + @Override + public ServerConfiguration newServerConfiguration() throws IOException { + this.conf = super.newServerConfiguration().setAlternateBookiePort(PortManager.nextFreePort()) + .setAlternateBookiePortEnabled(true); + return this.conf; + } + + @Test + public void testUsesSeparatePorts() throws Exception { + // sanity check that read and write are separate ports + assertTrue(conf.isAlternateBookiePortEnabled()); + assertFalse(conf.getBookiePort() == conf.getAlternateBookiePort()); + + // test server + BookieSocketAddress serverAddress = getBookieAddress(0); + assertEquals(conf.getBookiePort(), serverAddress.getWriteAddress().getPort()); + assertEquals(conf.getAlternateBookiePort(), serverAddress.getReadAddress().getPort()); + + // test client + BookieSocketAddress clientAddress = bkc.bookieWatcher.getBookieAddress(getBookie(0)); + assertEquals(conf.getBookiePort(), clientAddress.getWriteAddress().getPort()); + assertEquals(conf.getAlternateBookiePort(), clientAddress.getReadAddress().getPort()); + } + + @Test + public void testWriteAndReadSeparatePorts() throws Exception { + LedgerHandle lh = bkc.createLedger(1, 1, BookKeeper.DigestType.CRC32, new byte[]{}); + + byte[] data = new byte[] {1, 2, 3}; + long entryId = lh.addEntry(data); + assertEquals(entryId, lh.getLastAddConfirmed()); + Enumeration ledgerEntryEnumeration = lh.readEntries(entryId, entryId); + LedgerEntry entry = ledgerEntryEnumeration.nextElement(); + assertEquals(entryId, entry.entryId); + Assert.assertArrayEquals(data, entry.getEntry()); + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSequenceRead.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSequenceRead.java index fa5358f7..fe5d71ec 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSequenceRead.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSequenceRead.java @@ -2,7 +2,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.conf.ClientConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; @@ -34,12 +34,12 @@ public TestSequenceRead() { private LedgerHandle createLedgerWithDuplicatedBookies() throws Exception { final LedgerHandle lh = bkc.createLedger(3, 3, 3, digestType, passwd); // introduce duplicated bookies in an ensemble. - SortedMap> ensembles = lh.getLedgerMetadata().getEnsembles(); - SortedMap> newEnsembles = new TreeMap>(); - for (Map.Entry> entry : ensembles.entrySet()) { - ArrayList newList = new ArrayList(entry.getValue().size()); - BookieSocketAddress firstBookie = entry.getValue().get(0); - for (BookieSocketAddress ignored : entry.getValue()) { + SortedMap> ensembles = lh.getLedgerMetadata().getEnsembles(); + SortedMap> newEnsembles = new TreeMap>(); + for (Map.Entry> entry : ensembles.entrySet()) { + ArrayList newList = new ArrayList(entry.getValue().size()); + BookieId firstBookie = entry.getValue().get(0); + for (BookieId ignored : entry.getValue()) { newList.add(firstBookie); } newEnsembles.put(entry.getKey(), newList); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java index abb8d50f..3489df0e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestSpeculativeRead.java @@ -29,7 +29,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.client.AsyncCallback.ReadCallback; import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BaseTestCase; import org.junit.Test; @@ -162,7 +162,7 @@ public void testSpeculativeRead() throws Exception { // sleep second bookie CountDownLatch sleepLatch = new CountDownLatch(1); - BookieSocketAddress second = lnospec.getLedgerMetadata().getEnsembles().get(0L).get(1); + BookieId second = lnospec.getLedgerMetadata().getEnsembles().get(0L).get(1); sleepBookie(second, sleepLatch); try { @@ -317,7 +317,7 @@ public void testSpeculativeReadScheduling() throws Exception { LedgerHandle l = bkspec.openLedger(id, digestType, passwd); - ArrayList ensemble = l.getLedgerMetadata().getEnsembles().get(0L); + ArrayList ensemble = l.getLedgerMetadata().getEnsembles().get(0L); BitSet allHosts = new BitSet(ensemble.size()); for (int i = 0; i < ensemble.size(); i++) { allHosts.set(i, true); @@ -567,7 +567,7 @@ public void testSpeculativeReadLastEntryAndOpScheduling() throws Exception { LedgerHandle l = bkspec.openLedger(id, digestType, passwd); - ArrayList ensemble = l.getLedgerMetadata().getEnsembles().get(0L); + ArrayList ensemble = l.getLedgerMetadata().getEnsembles().get(0L); BitSet allHosts = new BitSet(ensemble.size()); for (int i = 0; i < ensemble.size(); i++) { allHosts.set(i, true); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java index fc6aaa73..bc94768c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestWatchEnsembleChange.java @@ -21,7 +21,7 @@ package org.apache.bookkeeper.client; import org.apache.bookkeeper.client.BookKeeper.DigestType; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.junit.Test; import org.slf4j.Logger; @@ -52,9 +52,9 @@ public void testWatchEnsembleChange() throws Exception { LedgerHandle readLh = bkc.openLedgerNoRecovery(lh.getId(), digestType, "".getBytes()); long lastLAC = readLh.getLastAddConfirmed(); assertEquals(numEntries - 2, lastLAC); - ArrayList ensemble = + ArrayList ensemble = lh.getLedgerMetadata().currentEnsemble; - for (BookieSocketAddress addr : ensemble) { + for (BookieId addr : ensemble) { killBookie(addr); } // write another batch of entries, which will trigger ensemble change diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java index 0b8071b5..e2ec21fa 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java @@ -21,6 +21,7 @@ * */ +import java.net.InetSocketAddress; import java.util.Arrays; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -35,6 +36,7 @@ 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.proto.PerChannelBookieClient.ConnectionState; @@ -77,9 +79,10 @@ public void testConnectCloseRace() { Executors.newCachedThreadPool()); OrderedSafeExecutor executor = new OrderedSafeExecutor(1); - BookieSocketAddress addr = getBookie(0); + BookieId addr = getBookie(0); for (int i = 0; i < 1000; i++) { - PerChannelBookieClient client = new PerChannelBookieClient(executor, channelFactory, addr); + PerChannelBookieClient client = new PerChannelBookieClient(executor, channelFactory, + new InetSocketAddress(addr.getHostName(), addr.getPort()), addr); client.connectIfNeededAndDoOp(new GenericCallback() { @Override public void operationComplete(int rc, PerChannelBookieClient client) { @@ -106,9 +109,10 @@ public void operationComplete(int rc, PerChannelBookieClient client) { = new NioClientSocketChannelFactory(Executors.newCachedThreadPool(), Executors.newCachedThreadPool()); OrderedSafeExecutor executor = new OrderedSafeExecutor(1); - BookieSocketAddress addr = getBookie(0); + BookieId addr = getBookie(0); - final PerChannelBookieClient client = new PerChannelBookieClient(executor, channelFactory, addr); + final PerChannelBookieClient client = new PerChannelBookieClient(executor, channelFactory, + new InetSocketAddress(addr.getHostName(), addr.getPort()), addr); final AtomicBoolean inconsistent = new AtomicBoolean(false); final AtomicBoolean running = new AtomicBoolean(true); Thread connectThread = new Thread() { @@ -173,7 +177,7 @@ public void run() { private static class SimpleWriteCallback implements WriteCallback { CountDownLatch done = new CountDownLatch(1); AtomicInteger rcHolder = new AtomicInteger(-1); - 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) { rcHolder.set(rc); done.countDown(); } @@ -210,9 +214,10 @@ public void testAsyncWriteToChannel() throws Exception { final ChannelBuffer bb = getDataAndDigest(1, 1); final SimpleWriteCallback wrcb = new SimpleWriteCallback(); - BookieSocketAddress addr = getBookie(0); + BookieId addr = getBookie(0); PerChannelBookieClient client = new PerChannelBookieClient( - conf, executor, channelFactory, addr, null, NullStatsLogger.INSTANCE, Optional.absent()); + conf, executor, channelFactory, new InetSocketAddress(addr.getHostName(), addr.getPort()), addr, + null, NullStatsLogger.INSTANCE, Optional.absent()); client.connectIfNeededAndDoOp(new GenericCallback() { @Override public void operationComplete(int rc, PerChannelBookieClient client) { @@ -240,7 +245,7 @@ public void testServerDigestWithGoodAndBadWrites() throws Exception { assertEquals(0, rc); } finally { if (null != bookie) { - killBookie(bookie.getLocalAddress()); + killBookie(bookie.getId()); } } } @@ -258,7 +263,7 @@ public void testServerDigestDisabledWithGoodAndBadWrites() throws Exception { assertEquals(0, rc); } finally { if (null != bookie) { - killBookie(bookie.getLocalAddress()); + killBookie(bookie.getId()); } } } @@ -284,7 +289,7 @@ public int issueWriteWithOptionalCorruption(BookieServer bookie, boolean injectC BookieSocketAddress addr = bookie.getLocalAddress(); PerChannelBookieClient client = new PerChannelBookieClient( - conf, executor, channelFactory, addr, null, NullStatsLogger.INSTANCE, Optional.absent()); + conf, executor, channelFactory, addr.getWriteAddress(), bookie.getId(), null, NullStatsLogger.INSTANCE, Optional.absent()); client.connectIfNeededAndDoOp(new GenericCallback() { @Override public void operationComplete(int rc, PerChannelBookieClient client) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java index aa6ab2d0..a9a763c2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java @@ -163,9 +163,9 @@ public void testShutdown() throws Exception { for (String child : children) { byte[] data = zkc.getData(electionPath + '/' + child, false, null); String bookieIP = new String(data); - String addr = auditor.getLocalAddress().toString(); + String id = auditor.getId().toString(); Assert.assertFalse("AuditorElection cleanup fails", bookieIP - .contains(addr)); + .contains(id)); } } @@ -178,7 +178,7 @@ public void testRestartAuditorBookieAfterCrashing() throws Exception { BookieServer auditor = verifyAuditor(); shutdownBookie(auditor); - String addr = auditor.getLocalAddress().toString(); + String id = auditor.getId().toString(); // restarting Bookie with same configurations. int indexOfDownBookie = bs.indexOf(auditor); @@ -187,12 +187,12 @@ public void testRestartAuditorBookieAfterCrashing() throws Exception { bs.remove(indexOfDownBookie); bsConfs.remove(indexOfDownBookie); tmpDirs.remove(indexOfDownBookie); - auditorElectors.remove(addr); + auditorElectors.remove(id); bs.add(startBookie(serverConfiguration)); // starting corresponding auditor elector - LOG.debug("Performing Auditor Election:" + addr); - startAuditorElector(addr); + LOG.debug("Performing Auditor Election:" + id); + startAuditorElector(id); // waiting for new auditor to come BookieServer newAuditor = waitForNewAuditor(auditor); @@ -200,8 +200,7 @@ public void testRestartAuditorBookieAfterCrashing() throws Exception { "Auditor re-election is not happened for auditor failure!", auditor, newAuditor); Assert.assertFalse("No relection after old auditor rejoins", auditor - .getLocalAddress().getPort() == newAuditor.getLocalAddress() - .getPort()); + .getId().getPort() == newAuditor.getId().getPort()); } private void startAuditorElector(String addr) throws Exception { @@ -217,8 +216,8 @@ private void startAuditorElector(String addr) throws Exception { private void startAuditorElectors() throws Exception { for (BookieServer bserver : bs) { - String addr = bserver.getLocalAddress().toString(); - startAuditorElector(addr); + String id = bserver.getId().toString(); + startAuditorElector(id); } } @@ -244,7 +243,7 @@ private List getAuditorBookie() throws KeeperException, byte[] data = zkc.getData(electionPath, false, null); Assert.assertNotNull("Auditor election failed", data); for (BookieServer bks : bs) { - if (new String(data).contains(bks.getLocalAddress().getPort() + "")) { + if (new String(data).contains(bks.getId().getPort() + "")) { auditors.add(bks); } } @@ -252,13 +251,13 @@ private List getAuditorBookie() throws KeeperException, } private void shutdownBookie(BookieServer bkServer) throws Exception { - String addr = bkServer.getLocalAddress().toString(); - LOG.debug("Shutting down bookie:" + addr); + String id = bkServer.getId().toString(); + LOG.debug("Shutting down bookie:" + id); // shutdown bookie which is an auditor bkServer.shutdown(); // stopping corresponding auditor elector - auditorElectors.get(addr).shutdown(); + auditorElectors.get(id).shutdown(); } private BookieServer waitForNewAuditor(BookieServer auditor) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java index 759de054..565d8ee1 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java @@ -113,10 +113,10 @@ public void tearDown() throws Exception { private void startAuditorElectors() throws Exception { for (BookieServer bserver : bs) { - String addr = bserver.getLocalAddress().toString(); - AuditorElector auditorElector = new AuditorElector(addr, + String id = bserver.getId().toString(); + AuditorElector auditorElector = new AuditorElector(id, baseConf, zkc); - auditorElectors.put(addr, auditorElector); + auditorElectors.put(id, auditorElector); auditorElector.start(); LOG.debug("Starting Auditor Elector"); } @@ -329,12 +329,12 @@ private void doLedgerRereplication(Long... ledgerIds) private String shutdownBookie(int bkShutdownIndex) throws IOException, InterruptedException { BookieServer bkServer = bs.get(bkShutdownIndex); - String bookieAddr = bkServer.getLocalAddress().toString(); - LOG.debug("Shutting down bookie:" + bookieAddr); + String id = bkServer.getId().toString(); + LOG.debug("Shutting down bookie:" + id); killBookie(bkShutdownIndex); - auditorElectors.get(bookieAddr).shutdown(); - auditorElectors.remove(bookieAddr); - return bookieAddr; + auditorElectors.get(id).shutdown(); + auditorElectors.remove(id); + return id; } private LedgerHandle createAndAddEntriesToLedger() throws BKException, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java index 444dcd6d..3c241ac5 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.replication; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestCallbacks; @@ -71,13 +71,13 @@ public void setUp() throws Exception { ServerConfiguration conf = new ServerConfiguration(bsConfs.get(0)); conf.setAuditorPeriodicBookieCheckInterval(CHECK_INTERVAL); conf.setAuditorStaleBookieInterval(CHECK_INTERVAL); - String addr = bs.get(0).getLocalAddress().toString(); + String id = bs.get(0).getId().toString(); ZooKeeperWatcherBase w = new ZooKeeperWatcherBase(10000); auditorZookeeper = ZkUtils.createConnectedZookeeperClient( zkUtil.getZooKeeperConnectString(), w); - auditorElector = new AuditorElector(addr, conf, + auditorElector = new AuditorElector(id, conf, auditorZookeeper); auditorElector.start(); } @@ -102,8 +102,8 @@ public void testPeriodicBookieCheckInterval() throws Exception { LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); LedgerMetadata md = LedgerHandleAdapter.getLedgerMetadata(lh); - List ensemble = md.getEnsembles().get(0L); - ensemble.set(0, new BookieSocketAddress("1.1.1.1", 1000)); + List ensemble = md.getEnsembles().get(0L); + ensemble.set(0, new BookieId("1.1.1.1", 1000)); TestCallbacks.GenericCallbackFuture cb = new TestCallbacks.GenericCallbackFuture(); ledgerManager.writeLedgerMetadata(lh.getId(), md, cb); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java index d8397546..6722ba6d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -74,15 +74,15 @@ public void setUp() throws Exception { ServerConfiguration conf = new ServerConfiguration(bsConfs.get(i)); conf.setAuditorPeriodicCheckInterval(CHECK_INTERVAL); - String addr = bs.get(i).getLocalAddress().toString(); + String id = bs.get(i).getId().toString(); ZooKeeper zk = ZooKeeperClient.createConnectedZooKeeper( zkUtil.getZooKeeperConnectString(), 10000); zkClients.add(zk); - AuditorElector auditorElector = new AuditorElector(addr, + AuditorElector auditorElector = new AuditorElector(id, conf, zk); - auditorElectors.put(addr, auditorElector); + auditorElectors.put(id, auditorElector); auditorElector.start(); LOG.debug("Starting Auditor Elector"); } @@ -169,7 +169,7 @@ public void testPeriodicCheckWhenLedgerDeleted() throws Exception { lh.close(); } final Auditor auditor = new Auditor( - Bookie.getBookieAddress(bsConfs.get(0)).toString(), + Bookie.getBookieAddress(bsConfs.get(0)).getBookieId().toString(), bsConfs.get(0), zkc); final AtomicBoolean exceptionCaught = new AtomicBoolean(false); final CountDownLatch latch = new CountDownLatch(1); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java index 32a5b7e0..fbe9c468 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java @@ -20,7 +20,7 @@ */ package org.apache.bookkeeper.replication; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestCallbacks; @@ -62,7 +62,7 @@ public void testAuditingDuringRollingRestart() throws Exception { underReplicationManager.pollLedgerToRereplicate(), -1); underReplicationManager.disableLedgerReplication(); - BookieSocketAddress auditor = AuditorElector.getCurrentAuditor(baseConf, zkc); + BookieId auditor = AuditorElector.getCurrentAuditor(baseConf, zkc); ServerConfiguration conf = killBookie(auditor); Thread.sleep(2000); startBookie(conf); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java index 7c274e7e..012d8073 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java @@ -34,7 +34,7 @@ import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; @@ -123,7 +123,7 @@ public void testOpenLedgers() throws Exception { List listOfLedgerHandle = createLedgersAndAddEntries(1, 5); LedgerHandle lh = listOfLedgerHandle.get(0); int ledgerReplicaIndex = 0; - BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter + BookieId replicaToKillAddr = LedgerHandleAdapter .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); final String urLedgerZNode = getUrLedgerZNode(lh); @@ -172,7 +172,7 @@ public void testClosedLedgers() throws Exception { closeLedgers(listOfLedgerHandle); LedgerHandle lhandle = listOfLedgerHandle.get(0); int ledgerReplicaIndex = 0; - BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter + BookieId replicaToKillAddr = LedgerHandleAdapter .getLedgerMetadata(lhandle).getEnsembles().get(0L).get(0); CountDownLatch latch = new CountDownLatch(listOfLedgerHandle.size()); @@ -234,7 +234,7 @@ public void testStopWhileReplicationInProgress() throws Exception { numberOfLedgers, 5); closeLedgers(listOfLedgerHandle); LedgerHandle handle = listOfLedgerHandle.get(0); - BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter + BookieId replicaToKillAddr = LedgerHandleAdapter .getLedgerMetadata(handle).getEnsembles().get(0L).get(0); LOG.info("Killing Bookie:" + replicaToKillAddr); @@ -313,7 +313,7 @@ public void testNoSuchLedgerExists() throws Exception { assertNull("UrLedger already exists!", watchUrLedgerNode(getUrLedgerZNode(lh), latch)); } - BookieSocketAddress replicaToKillAddr = LedgerHandleAdapter + BookieId replicaToKillAddr = LedgerHandleAdapter .getLedgerMetadata(listOfLedgerHandle.get(0)).getEnsembles() .get(0L).get(0); killBookie(replicaToKillAddr); @@ -346,7 +346,7 @@ public void testNoSuchLedgerExists() throws Exception { } /** - * Test that if a empty ledger loses the bookie not in the quorum for entry 0, it will + * Test that if an empty ledger loses the bookie not in the quorum for entry 0, it will * still be openable when it loses enough bookies to lose a whole quorum. */ @Test(timeout = 30000) @@ -357,7 +357,7 @@ public void testEmptyLedgerLosesQuorumEventually() throws Exception { String urZNode = getUrLedgerZNode(lh); watchUrLedgerNode(urZNode, latch); - BookieSocketAddress replicaToKill = LedgerHandleAdapter + BookieId replicaToKill = LedgerHandleAdapter .getLedgerMetadata(lh).getEnsembles().get(0L).get(2); LOG.info("Killing last bookie, {}, in ensemble {}", replicaToKill, LedgerHandleAdapter.getLedgerMetadata(lh).getEnsembles().get(0L)); @@ -402,11 +402,11 @@ public void testEmptyLedgerLosesQuorumEventually() throws Exception { } private int getReplicaIndexInLedger(LedgerHandle lh, - BookieSocketAddress replicaToKill) { - SortedMap> ensembles = LedgerHandleAdapter + BookieId replicaToKill) { + SortedMap> ensembles = LedgerHandleAdapter .getLedgerMetadata(lh).getEnsembles(); int ledgerReplicaIndex = -1; - for (BookieSocketAddress addr : ensembles.get(0L)) { + for (BookieId addr : ensembles.get(0L)) { ++ledgerReplicaIndex; if (addr.equals(replicaToKill)) { break; @@ -421,12 +421,12 @@ private void verifyLedgerEnsembleMetadataAfterReplication( LedgerHandle openLedger = bkc .openLedger(lh.getId(), digestType, PASSWD); - BookieSocketAddress BookieSocketAddress = LedgerHandleAdapter + BookieId BookieId = LedgerHandleAdapter .getLedgerMetadata(openLedger).getEnsembles().get(0L) .get(ledgerReplicaIndex); assertEquals("Rereplication has been failed and ledgerReplicaIndex :" - + ledgerReplicaIndex, newBookieServer.getLocalAddress(), - BookieSocketAddress); + + ledgerReplicaIndex, newBookieServer.getId(), + BookieId); } private void closeLedgers(List listOfLedgerHandle) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java index af6ea91e..4a497cc1 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java @@ -32,7 +32,7 @@ import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.replication.ReplicationException.BKAuditException; import org.apache.bookkeeper.test.MultiLedgerManagerTestCase; import org.apache.commons.io.FileUtils; @@ -112,7 +112,7 @@ public void testSimpleBookieLedgerMapping() throws Exception { BookieLedgerIndexer bookieLedgerIndex = new BookieLedgerIndexer( ledgerManager); - Map> bookieToLedgerIndex = bookieLedgerIndex + Map> bookieToLedgerIndex = bookieLedgerIndex .getBookieToLedgerIndex(); assertEquals("Missed few bookies in the bookie-ledger mapping!", 3, @@ -174,7 +174,7 @@ public void testEnsembleReformation() throws Exception { BookieLedgerIndexer bookieLedgerIndex = new BookieLedgerIndexer( ledgerManager); - Map> bookieToLedgerIndex = bookieLedgerIndex + Map> bookieToLedgerIndex = bookieLedgerIndex .getBookieToLedgerIndex(); assertEquals("Missed few bookies in the bookie-ledger mapping!", 4, bookieToLedgerIndex.size()); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java index 768cfb61..a975b5d9 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java @@ -37,7 +37,7 @@ import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieProtocol; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.test.MultiLedgerManagerTestCase; @@ -111,7 +111,7 @@ public void testRWShouldReplicateFragmentsToTargetBookie() throws Exception { for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = LedgerHandleAdapter + BookieId replicaToKill = LedgerHandleAdapter .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); @@ -122,7 +122,7 @@ public void testRWShouldReplicateFragmentsToTargetBookie() throws Exception { lh.addEntry(data); } - BookieSocketAddress newBkAddr = new BookieSocketAddress(InetAddress + BookieId newBkAddr = new BookieId(InetAddress .getLocalHost().getHostAddress(), startNewBookie); LOG.info("New Bookie addr :" + newBkAddr); @@ -161,13 +161,13 @@ public void testRWShouldRetryUntilThereAreEnoughBksAvailableForReplication() lh.addEntry(data); } lh.close(); - BookieSocketAddress replicaToKill = LedgerHandleAdapter + BookieId replicaToKill = LedgerHandleAdapter .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); ServerConfiguration killedBookieConfig = killBookie(replicaToKill); int startNewBookie = startNewBookie(); - BookieSocketAddress newBkAddr = new BookieSocketAddress(InetAddress + BookieId newBkAddr = new BookieId(InetAddress .getLocalHost().getHostAddress(), startNewBookie); LOG.info("New Bookie addr :" + newBkAddr); @@ -211,7 +211,7 @@ public void test2RWsShouldCompeteForReplicationOf2FragmentsAndCompleteReplicatio lh.addEntry(data); } lh.close(); - BookieSocketAddress replicaToKill = LedgerHandleAdapter + BookieId replicaToKill = LedgerHandleAdapter .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); ServerConfiguration killedBookieConfig = killBookie(replicaToKill); @@ -219,14 +219,14 @@ public void test2RWsShouldCompeteForReplicationOf2FragmentsAndCompleteReplicatio killAllBookies(lh, null); // Starte RW1 int startNewBookie1 = startNewBookie(); - BookieSocketAddress newBkAddr1 = new BookieSocketAddress(InetAddress + BookieId newBkAddr1 = new BookieId(InetAddress .getLocalHost().getHostAddress(), startNewBookie1); LOG.info("New Bookie addr :" + newBkAddr1); ReplicationWorker rw1 = new ReplicationWorker(zkc, baseConf); // Starte RW2 int startNewBookie2 = startNewBookie(); - BookieSocketAddress newBkAddr2 = new BookieSocketAddress(InetAddress + BookieId newBkAddr2 = new BookieId(InetAddress .getLocalHost().getHostAddress(), startNewBookie2); LOG.info("New Bookie addr :" + newBkAddr2); ZooKeeper zkc1 = ZooKeeperClient.createConnectedZooKeeperClient( @@ -273,13 +273,13 @@ public void testRWShouldCleanTheLedgerFromUnderReplicationIfLedgerAlreadyDeleted lh.addEntry(data); } lh.close(); - BookieSocketAddress replicaToKill = LedgerHandleAdapter + BookieId replicaToKill = LedgerHandleAdapter .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); killBookie(replicaToKill); int startNewBookie = startNewBookie(); - BookieSocketAddress newBkAddr = new BookieSocketAddress(InetAddress + BookieId newBkAddr = new BookieId(InetAddress .getLocalHost().getHostAddress(), startNewBookie); LOG.info("New Bookie addr :" + newBkAddr); ReplicationWorker rw = new ReplicationWorker(zkc, baseConf); @@ -309,7 +309,7 @@ public void testMultipleLedgerReplicationWithReplicationWorker() for (int i = 0; i < 10; i++) { lh1.addEntry(data); } - BookieSocketAddress replicaToKillFromFirstLedger = LedgerHandleAdapter + BookieId replicaToKillFromFirstLedger = LedgerHandleAdapter .getLedgerMetadata(lh1).getEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKillFromFirstLedger); @@ -321,7 +321,7 @@ public void testMultipleLedgerReplicationWithReplicationWorker() for (int i = 0; i < 10; i++) { lh2.addEntry(data); } - BookieSocketAddress replicaToKillFromSecondLedger = LedgerHandleAdapter + BookieId replicaToKillFromSecondLedger = LedgerHandleAdapter .getLedgerMetadata(lh2).getEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKillFromSecondLedger); @@ -335,7 +335,7 @@ public void testMultipleLedgerReplicationWithReplicationWorker() int startNewBookie = startNewBookie(); - BookieSocketAddress newBkAddr = new BookieSocketAddress(InetAddress + BookieId newBkAddr = new BookieId(InetAddress .getLocalHost().getHostAddress(), startNewBookie); LOG.info("New Bookie addr :" + newBkAddr); @@ -382,7 +382,7 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsUR() for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = LedgerHandleAdapter + BookieId replicaToKill = LedgerHandleAdapter .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); @@ -390,7 +390,7 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsUR() int startNewBookie = startNewBookie(); - BookieSocketAddress newBkAddr = new BookieSocketAddress(InetAddress + BookieId newBkAddr = new BookieId(InetAddress .getLocalHost().getHostAddress(), startNewBookie); LOG.info("New Bookie addr :" + newBkAddr); @@ -437,7 +437,7 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsNotUR() for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = LedgerHandleAdapter + BookieId replicaToKill = LedgerHandleAdapter .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); @@ -451,7 +451,7 @@ public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsNotUR() lh.addEntry(data); } - BookieSocketAddress newBkAddr = new BookieSocketAddress(InetAddress + BookieId newBkAddr = new BookieId(InetAddress .getLocalHost().getHostAddress(), startNewBookie); LOG.info("New Bookie addr :" + newBkAddr); @@ -498,7 +498,7 @@ public void testRWShutdownOnLocalBookieReadonlyTransition() throws Exception { for (int i = 0; i < 10; i++) { lh.addEntry(data); } - BookieSocketAddress replicaToKill = LedgerHandleAdapter.getLedgerMetadata(lh).getEnsembles().get(0L).get(0); + BookieId replicaToKill = LedgerHandleAdapter.getLedgerMetadata(lh).getEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); killBookie(replicaToKill); @@ -508,7 +508,7 @@ public void testRWShutdownOnLocalBookieReadonlyTransition() throws Exception { lh.addEntry(data); } - BookieSocketAddress newBkAddr = new BookieSocketAddress(InetAddress.getLocalHost().getHostAddress(), newBkPort); + BookieId newBkAddr = new BookieId(InetAddress.getLocalHost().getHostAddress(), newBkPort); LOG.info("New Bookie addr :" + newBkAddr); ReplicationWorker rw = new ReplicationWorker(zkc, baseConf); @@ -577,7 +577,7 @@ public void testRWReplicateClosedLedgerWithNoEntry() throws Exception { // closed empty ledger should have lastEntryId=-1 assertEquals(metadata.getLastEntryId(), BookieProtocol.INVALID_ENTRY_ID); // now let's kill a bookie to make it lost - BookieSocketAddress replicaToKill = LedgerHandleAdapter + BookieId replicaToKill = LedgerHandleAdapter .getLedgerMetadata(lh).getEnsembles().get(0L).get(0); LOG.info("Killing Bookie", replicaToKill); @@ -591,7 +591,7 @@ public void testRWReplicateClosedLedgerWithNoEntry() throws Exception { assertEquals(fragment.getAddress(0), replicaToKill); int startNewBookie = startNewBookie(); - BookieSocketAddress newBkAddr = new BookieSocketAddress(InetAddress + BookieId newBkAddr = new BookieId(InetAddress .getLocalHost().getHostAddress(), startNewBookie); LOG.info("New Bookie addr :" + newBkAddr); @@ -607,7 +607,7 @@ public void testRWReplicateClosedLedgerWithNoEntry() throws Exception { // verify the bad bookie is replaced by new bookie lh = bkc.openLedger(ledgerId, BookKeeper.DigestType.CRC32, TESTPASSWD); metadata = LedgerHandleAdapter.getLedgerMetadata(lh); - List newEnsemble = metadata.getEnsembles().values().iterator().next(); + List newEnsemble = metadata.getEnsembles().values().iterator().next(); assertTrue(newEnsemble.contains(newBkAddr)); assertFalse(newEnsemble.contains(replicaToKill)); } finally { @@ -616,14 +616,14 @@ public void testRWReplicateClosedLedgerWithNoEntry() throws Exception { } - private void killAllBookies(LedgerHandle lh, BookieSocketAddress excludeBK) + private void killAllBookies(LedgerHandle lh, BookieId excludeBK) throws InterruptedException { // Killing all bookies except newly replicated bookie - Set>> entrySet = LedgerHandleAdapter + Set>> entrySet = LedgerHandleAdapter .getLedgerMetadata(lh).getEnsembles().entrySet(); - for (Entry> entry : entrySet) { - ArrayList bookies = entry.getValue(); - for (BookieSocketAddress bookie : bookies) { + for (Entry> entry : entrySet) { + ArrayList bookies = entry.getValue(); + for (BookieId bookie : bookies) { if (bookie.equals(excludeBK)) { continue; } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java index 552e6284..bf58b977 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java @@ -41,6 +41,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.proto.ReadOnlyBookieServer; @@ -176,7 +177,7 @@ protected void stopBKCluster() throws Exception { if (autoRecovery != null && isAutoRecoveryEnabled()) { autoRecovery.shutdown(); LOG.debug("Shutdown auto recovery for bookieserver:" - + server.getLocalAddress()); + + server.getId()); } } bs.clear(); @@ -226,11 +227,19 @@ protected void startAllBookies() throws Exception { /** * Get bookie address for bookie at index */ - public BookieSocketAddress getBookie(int index) throws IllegalArgumentException { + public BookieId getBookie(int index) throws IllegalArgumentException { if (bs.size() <= index || index < 0) { throw new IllegalArgumentException("Invalid index, there are only " + bs.size() + " bookies. Asked for " + index); } + return bs.get(index).getId(); + } + + public BookieSocketAddress getBookieAddress(int index) throws IllegalArgumentException { + if (bs.size() <= index || index < 0) { + throw new IllegalArgumentException("Invalid index, there are only " + bs.size() + + " bookies. Asked for " + index); + } return bs.get(index).getLocalAddress(); } @@ -243,11 +252,11 @@ public BookieSocketAddress getBookie(int index) throws IllegalArgumentException * @return the configuration of killed bookie * @throws InterruptedException */ - public ServerConfiguration killBookie(BookieSocketAddress addr) throws InterruptedException { + public ServerConfiguration killBookie(BookieId id) throws InterruptedException { BookieServer toRemove = null; int toRemoveIndex = 0; for (BookieServer server : bs) { - if (server.getLocalAddress().equals(addr)) { + if (server.getId().equals(id)) { server.shutdown(); toRemove = server; break; @@ -270,9 +279,9 @@ public ServerConfiguration killBookie(BookieSocketAddress addr) throws Interrupt * @return the configuration of killed bookie * @throws InterruptedException */ - public void setBookieToReadOnly(BookieSocketAddress addr) throws InterruptedException { + public void setBookieToReadOnly(BookieId id) throws InterruptedException { for (BookieServer server : bs) { - if (server.getLocalAddress().equals(addr)) { + if (server.getId().equals(id)) { server.getBookie().doTransitionToReadOnlyMode(); break; } @@ -311,10 +320,10 @@ public ServerConfiguration killBookie(int index) throws InterruptedException, IO * @throws InterruptedException * @throws IOException */ - public CountDownLatch sleepBookie(BookieSocketAddress addr, final int seconds) + public CountDownLatch sleepBookie(BookieId id, final int seconds) throws InterruptedException, IOException { for (final BookieServer bookie : bs) { - if (bookie.getLocalAddress().equals(addr)) { + if (bookie.getId().equals(id)) { final CountDownLatch l = new CountDownLatch(1); Thread sleeper = new Thread() { @Override @@ -346,18 +355,18 @@ public void run() { * @throws InterruptedException * @throws IOException */ - public void sleepBookie(BookieSocketAddress addr, final CountDownLatch l) + public void sleepBookie(BookieId addr, final CountDownLatch l) throws InterruptedException, IOException { final CountDownLatch suspendLatch = new CountDownLatch(1); sleepBookie(addr, l, suspendLatch); suspendLatch.await(); } - public void sleepBookie(BookieSocketAddress addr, final CountDownLatch l, final CountDownLatch suspendLatch) + public void sleepBookie(BookieId id, final CountDownLatch l, final CountDownLatch suspendLatch) throws InterruptedException, IOException { for (final BookieServer bookie : bs) { - if (bookie.getLocalAddress().equals(addr)) { - LOG.info("Sleep bookie {}.", addr); + if (bookie.getId().equals(id)) { + LOG.info("Sleep bookie {}.", id); Thread sleeper = new Thread() { @Override public void run() { @@ -563,7 +572,7 @@ private void startAutoRecovery(BookieServer bserver, autoRecoveryProcess.start(); autoRecoveryProcesses.put(bserver, autoRecoveryProcess); LOG.debug("Starting Auditor Recovery for the bookie:" - + bserver.getLocalAddress()); + + bserver.getId()); } } @@ -573,7 +582,7 @@ private void stopAutoRecoveryService(BookieServer toRemove) { if (null != autoRecoveryMain && isAutoRecoveryEnabled()) { autoRecoveryMain.shutdown(); LOG.debug("Shutdown auto recovery for bookieserver:" - + toRemove.getLocalAddress()); + + toRemove.getId()); } } @@ -614,7 +623,7 @@ public void stopReplicationService() { .entrySet()) { autoRecoveryProcess.getValue().shutdown(); LOG.debug("Shutdown Auditor Recovery for the bookie:" - + autoRecoveryProcess.getKey().getLocalAddress()); + + autoRecoveryProcess.getKey().getId()); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java index 8ecbdb46..74432bba 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java @@ -27,6 +27,8 @@ import java.util.concurrent.Executors; import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.net.BookieIdToAddressMapping; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.util.IOUtils; import org.jboss.netty.buffer.ChannelBuffer; @@ -115,7 +117,7 @@ public void readEntryComplete(int rc, long ledgerId, long entryId, ChannelBuffer }; WriteCallback wrcb = new WriteCallback() { - public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddress addr, Object ctx) { + public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { if (ctx != null) { synchronized (ctx) { if (ctx instanceof ResultStruct) { @@ -133,10 +135,16 @@ public void testWriteGaps() throws Exception { final Object notifyObject = new Object(); byte[] passwd = new byte[20]; Arrays.fill(passwd, (byte) 'a'); - BookieSocketAddress addr = new BookieSocketAddress("127.0.0.1", port); + + BookieId addr = new BookieId("127.0.0.1", port); ResultStruct arc = new ResultStruct(); - BookieClient bc = new BookieClient(new ClientConfiguration(), channelFactory, executor); + BookieClient bc = new BookieClient(new ClientConfiguration(), channelFactory, executor, new BookieIdToAddressMapping() { + @Override + public BookieSocketAddress getBookieAddress(BookieId bookieId) { + return new BookieSocketAddress(bookieId.getHostName(), bookieId.getPort(), bookieId.getPort()); + } + }); ChannelBuffer bb; bb = createByteBuffer(1, 1, 1); bc.addEntry(addr, 1, passwd, 1, bb, wrcb, arc, BookieProtocol.FLAG_NONE); @@ -238,8 +246,13 @@ private ChannelBuffer createByteBuffer(int i, long lid, long eid) { @Test public void testNoLedger() throws Exception { ResultStruct arc = new ResultStruct(); - BookieSocketAddress addr = new BookieSocketAddress("127.0.0.1", port); - BookieClient bc = new BookieClient(new ClientConfiguration(), channelFactory, executor); + BookieId addr = new BookieId("127.0.0.1", port); + BookieClient bc = new BookieClient(new ClientConfiguration(), channelFactory, executor, new BookieIdToAddressMapping() { + @Override + public BookieSocketAddress getBookieAddress(BookieId bookieId) { + return new BookieSocketAddress(bookieId.getHostName(), bookieId.getPort(), bookieId.getPort()); + } + }); synchronized (arc) { bc.readEntry(addr, 2, 13, recb, arc); arc.wait(1000); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java index 8c0fe438..45fdc986 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ConcurrentLedgerTest.java @@ -33,7 +33,7 @@ import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; -import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; import org.junit.After; import org.junit.Before; @@ -168,7 +168,7 @@ private long doWrites(int ledgers, int size, int totalwrites) WriteCallback cb = new WriteCallback() { @Override public void writeComplete(int rc, long ledgerId, long entryId, - BookieSocketAddress addr, Object ctx) { + BookieId addr, Object ctx) { AtomicInteger counter = (AtomicInteger)ctx; counter.getAndIncrement(); throttle.release(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LoopbackClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LoopbackClient.java index bf1d4551..de55075e 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LoopbackClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LoopbackClient.java @@ -26,6 +26,8 @@ import java.util.concurrent.Executors; import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.net.BookieIdToAddressMapping; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieClient; import org.apache.bookkeeper.proto.BookieProtocol; @@ -64,11 +66,16 @@ synchronized void increment() { } LoopbackClient(ClientSocketChannelFactory channelFactory, OrderedSafeExecutor executor, long begin, int limit) throws IOException { - this.client = new BookieClient(new ClientConfiguration(), channelFactory, executor); + this.client = new BookieClient(new ClientConfiguration(), channelFactory, executor, new BookieIdToAddressMapping() { + @Override + public BookieSocketAddress getBookieAddress(BookieId bookieId) { + return new BookieSocketAddress(bookieId.getHostName(), bookieId.getPort(), bookieId.getPort()); + } + }); this.begin = begin; } - void write(long ledgerId, long entry, byte[] data, BookieSocketAddress addr, WriteCallback cb, Object ctx) + void write(long ledgerId, long entry, byte[] data, BookieId addr, WriteCallback cb, Object ctx) throws IOException, InterruptedException { LOG.info("Ledger id: " + ledgerId + ", Entry: " + entry); byte[] passwd = new byte[20]; @@ -78,7 +85,7 @@ void write(long ledgerId, long entry, byte[] data, BookieSocketAddress addr, Wri } @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) { Counter counter = (Counter) ctx; counter.increment(); } @@ -95,7 +102,7 @@ public static void main(String args[]) { .newCachedThreadPool()); OrderedSafeExecutor executor = new OrderedSafeExecutor(2); try { - BookieSocketAddress addr = new BookieSocketAddress("127.0.0.1", Integer.valueOf(args[2]).intValue()); + BookieId addr = new BookieId("127.0.0.1", Integer.valueOf(args[2]).intValue()); lb = new LoopbackClient(channelFactory, executor, begin, limit.intValue()); for (int i = 0; i < limit; i++) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/MultipleThreadReadTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/MultipleThreadReadTest.java index fb6a0802..682939e2 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/MultipleThreadReadTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/MultipleThreadReadTest.java @@ -45,6 +45,7 @@ public MultipleThreadReadTest() { super(6); this.digestType = BookKeeper.DigestType.CRC32; baseClientConf.setAddEntryTimeout(20); + baseClientConf.setReadEntryTimeout(20); } private void createClients(int numClients) { @@ -260,7 +261,6 @@ public void multiLedgerMultiThreadRead(final int numLedgers, // Create clients used for reading. Each client is responsible for a disjoint range of numLedgers // threads. Client X will be used by threads [numLedgers*X .. numLedgers*(X+1)) - closeClientsAndClear(); createClients(numThreads/numLedgers); List readResults = new ArrayList(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/zookeeper/TestZooKeeperClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/zookeeper/TestZooKeeperClient.java index d3814d21..6af2e13b 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/zookeeper/TestZooKeeperClient.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/zookeeper/TestZooKeeperClient.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -225,6 +226,7 @@ public void testRetrySyncOperations() throws Exception { expireZooKeeperSession(client, timeout); List children = client.getChildren(path, false, newStat); + Collections.sort(children); Assert.assertEquals(2, children.size()); Assert.assertTrue(children.contains("children")); Assert.assertTrue(children.contains("children2")); diff --git a/bookkeeper-server/src/test/resources/log4j.properties b/bookkeeper-server/src/test/resources/log4j.properties index df338d58..a705ef9d 100644 --- a/bookkeeper-server/src/test/resources/log4j.properties +++ b/bookkeeper-server/src/test/resources/log4j.properties @@ -29,7 +29,7 @@ #log4j.rootLogger=INFO, CONSOLE # Example with rolling log file -log4j.rootLogger=INFO, ROLLINGFILE +log4j.rootLogger=DEBUG, ROLLINGFILE # Example with rolling log file and tracing #log4j.rootLogger=TRACE, CONSOLE, ROLLINGFILE, TRACEFILE