Skip to content

Commit 520043b

Browse files
author
Colin Patrick Mccabe
committed
HDFS-7996. After swapping a volume, BlockReceiver reports ReplicaNotFoundException (Lei (Eddy) Xu via Colin P. McCabe)
(cherry picked from commit 023133c) (cherry picked from commit 0c5069c)
1 parent 6855bbc commit 520043b

File tree

3 files changed

+64
-20
lines changed

3 files changed

+64
-20
lines changed

hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -959,6 +959,9 @@ Release 2.7.0 - UNRELEASED
959959
HDFS-8039. Fix TestDebugAdmin#testRecoverLease and
960960
testVerifyBlockChecksumCommand on Windows. (Xiaoyu Yao via cnauroth)
961961

962+
HDFS-7996. After swapping a volume, BlockReceiver reports
963+
ReplicaNotFoundException (Lei (Eddy) Xu via Colin P. McCabe)
964+
962965
BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
963966

964967
HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

Lines changed: 32 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -281,7 +281,7 @@ String getStorageUuid() {
281281
}
282282

283283
/**
284-
* close files.
284+
* close files and release volume reference.
285285
*/
286286
@Override
287287
public void close() throws IOException {
@@ -798,17 +798,20 @@ void receiveBlock(
798798
// then finalize block or convert temporary to RBW.
799799
// For client-writes, the block is finalized in the PacketResponder.
800800
if (isDatanode || isTransfer) {
801-
// close the block/crc files
802-
close();
803-
block.setNumBytes(replicaInfo.getNumBytes());
804-
805-
if (stage == BlockConstructionStage.TRANSFER_RBW) {
806-
// for TRANSFER_RBW, convert temporary to RBW
807-
datanode.data.convertTemporaryToRbw(block);
808-
} else {
809-
// for isDatnode or TRANSFER_FINALIZED
810-
// Finalize the block.
811-
datanode.data.finalizeBlock(block);
801+
// Hold a volume reference to finalize block.
802+
try (ReplicaHandler handler = claimReplicaHandler()) {
803+
// close the block/crc files
804+
close();
805+
block.setNumBytes(replicaInfo.getNumBytes());
806+
807+
if (stage == BlockConstructionStage.TRANSFER_RBW) {
808+
// for TRANSFER_RBW, convert temporary to RBW
809+
datanode.data.convertTemporaryToRbw(block);
810+
} else {
811+
// for isDatnode or TRANSFER_FINALIZED
812+
// Finalize the block.
813+
datanode.data.finalizeBlock(block);
814+
}
812815
}
813816
datanode.metrics.incrBlocksWritten();
814817
}
@@ -980,7 +983,14 @@ private Checksum computePartialChunkCrc(long blkoff, long ckoff)
980983
}
981984
return partialCrc;
982985
}
983-
986+
987+
/** The caller claims the ownership of the replica handler. */
988+
private ReplicaHandler claimReplicaHandler() {
989+
ReplicaHandler handler = replicaHandler;
990+
replicaHandler = null;
991+
return handler;
992+
}
993+
984994
private static enum PacketResponderType {
985995
NON_PIPELINE, LAST_IN_PIPELINE, HAS_DOWNSTREAM_IN_PIPELINE
986996
}
@@ -1280,12 +1290,15 @@ public void run() {
12801290
* @param startTime time when BlockReceiver started receiving the block
12811291
*/
12821292
private void finalizeBlock(long startTime) throws IOException {
1283-
BlockReceiver.this.close();
1284-
final long endTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime()
1285-
: 0;
1286-
block.setNumBytes(replicaInfo.getNumBytes());
1287-
datanode.data.finalizeBlock(block);
1288-
1293+
long endTime = 0;
1294+
// Hold a volume reference to finalize block.
1295+
try (ReplicaHandler handler = BlockReceiver.this.claimReplicaHandler()) {
1296+
BlockReceiver.this.close();
1297+
endTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
1298+
block.setNumBytes(replicaInfo.getNumBytes());
1299+
datanode.data.finalizeBlock(block);
1300+
}
1301+
12891302
if (pinning) {
12901303
datanode.data.setPinning(block);
12911304
}

hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java

Lines changed: 29 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@
3434
import org.apache.hadoop.hdfs.MiniDFSCluster;
3535
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
3636
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
37+
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
3738
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
3839
import org.apache.hadoop.hdfs.server.common.Storage;
3940
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
@@ -64,6 +65,8 @@
6465
import org.apache.commons.logging.Log;
6566
import org.apache.commons.logging.LogFactory;
6667
import org.mockito.Mockito;
68+
import org.mockito.invocation.InvocationOnMock;
69+
import org.mockito.stubbing.Answer;
6770

6871
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
6972
import static org.hamcrest.CoreMatchers.anyOf;
@@ -77,6 +80,7 @@
7780
import static org.junit.Assert.fail;
7881
import static org.mockito.Matchers.any;
7982
import static org.mockito.Matchers.anyString;
83+
import static org.mockito.Mockito.doAnswer;
8084
import static org.mockito.Mockito.timeout;
8185

8286
public class TestDataNodeHotSwapVolumes {
@@ -577,6 +581,7 @@ private void testRemoveVolumeBeingWrittenForDatanode(int dataNodeIdx)
577581
final DataNode dn = cluster.getDataNodes().get(dataNodeIdx);
578582
final FileSystem fs = cluster.getFileSystem();
579583
final Path testFile = new Path("/test");
584+
final long lastTimeDiskErrorCheck = dn.getLastDiskErrorCheck();
580585

581586
FSDataOutputStream out = fs.create(testFile, REPLICATION);
582587

@@ -586,6 +591,23 @@ private void testRemoveVolumeBeingWrittenForDatanode(int dataNodeIdx)
586591
out.write(writeBuf);
587592
out.hflush();
588593

594+
// Make FsDatasetSpi#finalizeBlock a time-consuming operation. So if the
595+
// BlockReceiver releases volume reference before finalizeBlock(), the blocks
596+
// on the volume will be removed, and finalizeBlock() throws IOE.
597+
final FsDatasetSpi<? extends FsVolumeSpi> data = dn.data;
598+
dn.data = Mockito.spy(data);
599+
doAnswer(new Answer<Object>() {
600+
public Object answer(InvocationOnMock invocation)
601+
throws IOException, InterruptedException {
602+
Thread.sleep(1000);
603+
// Bypass the argument to FsDatasetImpl#finalizeBlock to verify that
604+
// the block is not removed, since the volume reference should not
605+
// be released at this point.
606+
data.finalizeBlock((ExtendedBlock) invocation.getArguments()[0]);
607+
return null;
608+
}
609+
}).when(dn.data).finalizeBlock(any(ExtendedBlock.class));
610+
589611
final CyclicBarrier barrier = new CyclicBarrier(2);
590612

591613
List<String> oldDirs = getDataDirs(dn);
@@ -612,13 +634,19 @@ public void run() {
612634
out.hflush();
613635
out.close();
614636

637+
reconfigThread.join();
638+
615639
// Verify the file has sufficient replications.
616640
DFSTestUtil.waitReplication(fs, testFile, REPLICATION);
617641
// Read the content back
618642
byte[] content = DFSTestUtil.readFileBuffer(fs, testFile);
619643
assertEquals(BLOCK_SIZE, content.length);
620644

621-
reconfigThread.join();
645+
// If an IOException thrown from BlockReceiver#run, it triggers
646+
// DataNode#checkDiskError(). So we can test whether checkDiskError() is called,
647+
// to see whether there is IOException in BlockReceiver#run().
648+
assertEquals(lastTimeDiskErrorCheck, dn.getLastDiskErrorCheck());
649+
622650
if (!exceptions.isEmpty()) {
623651
throw new IOException(exceptions.get(0).getCause());
624652
}

0 commit comments

Comments
 (0)