HDFS-8051. FsVolumeList#addVolume should release volume reference if not put it into BlockScanner. (Lei (Eddy) Xu via Colin P. McCabe)

(cherry picked from commit ef591b1d6a)
This commit is contained in:
Colin Patrick Mccabe 2015-04-03 16:34:23 -07:00
parent 0c5069c432
commit b26ba22a90
3 changed files with 27 additions and 0 deletions

View File

@ -1075,6 +1075,9 @@ Release 2.7.0 - UNRELEASED
HDFS-7996. After swapping a volume, BlockReceiver reports
ReplicaNotFoundException (Lei (Eddy) Xu via Colin P. McCabe)
HDFS-8051. FsVolumeList#addVolume should release volume reference if not
put it into BlockScanner. (Lei (Eddy) Xu via Colin P. McCabe)
BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
import org.apache.hadoop.util.Time;
@ -292,6 +293,10 @@ class FsVolumeList {
}
if (blockScanner != null) {
blockScanner.addVolumeScanner(ref);
} else {
// If the volume is not put into a volume scanner, it does not need to
// hold the reference.
IOUtils.cleanup(FsDatasetImpl.LOG, ref);
}
// If the volume is used to replace a failed volume, it needs to reset the
// volume failure info for this volume.

View File

@ -35,6 +35,7 @@ import java.util.Collections;
import java.util.List;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
public class TestFsVolumeList {
@ -101,4 +102,22 @@ public class TestFsVolumeList {
// checkDirs() should ignore the 2nd volume since it is closed.
volumeList.checkDirs();
}
@Test
public void testReleaseVolumeRefIfNoBlockScanner() throws IOException {
FsVolumeList volumeList = new FsVolumeList(
Collections.<VolumeFailureInfo>emptyList(), null, blockChooser);
File volDir = new File(baseDir, "volume-0");
volDir.mkdirs();
FsVolumeImpl volume = new FsVolumeImpl(dataset, "storage-id", volDir,
conf, StorageType.DEFAULT);
FsVolumeReference ref = volume.obtainReference();
volumeList.addVolume(ref);
try {
ref.close();
fail("Should throw exception because the reference is closed in "
+ "VolumeList#addVolume().");
} catch (IllegalStateException e) {
}
}
}