Browse Source

HDFS-8314. Move HdfsServerConstants#IO_FILE_BUFFER_SIZE and SMALL_BUFFER_SIZE to the users. Contributed by Li Lu.

Haohui Mai 10 năm trước cách đây
mục cha
commit
4da8490b51
16 tập tin đã thay đổi với 95 bổ sung54 xóa
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  3. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  4. 3 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
  5. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
  6. 0 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
  7. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
  8. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  9. 10 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
  10. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  11. 16 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  12. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
  13. 16 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  14. 6 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
  15. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
  16. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -513,6 +513,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7847. Modify NNThroughputBenchmark to be able to operate on a remote
     NameNode (Charles Lamb via Colin P. McCabe)
 
+    HDFS-8314. Move HdfsServerConstants#IO_FILE_BUFFER_SIZE and
+    SMALL_BUFFER_SIZE to the users. (Li Lu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -238,6 +238,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       new DFSHedgedReadMetrics();
   private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
   private final Sampler<?> traceSampler;
+  private final int smallBufferSize;
 
   public DfsClientConf getConf() {
     return dfsClientConf;
@@ -309,6 +310,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     this.stats = stats;
     this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
     this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
+    this.smallBufferSize = DFSUtil.getSmallBufferSize(conf);
 
     this.ugi = UserGroupInformation.getCurrentUser();
     
@@ -1902,7 +1904,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           //connect to a datanode
           IOStreamPair pair = connectToDN(datanodes[j], timeout, lb);
           out = new DataOutputStream(new BufferedOutputStream(pair.out,
-              HdfsServerConstants.SMALL_BUFFER_SIZE));
+              smallBufferSize));
           in = new DataInputStream(pair.in);
 
           if (LOG.isDebugEnabled()) {
@@ -2067,7 +2069,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
 
     try {
       DataOutputStream out = new DataOutputStream(new BufferedOutputStream(pair.out,
-          HdfsServerConstants.SMALL_BUFFER_SIZE));
+          smallBufferSize));
       DataInputStream in = new DataInputStream(pair.in);
   
       new Sender(out).readBlock(lb.getBlock(), lb.getBlockToken(), clientName,

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -70,6 +70,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderFactory;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
@@ -1514,4 +1515,14 @@ public class DFSUtil {
         .createKeyProviderCryptoExtension(keyProvider);
     return cryptoProvider;
   }
+
+  public static int getIoFileBufferSize(Configuration conf) {
+    return conf.getInt(
+      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
+      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
+  }
+
+  public static int getSmallBufferSize(Configuration conf) {
+    return Math.min(getIoFileBufferSize(conf) / 2, 512);
+  }
 }

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java

@@ -71,7 +71,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
@@ -92,7 +91,6 @@ import org.apache.htrace.Trace;
 import org.apache.htrace.TraceInfo;
 import org.apache.htrace.TraceScope;
 
-import com.google.common.base.Preconditions;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
@@ -123,6 +121,7 @@ import com.google.common.cache.RemovalNotification;
 @InterfaceAudience.Private
 class DataStreamer extends Daemon {
   static final Log LOG = LogFactory.getLog(DataStreamer.class);
+
   /**
    * Create a socket for a write pipeline
    *
@@ -1145,7 +1144,7 @@ class DataStreamer extends Daemon {
       unbufOut = saslStreams.out;
       unbufIn = saslStreams.in;
       out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-          HdfsServerConstants.SMALL_BUFFER_SIZE));
+          DFSUtil.getSmallBufferSize(dfsClient.getConfiguration())));
       in = new DataInputStream(unbufIn);
 
       //send the TRANSFER_BLOCK request
@@ -1425,7 +1424,7 @@ class DataStreamer extends Daemon {
         unbufOut = saslStreams.out;
         unbufIn = saslStreams.in;
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            HdfsServerConstants.SMALL_BUFFER_SIZE));
+            DFSUtil.getSmallBufferSize(dfsClient.getConfiguration())));
         blockReplyStream = new DataInputStream(unbufIn);
 
         //

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java

@@ -118,6 +118,8 @@ public class Dispatcher {
   /** The maximum number of concurrent blocks moves at a datanode */
   private final int maxConcurrentMovesPerNode;
 
+  private final int ioFileBufferSize;
+
   private static class GlobalBlockMap {
     private final Map<Block, DBlock> map = new HashMap<Block, DBlock>();
 
@@ -308,9 +310,9 @@ public class Dispatcher {
         unbufOut = saslStreams.out;
         unbufIn = saslStreams.in;
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+            ioFileBufferSize));
         in = new DataInputStream(new BufferedInputStream(unbufIn,
-            HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+            ioFileBufferSize));
 
         sendRequest(out, eb, accessToken);
         receiveResponse(in);
@@ -801,6 +803,7 @@ public class Dispatcher {
     this.saslClient = new SaslDataTransferClient(conf,
         DataTransferSaslUtil.getSaslPropertiesResolver(conf),
         TrustedChannelResolver.getInstance(conf), nnc.fallbackToSimpleAuth);
+    this.ioFileBufferSize = DFSUtil.getIoFileBufferSize(conf);
   }
 
   public DistributedFileSystem getDistributedFileSystem() {

+ 0 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java

@@ -24,9 +24,7 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@@ -56,12 +54,6 @@ public interface HdfsServerConstants {
   // to 1k.
   int MAX_PATH_LENGTH = 8000;
   int MAX_PATH_DEPTH = 1000;
-  int IO_FILE_BUFFER_SIZE = new HdfsConfiguration().getInt(
-      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
-      CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
-  // Used for writing header etc.
-  int SMALL_BUFFER_SIZE = Math.min(IO_FILE_BUFFER_SIZE / 2,
-      512);
   // An invalid transaction ID that will never be seen in a real namesystem.
   long INVALID_TXID = -12345;
   // Number of generation stamps reserved for legacy blocks.

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java

@@ -33,7 +33,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
 
@@ -60,6 +61,8 @@ public class BlockMetadataHeader {
    */
   private final short version;
   private DataChecksum checksum = null;
+
+  private static final HdfsConfiguration conf = new HdfsConfiguration();
     
   @VisibleForTesting
   public BlockMetadataHeader(short version, DataChecksum checksum) {
@@ -85,7 +88,7 @@ public class BlockMetadataHeader {
     DataInputStream in = null;
     try {
       in = new DataInputStream(new BufferedInputStream(
-        new FileInputStream(metaFile), HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+        new FileInputStream(metaFile), DFSUtil.getIoFileBufferSize(conf)));
       return readDataChecksum(in, metaFile);
     } finally {
       IOUtils.closeStream(in);

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -39,6 +39,7 @@ import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FSOutputSummer;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
@@ -47,7 +48,6 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaInputStreams;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.ReplicaOutputStreams;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -246,7 +246,8 @@ class BlockReceiver implements Closeable {
             out.getClass());
       }
       this.checksumOut = new DataOutputStream(new BufferedOutputStream(
-          streams.getChecksumOut(), HdfsServerConstants.SMALL_BUFFER_SIZE));
+          streams.getChecksumOut(), DFSUtil.getSmallBufferSize(
+          datanode.getConf())));
       // write data chunk header if creating a new replica
       if (isCreate) {
         BlockMetadataHeader.writeHeader(checksumOut, diskChecksum);

+ 10 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java

@@ -34,9 +34,10 @@ import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
@@ -104,8 +105,13 @@ class BlockSender implements java.io.Closeable {
    * not sure if there will be much more improvement.
    */
   private static final int MIN_BUFFER_WITH_TRANSFERTO = 64*1024;
+  private static final int IO_FILE_BUFFER_SIZE;
+  static {
+    HdfsConfiguration conf = new HdfsConfiguration();
+    IO_FILE_BUFFER_SIZE = DFSUtil.getIoFileBufferSize(conf);
+  }
   private static final int TRANSFERTO_BUFFER_SIZE = Math.max(
-      HdfsServerConstants.IO_FILE_BUFFER_SIZE, MIN_BUFFER_WITH_TRANSFERTO);
+      IO_FILE_BUFFER_SIZE, MIN_BUFFER_WITH_TRANSFERTO);
   
   /** the block to read from */
   private final ExtendedBlock block;
@@ -298,7 +304,7 @@ class BlockSender implements java.io.Closeable {
             // storage and computes the checksum.
             if (metaIn.getLength() > BlockMetadataHeader.getHeaderSize()) {
               checksumIn = new DataInputStream(new BufferedInputStream(
-                  metaIn, HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+                  metaIn, IO_FILE_BUFFER_SIZE));
   
               csum = BlockMetadataHeader.readDataChecksum(checksumIn, block);
               keepMetaInOpen = true;
@@ -747,7 +753,7 @@ class BlockSender implements java.io.Closeable {
         pktBufSize += checksumSize * maxChunksPerPacket;
       } else {
         maxChunksPerPacket = Math.max(1,
-            numberOfChunks(HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+            numberOfChunks(IO_FILE_BUFFER_SIZE));
         // Packet size includes both checksum and data
         pktBufSize += (chunkSize + checksumSize) * maxChunksPerPacket;
       }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -2156,7 +2156,7 @@ public class DataNode extends ReconfigurableBase
         unbufIn = saslStreams.in;
         
         out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            HdfsServerConstants.SMALL_BUFFER_SIZE));
+            DFSUtil.getSmallBufferSize(conf)));
         in = new DataInputStream(unbufIn);
         blockSender = new BlockSender(b, 0, b.getNumBytes(), 
             false, false, true, DataNode.this, null, cachingStrategy);

+ 16 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -48,7 +48,9 @@ import java.util.Arrays;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -109,7 +111,9 @@ class DataXceiver extends Receiver implements Runnable {
   private final InputStream socketIn;
   private OutputStream socketOut;
   private BlockReceiver blockReceiver = null;
-  
+  private final int ioFileBufferSize;
+  private final int smallBufferSize;
+
   /**
    * Client Name used in previous operation. Not available on first request
    * on the socket.
@@ -131,6 +135,8 @@ class DataXceiver extends Receiver implements Runnable {
     this.datanode = datanode;
     this.dataXceiverServer = dataXceiverServer;
     this.connectToDnViaHostname = datanode.getDnConf().connectToDnViaHostname;
+    this.ioFileBufferSize = DFSUtil.getIoFileBufferSize(datanode.getConf());
+    this.smallBufferSize = DFSUtil.getSmallBufferSize(datanode.getConf());
     remoteAddress = peer.getRemoteAddressString();
     final int colonIdx = remoteAddress.indexOf(':');
     remoteAddressWithoutPort =
@@ -191,7 +197,7 @@ class DataXceiver extends Receiver implements Runnable {
           socketIn, datanode.getXferAddress().getPort(),
           datanode.getDatanodeId());
         input = new BufferedInputStream(saslStreams.in,
-          HdfsServerConstants.SMALL_BUFFER_SIZE);
+            smallBufferSize);
         socketOut = saslStreams.out;
       } catch (InvalidMagicNumberException imne) {
         if (imne.isHandshake4Encryption()) {
@@ -514,7 +520,7 @@ class DataXceiver extends Receiver implements Runnable {
     long read = 0;
     OutputStream baseStream = getOutputStream();
     DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
-        baseStream, HdfsServerConstants.SMALL_BUFFER_SIZE));
+        baseStream, smallBufferSize));
     checkAccess(out, true, block, blockToken,
         Op.READ_BLOCK, BlockTokenIdentifier.AccessMode.READ);
   
@@ -658,7 +664,7 @@ class DataXceiver extends Receiver implements Runnable {
     final DataOutputStream replyOut = new DataOutputStream(
         new BufferedOutputStream(
             getOutputStream(),
-            HdfsServerConstants.SMALL_BUFFER_SIZE));
+            smallBufferSize));
     checkAccess(replyOut, isClient, block, blockToken,
         Op.WRITE_BLOCK, BlockTokenIdentifier.AccessMode.WRITE);
 
@@ -717,7 +723,7 @@ class DataXceiver extends Receiver implements Runnable {
           unbufMirrorOut = saslStreams.out;
           unbufMirrorIn = saslStreams.in;
           mirrorOut = new DataOutputStream(new BufferedOutputStream(unbufMirrorOut,
-              HdfsServerConstants.SMALL_BUFFER_SIZE));
+              smallBufferSize));
           mirrorIn = new DataInputStream(unbufMirrorIn);
 
           // Do not propagate allowLazyPersist to downstream DataNodes.
@@ -932,7 +938,7 @@ class DataXceiver extends Receiver implements Runnable {
         .getMetaDataInputStream(block);
     
     final DataInputStream checksumIn = new DataInputStream(
-        new BufferedInputStream(metadataIn, HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+        new BufferedInputStream(metadataIn, ioFileBufferSize));
     updateCurrentThreadName("Getting checksum for block " + block);
     try {
       //read metadata file
@@ -1024,7 +1030,7 @@ class DataXceiver extends Receiver implements Runnable {
       // set up response stream
       OutputStream baseStream = getOutputStream();
       reply = new DataOutputStream(new BufferedOutputStream(
-          baseStream, HdfsServerConstants.SMALL_BUFFER_SIZE));
+          baseStream, smallBufferSize));
 
       // send status first
       writeSuccessWithChecksumInfo(blockSender, reply);
@@ -1131,10 +1137,10 @@ class DataXceiver extends Receiver implements Runnable {
         unbufProxyOut = saslStreams.out;
         unbufProxyIn = saslStreams.in;
         
-        proxyOut = new DataOutputStream(new BufferedOutputStream(unbufProxyOut, 
-            HdfsServerConstants.SMALL_BUFFER_SIZE));
+        proxyOut = new DataOutputStream(new BufferedOutputStream(unbufProxyOut,
+            smallBufferSize));
         proxyReply = new DataInputStream(new BufferedInputStream(unbufProxyIn,
-            HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+            ioFileBufferSize));
         
         /* send request to the proxy */
         IoeDuringCopyBlockOperation = true;

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java

@@ -38,10 +38,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DU;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil;
@@ -76,6 +76,7 @@ class BlockPoolSlice {
   private final File lazypersistDir;
   private final File rbwDir; // directory store RBW replica
   private final File tmpDir; // directory store Temporary replica
+  private final int ioFileBufferSize;
   private static final String DU_CACHE_FILE = "dfsUsed";
   private volatile boolean dfsUsedSaved = false;
   private static final int SHUTDOWN_HOOK_PRIORITY = 30;
@@ -108,6 +109,8 @@ class BlockPoolSlice {
       }
     }
 
+    this.ioFileBufferSize = DFSUtil.getIoFileBufferSize(conf);
+
     this.deleteDuplicateReplicas = conf.getBoolean(
         DFSConfigKeys.DFS_DATANODE_DUPLICATE_REPLICA_DELETION,
         DFSConfigKeys.DFS_DATANODE_DUPLICATE_REPLICA_DELETION_DEFAULT);
@@ -612,7 +615,7 @@ class BlockPoolSlice {
       }
       checksumIn = new DataInputStream(
           new BufferedInputStream(new FileInputStream(metaFile),
-              HdfsServerConstants.IO_FILE_BUFFER_SIZE));
+              ioFileBufferSize));
 
       // read and handle the common header here. For now just a version
       final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(

+ 16 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -58,7 +58,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
@@ -66,7 +68,6 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
@@ -247,6 +248,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
   private static final int MAX_BLOCK_EVICTIONS_PER_ITERATION = 3;
 
+  private final int smallBufferSize;
 
   // Used for synchronizing access to usage stats
   private final Object statsLock = new Object();
@@ -264,6 +266,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     this.datanode = datanode;
     this.dataStorage = storage;
     this.conf = conf;
+    this.smallBufferSize = DFSUtil.getSmallBufferSize(conf);
     // The number of volumes required for operation is the total number 
     // of volumes minus the number of failed volumes we can tolerate.
     final int volFailuresTolerated =
@@ -837,19 +840,21 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
    * @throws IOException
    */
   static File[] copyBlockFiles(long blockId, long genStamp, File srcMeta,
-      File srcFile, File destRoot, boolean calculateChecksum)
-      throws IOException {
+      File srcFile, File destRoot, boolean calculateChecksum,
+      int smallBufferSize) throws IOException {
     final File destDir = DatanodeUtil.idToBlockDir(destRoot, blockId);
     final File dstFile = new File(destDir, srcFile.getName());
     final File dstMeta = FsDatasetUtil.getMetaFile(dstFile, genStamp);
-    return copyBlockFiles(srcMeta, srcFile, dstMeta, dstFile, calculateChecksum);
+    return copyBlockFiles(srcMeta, srcFile, dstMeta, dstFile, calculateChecksum,
+        smallBufferSize);
   }
 
   static File[] copyBlockFiles(File srcMeta, File srcFile, File dstMeta,
-                               File dstFile, boolean calculateChecksum)
+                               File dstFile, boolean calculateChecksum,
+                               int smallBufferSize)
       throws IOException {
     if (calculateChecksum) {
-      computeChecksum(srcMeta, dstMeta, srcFile);
+      computeChecksum(srcMeta, dstMeta, srcFile, smallBufferSize);
     } else {
       try {
         Storage.nativeCopyFileUnbuffered(srcMeta, dstMeta, true);
@@ -913,7 +918,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       File[] blockFiles = copyBlockFiles(block.getBlockId(),
           block.getGenerationStamp(), oldMetaFile, oldBlockFile,
           targetVolume.getTmpDir(block.getBlockPoolId()),
-          replicaInfo.isOnTransientStorage());
+          replicaInfo.isOnTransientStorage(), smallBufferSize);
 
       ReplicaInfo newReplicaInfo = new ReplicaInPipeline(
           replicaInfo.getBlockId(), replicaInfo.getGenerationStamp(),
@@ -941,7 +946,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
    * @param blockFile block file for which the checksum will be computed
    * @throws IOException
    */
-  private static void computeChecksum(File srcMeta, File dstMeta, File blockFile)
+  private static void computeChecksum(File srcMeta, File dstMeta,
+      File blockFile, int smallBufferSize)
       throws IOException {
     final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(srcMeta);
     final byte[] data = new byte[1 << 16];
@@ -957,7 +963,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         }
       }
       metaOut = new DataOutputStream(new BufferedOutputStream(
-          new FileOutputStream(dstMeta), HdfsServerConstants.SMALL_BUFFER_SIZE));
+          new FileOutputStream(dstMeta), smallBufferSize));
       BlockMetadataHeader.writeHeader(metaOut, checksum);
 
       int offset = 0;
@@ -2480,7 +2486,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       final File dstMetaFile = FsDatasetUtil.getMetaFile(dstBlockFile, newGS);
       return copyBlockFiles(replicaInfo.getMetaFile(),
           replicaInfo.getBlockFile(),
-          dstMetaFile, dstBlockFile, true);
+          dstMetaFile, dstBlockFile, true, smallBufferSize);
     }
   }
 

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java

@@ -20,10 +20,11 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
 
-import javax.ws.rs.HEAD;
 import java.io.File;
 import java.io.IOException;
 import java.util.HashMap;
@@ -55,6 +56,7 @@ class RamDiskAsyncLazyPersistService {
   private final ThreadGroup threadGroup;
   private Map<File, ThreadPoolExecutor> executors
       = new HashMap<File, ThreadPoolExecutor>();
+  private final static HdfsConfiguration EMPTY_HDFS_CONF = new HdfsConfiguration();
 
   /**
    * Create a RamDiskAsyncLazyPersistService with a set of volumes (specified by their
@@ -234,9 +236,11 @@ class RamDiskAsyncLazyPersistService {
       boolean succeeded = false;
       final FsDatasetImpl dataset = (FsDatasetImpl)datanode.getFSDataset();
       try (FsVolumeReference ref = this.targetVolume) {
+        int smallBufferSize = DFSUtil.getSmallBufferSize(EMPTY_HDFS_CONF);
         // No FsDatasetImpl lock for the file copy
         File targetFiles[] = FsDatasetImpl.copyBlockFiles(
-            blockId, genStamp, metaFile, blockFile, lazyPersistDir, true);
+            blockId, genStamp, metaFile, blockFile, lazyPersistDir, true,
+            smallBufferSize);
 
         // Lock FsDataSetImpl during onCompleteLazyPersist callback
         dataset.onCompleteLazyPersist(bpId, blockId,

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java

@@ -43,8 +43,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
@@ -77,6 +77,7 @@ public class TransferFsImage {
 
   private final static String CONTENT_TYPE = "Content-Type";
   private final static String CONTENT_TRANSFER_ENCODING = "Content-Transfer-Encoding";
+  private final static int IO_FILE_BUFFER_SIZE;
 
   @VisibleForTesting
   static int timeout = 0;
@@ -88,6 +89,7 @@ public class TransferFsImage {
     connectionFactory = URLConnectionFactory
         .newDefaultURLConnectionFactory(conf);
     isSpnegoEnabled = UserGroupInformation.isSecurityEnabled();
+    IO_FILE_BUFFER_SIZE = DFSUtil.getIoFileBufferSize(conf);
   }
 
   private static final Log LOG = LogFactory.getLog(TransferFsImage.class);
@@ -336,7 +338,7 @@ public class TransferFsImage {
   private static void copyFileToStream(OutputStream out, File localfile,
       FileInputStream infile, DataTransferThrottler throttler,
       Canceler canceler) throws IOException {
-    byte buf[] = new byte[HdfsServerConstants.IO_FILE_BUFFER_SIZE];
+    byte buf[] = new byte[IO_FILE_BUFFER_SIZE];
     try {
       CheckpointFaultInjector.getInstance()
           .aboutToSendFile(localfile);
@@ -345,7 +347,7 @@ public class TransferFsImage {
             shouldSendShortFile(localfile)) {
           // Test sending image shorter than localfile
           long len = localfile.length();
-          buf = new byte[(int)Math.min(len/2, HdfsServerConstants.IO_FILE_BUFFER_SIZE)];
+          buf = new byte[(int)Math.min(len/2, IO_FILE_BUFFER_SIZE)];
           // This will read at most half of the image
           // and the rest of the image will be sent over the wire
           infile.read(buf);
@@ -510,7 +512,7 @@ public class TransferFsImage {
       }
       
       int num = 1;
-      byte[] buf = new byte[HdfsServerConstants.IO_FILE_BUFFER_SIZE];
+      byte[] buf = new byte[IO_FILE_BUFFER_SIZE];
       while (num > 0) {
         num = stream.read(buf);
         if (num > 0) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -956,7 +956,7 @@ public class DFSTestUtil {
     final long writeTimeout = dfsClient.getDatanodeWriteTimeout(datanodes.length);
     final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
         NetUtils.getOutputStream(s, writeTimeout),
-        HdfsServerConstants.SMALL_BUFFER_SIZE));
+        DFSUtil.getSmallBufferSize(dfsClient.getConfiguration())));
     final DataInputStream in = new DataInputStream(NetUtils.getInputStream(s));
 
     // send the request