|
@@ -18,7 +18,6 @@
|
|
|
package org.apache.hadoop.hdfs;
|
|
|
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
-import static org.junit.Assert.assertNull;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
|
|
@@ -30,12 +29,7 @@ import java.util.Iterator;
|
|
|
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;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
@@ -48,8 +42,16 @@ 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.hdfs.server.namenode.NameNodeAdapter;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
+import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
+
|
|
|
+import com.google.common.base.Supplier;
|
|
|
+import com.google.common.collect.Lists;
|
|
|
|
|
|
/**
|
|
|
* This class tests node maintenance.
|
|
@@ -125,8 +127,8 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
|
|
|
// When node is in ENTERING_MAINTENANCE state, it can still serve read
|
|
|
// requests
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas, null,
|
|
|
- nodeOutofService));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas, null,
|
|
|
+ nodeOutofService);
|
|
|
|
|
|
putNodeInService(0, nodeOutofService.getDatanodeUuid());
|
|
|
|
|
@@ -387,8 +389,8 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
|
|
|
// The block should be replicated to another datanode to meet
|
|
|
// expected replication count.
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, expectedReplicasInRead,
|
|
|
- nodeOutofService));
|
|
|
+ checkWithRetry(ns, fileSys, file, expectedReplicasInRead,
|
|
|
+ nodeOutofService);
|
|
|
|
|
|
cleanupFile(fileSys, file);
|
|
|
teardown();
|
|
@@ -548,19 +550,19 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
client.datanodeReport(DatanodeReportType.LIVE).length);
|
|
|
|
|
|
// test 1, verify the replica in IN_MAINTENANCE state isn't in LocatedBlock
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas - 1,
|
|
|
- nodeOutofService));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas - 1,
|
|
|
+ nodeOutofService);
|
|
|
|
|
|
takeNodeOutofService(0, nodeOutofService.getDatanodeUuid(), 0, null,
|
|
|
AdminStates.DECOMMISSIONED);
|
|
|
|
|
|
// test 2 after decommission has completed, the replication count is
|
|
|
// replicas + 1 which includes the decommissioned node.
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas + 1, null));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas + 1, null);
|
|
|
|
|
|
// test 3, put the node in service, replication count should restore.
|
|
|
putNodeInService(0, nodeOutofService.getDatanodeUuid());
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas, null));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas, null);
|
|
|
|
|
|
cleanupFile(fileSys, file);
|
|
|
}
|
|
@@ -587,8 +589,8 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
takeNodeOutofService(0, nodeOutofService.getDatanodeUuid(), Long.MAX_VALUE,
|
|
|
null, AdminStates.IN_MAINTENANCE);
|
|
|
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas - 1,
|
|
|
- nodeOutofService));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas - 1,
|
|
|
+ nodeOutofService);
|
|
|
|
|
|
cleanupFile(fileSys, file);
|
|
|
}
|
|
@@ -631,10 +633,10 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
takeNodeOutofService(0, decommissionDNUuid, 0, null, maintenanceNodes,
|
|
|
AdminStates.DECOMMISSIONED);
|
|
|
// Out of the replicas returned, one is the decommissioned node.
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, repl, maintenanceDN));
|
|
|
+ checkWithRetry(ns, fileSys, file, repl, maintenanceDN);
|
|
|
|
|
|
putNodeInService(0, maintenanceDN);
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, repl + 1, null));
|
|
|
+ checkWithRetry(ns, fileSys, file, repl + 1, null);
|
|
|
|
|
|
cleanupFile(fileSys, file);
|
|
|
teardown();
|
|
@@ -663,7 +665,7 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
AdminStates.IN_MAINTENANCE);
|
|
|
|
|
|
// Verify file replication matches maintenance state min replication
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, 1, null, nodes[0]));
|
|
|
+ checkWithRetry(ns, fileSys, file, 1, null, nodes[0]);
|
|
|
|
|
|
// Put the maintenance nodes back in service
|
|
|
for (DatanodeInfo datanodeInfo : maintenanceDN) {
|
|
@@ -671,7 +673,7 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
}
|
|
|
|
|
|
// Verify file replication catching up to the old state
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, repl, null));
|
|
|
+ checkWithRetry(ns, fileSys, file, repl, null);
|
|
|
|
|
|
cleanupFile(fileSys, file);
|
|
|
}
|
|
@@ -720,19 +722,19 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
|
|
|
// Verify that the nodeOutofService remains in blocksMap and
|
|
|
// # of live replicas For read operation is expected.
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, oldFactor - 1,
|
|
|
- nodeOutofService));
|
|
|
+ checkWithRetry(ns, fileSys, file, oldFactor - 1,
|
|
|
+ nodeOutofService);
|
|
|
|
|
|
final DFSClient client = getDfsClient(0);
|
|
|
client.setReplication(file.toString(), (short)newFactor);
|
|
|
|
|
|
// Verify that the nodeOutofService remains in blocksMap and
|
|
|
// # of live replicas for read operation.
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, expectedLiveReplicas,
|
|
|
- nodeOutofService));
|
|
|
+ checkWithRetry(ns, fileSys, file, expectedLiveReplicas,
|
|
|
+ nodeOutofService);
|
|
|
|
|
|
putNodeInService(0, nodeOutofService.getDatanodeUuid());
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, newFactor, null));
|
|
|
+ checkWithRetry(ns, fileSys, file, newFactor, null);
|
|
|
|
|
|
cleanupFile(fileSys, file);
|
|
|
teardown();
|
|
@@ -765,8 +767,8 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
getFirstBlockFirstReplicaUuid(fileSys, file), Long.MAX_VALUE, null,
|
|
|
AdminStates.IN_MAINTENANCE);
|
|
|
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas - 1,
|
|
|
- nodeOutofService));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas - 1,
|
|
|
+ nodeOutofService);
|
|
|
|
|
|
final DFSClient client = getDfsClient(0);
|
|
|
assertEquals("All datanodes must be alive", numDatanodes,
|
|
@@ -779,16 +781,16 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
client.datanodeReport(DatanodeReportType.LIVE).length);
|
|
|
|
|
|
// Dead maintenance node's blocks should remain in block map.
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas - 1,
|
|
|
- nodeOutofService));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas - 1,
|
|
|
+ nodeOutofService);
|
|
|
|
|
|
// When dead maintenance mode is transitioned to out of maintenance mode,
|
|
|
// its blocks should be removed from block map.
|
|
|
// This will then trigger replication to restore the live replicas back
|
|
|
// to replication factor.
|
|
|
putNodeInService(0, nodeOutofService.getDatanodeUuid());
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas, nodeOutofService,
|
|
|
- null));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas, nodeOutofService,
|
|
|
+ null);
|
|
|
|
|
|
cleanupFile(fileSys, file);
|
|
|
}
|
|
@@ -821,8 +823,8 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
getFirstBlockFirstReplicaUuid(fileSys, file), Long.MAX_VALUE, null,
|
|
|
AdminStates.IN_MAINTENANCE);
|
|
|
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas - 1,
|
|
|
- nodeOutofService));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas - 1,
|
|
|
+ nodeOutofService);
|
|
|
|
|
|
DFSClient client = getDfsClient(0);
|
|
|
assertEquals("All datanodes must be alive", numDatanodes,
|
|
@@ -836,23 +838,23 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
client.datanodeReport(DatanodeReportType.LIVE).length);
|
|
|
|
|
|
// Dead maintenance node's blocks should remain in block map.
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas - 1,
|
|
|
- nodeOutofService));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas - 1,
|
|
|
+ nodeOutofService);
|
|
|
|
|
|
// restart nn, nn will restore 3 live replicas given it doesn't
|
|
|
// know the maintenance node has the replica.
|
|
|
getCluster().restartNameNode(0);
|
|
|
ns = getCluster().getNamesystem(0);
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas, null));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas, null);
|
|
|
|
|
|
// restart dn, nn has 1 maintenance replica and 3 live replicas.
|
|
|
getCluster().restartDataNode(dnProp, true);
|
|
|
getCluster().waitActive();
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas, nodeOutofService));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas, nodeOutofService);
|
|
|
|
|
|
// Put the node in service, a redundant replica should be removed.
|
|
|
putNodeInService(0, nodeOutofService.getDatanodeUuid());
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas, null));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas, null);
|
|
|
|
|
|
cleanupFile(fileSys, file);
|
|
|
}
|
|
@@ -878,12 +880,12 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
writeFile(fileSys, file, replicas, 2);
|
|
|
|
|
|
// Verify nodeOutofService wasn't chosen for write operation.
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas - 1,
|
|
|
- nodeOutofService, null));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas - 1,
|
|
|
+ nodeOutofService, null);
|
|
|
|
|
|
// Put the node back to service, live replicas should be restored.
|
|
|
putNodeInService(0, nodeOutofService.getDatanodeUuid());
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, replicas, null));
|
|
|
+ checkWithRetry(ns, fileSys, file, replicas, null);
|
|
|
|
|
|
cleanupFile(fileSys, file);
|
|
|
}
|
|
@@ -934,12 +936,12 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
client.setReplication(file.toString(), (short) 1);
|
|
|
|
|
|
// Verify the nodeOutofService remains in blocksMap.
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, 1, nodeOutofService));
|
|
|
+ checkWithRetry(ns, fileSys, file, 1, nodeOutofService);
|
|
|
|
|
|
// Restart NN and verify the nodeOutofService remains in blocksMap.
|
|
|
getCluster().restartNameNode(0);
|
|
|
ns = getCluster().getNamesystem(0);
|
|
|
- assertNull(checkWithRetry(ns, fileSys, file, 1, nodeOutofService));
|
|
|
+ checkWithRetry(ns, fileSys, file, 1, nodeOutofService);
|
|
|
|
|
|
cleanupFile(fileSys, file);
|
|
|
}
|
|
@@ -1081,30 +1083,32 @@ public class TestMaintenanceState extends AdminStatesBaseTest {
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
- static String checkWithRetry(FSNamesystem ns, FileSystem fileSys,
|
|
|
- Path name, int repl, DatanodeInfo inMaintenanceNode)
|
|
|
- throws IOException {
|
|
|
- return checkWithRetry(ns, fileSys, name, repl, inMaintenanceNode,
|
|
|
+ static void checkWithRetry(FSNamesystem ns, FileSystem fileSys, Path name,
|
|
|
+ int repl, DatanodeInfo inMaintenanceNode) {
|
|
|
+ checkWithRetry(ns, fileSys, name, repl, inMaintenanceNode,
|
|
|
inMaintenanceNode);
|
|
|
}
|
|
|
|
|
|
- static String checkWithRetry(FSNamesystem ns, FileSystem fileSys,
|
|
|
- Path name, int repl, DatanodeInfo excludedNode,
|
|
|
- DatanodeInfo underMaintenanceNode) throws IOException {
|
|
|
- int tries = 0;
|
|
|
- String output = null;
|
|
|
- while (tries++ < 200) {
|
|
|
- try {
|
|
|
- Thread.sleep(100);
|
|
|
- output = checkFile(ns, fileSys, name, repl, excludedNode,
|
|
|
- underMaintenanceNode);
|
|
|
- if (output == null) {
|
|
|
- break;
|
|
|
+ static void checkWithRetry(final FSNamesystem ns, final FileSystem fileSys,
|
|
|
+ final Path name, final int repl, final DatanodeInfo excludedNode,
|
|
|
+ final DatanodeInfo underMaintenanceNode) {
|
|
|
+ try {
|
|
|
+ GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public Boolean get() {
|
|
|
+ String output = null;
|
|
|
+ try {
|
|
|
+ output = checkFile(ns, fileSys, name, repl, excludedNode,
|
|
|
+ underMaintenanceNode);
|
|
|
+ } catch (Exception ignored) {
|
|
|
+ }
|
|
|
+
|
|
|
+ return (output == null);
|
|
|
}
|
|
|
- } catch (InterruptedException ie) {
|
|
|
- }
|
|
|
+ }, 100, 60000);
|
|
|
+ } catch (Exception ignored) {
|
|
|
}
|
|
|
- return output;
|
|
|
}
|
|
|
|
|
|
static private DatanodeInfo[] getFirstBlockReplicasDatanodeInfos(
|