HDFS-3474. Cleanup Exception handling in BookKeeper journal manager. Contributed by Ivan Kelly.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1344386 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0a80f82a30
commit
7a4c33ae86
@ -28,6 +28,7 @@
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader;
|
||||
import org.apache.bookkeeper.client.LedgerHandle;
|
||||
import org.apache.bookkeeper.client.LedgerEntry;
|
||||
import org.apache.bookkeeper.client.BKException;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
@ -104,8 +105,10 @@ protected FSEditLogOp nextOp() throws IOException {
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
lh.close();
|
||||
} catch (Exception e) {
|
||||
} catch (BKException e) {
|
||||
throw new IOException("Exception closing ledger", e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException("Interrupted closing ledger", e);
|
||||
}
|
||||
}
|
||||
|
||||
@ -168,11 +171,8 @@ private static class LedgerInputStream extends InputStream {
|
||||
throws IOException {
|
||||
this.lh = lh;
|
||||
readEntries = firstBookKeeperEntry;
|
||||
try {
|
||||
|
||||
maxEntry = lh.getLastAddConfirmed();
|
||||
} catch (Exception e) {
|
||||
throw new IOException("Error reading last entry id", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
@ -193,8 +193,10 @@ private InputStream nextStream() throws IOException {
|
||||
assert !entries.hasMoreElements();
|
||||
return e.getEntryInputStream();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
} catch (BKException e) {
|
||||
throw new IOException("Error reading entries from bookkeeper", e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException("Interrupted reading entries from bookkeeper", e);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -188,8 +188,11 @@ public BookKeeperJournalManager(Configuration conf, URI uri)
|
||||
|
||||
bkc = new BookKeeper(new ClientConfiguration(),
|
||||
zkc);
|
||||
} catch (Exception e) {
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Error initializing zk", e);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted while initializing bk journal manager",
|
||||
ie);
|
||||
}
|
||||
|
||||
ci = new CurrentInprogress(zkc, currentInprogressNodePath);
|
||||
@ -211,6 +214,7 @@ public EditLogOutputStream startLogSegment(long txId) throws IOException {
|
||||
throw new IOException("We've already seen " + txId
|
||||
+ ". A new stream cannot be created with it");
|
||||
}
|
||||
|
||||
try {
|
||||
String existingInprogressNode = ci.read();
|
||||
if (null != existingInprogressNode
|
||||
@ -224,6 +228,15 @@ public EditLogOutputStream startLogSegment(long txId) throws IOException {
|
||||
currentLedger = bkc.createLedger(ensembleSize, quorumSize,
|
||||
BookKeeper.DigestType.MAC,
|
||||
digestpw.getBytes());
|
||||
} catch (BKException bke) {
|
||||
throw new IOException("Error creating ledger", bke);
|
||||
} catch (KeeperException ke) {
|
||||
throw new IOException("Error in zookeeper while creating ledger", ke);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted creating ledger", ie);
|
||||
}
|
||||
|
||||
try {
|
||||
String znodePath = inprogressZNode(txId);
|
||||
EditLogLedgerMetadata l = new EditLogLedgerMetadata(znodePath,
|
||||
HdfsConstants.LAYOUT_VERSION, currentLedger.getId(), txId);
|
||||
@ -239,21 +252,27 @@ public EditLogOutputStream startLogSegment(long txId) throws IOException {
|
||||
maxTxId.store(txId);
|
||||
ci.update(znodePath);
|
||||
return new BookKeeperEditLogOutputStream(conf, currentLedger);
|
||||
} catch (Exception e) {
|
||||
if (currentLedger != null) {
|
||||
} catch (KeeperException ke) {
|
||||
cleanupLedger(currentLedger);
|
||||
throw new IOException("Error storing ledger metadata", ke);
|
||||
}
|
||||
}
|
||||
|
||||
private void cleanupLedger(LedgerHandle lh) {
|
||||
try {
|
||||
long id = currentLedger.getId();
|
||||
currentLedger.close();
|
||||
bkc.deleteLedger(id);
|
||||
} catch (Exception e2) {
|
||||
} catch (BKException bke) {
|
||||
//log & ignore, an IOException will be thrown soon
|
||||
LOG.error("Error closing ledger", e2);
|
||||
}
|
||||
}
|
||||
throw new IOException("Error creating ledger", e);
|
||||
LOG.error("Error closing ledger", bke);
|
||||
} catch (InterruptedException ie) {
|
||||
LOG.warn("Interrupted while closing ledger", ie);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Finalize a log segment. If the journal manager is currently
|
||||
* writing to a ledger, ensure that this is the ledger of the log segment
|
||||
@ -347,8 +366,11 @@ EditLogInputStream getInputStream(long fromTxId, boolean inProgressOk)
|
||||
l);
|
||||
s.skipTo(fromTxId);
|
||||
return s;
|
||||
} catch (Exception e) {
|
||||
} catch (BKException e) {
|
||||
throw new IOException("Could not open ledger for " + fromTxId, e);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted opening ledger for "
|
||||
+ fromTxId, ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -479,8 +501,10 @@ public void close() throws IOException {
|
||||
try {
|
||||
bkc.close();
|
||||
zkc.close();
|
||||
} catch (Exception e) {
|
||||
throw new IOException("Couldn't close zookeeper client", e);
|
||||
} catch (BKException bke) {
|
||||
throw new IOException("Couldn't close bookkeeper client", bke);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted while closing journal manager", ie);
|
||||
}
|
||||
}
|
||||
|
||||
@ -525,9 +549,12 @@ private long recoverLastTxId(EditLogLedgerMetadata l, boolean fence)
|
||||
op = in.readOp();
|
||||
}
|
||||
return endTxId;
|
||||
} catch (Exception e) {
|
||||
} catch (BKException e) {
|
||||
throw new IOException("Exception retreiving last tx id for ledger " + l,
|
||||
e);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted while retreiving last tx id "
|
||||
+ "for ledger " + l, ie);
|
||||
}
|
||||
}
|
||||
|
||||
@ -542,8 +569,10 @@ private List<EditLogLedgerMetadata> getLedgerList() throws IOException {
|
||||
for (String n : ledgerNames) {
|
||||
ledgers.add(EditLogLedgerMetadata.read(zkc, ledgerPath + "/" + n));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Exception reading ledger list from zk", e);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted getting list of ledgers from zk", ie);
|
||||
}
|
||||
|
||||
Collections.sort(ledgers, EditLogLedgerMetadata.COMPARATOR);
|
||||
|
@ -68,8 +68,10 @@ class CurrentInprogress {
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Exception accessing Zookeeper", e);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted accessing Zookeeper", ie);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -130,8 +130,10 @@ static EditLogLedgerMetadata read(ZooKeeper zkc, String path)
|
||||
}
|
||||
} catch(KeeperException.NoNodeException nne) {
|
||||
throw nne;
|
||||
} catch(Exception e) {
|
||||
throw new IOException("Error reading from zookeeper", e);
|
||||
} catch(KeeperException ke) {
|
||||
throw new IOException("Error reading from zookeeper", ke);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted reading from zookeeper", ie);
|
||||
}
|
||||
}
|
||||
|
||||
@ -151,8 +153,10 @@ void write(ZooKeeper zkc, String path)
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
} catch (KeeperException.NodeExistsException nee) {
|
||||
throw nee;
|
||||
} catch (Exception e) {
|
||||
throw new IOException("Error creating ledger znode");
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Error creating ledger znode", e);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted creating ledger znode", ie);
|
||||
}
|
||||
}
|
||||
|
||||
@ -164,9 +168,12 @@ boolean verify(ZooKeeper zkc, String path) {
|
||||
+ " against " + other);
|
||||
}
|
||||
return other == this;
|
||||
} catch (Exception e) {
|
||||
} catch (KeeperException e) {
|
||||
LOG.error("Couldn't verify data in " + path, e);
|
||||
return false;
|
||||
} catch (IOException ie) {
|
||||
LOG.error("Couldn't verify data in " + path, ie);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -20,6 +20,7 @@
|
||||
import java.io.IOException;
|
||||
import org.apache.zookeeper.ZooKeeper;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.zookeeper.ZooDefs.Ids;
|
||||
import org.apache.zookeeper.data.Stat;
|
||||
|
||||
@ -58,8 +59,10 @@ synchronized void store(long maxTxId) throws IOException {
|
||||
zkc.create(path, txidStr.getBytes("UTF-8"),
|
||||
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Error writing max tx id", e);
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException("Interrupted while writing max tx id", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -74,8 +77,10 @@ synchronized long get() throws IOException {
|
||||
String txidString = new String(bytes, "UTF-8");
|
||||
return Long.valueOf(txidString);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Error reading the max tx id from zk", e);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new IOException("Interrupted while reading thr max tx id", ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user