HDFS-4201. NPE in BPServiceActor#sendHeartBeat (jxiang via cmccabe)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1550269 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Colin McCabe 2013-12-11 21:30:16 +00:00
parent ed4d318d68
commit 9678020e59
3 changed files with 63 additions and 3 deletions

View File

@ -813,6 +813,8 @@ Release 2.3.0 - UNRELEASED
HDFS-5074. Allow starting up from an fsimage checkpoint in the middle of a
segment. (Todd Lipcon via atm)
HDFS-4201. NPE in BPServiceActor#sendHeartBeat. (jxiang via cmccabe)
Release 2.2.0 - 2013-10-13
INCOMPATIBLE CHANGES

View File

@ -274,12 +274,22 @@ DataNode getDataNode() {
synchronized void verifyAndSetNamespaceInfo(NamespaceInfo nsInfo) throws IOException {
if (this.bpNSInfo == null) {
this.bpNSInfo = nsInfo;
boolean success = false;
// Now that we know the namespace ID, etc, we can pass this to the DN.
// The DN can now initialize its local storage if we are the
// first BP to handshake, etc.
dn.initBlockPool(this);
return;
try {
dn.initBlockPool(this);
success = true;
} finally {
if (!success) {
// The datanode failed to initialize the BP. We need to reset
// the namespace info so that other BPService actors still have
// a chance to set it, and re-initialize the datanode.
this.bpNSInfo = null;
}
}
} else {
checkNSEquality(bpNSInfo.getBlockPoolID(), nsInfo.getBlockPoolID(),
"Blockpool ID");

View File

@ -25,7 +25,9 @@
import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -294,6 +296,47 @@ public void testPickActiveNameNode() throws Exception {
}
}
/**
* Test datanode block pool initialization error handling.
* Failure in initializing a block pool should not cause NPE.
*/
@Test
public void testBPInitErrorHandling() throws Exception {
final DataNode mockDn = Mockito.mock(DataNode.class);
Mockito.doReturn(true).when(mockDn).shouldRun();
Configuration conf = new Configuration();
File dnDataDir = new File(
new File(TEST_BUILD_DATA, "testBPInitErrorHandling"), "data");
conf.set(DFS_DATANODE_DATA_DIR_KEY, dnDataDir.toURI().toString());
Mockito.doReturn(conf).when(mockDn).getConf();
Mockito.doReturn(new DNConf(conf)).when(mockDn).getDnConf();
Mockito.doReturn(DataNodeMetrics.create(conf, "fake dn")).
when(mockDn).getMetrics();
final AtomicInteger count = new AtomicInteger();
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
if (count.getAndIncrement() == 0) {
throw new IOException("faked initBlockPool exception");
}
// The initBlockPool is called again. Now mock init is done.
Mockito.doReturn(mockFSDataset).when(mockDn).getFSDataset();
return null;
}
}).when(mockDn).initBlockPool(Mockito.any(BPOfferService.class));
BPOfferService bpos = setupBPOSForNNs(mockDn, mockNN1, mockNN2);
bpos.start();
try {
waitForInitialization(bpos);
List<BPServiceActor> actors = bpos.getBPServiceActors();
assertEquals(1, actors.size());
BPServiceActor actor = actors.get(0);
waitForBlockReport(actor.getNameNodeProxy());
} finally {
bpos.stop();
}
}
private void waitForOneToFail(final BPOfferService bpos)
throws Exception {
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@ -311,6 +354,11 @@ public Boolean get() {
*/
private BPOfferService setupBPOSForNNs(
DatanodeProtocolClientSideTranslatorPB ... nns) throws IOException {
return setupBPOSForNNs(mockDn, nns);
}
private BPOfferService setupBPOSForNNs(DataNode mockDn,
DatanodeProtocolClientSideTranslatorPB ... nns) throws IOException {
// Set up some fake InetAddresses, then override the connectToNN
// function to return the corresponding proxies.