Skip to content
This repository has been archived by the owner on Apr 30, 2019. It is now read-only.

Commit

Permalink
Merge pull request apache#185 from evans/YBK-3
Browse files Browse the repository at this point in the history
YBK-3: fix issues in AuditorLedgerCheckerTest
  • Loading branch information
Bobby Evans authored and GitHub Enterprise committed Oct 13, 2017
2 parents 361a88e + 126681f commit 74feb75
Show file tree
Hide file tree
Showing 7 changed files with 83 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -44,8 +44,6 @@
import org.apache.bookkeeper.bookie.Journal.JournalScanner;
import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage;
import org.apache.bookkeeper.bookie.storage.ldb.EntryLocationIndex;
import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageFactory.DbConfigType;
import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageRocksDB;
import org.apache.bookkeeper.bookie.storage.ldb.LocationsIndexRebuildOp;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
Expand Down Expand Up @@ -1128,7 +1126,7 @@ public int runCmd(CommandLine cmdLine) throws Exception {
bookies.addAll(availableBookies);
} else if (cmdLine.hasOption("ro")) {
Collection<BookieSocketAddress> roBookies = bka
.getReadOnlyBookies();
.getReadOnlyBookiesAsync();
bookies.addAll(roBookies);
}
for (BookieSocketAddress b : bookies) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -193,12 +193,22 @@ public Collection<BookieSocketAddress> getAvailableBookies()
}

