Merge r:1344386 HDFS-3474. Cleanup Exception handling in BookKeeper journal manager. Contributed by Ivan Kelly.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1344397 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Uma Maheswara Rao G 2012-05-30 18:37:13 +00:00
parent ed20ba2e49
commit 7dbeea099f
6 changed files with 83 additions and 35 deletions

View File

@ -124,6 +124,9 @@ Release 2.0.1-alpha - UNRELEASED
HDFS-3398. Client will not retry when primaryDN is down once it's just got pipeline. HDFS-3398. Client will not retry when primaryDN is down once it's just got pipeline.
(Amith D K via umamahesh) (Amith D K via umamahesh)
HDFS-3474. Cleanup Exception handling in BookKeeper journal manager.
(Ivan Kelly via umamahesh)
Release 2.0.0-alpha - UNRELEASED Release 2.0.0-alpha - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader; import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader;
import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerEntry;
import org.apache.bookkeeper.client.BKException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -104,8 +105,10 @@ class BookKeeperEditLogInputStream extends EditLogInputStream {
public void close() throws IOException { public void close() throws IOException {
try { try {
lh.close(); lh.close();
} catch (Exception e) { } catch (BKException e) {
throw new IOException("Exception closing ledger", e); throw new IOException("Exception closing ledger", e);
} catch (InterruptedException e) {
throw new IOException("Interrupted closing ledger", e);
} }
} }
@ -168,11 +171,8 @@ class BookKeeperEditLogInputStream extends EditLogInputStream {
throws IOException { throws IOException {
this.lh = lh; this.lh = lh;
readEntries = firstBookKeeperEntry; readEntries = firstBookKeeperEntry;
try {
maxEntry = lh.getLastAddConfirmed(); maxEntry = lh.getLastAddConfirmed();
} catch (Exception e) {
throw new IOException("Error reading last entry id", e);
}
} }
/** /**
@ -193,8 +193,10 @@ class BookKeeperEditLogInputStream extends EditLogInputStream {
assert !entries.hasMoreElements(); assert !entries.hasMoreElements();
return e.getEntryInputStream(); return e.getEntryInputStream();
} }
} catch (Exception e) { } catch (BKException e) {
throw new IOException("Error reading entries from bookkeeper", e); throw new IOException("Error reading entries from bookkeeper", e);
} catch (InterruptedException e) {
throw new IOException("Interrupted reading entries from bookkeeper", e);
} }
return null; return null;
} }

View File

@ -188,8 +188,11 @@ public class BookKeeperJournalManager implements JournalManager {
bkc = new BookKeeper(new ClientConfiguration(), bkc = new BookKeeper(new ClientConfiguration(),
zkc); zkc);
} catch (Exception e) { } catch (KeeperException e) {
throw new IOException("Error initializing zk", 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); ci = new CurrentInprogress(zkc, currentInprogressNodePath);
@ -211,6 +214,7 @@ public class BookKeeperJournalManager implements JournalManager {
throw new IOException("We've already seen " + txId throw new IOException("We've already seen " + txId
+ ". A new stream cannot be created with it"); + ". A new stream cannot be created with it");
} }
try { try {
String existingInprogressNode = ci.read(); String existingInprogressNode = ci.read();
if (null != existingInprogressNode if (null != existingInprogressNode
@ -224,6 +228,15 @@ public class BookKeeperJournalManager implements JournalManager {
currentLedger = bkc.createLedger(ensembleSize, quorumSize, currentLedger = bkc.createLedger(ensembleSize, quorumSize,
BookKeeper.DigestType.MAC, BookKeeper.DigestType.MAC,
digestpw.getBytes()); 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); String znodePath = inprogressZNode(txId);
EditLogLedgerMetadata l = new EditLogLedgerMetadata(znodePath, EditLogLedgerMetadata l = new EditLogLedgerMetadata(znodePath,
HdfsConstants.LAYOUT_VERSION, currentLedger.getId(), txId); HdfsConstants.LAYOUT_VERSION, currentLedger.getId(), txId);
@ -239,21 +252,27 @@ public class BookKeeperJournalManager implements JournalManager {
maxTxId.store(txId); maxTxId.store(txId);
ci.update(znodePath); ci.update(znodePath);
return new BookKeeperEditLogOutputStream(conf, currentLedger); return new BookKeeperEditLogOutputStream(conf, currentLedger);
} catch (Exception e) { } catch (KeeperException ke) {
if (currentLedger != null) { cleanupLedger(currentLedger);
throw new IOException("Error storing ledger metadata", ke);
}
}
private void cleanupLedger(LedgerHandle lh) {
try { try {
long id = currentLedger.getId(); long id = currentLedger.getId();
currentLedger.close(); currentLedger.close();
bkc.deleteLedger(id); bkc.deleteLedger(id);
} catch (Exception e2) { } catch (BKException bke) {
//log & ignore, an IOException will be thrown soon //log & ignore, an IOException will be thrown soon
LOG.error("Error closing ledger", e2); LOG.error("Error closing ledger", bke);
} } catch (InterruptedException ie) {
} LOG.warn("Interrupted while closing ledger", ie);
throw new IOException("Error creating ledger", e);
} }
} }
/** /**
* Finalize a log segment. If the journal manager is currently * Finalize a log segment. If the journal manager is currently
* writing to a ledger, ensure that this is the ledger of the log segment * writing to a ledger, ensure that this is the ledger of the log segment
@ -347,8 +366,11 @@ public class BookKeeperJournalManager implements JournalManager {
l); l);
s.skipTo(fromTxId); s.skipTo(fromTxId);
return s; return s;
} catch (Exception e) { } catch (BKException e) {
throw new IOException("Could not open ledger for " + fromTxId, 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 class BookKeeperJournalManager implements JournalManager {
try { try {
bkc.close(); bkc.close();
zkc.close(); zkc.close();
} catch (Exception e) { } catch (BKException bke) {
throw new IOException("Couldn't close zookeeper client", e); 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 @@ public class BookKeeperJournalManager implements JournalManager {
op = in.readOp(); op = in.readOp();
} }
return endTxId; return endTxId;
} catch (Exception e) { } catch (BKException e) {
throw new IOException("Exception retreiving last tx id for ledger " + l, throw new IOException("Exception retreiving last tx id for ledger " + l,
e); e);
} catch (InterruptedException ie) {
throw new IOException("Interrupted while retreiving last tx id "
+ "for ledger " + l, ie);
} }
} }
@ -542,8 +569,10 @@ public class BookKeeperJournalManager implements JournalManager {
for (String n : ledgerNames) { for (String n : ledgerNames) {
ledgers.add(EditLogLedgerMetadata.read(zkc, ledgerPath + "/" + n)); ledgers.add(EditLogLedgerMetadata.read(zkc, ledgerPath + "/" + n));
} }
} catch (Exception e) { } catch (KeeperException e) {
throw new IOException("Exception reading ledger list from zk", 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); Collections.sort(ledgers, EditLogLedgerMetadata.COMPARATOR);

View File

@ -68,8 +68,10 @@ class CurrentInprogress {
} }
} }
} }
} catch (Exception e) { } catch (KeeperException e) {
throw new IOException("Exception accessing Zookeeper", e); throw new IOException("Exception accessing Zookeeper", e);
} catch (InterruptedException ie) {
throw new IOException("Interrupted accessing Zookeeper", ie);
} }
} }

View File

@ -130,8 +130,10 @@ public class EditLogLedgerMetadata {
} }
} catch(KeeperException.NoNodeException nne) { } catch(KeeperException.NoNodeException nne) {
throw nne; throw nne;
} catch(Exception e) { } catch(KeeperException ke) {
throw new IOException("Error reading from zookeeper", e); throw new IOException("Error reading from zookeeper", ke);
} catch (InterruptedException ie) {
throw new IOException("Interrupted reading from zookeeper", ie);
} }
} }
@ -151,8 +153,10 @@ public class EditLogLedgerMetadata {
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
} catch (KeeperException.NodeExistsException nee) { } catch (KeeperException.NodeExistsException nee) {
throw nee; throw nee;
} catch (Exception e) { } catch (KeeperException e) {
throw new IOException("Error creating ledger znode"); throw new IOException("Error creating ledger znode", e);
} catch (InterruptedException ie) {
throw new IOException("Interrupted creating ledger znode", ie);
} }
} }
@ -164,9 +168,12 @@ public class EditLogLedgerMetadata {
+ " against " + other); + " against " + other);
} }
return other == this; return other == this;
} catch (Exception e) { } catch (KeeperException e) {
LOG.error("Couldn't verify data in " + path, e); LOG.error("Couldn't verify data in " + path, e);
return false; return false;
} catch (IOException ie) {
LOG.error("Couldn't verify data in " + path, ie);
return false;
} }
} }

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.contrib.bkjournal;
import java.io.IOException; import java.io.IOException;
import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.data.Stat;
@ -58,8 +59,10 @@ class MaxTxId {
zkc.create(path, txidStr.getBytes("UTF-8"), zkc.create(path, txidStr.getBytes("UTF-8"),
Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
} }
} catch (Exception e) { } catch (KeeperException e) {
throw new IOException("Error writing max tx id", 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 @@ class MaxTxId {
String txidString = new String(bytes, "UTF-8"); String txidString = new String(bytes, "UTF-8");
return Long.valueOf(txidString); return Long.valueOf(txidString);
} }
} catch (Exception e) { } catch (KeeperException e) {
throw new IOException("Error reading the max tx id from zk", 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);
} }
} }
} }