|
@@ -34,12 +34,16 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
|
|
|
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
|
|
import org.apache.hadoop.hdfs.server.common.Storage;
|
|
import org.apache.hadoop.hdfs.server.common.Storage;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
|
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
|
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
|
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
@@ -59,6 +63,7 @@ import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
|
+import org.mockito.Mockito;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
|
|
import static org.hamcrest.CoreMatchers.anyOf;
|
|
import static org.hamcrest.CoreMatchers.anyOf;
|
|
@@ -70,6 +75,9 @@ import static org.junit.Assert.assertFalse;
|
|
import static org.junit.Assert.assertThat;
|
|
import static org.junit.Assert.assertThat;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.fail;
|
|
import static org.junit.Assert.fail;
|
|
|
|
+import static org.mockito.Matchers.any;
|
|
|
|
+import static org.mockito.Matchers.anyString;
|
|
|
|
+import static org.mockito.Mockito.timeout;
|
|
|
|
|
|
public class TestDataNodeHotSwapVolumes {
|
|
public class TestDataNodeHotSwapVolumes {
|
|
private static final Log LOG = LogFactory.getLog(
|
|
private static final Log LOG = LogFactory.getLog(
|
|
@@ -702,4 +710,38 @@ public class TestDataNodeHotSwapVolumes {
|
|
// More data has been written to this volume.
|
|
// More data has been written to this volume.
|
|
assertTrue(restoredVolume.getDfsUsed() > used);
|
|
assertTrue(restoredVolume.getDfsUsed() > used);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ /** Test that a full block report is sent after hot swapping volumes */
|
|
|
|
+ @Test(timeout=100000)
|
|
|
|
+ public void testFullBlockReportAfterRemovingVolumes()
|
|
|
|
+ throws IOException, ReconfigurationException {
|
|
|
|
+
|
|
|
|
+ Configuration conf = new Configuration();
|
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
|
|
|
+
|
|
|
|
+ // Similar to TestTriggerBlockReport, set a really long value for
|
|
|
|
+ // dfs.heartbeat.interval, so that incremental block reports and heartbeats
|
|
|
|
+ // won't be sent during this test unless they're triggered
|
|
|
|
+ // manually.
|
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10800000L);
|
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1080L);
|
|
|
|
+
|
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
|
|
|
+ cluster.waitActive();
|
|
|
|
+
|
|
|
|
+ final DataNode dn = cluster.getDataNodes().get(0);
|
|
|
|
+ DatanodeProtocolClientSideTranslatorPB spy =
|
|
|
|
+ DataNodeTestUtils.spyOnBposToNN(dn, cluster.getNameNode());
|
|
|
|
+
|
|
|
|
+ // Remove a data dir from datanode
|
|
|
|
+ File dataDirToKeep = new File(cluster.getDataDirectory(), "data1");
|
|
|
|
+ dn.reconfigurePropertyImpl(DFS_DATANODE_DATA_DIR_KEY, dataDirToKeep.toString());
|
|
|
|
+
|
|
|
|
+ // We should get 1 full report
|
|
|
|
+ Mockito.verify(spy, timeout(60000).times(1)).blockReport(
|
|
|
|
+ any(DatanodeRegistration.class),
|
|
|
|
+ anyString(),
|
|
|
|
+ any(StorageBlockReport[].class),
|
|
|
|
+ any(BlockReportContext.class));
|
|
|
|
+ }
|
|
}
|
|
}
|