|
@@ -31,13 +31,19 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
|
|
+import org.apache.hadoop.hdfs.server.common.Storage;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
|
|
+import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
+import java.io.RandomAccessFile;
|
|
|
+import java.nio.channels.FileChannel;
|
|
|
+import java.nio.channels.FileLock;
|
|
|
+import java.nio.channels.OverlappingFileLockException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collection;
|
|
@@ -46,13 +52,19 @@ import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
+import org.apache.commons.logging.Log;
|
|
|
+import org.apache.commons.logging.LogFactory;
|
|
|
+
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertFalse;
|
|
|
+import static org.junit.Assert.assertNotNull;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
|
|
|
public class TestDataNodeHotSwapVolumes {
|
|
|
+ private static final Log LOG = LogFactory.getLog(
|
|
|
+ TestDataNodeHotSwapVolumes.class);
|
|
|
private static final int BLOCK_SIZE = 512;
|
|
|
private MiniDFSCluster cluster;
|
|
|
|
|
@@ -179,8 +191,10 @@ public class TestDataNodeHotSwapVolumes {
|
|
|
DataNode.ChangedVolumes changedVolumes =dn.parseChangedVolumes();
|
|
|
List<StorageLocation> newVolumes = changedVolumes.newLocations;
|
|
|
assertEquals(2, newVolumes.size());
|
|
|
- assertEquals("/foo/path1", newVolumes.get(0).getFile().getAbsolutePath());
|
|
|
- assertEquals("/foo/path2", newVolumes.get(1).getFile().getAbsolutePath());
|
|
|
+ assertEquals(new File("/foo/path1").getAbsolutePath(),
|
|
|
+ newVolumes.get(0).getFile().getAbsolutePath());
|
|
|
+ assertEquals(new File("/foo/path2").getAbsolutePath(),
|
|
|
+ newVolumes.get(1).getFile().getAbsolutePath());
|
|
|
|
|
|
List<StorageLocation> removedVolumes = changedVolumes.deactivateLocations;
|
|
|
assertEquals(oldLocations.size(), removedVolumes.size());
|
|
@@ -371,6 +385,8 @@ public class TestDataNodeHotSwapVolumes {
|
|
|
String newDirs = oldDirs.iterator().next(); // Keep the first volume.
|
|
|
dn.reconfigurePropertyImpl(
|
|
|
DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
|
|
|
+ assertFileLocksReleased(
|
|
|
+ new ArrayList<String>(oldDirs).subList(1, oldDirs.size()));
|
|
|
dn.scheduleAllBlockReport(0);
|
|
|
|
|
|
try {
|
|
@@ -409,6 +425,8 @@ public class TestDataNodeHotSwapVolumes {
|
|
|
String newDirs = oldDirs.iterator().next(); // Keep the first volume.
|
|
|
dn.reconfigurePropertyImpl(
|
|
|
DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
|
|
|
+ assertFileLocksReleased(
|
|
|
+ new ArrayList<String>(oldDirs).subList(1, oldDirs.size()));
|
|
|
|
|
|
// Force DataNode to report missing blocks.
|
|
|
dn.scheduleAllBlockReport(0);
|
|
@@ -420,4 +438,44 @@ public class TestDataNodeHotSwapVolumes {
|
|
|
// Wait NameNode to replica missing blocks.
|
|
|
DFSTestUtil.waitReplication(fs, testFile, replFactor);
|
|
|
}
|
|
|
-}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Asserts that the storage lock file in each given directory has been
|
|
|
+ * released. This method works by trying to acquire the lock file itself. If
|
|
|
+ * locking fails here, then the main code must have failed to release it.
|
|
|
+ *
|
|
|
+ * @param dirs every storage directory to check
|
|
|
+ * @throws IOException if there is an unexpected I/O error
|
|
|
+ */
|
|
|
+ private static void assertFileLocksReleased(Collection<String> dirs)
|
|
|
+ throws IOException {
|
|
|
+ for (String dir: dirs) {
|
|
|
+ StorageLocation sl = StorageLocation.parse(dir);
|
|
|
+ File lockFile = new File(sl.getFile(), Storage.STORAGE_FILE_LOCK);
|
|
|
+ RandomAccessFile raf = null;
|
|
|
+ FileChannel channel = null;
|
|
|
+ FileLock lock = null;
|
|
|
+ try {
|
|
|
+ raf = new RandomAccessFile(lockFile, "rws");
|
|
|
+ channel = raf.getChannel();
|
|
|
+ lock = channel.tryLock();
|
|
|
+ assertNotNull(String.format(
|
|
|
+ "Lock file at %s appears to be held by a different process.",
|
|
|
+ lockFile.getAbsolutePath()), lock);
|
|
|
+ } catch (OverlappingFileLockException e) {
|
|
|
+ fail(String.format("Must release lock file at %s.",
|
|
|
+ lockFile.getAbsolutePath()));
|
|
|
+ } finally {
|
|
|
+ if (lock != null) {
|
|
|
+ try {
|
|
|
+ lock.release();
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn(String.format("I/O error releasing file lock %s.",
|
|
|
+ lockFile.getAbsolutePath()), e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ IOUtils.cleanup(null, channel, raf);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+}
|