|
@@ -31,6 +31,8 @@ import java.util.List;
|
|
|
import java.util.Map;
|
|
|
|
|
|
import com.google.common.collect.Lists;
|
|
|
+import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
|
import org.junit.Assert;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -41,8 +43,10 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
import org.junit.Test;
|
|
@@ -940,6 +944,53 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
cleanupFile(fileSys, file);
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout = 120000)
|
|
|
+ public void testFileCloseAfterEnteringMaintenance() throws Exception {
|
|
|
+ LOG.info("Starting testFileCloseAfterEnteringMaintenance");
|
|
|
+ int expirationInMs = 30 * 1000;
|
|
|
+ int numDataNodes = 3;
|
|
|
+ int numNameNodes = 1;
|
|
|
+ getConf().setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY, 2);
|
|
|
+
|
|
|
+ startCluster(numNameNodes, numDataNodes);
|
|
|
+ getCluster().waitActive();
|
|
|
+
|
|
|
+ FSNamesystem fsn = getCluster().getNameNode().getNamesystem();
|
|
|
+ List<String> hosts = new ArrayList<>();
|
|
|
+ for (DataNode dn : getCluster().getDataNodes()) {
|
|
|
+ hosts.add(dn.getDisplayName());
|
|
|
+ putNodeInService(0, dn.getDatanodeUuid());
|
|
|
+ }
|
|
|
+ assertEquals(numDataNodes, fsn.getNumLiveDataNodes());
|
|
|
+
|
|
|
+ Path openFile = new Path("/testClosingFileInMaintenance.dat");
|
|
|
+ // Lets write 2 blocks of data to the openFile
|
|
|
+ writeFile(getCluster().getFileSystem(), openFile, (short) 3);
|
|
|
+
|
|
|
+ // Lets write some more data and keep the file open
|
|
|
+ FSDataOutputStream fsDataOutputStream = getCluster().getFileSystem()
|
|
|
+ .append(openFile);
|
|
|
+ byte[] bytes = new byte[1024];
|
|
|
+ fsDataOutputStream.write(bytes);
|
|
|
+ fsDataOutputStream.hsync();
|
|
|
+
|
|
|
+ LocatedBlocks lbs = NameNodeAdapter.getBlockLocations(
|
|
|
+ getCluster().getNameNode(0), openFile.toString(), 0, 3 * blockSize);
|
|
|
+ DatanodeInfo[] dnInfos4LastBlock = lbs.getLastLocatedBlock().getLocations();
|
|
|
+
|
|
|
+ // Request maintenance for DataNodes 1 and 2 which has the last block.
|
|
|
+ takeNodeOutofService(0,
|
|
|
+ Lists.newArrayList(dnInfos4LastBlock[0].getDatanodeUuid(),
|
|
|
+ dnInfos4LastBlock[1].getDatanodeUuid()),
|
|
|
+ Time.now() + expirationInMs,
|
|
|
+ null, null, AdminStates.ENTERING_MAINTENANCE);
|
|
|
+
|
|
|
+ // Closing the file should succeed even when the
|
|
|
+ // last blocks' nodes are entering maintenance.
|
|
|
+ fsDataOutputStream.close();
|
|
|
+ cleanupFile(getCluster().getFileSystem(), openFile);
|
|
|
+ }
|
|
|
+
|
|
|
static String getFirstBlockFirstReplicaUuid(FileSystem fileSys,
|
|
|
Path name) throws IOException {
|
|
|
DatanodeInfo[] nodes = getFirstBlockReplicasDatanodeInfos(fileSys, name);
|