|
@@ -41,9 +41,12 @@ import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
+import org.apache.hadoop.hdfs.DFSClient.DFSOutputStream;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.DataTransferProtocol;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DataTransferProtocol.BlockConstructionStage;
|
|
|
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
@@ -51,6 +54,7 @@ import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.security.AccessToken;
|
|
|
import org.apache.hadoop.util.DataChecksum;
|
|
|
+import org.junit.Test;
|
|
|
|
|
|
/**
|
|
|
* This tests data transfer protocol handling in the Datanode. It sends
|
|
@@ -94,6 +98,7 @@ public class TestDataTransferProtocol extends TestCase {
|
|
|
|
|
|
DataInputStream in = new DataInputStream(sock.getInputStream());
|
|
|
out.write(sendBuf.toByteArray());
|
|
|
+ out.flush();
|
|
|
try {
|
|
|
in.readFully(retBuf);
|
|
|
} catch (EOFException eof) {
|
|
@@ -137,7 +142,139 @@ public class TestDataTransferProtocol extends TestCase {
|
|
|
in.readFully(arr);
|
|
|
}
|
|
|
|
|
|
- public void testDataTransferProtocol() throws IOException {
|
|
|
+ private void writeZeroLengthPacket(Block block, String description)
|
|
|
+ throws IOException {
|
|
|
+ sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
|
|
|
+ sendOut.writeInt(512); // checksum size
|
|
|
+ sendOut.writeInt(8); // size of packet
|
|
|
+ sendOut.writeLong(block.getNumBytes()); // OffsetInBlock
|
|
|
+ sendOut.writeLong(100); // sequencenumber
|
|
|
+ sendOut.writeBoolean(true); // lastPacketInBlock
|
|
|
+
|
|
|
+ sendOut.writeInt(0); // chunk length
|
|
|
+ sendOut.writeInt(0); // zero checksum
|
|
|
+
|
|
|
+ // mark the end of block
|
|
|
+ sendOut.writeInt(0);
|
|
|
+
|
|
|
+ //ok finally write a block with 0 len
|
|
|
+ SUCCESS.write(recvOut);
|
|
|
+ Text.writeString(recvOut, ""); // first bad node
|
|
|
+ recvOut.writeLong(100); // sequencenumber
|
|
|
+ SUCCESS.write(recvOut);
|
|
|
+ sendRecvData(description, false);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void testWrite(Block block, BlockConstructionStage stage,
|
|
|
+ String description, Boolean eofExcepted) throws IOException {
|
|
|
+ sendBuf.reset();
|
|
|
+ recvBuf.reset();
|
|
|
+ DataTransferProtocol.Sender.opWriteBlock(sendOut,
|
|
|
+ block.getBlockId(), block.getGenerationStamp(), 0,
|
|
|
+ stage, 0L, 0L, 0L, "cl", null,
|
|
|
+ new DatanodeInfo[1], AccessToken.DUMMY_TOKEN);
|
|
|
+ if (eofExcepted) {
|
|
|
+ ERROR.write(recvOut);
|
|
|
+ sendRecvData(description, true);
|
|
|
+ } else {
|
|
|
+ writeZeroLengthPacket(block, description);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test public void testOpWrite() throws IOException {
|
|
|
+ int numDataNodes = 1;
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setBoolean("dfs.support.append", true);
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster(conf, numDataNodes, true, null);
|
|
|
+ try {
|
|
|
+ cluster.waitActive();
|
|
|
+ datanode = cluster.getDataNodes().get(0).dnRegistration;
|
|
|
+ dnAddr = NetUtils.createSocketAddr(datanode.getName());
|
|
|
+ FileSystem fileSys = cluster.getFileSystem();
|
|
|
+
|
|
|
+ /* Test writing to finalized replicas */
|
|
|
+ Path file = new Path("dataprotocol.dat");
|
|
|
+ DFSTestUtil.createFile(fileSys, file, 1L, (short)numDataNodes, 0L);
|
|
|
+ // get the first blockid for the file
|
|
|
+ Block firstBlock = DFSTestUtil.getFirstBlock(fileSys, file);
|
|
|
+ // test PIPELINE_SETUP_CREATE on a finalized block
|
|
|
+ testWrite(firstBlock, BlockConstructionStage.PIPELINE_SETUP_CREATE,
|
|
|
+ "Cannot create an existing block", true);
|
|
|
+ // test PIPELINE_DATA_STREAMING on a finalized block
|
|
|
+ testWrite(firstBlock, BlockConstructionStage.DATA_STREAMING,
|
|
|
+ "Unexpected stage", true);
|
|
|
+ // test PIPELINE_SETUP_STREAMING_RECOVERY on an existing block
|
|
|
+ testWrite(firstBlock,
|
|
|
+ BlockConstructionStage.PIPELINE_SETUP_STREAMING_RECOVERY,
|
|
|
+ "Successful for now", false);
|
|
|
+ // test PIPELINE_SETUP_APPEND on an existing block
|
|
|
+ testWrite(firstBlock,
|
|
|
+ BlockConstructionStage.PIPELINE_SETUP_APPEND,
|
|
|
+ "Append to a finalized replica", false);
|
|
|
+ // test PIPELINE_SETUP_APPEND on an existing block
|
|
|
+ testWrite(firstBlock,
|
|
|
+ BlockConstructionStage.PIPELINE_SETUP_APPEND_RECOVERY,
|
|
|
+ "Recover appending to a finalized replica", false);
|
|
|
+
|
|
|
+ /* Test writing to a new block */
|
|
|
+ long newBlockId = firstBlock.getBlockId() + 1;
|
|
|
+ Block newBlock = new Block(newBlockId, 0,
|
|
|
+ firstBlock.getGenerationStamp());
|
|
|
+
|
|
|
+ // test PIPELINE_SETUP_CREATE on a new block
|
|
|
+ testWrite(newBlock, BlockConstructionStage.PIPELINE_SETUP_CREATE,
|
|
|
+ "Create a new block", false);
|
|
|
+ // test PIPELINE_SETUP_STREAMING_RECOVERY on a new block
|
|
|
+ newBlock.setBlockId(newBlock.getBlockId()+1);
|
|
|
+ testWrite(newBlock,
|
|
|
+ BlockConstructionStage.PIPELINE_SETUP_STREAMING_RECOVERY,
|
|
|
+ "Recover a new block", false);
|
|
|
+ // test PIPELINE_SETUP_APPEND on a new block
|
|
|
+ newBlock.setBlockId(newBlock.getBlockId()+1);
|
|
|
+ testWrite(newBlock,
|
|
|
+ BlockConstructionStage.PIPELINE_SETUP_APPEND,
|
|
|
+ "Cannot append to a new block", true);
|
|
|
+ // test PIPELINE_SETUP_APPEND_RECOVERY on a new block
|
|
|
+ newBlock.setBlockId(newBlock.getBlockId()+1);
|
|
|
+ testWrite(newBlock,
|
|
|
+ BlockConstructionStage.PIPELINE_SETUP_APPEND_RECOVERY,
|
|
|
+ "Cannot append to a new block", true);
|
|
|
+
|
|
|
+ /* Test writing to RBW replicas */
|
|
|
+ // change first block to a RBW
|
|
|
+ DFSOutputStream out = (DFSOutputStream)(fileSys.append(file).
|
|
|
+ getWrappedStream());
|
|
|
+ out.write(1);
|
|
|
+ out.hflush();
|
|
|
+ FSDataInputStream in = fileSys.open(file);
|
|
|
+ firstBlock = DFSTestUtil.getAllBlocks(in).get(0).getBlock();
|
|
|
+
|
|
|
+ try {
|
|
|
+ // test PIPELINE_SETUP_CREATE on a RBW block
|
|
|
+ testWrite(firstBlock, BlockConstructionStage.PIPELINE_SETUP_CREATE,
|
|
|
+ "Cannot create a RBW block", true);
|
|
|
+ // test PIPELINE_SETUP_APPEND on an existing block
|
|
|
+ testWrite(firstBlock, BlockConstructionStage.PIPELINE_SETUP_APPEND,
|
|
|
+ "Cannot append to a RBW replica", true);
|
|
|
+ // test PIPELINE_SETUP_APPEND on an existing block
|
|
|
+ testWrite(firstBlock,
|
|
|
+ BlockConstructionStage.PIPELINE_SETUP_APPEND_RECOVERY,
|
|
|
+ "Cannot append to a RBW replica", true);
|
|
|
+ // test PIPELINE_SETUP_STREAMING_RECOVERY on a RBW block
|
|
|
+ testWrite(firstBlock,
|
|
|
+ BlockConstructionStage.PIPELINE_SETUP_STREAMING_RECOVERY,
|
|
|
+ "Recover a RBW replica", false);
|
|
|
+ } finally {
|
|
|
+ IOUtils.closeStream(in);
|
|
|
+ IOUtils.closeStream(out);
|
|
|
+ }
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+@Test public void testDataTransferProtocol() throws IOException {
|
|
|
Random random = new Random();
|
|
|
int oneMil = 1024*1024;
|
|
|
Path file = new Path("dataprotocol.dat");
|
|
@@ -146,6 +283,7 @@ public class TestDataTransferProtocol extends TestCase {
|
|
|
Configuration conf = new Configuration();
|
|
|
conf.setInt("dfs.replication", numDataNodes);
|
|
|
MiniDFSCluster cluster = new MiniDFSCluster(conf, numDataNodes, true, null);
|
|
|
+ try {
|
|
|
cluster.waitActive();
|
|
|
DFSClient dfsClient = new DFSClient(
|
|
|
new InetSocketAddress("localhost", cluster.getNameNodePort()),
|
|
@@ -178,16 +316,10 @@ public class TestDataTransferProtocol extends TestCase {
|
|
|
|
|
|
/* Test OP_WRITE_BLOCK */
|
|
|
sendBuf.reset();
|
|
|
- sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
|
|
|
- WRITE_BLOCK.write(sendOut);
|
|
|
- sendOut.writeLong(newBlockId); // block id
|
|
|
- sendOut.writeLong(0); // generation stamp
|
|
|
- sendOut.writeInt(0); // targets in pipeline
|
|
|
- sendOut.writeBoolean(false); // recoveryFlag
|
|
|
- Text.writeString(sendOut, "cl");// clientID
|
|
|
- sendOut.writeBoolean(false); // no src node info
|
|
|
- sendOut.writeInt(0); // number of downstream targets
|
|
|
- AccessToken.DUMMY_TOKEN.write(sendOut);
|
|
|
+ DataTransferProtocol.Sender.opWriteBlock(sendOut,
|
|
|
+ newBlockId, 0L, 0,
|
|
|
+ BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
|
|
|
+ new DatanodeInfo[1], AccessToken.DUMMY_TOKEN);
|
|
|
sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
|
|
|
|
|
|
// bad bytes per checksum
|
|
@@ -198,32 +330,10 @@ public class TestDataTransferProtocol extends TestCase {
|
|
|
|
|
|
sendBuf.reset();
|
|
|
recvBuf.reset();
|
|
|
- sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
|
|
|
- WRITE_BLOCK.write(sendOut);
|
|
|
- sendOut.writeLong(newBlockId);
|
|
|
- sendOut.writeLong(0); // generation stamp
|
|
|
- sendOut.writeInt(0); // targets in pipeline
|
|
|
- sendOut.writeBoolean(false); // recoveryFlag
|
|
|
- Text.writeString(sendOut, "cl");// clientID
|
|
|
- sendOut.writeBoolean(false); // no src node info
|
|
|
-
|
|
|
- // bad number of targets
|
|
|
- sendOut.writeInt(-1-random.nextInt(oneMil));
|
|
|
- ERROR.write(recvOut);
|
|
|
- sendRecvData("bad targets len while writing block " + newBlockId, true);
|
|
|
-
|
|
|
- sendBuf.reset();
|
|
|
- recvBuf.reset();
|
|
|
- sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
|
|
|
- WRITE_BLOCK.write(sendOut);
|
|
|
- sendOut.writeLong(++newBlockId);
|
|
|
- sendOut.writeLong(0); // generation stamp
|
|
|
- sendOut.writeInt(0); // targets in pipeline
|
|
|
- sendOut.writeBoolean(false); // recoveryFlag
|
|
|
- Text.writeString(sendOut, "cl");// clientID
|
|
|
- sendOut.writeBoolean(false); // no src node info
|
|
|
- sendOut.writeInt(0);
|
|
|
- AccessToken.DUMMY_TOKEN.write(sendOut);
|
|
|
+ DataTransferProtocol.Sender.opWriteBlock(sendOut,
|
|
|
+ ++newBlockId, 0L, 0,
|
|
|
+ BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
|
|
|
+ new DatanodeInfo[1], AccessToken.DUMMY_TOKEN);
|
|
|
sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
|
|
|
sendOut.writeInt(512);
|
|
|
sendOut.writeInt(4); // size of packet
|
|
@@ -243,16 +353,10 @@ public class TestDataTransferProtocol extends TestCase {
|
|
|
// test for writing a valid zero size block
|
|
|
sendBuf.reset();
|
|
|
recvBuf.reset();
|
|
|
- sendOut.writeShort((short)DataTransferProtocol.DATA_TRANSFER_VERSION);
|
|
|
- WRITE_BLOCK.write(sendOut);
|
|
|
- sendOut.writeLong(++newBlockId);
|
|
|
- sendOut.writeLong(0); // generation stamp
|
|
|
- sendOut.writeInt(0); // targets in pipeline
|
|
|
- sendOut.writeBoolean(false); // recoveryFlag
|
|
|
- Text.writeString(sendOut, "cl");// clientID
|
|
|
- sendOut.writeBoolean(false); // no src node info
|
|
|
- sendOut.writeInt(0);
|
|
|
- AccessToken.DUMMY_TOKEN.write(sendOut);
|
|
|
+ DataTransferProtocol.Sender.opWriteBlock(sendOut,
|
|
|
+ ++newBlockId, 0L, 0,
|
|
|
+ BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L, 0L, 0L, "cl", null,
|
|
|
+ new DatanodeInfo[1], AccessToken.DUMMY_TOKEN);
|
|
|
sendOut.writeByte((byte)DataChecksum.CHECKSUM_CRC32);
|
|
|
sendOut.writeInt(512); // checksum size
|
|
|
sendOut.writeInt(8); // size of packet
|
|
@@ -262,6 +366,7 @@ public class TestDataTransferProtocol extends TestCase {
|
|
|
|
|
|
sendOut.writeInt(0); // chunk length
|
|
|
sendOut.writeInt(0); // zero checksum
|
|
|
+ sendOut.flush();
|
|
|
//ok finally write a block with 0 len
|
|
|
SUCCESS.write(recvOut);
|
|
|
Text.writeString(recvOut, ""); // first bad node
|
|
@@ -353,5 +458,8 @@ public class TestDataTransferProtocol extends TestCase {
|
|
|
Text.writeString(sendOut, "cl");
|
|
|
AccessToken.DUMMY_TOKEN.write(sendOut);
|
|
|
readFile(fileSys, file, fileLen);
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
}
|
|
|
}
|