/**
* Get a list of readonly bookies
* Get a list of readonly bookies (synchronous).
*
* @return a collection of bookie addresses
* @throws BKException on any error trying to read the bookies
*/
public Collection<BookieSocketAddress> getReadOnlyBookies() {
return bkc.bookieWatcher.getReadOnlyBookies();
public Collection<BookieSocketAddress> getReadOnlyBookiesSync() throws BKException {
return bkc.bookieWatcher.getReadOnlyBookiesSync();
}

/**
* Get a list of readonly bookies (asynchronous).
*
* @return a collection of bookie addresses
*/
public Collection<BookieSocketAddress> getReadOnlyBookiesAsync() {
return bkc.bookieWatcher.getReadOnlyBookiesAsync();
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,6 +129,21 @@ void notifyReadOnlyBookiesChanged(final BookiesListener listener) throws BKExcep
readOnlyBookieWatcher.notifyBookiesChanged(listener);
}

public Collection<BookieSocketAddress> getReadOnlyBookiesSync() throws BKException {
try {
String znode = this.bookieRegistrationPath + "/" + BookKeeperConstants.READONLY;
List<String> children = bk.getZkHandle().getChildren(znode, false);
return convertToBookieAddresses(children);
} catch (KeeperException ke) {
logger.error("Failed to get read only bookie list : ", ke);
throw new BKException.ZKException();
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
logger.error("Interrupted reading read only bookie list", ie);
throw new BKException.BKInterruptedException();
}
}

public Collection<BookieSocketAddress> getBookies() throws BKException {
try {
List<String> children = bk.getZkHandle().getChildren(this.bookieRegistrationPath, false);
Expand All @@ -144,7 +159,7 @@ public Collection<BookieSocketAddress> getBookies() throws BKException {
}
}

Collection<BookieSocketAddress> getReadOnlyBookies() {
Collection<BookieSocketAddress> getReadOnlyBookiesAsync() {
return new HashSet<BookieSocketAddress>(readOnlyBookieWatcher.getReadOnlyBookies());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -303,6 +303,11 @@ public Long getSample() {
}
}

@VisibleForTesting
public AutoRecoveryMain getAutoRecoveryMain() {
return autoRecoveryMain;
}

protected Bookie newBookie(ServerConfiguration conf)
throws IOException, KeeperException, InterruptedException, BookieException {
return new Bookie(conf, statsLogger.scope(BOOKIE_SCOPE));
Expand Down Expand Up @@ -602,4 +607,16 @@ public void run() {
System.exit(ExitCode.SERVER_EXCEPTION);
}
}

@Override
public String toString() {
String id = "UNKNOWN";

try {
id = Bookie.getBookieAddress(conf).toString();
} catch (UnknownHostException e) {
//Ignored...
}
return "Bookie Server listening on " + id;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -143,6 +143,16 @@ public void run() {
}
}

@VisibleForTesting
synchronized Future<?> forceAudit() {
if (executor.isShutdown()) {
SettableFuture<Void> f = SettableFuture.<Void>create();
f.setException(new BKAuditException("Auditor shutting down"));
return f;
}
return executor.submit(BOOKIE_CHECK);
}

@VisibleForTesting
synchronized Future<?> submitAuditTask() {
if (executor.isShutdown()) {
Expand Down Expand Up @@ -267,7 +277,7 @@ private void waitIfLedgerReplicationDisabled() throws UnavailableException,
private List<String> getAvailableBookies() throws BKException {
// Get the available bookies
Collection<BookieSocketAddress> availableBkAddresses = admin.getAvailableBookies();
Collection<BookieSocketAddress> readOnlyBkAddresses = admin.getReadOnlyBookies();
Collection<BookieSocketAddress> readOnlyBkAddresses = admin.getReadOnlyBookiesSync();
availableBkAddresses.addAll(readOnlyBkAddresses);

List<String> availableBookies = new ArrayList<String>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -359,6 +359,11 @@ public boolean isRunning() {
return running.get();
}

@Override
public String toString() {
return "AuditorElector for " + bookieId;
}

/**
* Compare the votes in the ascending order of the sequence number. Vote
* format is 'V_sequencenumber', comparator will do sorting based on the
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,9 @@
import java.util.Random;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;

import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
Expand All @@ -49,6 +51,7 @@
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.Watcher;
import org.apache.zookeeper.data.Stat;
import org.junit.Before;
import org.junit.Test;
import org.junit.Ignore;
Expand Down Expand Up @@ -149,6 +152,7 @@ public void testSimpleLedger() throws Exception {

// grace period for publishing the bk-ledger
LOG.debug("Waiting for ledgers to be marked as under replicated");
forceAudit();
underReplicaLatch.await(5, TimeUnit.SECONDS);
Map<Long, String> urLedgerData = getUrLedgerData(urLedgerList);
assertEquals("Missed identifying under replicated ledgers", 1,
Expand Down Expand Up @@ -277,6 +281,7 @@ public void testReadOnlyBookieExclusionFromURLedgersCheck() throws Exception {

// grace period for publishing the bk-ledger
LOG.debug("Waiting for Auditor to finish ledger check.");
forceAudit();
assertFalse("latch should not have completed", underReplicaLatch.await(5, TimeUnit.SECONDS));
}

Expand All @@ -294,19 +299,22 @@ public void testReadOnlyBookieShutdown() throws Exception {
final CountDownLatch underReplicaLatch = registerUrLedgerWatcher(count);

int bkIndex = bs.size() - 1;
LOG.debug("Moving bookie {} {} to read only...", bkIndex, bs.get(bkIndex));
ServerConfiguration bookieConf = bsConfs.get(bkIndex);
BookieServer bk = bs.get(bkIndex);
bookieConf.setReadOnlyModeEnabled(true);
bk.getBookie().transitionToReadOnlyMode();

// grace period for publishing the bk-ledger
LOG.debug("Waiting for Auditor to finish ledger check.");
assertFalse("latch should not have completed", underReplicaLatch.await(5, TimeUnit.SECONDS));
forceAudit();
assertFalse("latch should not have completed", underReplicaLatch.await(1, TimeUnit.SECONDS));

String shutdownBookie = shutdownBookie(bkIndex);

// grace period for publishing the bk-ledger
LOG.debug("Waiting for ledgers to be marked as under replicated");
forceAudit();
underReplicaLatch.await(5, TimeUnit.SECONDS);
Map<Long, String> urLedgerData = getUrLedgerData(urLedgerList);
assertEquals("Missed identifying under replicated ledgers", 1, urLedgerList.size());
Expand All @@ -322,6 +330,17 @@ public void testReadOnlyBookieShutdown() throws Exception {
data.contains(shutdownBookie));
}

private void forceAudit() throws InterruptedException, ExecutionException, TimeoutException {
for (AuditorElector ae : auditorElectors.values()) {
Auditor auditor = ae.getAuditor();
if (auditor != null) {
LOG.debug("Forcing audit {}", ae);
ae.getAuditor().forceAudit().get(5, TimeUnit.SECONDS);
LOG.debug("Audit complete...");
}
}
}

/**
* Wait for ledger to be underreplicated, and to be missing all replicas specified
*/
Expand Down

0 comments on commit 74feb75

Please sign in to comment.