HDFS-11817. A faulty node can cause a lease leak and NPE on accessing data. Contributed by Kihwal Lee.
This commit is contained in:
parent
fc6cb4b2dd
commit
2cb63433ab
|
@ -69,11 +69,24 @@ public class BlockUnderConstructionFeature {
|
|||
|
||||
/** Set expected locations */
|
||||
public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets) {
|
||||
int numLocations = targets == null ? 0 : targets.length;
|
||||
if (targets == null) {
|
||||
return;
|
||||
}
|
||||
int numLocations = 0;
|
||||
for (DatanodeStorageInfo target : targets) {
|
||||
if (target != null) {
|
||||
numLocations++;
|
||||
}
|
||||
}
|
||||
|
||||
this.replicas = new ReplicaUnderConstruction[numLocations];
|
||||
for(int i = 0; i < numLocations; i++) {
|
||||
replicas[i] = new ReplicaUnderConstruction(block, targets[i],
|
||||
ReplicaState.RBW);
|
||||
int offset = 0;
|
||||
for(int i = 0; i < targets.length; i++) {
|
||||
// Only store non-null DatanodeStorageInfo.
|
||||
if (targets[i] != null) {
|
||||
replicas[i] = new ReplicaUnderConstruction(block,
|
||||
targets[i], ReplicaState.RBW);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -142,10 +155,17 @@ public class BlockUnderConstructionFeature {
|
|||
* Initialize lease recovery for this block.
|
||||
* Find the first alive data-node starting from the previous primary and
|
||||
* make it primary.
|
||||
* @param blockInfo Block to be recovered
|
||||
* @param recoveryId Recovery ID (new gen stamp)
|
||||
* @param startRecovery Issue recovery command to datanode if true.
|
||||
*/
|
||||
public void initializeBlockRecovery(BlockInfo blockInfo, long recoveryId) {
|
||||
public void initializeBlockRecovery(BlockInfo blockInfo, long recoveryId,
|
||||
boolean startRecovery) {
|
||||
setBlockUCState(BlockUCState.UNDER_RECOVERY);
|
||||
blockRecoveryId = recoveryId;
|
||||
if (!startRecovery) {
|
||||
return;
|
||||
}
|
||||
if (replicas.length == 0) {
|
||||
NameNode.blockStateChangeLog.warn("BLOCK*" +
|
||||
" BlockUnderConstructionFeature.initializeBlockRecovery:" +
|
||||
|
|
|
@ -561,10 +561,11 @@ public class DatanodeManager {
|
|||
DatanodeID[] datanodeID, String[] storageIDs,
|
||||
String format, Object... args) throws UnregisteredNodeException {
|
||||
if (datanodeID.length != storageIDs.length) {
|
||||
// Error for pre-2.0.0-alpha clients.
|
||||
final String err = (storageIDs.length == 0?
|
||||
"Missing storageIDs: It is likely that the HDFS client,"
|
||||
+ " who made this call, is running in an older version of Hadoop"
|
||||
+ " which does not support storageIDs."
|
||||
+ "(pre-2.0.0-alpha) which does not support storageIDs."
|
||||
: "Length mismatched: storageIDs.length=" + storageIDs.length + " != "
|
||||
) + " datanodeID.length=" + datanodeID.length;
|
||||
throw new HadoopIllegalArgumentException(
|
||||
|
|
|
@ -259,7 +259,7 @@ final class FSDirTruncateOp {
|
|||
}
|
||||
if (shouldRecoverNow) {
|
||||
truncatedBlockUC.getUnderConstructionFeature().initializeBlockRecovery(
|
||||
truncatedBlockUC, newBlock.getGenerationStamp());
|
||||
truncatedBlockUC, newBlock.getGenerationStamp(), true);
|
||||
}
|
||||
|
||||
return newBlock;
|
||||
|
|
|
@ -3203,7 +3203,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|||
} else if(truncateRecovery) {
|
||||
recoveryBlock.setGenerationStamp(blockRecoveryId);
|
||||
}
|
||||
uc.initializeBlockRecovery(lastBlock, blockRecoveryId);
|
||||
uc.initializeBlockRecovery(lastBlock, blockRecoveryId, true);
|
||||
leaseManager.renewLease(lease);
|
||||
// Cannot close file right now, since the last block requires recovery.
|
||||
// This may potentially cause infinite loop in lease recovery
|
||||
|
|
|
@ -380,9 +380,16 @@ public class LeaseManager {
|
|||
if (!p.startsWith("/")) {
|
||||
throw new IOException("Invalid path in the lease " + p);
|
||||
}
|
||||
boolean completed = fsnamesystem.internalReleaseLease(
|
||||
leaseToCheck, p, iip,
|
||||
HdfsServerConstants.NAMENODE_LEASE_HOLDER);
|
||||
boolean completed = false;
|
||||
try {
|
||||
completed = fsnamesystem.internalReleaseLease(
|
||||
leaseToCheck, p, iip,
|
||||
HdfsServerConstants.NAMENODE_LEASE_HOLDER);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Cannot release the path " + p + " in the lease "
|
||||
+ leaseToCheck + ". It will be retried.", e);
|
||||
continue;
|
||||
}
|
||||
if (LOG.isDebugEnabled()) {
|
||||
if (completed) {
|
||||
LOG.debug("Lease recovery for inode " + id + " is complete. " +
|
||||
|
@ -396,7 +403,7 @@ public class LeaseManager {
|
|||
needSync = true;
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Cannot release the path " + p + " in the lease "
|
||||
LOG.warn("Removing lease with an invalid path: " + p + ","
|
||||
+ leaseToCheck, e);
|
||||
removing.add(id);
|
||||
}
|
||||
|
|
|
@ -50,7 +50,7 @@ public class TestBlockUnderConstructionFeature {
|
|||
DFSTestUtil.resetLastUpdatesWithOffset(dd1, -3 * 1000);
|
||||
DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
|
||||
DFSTestUtil.resetLastUpdatesWithOffset(dd3, -2 * 1000);
|
||||
blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 1);
|
||||
blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 1, true);
|
||||
BlockInfo[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
|
||||
assertEquals(blockInfoRecovery[0], blockInfo);
|
||||
|
||||
|
@ -58,7 +58,7 @@ public class TestBlockUnderConstructionFeature {
|
|||
DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000);
|
||||
DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
|
||||
DFSTestUtil.resetLastUpdatesWithOffset(dd3, -3 * 1000);
|
||||
blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 2);
|
||||
blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 2, true);
|
||||
blockInfoRecovery = dd1.getLeaseRecoveryCommand(1);
|
||||
assertEquals(blockInfoRecovery[0], blockInfo);
|
||||
|
||||
|
@ -66,7 +66,7 @@ public class TestBlockUnderConstructionFeature {
|
|||
DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000);
|
||||
DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
|
||||
DFSTestUtil.resetLastUpdatesWithOffset(dd3, -3 * 1000);
|
||||
blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 3);
|
||||
blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 3, true);
|
||||
blockInfoRecovery = dd3.getLeaseRecoveryCommand(1);
|
||||
assertEquals(blockInfoRecovery[0], blockInfo);
|
||||
|
||||
|
@ -75,7 +75,7 @@ public class TestBlockUnderConstructionFeature {
|
|||
DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000);
|
||||
DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
|
||||
DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0);
|
||||
blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 3);
|
||||
blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo, 3, true);
|
||||
blockInfoRecovery = dd3.getLeaseRecoveryCommand(1);
|
||||
assertEquals(blockInfoRecovery[0], blockInfo);
|
||||
}
|
||||
|
|
|
@ -38,12 +38,16 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
|
||||
public class TestBlockUnderConstruction {
|
||||
static final String BASE_DIR = "/test/TestBlockUnderConstruction";
|
||||
static final int BLOCK_SIZE = 8192; // same as TestFileCreation.blocksize
|
||||
|
@ -183,4 +187,45 @@ public class TestBlockUnderConstruction {
|
|||
// close file
|
||||
out.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* A storage ID can be invalid if the storage failed or the node
|
||||
* reregisters. When the node heart-beats, the storage report in it
|
||||
* causes storage volumes to be added back. An invalid storage ID
|
||||
* should not cause an NPE.
|
||||
*/
|
||||
@Test
|
||||
public void testEmptyExpectedLocations() throws Exception {
|
||||
final NamenodeProtocols namenode = cluster.getNameNodeRpc();
|
||||
final FSNamesystem fsn = cluster.getNamesystem();
|
||||
final BlockManager bm = fsn.getBlockManager();
|
||||
final Path p = new Path(BASE_DIR, "file2.dat");
|
||||
final String src = p.toString();
|
||||
final FSDataOutputStream out = TestFileCreation.createFile(hdfs, p, 1);
|
||||
writeFile(p, out, 256);
|
||||
out.hflush();
|
||||
|
||||
// make sure the block is readable
|
||||
LocatedBlocks lbs = namenode.getBlockLocations(src, 0, 256);
|
||||
LocatedBlock lastLB = lbs.getLocatedBlocks().get(0);
|
||||
final Block b = lastLB.getBlock().getLocalBlock();
|
||||
|
||||
// fake a block recovery
|
||||
long blockRecoveryId = bm.nextGenerationStamp(false);
|
||||
BlockUnderConstructionFeature uc = bm.getStoredBlock(b).
|
||||
getUnderConstructionFeature();
|
||||
uc.initializeBlockRecovery(null, blockRecoveryId, false);
|
||||
|
||||
try {
|
||||
String[] storages = { "invalid-storage-id1" };
|
||||
fsn.commitBlockSynchronization(lastLB.getBlock(), blockRecoveryId, 256L,
|
||||
true, false, lastLB.getLocations(), storages);
|
||||
} catch (java.lang.IllegalStateException ise) {
|
||||
// Although a failure is expected as of now, future commit policy
|
||||
// changes may make it not fail. This is not critical to the test.
|
||||
}
|
||||
|
||||
// Invalid storage should not trigger an exception.
|
||||
lbs = namenode.getBlockLocations(src, 0, 256);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -73,7 +73,7 @@ public class TestCommitBlockSynchronization {
|
|||
blockInfo.setBlockCollectionId(file.getId());
|
||||
blockInfo.setGenerationStamp(genStamp);
|
||||
blockInfo.getUnderConstructionFeature().initializeBlockRecovery(blockInfo,
|
||||
genStamp);
|
||||
genStamp, true);
|
||||
doReturn(blockInfo).when(file).removeLastBlock(any(Block.class));
|
||||
doReturn(true).when(file).isUnderConstruction();
|
||||
doReturn(new BlockInfoContiguous[1]).when(file).getBlocks();
|
||||
|
|
|
@ -73,6 +73,7 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
|
|||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
||||
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
|
||||
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
|
||||
|
@ -737,7 +738,13 @@ public class TestRetryCacheWithHA {
|
|||
DatanodeInfo[] newNodes = new DatanodeInfo[2];
|
||||
newNodes[0] = nodes[0];
|
||||
newNodes[1] = nodes[1];
|
||||
String[] storageIDs = {"s0", "s1"};
|
||||
final DatanodeManager dm = cluster.getNamesystem(0).getBlockManager()
|
||||
.getDatanodeManager();
|
||||
final String storageID1 = dm.getDatanode(newNodes[0]).getStorageInfos()[0]
|
||||
.getStorageID();
|
||||
final String storageID2 = dm.getDatanode(newNodes[1]).getStorageInfos()[0]
|
||||
.getStorageID();
|
||||
String[] storageIDs = {storageID1, storageID2};
|
||||
|
||||
client.getNamenode().updatePipeline(client.getClientName(), oldBlock,
|
||||
newBlock, newNodes, storageIDs);
|
||||
|
|
Loading…
Reference in New Issue