|
@@ -49,6 +49,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
|
|
|
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
@@ -58,6 +59,7 @@ import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.test.PathUtils;
|
|
|
import org.apache.hadoop.test.Whitebox;
|
|
|
+import org.apache.hadoop.util.DataChecksum;
|
|
|
import org.junit.AfterClass;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.BeforeClass;
|
|
@@ -508,6 +510,45 @@ public class TestDFSOutputStream {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout=60000)
|
|
|
+ public void testFirstPacketSizeInNewBlocks() throws IOException {
|
|
|
+ final long blockSize = (long) 1024 * 1024;
|
|
|
+ MiniDFSCluster dfsCluster = cluster;
|
|
|
+ DistributedFileSystem fs = dfsCluster.getFileSystem();
|
|
|
+ Configuration dfsConf = fs.getConf();
|
|
|
+
|
|
|
+ EnumSet<CreateFlag> flags = EnumSet.of(CreateFlag.CREATE);
|
|
|
+ try(FSDataOutputStream fos = fs.create(new Path("/testfile.dat"),
|
|
|
+ FsPermission.getDefault(),
|
|
|
+ flags, 512, (short)3, blockSize, null)) {
|
|
|
+
|
|
|
+ DataChecksum crc32c = DataChecksum.newDataChecksum(
|
|
|
+ DataChecksum.Type.CRC32C, 512);
|
|
|
+
|
|
|
+ long loop = 0;
|
|
|
+ Random r = new Random();
|
|
|
+ byte[] buf = new byte[(int) blockSize];
|
|
|
+ r.nextBytes(buf);
|
|
|
+ fos.write(buf);
|
|
|
+ fos.hflush();
|
|
|
+
|
|
|
+ int chunkSize = crc32c.getBytesPerChecksum() + crc32c.getChecksumSize();
|
|
|
+ int packetContentSize = (dfsConf.getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
|
|
|
+ DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT) -
|
|
|
+ PacketHeader.PKT_MAX_HEADER_LEN) / chunkSize * chunkSize;
|
|
|
+
|
|
|
+ while (loop < 20) {
|
|
|
+ r.nextBytes(buf);
|
|
|
+ fos.write(buf);
|
|
|
+ fos.hflush();
|
|
|
+ loop++;
|
|
|
+ Assert.assertEquals(((DFSOutputStream) fos.getWrappedStream()).packetSize,
|
|
|
+ packetContentSize);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ fs.delete(new Path("/testfile.dat"), true);
|
|
|
+ }
|
|
|
+
|
|
|
@AfterClass
|
|
|
public static void tearDown() {
|
|
|
if (cluster != null) {
|