hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From z..@apache.org
Subject [23/50] [abbrv] hadoop git commit: HDFS-7308. Change the packet chunk size computation in DFSOutputStream in order to enforce packet size <= 64kB. Contributed by Takuya Fukudome
Date Mon, 02 Mar 2015 17:15:39 GMT
HDFS-7308. Change the packet chunk size computation in DFSOutputStream in order to enforce
packet size <= 64kB.  Contributed by Takuya Fukudome


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6eb34243
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6eb34243
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6eb34243

Branch: refs/heads/HDFS-7285
Commit: 6eb3424388a2beeb290dfd670b5f6cf6b720080f
Parents: 54bcb5f
Author: Tsz-Wo Nicholas Sze <szetszwo@hortonworks.com>
Authored: Fri Feb 27 23:45:37 2015 +0800
Committer: Zhe Zhang <zhezhang@cloudera.com>
Committed: Mon Mar 2 09:13:53 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  3 +-
 .../apache/hadoop/hdfs/TestDFSOutputStream.java | 31 ++++++++++++++++++++
 3 files changed, 36 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6eb34243/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 8556afd..b2422d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -682,6 +682,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7819. Log WARN message for the blocks which are not in Block ID based
     layout (Rakesh R via Colin P. McCabe)
 
+    HDFS-7308. Change the packet chunk size computation in DFSOutputStream in
+    order to enforce packet size <= 64kB.  (Takuya Fukudome via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6eb34243/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 9d7dca9..b3e8c97 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -1851,8 +1851,9 @@ public class DFSOutputStream extends FSOutputSummer
   }
 
   private void computePacketChunkSize(int psize, int csize) {
+    final int bodySize = psize - PacketHeader.PKT_MAX_HEADER_LEN;
     final int chunkSize = csize + getChecksumSize();
-    chunksPerPacket = Math.max(psize/chunkSize, 1);
+    chunksPerPacket = Math.max(bodySize/chunkSize, 1);
     packetSize = chunkSize*chunksPerPacket;
     if (DFSClient.LOG.isDebugEnabled()) {
       DFSClient.LOG.debug("computePacketChunkSize: src=" + src +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6eb34243/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
----------------------------------------------------------------------
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 678a3b8..7269e39 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
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.conf.Configuration;
@@ -66,6 +68,35 @@ public class TestDFSOutputStream {
     dos.close();
   }
 
+  /**
+   * The computePacketChunkSize() method of DFSOutputStream should set the actual
+   * packet size < 64kB. See HDFS-7308 for details.
+   */
+  @Test
+  public void testComputePacketChunkSize()
+      throws Exception {
+    DistributedFileSystem fs = cluster.getFileSystem();
+    FSDataOutputStream os = fs.create(new Path("/test"));
+    DFSOutputStream dos = (DFSOutputStream) Whitebox.getInternalState(os,
+        "wrappedStream");
+
+    final int packetSize = 64*1024;
+    final int bytesPerChecksum = 512;
+
+    Method method = dos.getClass().getDeclaredMethod("computePacketChunkSize",
+        int.class, int.class);
+    method.setAccessible(true);
+    method.invoke(dos, packetSize, bytesPerChecksum);
+
+    Field field = dos.getClass().getDeclaredField("packetSize");
+    field.setAccessible(true);
+
+    Assert.assertTrue((Integer) field.get(dos) + 33 < packetSize);
+    // If PKT_MAX_HEADER_LEN is 257, actual packet size come to over 64KB
+    // without a fix on HDFS-7308.
+    Assert.assertTrue((Integer) field.get(dos) + 257 < packetSize);
+  }
+
   @AfterClass
   public static void tearDown() {
     cluster.shutdown();


Mime
View raw message