diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index a1bfb7f5d594e..9b045824c570a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -542,6 +542,11 @@ protected void adjustChunkBoundary() { } else { psize = (int) Math .min(blockSize - getStreamer().getBytesCurBlock(), writePacketSize); + if (psize < writePacketSize) { + final int chunkSize = bytesPerChecksum + getChecksumSize(); + int numPackets = (psize + bytesPerChecksum - 1) / bytesPerChecksum; + psize = PacketHeader.PKT_MAX_HEADER_LEN + numPackets * chunkSize; + } } computePacketChunkSize(psize, bytesPerChecksum); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java index bdb91f91bc5e4..fe88ee281b3b6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java @@ -549,6 +549,45 @@ public void testFirstPacketSizeInNewBlocks() throws IOException { fs.delete(new Path("/testfile.dat"), true); } + @Test(timeout = 60000) + public void testLastPacketSizeInBlocks() throws IOException { + final long blockSize = (long) 1024 * 1024; + MiniDFSCluster dfsCluster = cluster; + DistributedFileSystem fs = dfsCluster.getFileSystem(); + Configuration dfsConf = fs.getConf(); + + EnumSet 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) {