|
@@ -15,22 +15,15 @@
|
|
* See the License for the specific language governing permissions and
|
|
* See the License for the specific language governing permissions and
|
|
* limitations under the License.
|
|
* limitations under the License.
|
|
*/
|
|
*/
|
|
-package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
+package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
|
|
|
|
|
|
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 java.io.IOException;
|
|
import java.io.IOException;
|
|
-import java.util.List;
|
|
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
-
|
|
|
|
import java.net.SocketTimeoutException;
|
|
import java.net.SocketTimeoutException;
|
|
-import org.apache.hadoop.io.Writable;
|
|
|
|
-import org.apache.hadoop.io.LongWritable;
|
|
|
|
-import org.apache.hadoop.ipc.RPC;
|
|
|
|
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
|
|
|
|
-import org.apache.hadoop.ipc.Server;
|
|
|
|
-import org.apache.hadoop.net.NetUtils;
|
|
|
|
|
|
+import java.util.List;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
@@ -41,16 +34,28 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
|
|
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
|
|
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
|
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
|
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
|
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
|
|
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
|
|
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
|
|
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
|
|
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
|
|
-import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
|
|
|
|
|
|
+import org.apache.hadoop.io.LongWritable;
|
|
|
|
+import org.apache.hadoop.io.Writable;
|
|
|
|
+import org.apache.hadoop.ipc.RPC;
|
|
|
|
+import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
|
|
|
|
+import org.apache.hadoop.ipc.Server;
|
|
|
|
+import org.apache.hadoop.net.NetUtils;
|
|
import org.junit.Assert;
|
|
import org.junit.Assert;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
|
|
|
|
@@ -103,14 +108,14 @@ public class TestInterDatanodeProtocol {
|
|
}
|
|
}
|
|
|
|
|
|
public static void checkMetaInfo(ExtendedBlock b, DataNode dn) throws IOException {
|
|
public static void checkMetaInfo(ExtendedBlock b, DataNode dn) throws IOException {
|
|
- Block metainfo = dn.data.getStoredBlock(b.getBlockPoolId(), b.getBlockId());
|
|
|
|
|
|
+ Block metainfo = DataNodeTestUtils.getFSDataset(dn).getStoredBlock(
|
|
|
|
+ b.getBlockPoolId(), b.getBlockId());
|
|
Assert.assertEquals(b.getBlockId(), metainfo.getBlockId());
|
|
Assert.assertEquals(b.getBlockId(), metainfo.getBlockId());
|
|
Assert.assertEquals(b.getNumBytes(), metainfo.getNumBytes());
|
|
Assert.assertEquals(b.getNumBytes(), metainfo.getNumBytes());
|
|
}
|
|
}
|
|
|
|
|
|
public static LocatedBlock getLastLocatedBlock(
|
|
public static LocatedBlock getLastLocatedBlock(
|
|
- ClientProtocol namenode, String src
|
|
|
|
- ) throws IOException {
|
|
|
|
|
|
+ ClientProtocol namenode, String src) throws IOException {
|
|
//get block info for the last block
|
|
//get block info for the last block
|
|
LocatedBlocks locations = namenode.getBlockLocations(src, 0, Long.MAX_VALUE);
|
|
LocatedBlocks locations = namenode.getBlockLocations(src, 0, Long.MAX_VALUE);
|
|
List<LocatedBlock> blocks = locations.getLocatedBlocks();
|
|
List<LocatedBlock> blocks = locations.getLocatedBlocks();
|
|
@@ -148,13 +153,11 @@ public class TestInterDatanodeProtocol {
|
|
|
|
|
|
//connect to a data node
|
|
//connect to a data node
|
|
DataNode datanode = cluster.getDataNode(datanodeinfo[0].getIpcPort());
|
|
DataNode datanode = cluster.getDataNode(datanodeinfo[0].getIpcPort());
|
|
- InterDatanodeProtocol idp = DataNode.createInterDataNodeProtocolProxy(
|
|
|
|
- datanodeinfo[0], conf, datanode.getDnConf().socketTimeout);
|
|
|
|
|
|
+ InterDatanodeProtocol idp = DataNodeTestUtils.createInterDatanodeProtocolProxy(
|
|
|
|
+ datanode, datanodeinfo[0], conf);
|
|
|
|
|
|
//stop block scanner, so we could compare lastScanTime
|
|
//stop block scanner, so we could compare lastScanTime
|
|
- if (datanode.blockScanner != null) {
|
|
|
|
- datanode.blockScanner.shutdown();
|
|
|
|
- }
|
|
|
|
|
|
+ DataNodeTestUtils.shutdownBlockScanner(datanode);
|
|
|
|
|
|
//verify BlockMetaDataInfo
|
|
//verify BlockMetaDataInfo
|
|
ExtendedBlock b = locatedblock.getBlock();
|
|
ExtendedBlock b = locatedblock.getBlock();
|
|
@@ -187,14 +190,14 @@ public class TestInterDatanodeProtocol {
|
|
}
|
|
}
|
|
|
|
|
|
/** Test
|
|
/** Test
|
|
- * {@link FSDataset#initReplicaRecovery(String, ReplicasMap, Block, long)}
|
|
|
|
|
|
+ * {@link FsDatasetImpl#initReplicaRecovery(String, ReplicaMap, Block, long)}
|
|
*/
|
|
*/
|
|
@Test
|
|
@Test
|
|
public void testInitReplicaRecovery() throws IOException {
|
|
public void testInitReplicaRecovery() throws IOException {
|
|
final long firstblockid = 10000L;
|
|
final long firstblockid = 10000L;
|
|
final long gs = 7777L;
|
|
final long gs = 7777L;
|
|
final long length = 22L;
|
|
final long length = 22L;
|
|
- final ReplicasMap map = new ReplicasMap(this);
|
|
|
|
|
|
+ final ReplicaMap map = new ReplicaMap(this);
|
|
String bpid = "BP-TEST";
|
|
String bpid = "BP-TEST";
|
|
final Block[] blocks = new Block[5];
|
|
final Block[] blocks = new Block[5];
|
|
for(int i = 0; i < blocks.length; i++) {
|
|
for(int i = 0; i < blocks.length; i++) {
|
|
@@ -208,7 +211,8 @@ public class TestInterDatanodeProtocol {
|
|
final ReplicaInfo originalInfo = map.get(bpid, b);
|
|
final ReplicaInfo originalInfo = map.get(bpid, b);
|
|
|
|
|
|
final long recoveryid = gs + 1;
|
|
final long recoveryid = gs + 1;
|
|
- final ReplicaRecoveryInfo recoveryInfo = FSDataset.initReplicaRecovery(bpid, map, blocks[0], recoveryid);
|
|
|
|
|
|
+ final ReplicaRecoveryInfo recoveryInfo = FsDatasetImpl.initReplicaRecovery(
|
|
|
|
+ bpid, map, blocks[0], recoveryid);
|
|
assertEquals(originalInfo, recoveryInfo);
|
|
assertEquals(originalInfo, recoveryInfo);
|
|
|
|
|
|
final ReplicaUnderRecovery updatedInfo = (ReplicaUnderRecovery)map.get(bpid, b);
|
|
final ReplicaUnderRecovery updatedInfo = (ReplicaUnderRecovery)map.get(bpid, b);
|
|
@@ -217,7 +221,7 @@ public class TestInterDatanodeProtocol {
|
|
|
|
|
|
//recover one more time
|
|
//recover one more time
|
|
final long recoveryid2 = gs + 2;
|
|
final long recoveryid2 = gs + 2;
|
|
- final ReplicaRecoveryInfo recoveryInfo2 = FSDataset.initReplicaRecovery(bpid, map, blocks[0], recoveryid2);
|
|
|
|
|
|
+ final ReplicaRecoveryInfo recoveryInfo2 = FsDatasetImpl.initReplicaRecovery(bpid, map, blocks[0], recoveryid2);
|
|
assertEquals(originalInfo, recoveryInfo2);
|
|
assertEquals(originalInfo, recoveryInfo2);
|
|
|
|
|
|
final ReplicaUnderRecovery updatedInfo2 = (ReplicaUnderRecovery)map.get(bpid, b);
|
|
final ReplicaUnderRecovery updatedInfo2 = (ReplicaUnderRecovery)map.get(bpid, b);
|
|
@@ -226,7 +230,7 @@ public class TestInterDatanodeProtocol {
|
|
|
|
|
|
//case RecoveryInProgressException
|
|
//case RecoveryInProgressException
|
|
try {
|
|
try {
|
|
- FSDataset.initReplicaRecovery(bpid, map, b, recoveryid);
|
|
|
|
|
|
+ FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid);
|
|
Assert.fail();
|
|
Assert.fail();
|
|
}
|
|
}
|
|
catch(RecoveryInProgressException ripe) {
|
|
catch(RecoveryInProgressException ripe) {
|
|
@@ -237,7 +241,7 @@ public class TestInterDatanodeProtocol {
|
|
{ // BlockRecoveryFI_01: replica not found
|
|
{ // BlockRecoveryFI_01: replica not found
|
|
final long recoveryid = gs + 1;
|
|
final long recoveryid = gs + 1;
|
|
final Block b = new Block(firstblockid - 1, length, gs);
|
|
final Block b = new Block(firstblockid - 1, length, gs);
|
|
- ReplicaRecoveryInfo r = FSDataset.initReplicaRecovery(bpid, map, b, recoveryid);
|
|
|
|
|
|
+ ReplicaRecoveryInfo r = FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid);
|
|
Assert.assertNull("Data-node should not have this replica.", r);
|
|
Assert.assertNull("Data-node should not have this replica.", r);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -245,7 +249,7 @@ public class TestInterDatanodeProtocol {
|
|
final long recoveryid = gs - 1;
|
|
final long recoveryid = gs - 1;
|
|
final Block b = new Block(firstblockid + 1, length, gs);
|
|
final Block b = new Block(firstblockid + 1, length, gs);
|
|
try {
|
|
try {
|
|
- FSDataset.initReplicaRecovery(bpid, map, b, recoveryid);
|
|
|
|
|
|
+ FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid);
|
|
Assert.fail();
|
|
Assert.fail();
|
|
}
|
|
}
|
|
catch(IOException ioe) {
|
|
catch(IOException ioe) {
|
|
@@ -258,7 +262,7 @@ public class TestInterDatanodeProtocol {
|
|
final long recoveryid = gs + 1;
|
|
final long recoveryid = gs + 1;
|
|
final Block b = new Block(firstblockid, length, gs+1);
|
|
final Block b = new Block(firstblockid, length, gs+1);
|
|
try {
|
|
try {
|
|
- FSDataset.initReplicaRecovery(bpid, map, b, recoveryid);
|
|
|
|
|
|
+ FsDatasetImpl.initReplicaRecovery(bpid, map, b, recoveryid);
|
|
fail("InitReplicaRecovery should fail because replica's " +
|
|
fail("InitReplicaRecovery should fail because replica's " +
|
|
"gs is less than the block's gs");
|
|
"gs is less than the block's gs");
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
@@ -270,7 +274,7 @@ public class TestInterDatanodeProtocol {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Test for
|
|
* Test for
|
|
- * {@link FSDataset#updateReplicaUnderRecovery(ExtendedBlock, long, long)}
|
|
|
|
|
|
+ * {@link FsDatasetImpl#updateReplicaUnderRecovery(ExtendedBlock, long, long)}
|
|
* */
|
|
* */
|
|
@Test
|
|
@Test
|
|
public void testUpdateReplicaUnderRecovery() throws IOException {
|
|
public void testUpdateReplicaUnderRecovery() throws IOException {
|
|
@@ -296,22 +300,22 @@ public class TestInterDatanodeProtocol {
|
|
//get DataNode and FSDataset objects
|
|
//get DataNode and FSDataset objects
|
|
final DataNode datanode = cluster.getDataNode(datanodeinfo[0].getIpcPort());
|
|
final DataNode datanode = cluster.getDataNode(datanodeinfo[0].getIpcPort());
|
|
Assert.assertTrue(datanode != null);
|
|
Assert.assertTrue(datanode != null);
|
|
- Assert.assertTrue(datanode.data instanceof FSDataset);
|
|
|
|
- final FSDataset fsdataset = (FSDataset)datanode.data;
|
|
|
|
|
|
|
|
//initReplicaRecovery
|
|
//initReplicaRecovery
|
|
final ExtendedBlock b = locatedblock.getBlock();
|
|
final ExtendedBlock b = locatedblock.getBlock();
|
|
final long recoveryid = b.getGenerationStamp() + 1;
|
|
final long recoveryid = b.getGenerationStamp() + 1;
|
|
final long newlength = b.getNumBytes() - 1;
|
|
final long newlength = b.getNumBytes() - 1;
|
|
|
|
+ final FsDatasetSpi<?> fsdataset = DataNodeTestUtils.getFSDataset(datanode);
|
|
final ReplicaRecoveryInfo rri = fsdataset.initReplicaRecovery(
|
|
final ReplicaRecoveryInfo rri = fsdataset.initReplicaRecovery(
|
|
new RecoveringBlock(b, null, recoveryid));
|
|
new RecoveringBlock(b, null, recoveryid));
|
|
|
|
|
|
//check replica
|
|
//check replica
|
|
- final ReplicaInfo replica = fsdataset.fetchReplicaInfo(bpid, b.getBlockId());
|
|
|
|
|
|
+ final ReplicaInfo replica = FsDatasetTestUtil.fetchReplicaInfo(
|
|
|
|
+ fsdataset, bpid, b.getBlockId());
|
|
Assert.assertEquals(ReplicaState.RUR, replica.getState());
|
|
Assert.assertEquals(ReplicaState.RUR, replica.getState());
|
|
|
|
|
|
//check meta data before update
|
|
//check meta data before update
|
|
- FSDataset.checkReplicaFiles(replica);
|
|
|
|
|
|
+ FsDatasetImpl.checkReplicaFiles(replica);
|
|
|
|
|
|
//case "THIS IS NOT SUPPOSED TO HAPPEN"
|
|
//case "THIS IS NOT SUPPOSED TO HAPPEN"
|
|
//with (block length) != (stored replica's on disk length).
|
|
//with (block length) != (stored replica's on disk length).
|