Browse Source

HDFS-8057 Move BlockReader implementation to the client implementation package. Contributed by Takanobu Asanuma

Tsz-Wo Nicholas Sze 9 years ago
parent
commit
f308561f1d
31 changed files with 170 additions and 117 deletions
  1. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  2. 14 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderFactory.java
  3. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderLocal.java
  4. 3 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderLocalLegacy.java
  5. 13 11
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderRemote.java
  6. 10 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderRemote2.java
  7. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderUtil.java
  8. 3 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/ExternalBlockReader.java
  9. 1 1
      hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
  10. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java
  11. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  12. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
  13. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java
  14. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestConnCache.java
  15. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDisableConnCache.java
  16. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java
  17. 11 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/BlockReaderTestUtil.java
  18. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderBase.java
  19. 16 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderFactory.java
  20. 47 41
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderLocal.java
  21. 10 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderLocalLegacy.java
  22. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderRemote.java
  23. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderRemote2.java
  24. 7 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestClientBlockVerification.java
  25. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java
  26. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
  27. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
  28. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCacheRevocation.java
  29. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
  30. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java
  31. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -56,6 +56,7 @@ import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSUtilClient.CorruptedBlocks;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderFactory;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;

+ 14 - 6
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderFactory.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
 
@@ -34,7 +34,15 @@ import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.BlockReader;
+import org.apache.hadoop.hdfs.ClientContext;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.ExtendedBlockId;
+import org.apache.hadoop.hdfs.RemotePeerFactory;
+import org.apache.hadoop.hdfs.ReplicaAccessor;
+import org.apache.hadoop.hdfs.ReplicaAccessorBuilder;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
 import org.apache.hadoop.hdfs.net.DomainPeer;
 import org.apache.hadoop.hdfs.net.Peer;
@@ -646,7 +654,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   }
 
   /**
-   * Get a RemoteBlockReader that communicates over a UNIX domain socket.
+   * Get a BlockReaderRemote that communicates over a UNIX domain socket.
    *
    * @return The new BlockReader, or null if we failed to create the block
    * reader.
@@ -709,7 +717,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   }
 
   /**
-   * Get a RemoteBlockReader that communicates over a TCP socket.
+   * Get a BlockReaderRemote that communicates over a TCP socket.
    *
    * @return The new BlockReader.  We will not return null, but instead throw
    *         an exception if this fails.
@@ -837,13 +845,13 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   private BlockReader getRemoteBlockReader(Peer peer) throws IOException {
     int networkDistance = clientContext.getNetworkDistance(datanode);
     if (conf.getShortCircuitConf().isUseLegacyBlockReader()) {
-      return RemoteBlockReader.newBlockReader(fileName,
+      return BlockReaderRemote.newBlockReader(fileName,
           block, token, startOffset, length, conf.getIoBufferSize(),
           verifyChecksum, clientName, peer, datanode,
           clientContext.getPeerCache(), cachingStrategy, tracer,
           networkDistance);
     } else {
-      return RemoteBlockReader2.newBlockReader(
+      return BlockReaderRemote2.newBlockReader(
           fileName, block, token, startOffset, length,
           verifyChecksum, clientName, peer, datanode,
           clientContext.getPeerCache(), cachingStrategy, tracer,

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderLocal.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -25,6 +25,7 @@ import java.util.EnumSet;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderLocalLegacy.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
 import java.io.DataInputStream;
 import java.io.File;
@@ -33,8 +33,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.BlockReader;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;

+ 13 - 11
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderRemote.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
@@ -29,6 +29,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FSInputChecker;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.ReadOption;
+import org.apache.hadoop.hdfs.BlockReader;
+import org.apache.hadoop.hdfs.PeerCache;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -54,13 +56,13 @@ import org.slf4j.LoggerFactory;
 
 /**
  * @deprecated this is an old implementation that is being left around
- * in case any issues spring up with the new {@link RemoteBlockReader2}
+ * in case any issues spring up with the new {@link BlockReaderRemote2}
  * implementation.
  * It will be removed in the next release.
  */
 @InterfaceAudience.Private
 @Deprecated
-public class RemoteBlockReader extends FSInputChecker implements BlockReader {
+public class BlockReaderRemote extends FSInputChecker implements BlockReader {
   static final Logger LOG = LoggerFactory.getLogger(FSInputChecker.class);
 
   private final Peer peer;
@@ -209,7 +211,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
       int len, byte[] checksumBuf)
       throws IOException {
     try (TraceScope ignored = tracer.newScope(
-        "RemoteBlockReader#readChunk(" + blockId + ")")) {
+        "BlockReaderRemote#readChunk(" + blockId + ")")) {
       return readChunkImpl(pos, buf, offset, len, checksumBuf);
     }
   }
@@ -335,7 +337,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
     return bytesToRead;
   }
 
-  private RemoteBlockReader(String file, String bpid, long blockId,
+  private BlockReaderRemote(String file, String bpid, long blockId,
       DataInputStream in, DataChecksum checksum, boolean verifyChecksum,
       long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
       DatanodeID datanodeID, PeerCache peerCache, Tracer tracer,
@@ -386,7 +388,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
    * @param clientName  Client name
    * @return New BlockReader instance, or null on error.
    */
-  public static RemoteBlockReader newBlockReader(String file,
+  public static BlockReaderRemote newBlockReader(String file,
       ExtendedBlock block,
       Token<BlockTokenIdentifier> blockToken,
       long startOffset, long len,
@@ -412,7 +414,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
 
     BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
         PBHelperClient.vintPrefixed(in));
-    RemoteBlockReader2.checkSuccess(status, peer, block, file);
+    BlockReaderRemote2.checkSuccess(status, peer, block, file);
     ReadOpChecksumInfoProto checksumInfo =
         status.getReadOpChecksumInfo();
     DataChecksum checksum = DataTransferProtoUtil.fromProto(
@@ -429,7 +431,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
           startOffset + " for file " + file);
     }
 
-    return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(),
+    return new BlockReaderRemote(file, block.getBlockPoolId(), block.getBlockId(),
         in, checksum, verifyChecksum, startOffset, firstChunkOffset, len,
         peer, datanodeID, peerCache, tracer, networkDistance);
   }
@@ -467,7 +469,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   void sendReadResult(Peer peer, Status statusCode) {
     assert !sentStatusCode : "already sent status code to " + peer;
     try {
-      RemoteBlockReader2.writeReadResult(peer.getOutputStream(), statusCode);
+      BlockReaderRemote2.writeReadResult(peer.getOutputStream(), statusCode);
       sentStatusCode = true;
     } catch (IOException e) {
       // It's ok not to be able to send this. But something is probably wrong.
@@ -478,14 +480,14 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
 
   @Override
   public int read(ByteBuffer buf) throws IOException {
-    throw new UnsupportedOperationException("readDirect unsupported in RemoteBlockReader");
+    throw new UnsupportedOperationException("readDirect unsupported in BlockReaderRemote");
   }
 
   @Override
   public int available() {
     // An optimistic estimate of how much data is available
     // to us without doing network I/O.
-    return RemoteBlockReader2.TCP_WINDOW_SIZE;
+    return BlockReaderRemote2.TCP_WINDOW_SIZE;
   }
 
   @Override

+ 10 - 8
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderRemote2.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
@@ -30,6 +30,8 @@ import java.util.UUID;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ReadOption;
+import org.apache.hadoop.hdfs.BlockReader;
+import org.apache.hadoop.hdfs.PeerCache;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -78,13 +80,13 @@ import org.slf4j.LoggerFactory;
  *
  * This is a new implementation introduced in Hadoop 0.23 which
  * is more efficient and simpler than the older BlockReader
- * implementation. It should be renamed to RemoteBlockReader
+ * implementation. It should be renamed to BlockReaderRemote
  * once we are confident in it.
  */
 @InterfaceAudience.Private
-public class RemoteBlockReader2  implements BlockReader {
+public class BlockReaderRemote2 implements BlockReader {
 
-  static final Logger LOG = LoggerFactory.getLogger(RemoteBlockReader2.class);
+  static final Logger LOG = LoggerFactory.getLogger(BlockReaderRemote2.class);
   static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB;
 
   final private Peer peer;
@@ -138,7 +140,7 @@ public class RemoteBlockReader2  implements BlockReader {
     if (curDataSlice == null ||
         curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
       try (TraceScope ignored = tracer.newScope(
-          "RemoteBlockReader2#readNextPacket(" + blockId + ")")) {
+          "BlockReaderRemote2#readNextPacket(" + blockId + ")")) {
         readNextPacket();
       }
     }
@@ -162,7 +164,7 @@ public class RemoteBlockReader2  implements BlockReader {
     if (curDataSlice == null ||
         (curDataSlice.remaining() == 0 && bytesNeededToFinish > 0)) {
       try (TraceScope ignored = tracer.newScope(
-          "RemoteBlockReader2#readNextPacket(" + blockId + ")")) {
+          "BlockReaderRemote2#readNextPacket(" + blockId + ")")) {
         readNextPacket();
       }
     }
@@ -273,7 +275,7 @@ public class RemoteBlockReader2  implements BlockReader {
     }
   }
 
-  protected RemoteBlockReader2(String file, long blockId,
+  protected BlockReaderRemote2(String file, long blockId,
       DataChecksum checksum, boolean verifyChecksum,
       long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
       DatanodeID datanodeID, PeerCache peerCache, Tracer tracer,
@@ -425,7 +427,7 @@ public class RemoteBlockReader2  implements BlockReader {
           startOffset + " for file " + file);
     }
 
-    return new RemoteBlockReader2(file, block.getBlockId(), checksum,
+    return new BlockReaderRemote2(file, block.getBlockId(), checksum,
         verifyChecksum, startOffset, firstChunkOffset, len, peer, datanodeID,
         peerCache, tracer, networkDistance);
   }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderUtil.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/BlockReaderUtil.java

@@ -15,9 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.BlockReader;
 
 import java.io.IOException;
 

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/ExternalBlockReader.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -23,6 +23,8 @@ import java.util.EnumSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ReadOption;
+import org.apache.hadoop.hdfs.BlockReader;
+import org.apache.hadoop.hdfs.ReplicaAccessor;
 import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
 import org.apache.hadoop.util.DataChecksum;
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml

@@ -133,7 +133,7 @@
 
      <!-- Don't complain about LocalDatanodeInfo's anonymous class -->
      <Match>
-       <Class name="org.apache.hadoop.hdfs.BlockReaderLocal$LocalDatanodeInfo$1" />
+       <Class name="org.apache.hadoop.hdfs.client.impl.BlockReaderLocal$LocalDatanodeInfo$1" />
        <Bug pattern="SE_BAD_FIELD_INNER_CLASS" />
      </Match>
      <!-- Only one method increments numFailedVolumes and it is synchronized -->

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReader.java

@@ -23,7 +23,7 @@ import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DFSUtilClient.CorruptedBlocks;
-import org.apache.hadoop.hdfs.RemoteBlockReader2;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderRemote2;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -112,7 +112,7 @@ class StripedBlockReader {
          *
          * TODO: add proper tracer
          */
-      return RemoteBlockReader2.newBlockReader(
+      return BlockReaderRemote2.newBlockReader(
           "dummy", block, blockToken, offsetInBlock,
           block.getNumBytes() - offsetInBlock, true,
           "", newConnectedPeer(block, dnAddr, blockToken, source), source,

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -43,7 +43,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.BlockReader;
-import org.apache.hadoop.hdfs.BlockReaderFactory;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderFactory;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java

@@ -38,7 +38,7 @@ import org.apache.commons.lang.SystemUtils;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hdfs.BlockReaderTestUtil;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
 import org.apache.hadoop.hdfs.ClientContext;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -48,7 +48,6 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;

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

@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;

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

@@ -26,6 +26,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
 import org.junit.Assert;
 import org.junit.Test;
 

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

@@ -24,6 +24,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
 import org.junit.Test;
 
 /**

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

@@ -28,6 +28,7 @@ import java.util.Random;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;

+ 11 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/BlockReaderTestUtil.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -32,6 +32,15 @@ import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.BlockReader;
+import org.apache.hadoop.hdfs.ClientContext;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.RemotePeerFactory;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -224,7 +233,7 @@ public class BlockReaderTestUtil {
     int ipcport = nodes[0].getIpcPort();
     return cluster.getDataNode(ipcport);
   }
-  
+
   public static void enableHdfsCachingTracing() {
     LogManager.getLogger(CacheReplicationMonitor.class.getName()).setLevel(
         Level.TRACE);

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderBase.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderBase.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
 import static org.junit.Assert.assertEquals;
 
@@ -24,6 +24,9 @@ import java.util.Random;
 
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.BlockReader;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.junit.After;
 import org.junit.Before;

+ 16 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderFactory.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT;
@@ -41,6 +41,11 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.BlockReader;
+import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -125,7 +130,7 @@ public class TestBlockReaderFactory {
     cluster.shutdown();
     sockDir.close();
   }
-  
+
   /**
    * Test the case where we have multiple threads waiting on the
    * ShortCircuitCache delivering a certain ShortCircuitReplica.
@@ -200,7 +205,7 @@ public class TestBlockReaderFactory {
    * Test the case where we have a failure to complete a short circuit read
    * that occurs, and then later on, we have a success.
    * Any thread waiting on a cache load should receive the failure (if it
-   * occurs);  however, the failure result should not be cached.  We want 
+   * occurs);  however, the failure result should not be cached.  We want
    * to be able to retry later and succeed.
    */
   @Test(timeout=60000)
@@ -244,7 +249,7 @@ public class TestBlockReaderFactory {
       public void run() {
         try {
           // First time should fail.
-          List<LocatedBlock> locatedBlocks = 
+          List<LocatedBlock> locatedBlocks =
               cluster.getNameNode().getRpcServer().getBlockLocations(
               TEST_FILE, 0, TEST_FILE_LEN).getLocatedBlocks();
           LocatedBlock lblock = locatedBlocks.get(0); // first block
@@ -253,7 +258,7 @@ public class TestBlockReaderFactory {
             blockReader = BlockReaderTestUtil.getBlockReader(
                 cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN);
             Assert.fail("expected getBlockReader to fail the first time.");
-          } catch (Throwable t) { 
+          } catch (Throwable t) {
             Assert.assertTrue("expected to see 'TCP reads were disabled " +
                 "for testing' in exception " + t, t.getMessage().contains(
                 "TCP reads were disabled for testing"));
@@ -267,7 +272,7 @@ public class TestBlockReaderFactory {
           try {
             blockReader = BlockReaderTestUtil.getBlockReader(
                 cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN);
-          } catch (Throwable t) { 
+          } catch (Throwable t) {
             LOG.error("error trying to retrieve a block reader " +
                 "the second time.", t);
             throw t;
@@ -327,7 +332,7 @@ public class TestBlockReaderFactory {
         calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
     Assert.assertTrue(Arrays.equals(contents, expected));
     final ShortCircuitCache cache =
-        fs.dfs.getClientContext().getShortCircuitCache();
+        fs.getClient().getClientContext().getShortCircuitCache();
     final DatanodeInfo datanode =
         new DatanodeInfo(cluster.getDataNodes().get(0).getDatanodeId());
     cache.getDfsClientShmManager().visit(new Visitor() {
@@ -344,7 +349,7 @@ public class TestBlockReaderFactory {
     cluster.shutdown();
     sockDir.close();
   }
- 
+
   /**
    * Test that a client which does not support short-circuit reads using
    * shared memory can talk with a server which supports it.
@@ -375,12 +380,12 @@ public class TestBlockReaderFactory {
         calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
     Assert.assertTrue(Arrays.equals(contents, expected));
     final ShortCircuitCache cache =
-        fs.dfs.getClientContext().getShortCircuitCache();
+        fs.getClient().getClientContext().getShortCircuitCache();
     Assert.assertEquals(null, cache.getDfsClientShmManager());
     cluster.shutdown();
     sockDir.close();
   }
-  
+
   /**
    * Test shutting down the ShortCircuitCache while there are things in it.
    */
@@ -407,7 +412,7 @@ public class TestBlockReaderFactory {
         calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
     Assert.assertTrue(Arrays.equals(contents, expected));
     final ShortCircuitCache cache =
-        fs.dfs.getClientContext().getShortCircuitCache();
+        fs.getClient().getClientContext().getShortCircuitCache();
     cache.close();
     Assert.assertTrue(cache.getDfsClientShmManager().
         getDomainSocketWatcher().isClosed());

+ 47 - 41
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocal.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderLocal.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
 import static org.hamcrest.CoreMatchers.equalTo;
 
@@ -32,9 +32,15 @@ import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.ClientContext;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.ExtendedBlockId;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache;
@@ -55,24 +61,24 @@ import org.junit.Test;
 
 public class TestBlockReaderLocal {
   private static TemporarySocketDirectory sockDir;
-  
+
   @BeforeClass
   public static void init() {
     sockDir = new TemporarySocketDirectory();
     DomainSocket.disableBindPathValidation();
   }
-  
+
   @AfterClass
   public static void shutdown() throws IOException {
     sockDir.close();
   }
-  
+
   public static void assertArrayRegionsEqual(byte []buf1, int off1, byte []buf2,
       int off2, int len) {
     for (int i = 0; i < len; i++) {
       if (buf1[off1 + i] != buf2[off2 + i]) {
-        Assert.fail("arrays differ at byte " +  i + ". " + 
-          "The first array has " + (int)buf1[off1 + i] + 
+        Assert.fail("arrays differ at byte " +  i + ". " +
+          "The first array has " + (int)buf1[off1 + i] +
           ", but the second array has " + (int)buf2[off2 + i]);
       }
     }
@@ -85,7 +91,7 @@ public class TestBlockReaderLocal {
    * @param buf              The ByteBuffer to read into
    * @param off              The offset in the buffer to read into
    * @param len              The number of bytes to read.
-   * 
+   *
    * @throws IOException     If it could not read the requested number of bytes
    */
   private static void readFully(BlockReaderLocal reader,
@@ -120,7 +126,7 @@ public class TestBlockReaderLocal {
       // default: no-op
     }
   }
-  
+
   public void runBlockReaderLocalTest(BlockReaderLocalTest test,
       boolean checksum, long readahead) throws IOException {
     Assume.assumeThat(DomainSocket.getLoadingFailureReason(), equalTo(null));
@@ -139,7 +145,7 @@ public class TestBlockReaderLocal {
     BlockReaderLocal blockReaderLocal = null;
     FSDataInputStream fsIn = null;
     byte original[] = new byte[BlockReaderLocalTest.TEST_LENGTH];
-    
+
     FileSystem fs = null;
     ShortCircuitShm shm = null;
     RandomAccessFile raf = null;
@@ -186,7 +192,7 @@ public class TestBlockReaderLocal {
       raf.setLength(8192);
       FileInputStream shmStream = new FileInputStream(raf.getFD());
       shm = new ShortCircuitShm(ShmId.createRandom(), shmStream);
-      ShortCircuitReplica replica = 
+      ShortCircuitReplica replica =
           new ShortCircuitReplica(key, dataIn, metaIn, shortCircuitCache,
               Time.now(), shm.allocAndRegisterSlot(
                   ExtendedBlockId.fromExtendedBlock(block)));
@@ -216,21 +222,21 @@ public class TestBlockReaderLocal {
       if (raf != null) raf.close();
     }
   }
-  
-  private static class TestBlockReaderLocalImmediateClose 
+
+  private static class TestBlockReaderLocalImmediateClose
       extends BlockReaderLocalTest {
   }
-  
+
   @Test
   public void testBlockReaderLocalImmediateClose() throws IOException {
     runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), true, 0);
     runBlockReaderLocalTest(new TestBlockReaderLocalImmediateClose(), false, 0);
   }
-  
-  private static class TestBlockReaderSimpleReads 
+
+  private static class TestBlockReaderSimpleReads
       extends BlockReaderLocalTest {
     @Override
-    public void doTest(BlockReaderLocal reader, byte original[]) 
+    public void doTest(BlockReaderLocal reader, byte original[])
         throws IOException {
       byte buf[] = new byte[TEST_LENGTH];
       reader.readFully(buf, 0, 512);
@@ -246,7 +252,7 @@ public class TestBlockReaderLocal {
           BlockReaderLocalTest.BYTES_PER_CHECKSUM);
     }
   }
-  
+
   @Test
   public void testBlockReaderSimpleReads() throws IOException {
     runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), true,
@@ -275,11 +281,11 @@ public class TestBlockReaderLocal {
       throws IOException {
     runBlockReaderLocalTest(new TestBlockReaderSimpleReads(), false, 0);
   }
-  
-  private static class TestBlockReaderLocalArrayReads2 
+
+  private static class TestBlockReaderLocalArrayReads2
       extends BlockReaderLocalTest {
     @Override
-    public void doTest(BlockReaderLocal reader, byte original[]) 
+    public void doTest(BlockReaderLocal reader, byte original[])
         throws IOException {
       byte buf[] = new byte[TEST_LENGTH];
       reader.readFully(buf, 0, 10);
@@ -296,7 +302,7 @@ public class TestBlockReaderLocal {
       assertArrayRegionsEqual(original, 1716, buf, 1716, 5);
     }
   }
-  
+
   @Test
   public void testBlockReaderLocalArrayReads2() throws IOException {
     runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(),
@@ -322,10 +328,10 @@ public class TestBlockReaderLocal {
     runBlockReaderLocalTest(new TestBlockReaderLocalArrayReads2(), false, 0);
   }
 
-  private static class TestBlockReaderLocalByteBufferReads 
+  private static class TestBlockReaderLocalByteBufferReads
       extends BlockReaderLocalTest {
     @Override
-    public void doTest(BlockReaderLocal reader, byte original[]) 
+    public void doTest(BlockReaderLocal reader, byte original[])
         throws IOException {
       ByteBuffer buf = ByteBuffer.wrap(new byte[TEST_LENGTH]);
       readFully(reader, buf, 0, 10);
@@ -339,7 +345,7 @@ public class TestBlockReaderLocal {
       assertArrayRegionsEqual(original, 811, buf.array(), 811, 5);
     }
   }
-  
+
   @Test
   public void testBlockReaderLocalByteBufferReads()
       throws IOException {
@@ -354,7 +360,7 @@ public class TestBlockReaderLocal {
         new TestBlockReaderLocalByteBufferReads(),
         false, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
   }
-  
+
   @Test
   public void testBlockReaderLocalByteBufferReadsNoReadahead()
       throws IOException {
@@ -373,7 +379,7 @@ public class TestBlockReaderLocal {
    * Test reads that bypass the bounce buffer (because they are aligned
    * and bigger than the readahead).
    */
-  private static class TestBlockReaderLocalByteBufferFastLaneReads 
+  private static class TestBlockReaderLocalByteBufferFastLaneReads
       extends BlockReaderLocalTest {
     @Override
     public void doTest(BlockReaderLocal reader, byte original[])
@@ -410,7 +416,7 @@ public class TestBlockReaderLocal {
           50);
     }
   }
-  
+
   @Test
   public void testBlockReaderLocalByteBufferFastLaneReads()
       throws IOException {
@@ -456,7 +462,7 @@ public class TestBlockReaderLocal {
       }
     }
 
-    public void doTest(BlockReaderLocal reader, byte original[]) 
+    public void doTest(BlockReaderLocal reader, byte original[])
         throws IOException {
       byte buf[] = new byte[TEST_LENGTH];
       if (usingChecksums) {
@@ -471,19 +477,19 @@ public class TestBlockReaderLocal {
       }
     }
   }
-  
+
   @Test
   public void testBlockReaderLocalReadCorruptStart()
       throws IOException {
     runBlockReaderLocalTest(new TestBlockReaderLocalReadCorruptStart(), true,
         HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
   }
-  
+
   private static class TestBlockReaderLocalReadCorrupt
       extends BlockReaderLocalTest {
     boolean usingChecksums = false;
     @Override
-    public void setup(File blockFile, boolean usingChecksums) 
+    public void setup(File blockFile, boolean usingChecksums)
         throws IOException {
       RandomAccessFile bf = null;
       this.usingChecksums = usingChecksums;
@@ -496,7 +502,7 @@ public class TestBlockReaderLocal {
       }
     }
 
-    public void doTest(BlockReaderLocal reader, byte original[]) 
+    public void doTest(BlockReaderLocal reader, byte original[])
         throws IOException {
       byte buf[] = new byte[TEST_LENGTH];
       try {
@@ -522,7 +528,7 @@ public class TestBlockReaderLocal {
       }
     }
   }
-  
+
   @Test
   public void testBlockReaderLocalReadCorrupt()
       throws IOException {
@@ -555,7 +561,7 @@ public class TestBlockReaderLocal {
     public void setup(File blockFile, boolean usingChecksums)
         throws IOException {
     }
-    
+
     @Override
     public void doTest(BlockReaderLocal reader, byte original[])
         throws IOException {
@@ -675,7 +681,7 @@ public class TestBlockReaderLocal {
     runBlockReaderLocalTest(new TestBlockReaderLocalOnFileWithoutChecksum(),
         false, 0);
   }
-  
+
   @Test
   public void testBlockReaderLocalReadZeroBytes()
       throws IOException {
@@ -703,7 +709,7 @@ public class TestBlockReaderLocal {
     runBlockReaderLocalTest(new TestBlockReaderLocalReadZeroBytes(),
         false, 0);
   }
-  
+
 
   @Test(timeout=60000)
   public void TestStatisticsForShortCircuitLocalRead() throws Exception {
@@ -714,7 +720,7 @@ public class TestBlockReaderLocal {
   public void TestStatisticsForLocalRead() throws Exception {
     testStatistics(false);
   }
-  
+
   private void testStatistics(boolean isShortCircuit) throws Exception {
     Assume.assumeTrue(DomainSocket.getLoadingFailureReason() == null);
     HdfsConfiguration conf = new HdfsConfiguration();
@@ -756,12 +762,12 @@ public class TestBlockReaderLocal {
       IOUtils.readFully(fsIn, original, 0,
           BlockReaderLocalTest.TEST_LENGTH);
       HdfsDataInputStream dfsIn = (HdfsDataInputStream)fsIn;
-      Assert.assertEquals(BlockReaderLocalTest.TEST_LENGTH, 
+      Assert.assertEquals(BlockReaderLocalTest.TEST_LENGTH,
           dfsIn.getReadStatistics().getTotalBytesRead());
-      Assert.assertEquals(BlockReaderLocalTest.TEST_LENGTH, 
+      Assert.assertEquals(BlockReaderLocalTest.TEST_LENGTH,
           dfsIn.getReadStatistics().getTotalLocalBytesRead());
       if (isShortCircuit) {
-        Assert.assertEquals(BlockReaderLocalTest.TEST_LENGTH, 
+        Assert.assertEquals(BlockReaderLocalTest.TEST_LENGTH,
             dfsIn.getReadStatistics().getTotalShortCircuitBytesRead());
       } else {
         Assert.assertEquals(0,

+ 10 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderLocalLegacy.java

@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -30,6 +30,13 @@ 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.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
@@ -52,7 +59,7 @@ public class TestBlockReaderLocalLegacy {
     DFSInputStream.tcpReadsDisabledForTesting = true;
     DomainSocket.disableBindPathValidation();
   }
-  
+
   private static HdfsConfiguration getConfiguration(
       TemporarySocketDirectory socketDir) throws IOException {
     HdfsConfiguration conf = new HdfsConfiguration();
@@ -84,7 +91,7 @@ public class TestBlockReaderLocalLegacy {
   public void testStablePositionAfterCorruptRead() throws Exception {
     final short REPL_FACTOR = 1;
     final long FILE_LENGTH = 512L;
-    
+
     HdfsConfiguration conf = getConfiguration(null);
     MiniDFSCluster cluster =
         new MiniDFSCluster.Builder(conf).numDataNodes(1).build();

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderRemote.java

@@ -15,11 +15,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
+import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 
-public class TestRemoteBlockReader extends TestBlockReaderBase {
+public class TestBlockReaderRemote extends TestBlockReaderBase {
 
   HdfsConfiguration createConf() {
     HdfsConfiguration conf = new HdfsConfiguration();

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRemoteBlockReader2.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestBlockReaderRemote2.java

@@ -15,9 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
-public class TestRemoteBlockReader2 extends TestBlockReaderBase {
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+
+public class TestBlockReaderRemote2 extends TestBlockReaderBase {
   HdfsConfiguration createConf() {
     HdfsConfiguration conf = new HdfsConfiguration();
     return conf;

+ 7 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientBlockVerification.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/client/impl/TestClientBlockVerification.java

@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.hdfs;
+package org.apache.hadoop.hdfs.client.impl;
 
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.spy;
@@ -25,6 +25,7 @@ import static org.mockito.Mockito.verify;
 import java.util.List;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -41,7 +42,7 @@ public class TestClientBlockVerification {
   static LocatedBlock testBlock = null;
 
   static {
-    GenericTestUtils.setLogLevel(RemoteBlockReader2.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(BlockReaderRemote2.LOG, Level.ALL);
   }
   @BeforeClass
   public static void setupCluster() throws Exception {
@@ -57,7 +58,7 @@ public class TestClientBlockVerification {
    */
   @Test
   public void testBlockVerification() throws Exception {
-    RemoteBlockReader2 reader = (RemoteBlockReader2)spy(
+    BlockReaderRemote2 reader = (BlockReaderRemote2)spy(
         util.getBlockReader(testBlock, 0, FILE_SIZE_K * 1024));
     util.readAndCheckEOS(reader, FILE_SIZE_K * 1024, true);
     verify(reader).sendReadResult(Status.CHECKSUM_OK);
@@ -69,7 +70,7 @@ public class TestClientBlockVerification {
    */
   @Test
   public void testIncompleteRead() throws Exception {
-    RemoteBlockReader2 reader = (RemoteBlockReader2)spy(
+    BlockReaderRemote2 reader = (BlockReaderRemote2)spy(
         util.getBlockReader(testBlock, 0, FILE_SIZE_K * 1024));
     util.readAndCheckEOS(reader, FILE_SIZE_K / 2 * 1024, false);
 
@@ -87,7 +88,7 @@ public class TestClientBlockVerification {
   @Test
   public void testCompletePartialRead() throws Exception {
     // Ask for half the file
-    RemoteBlockReader2 reader = (RemoteBlockReader2)spy(
+    BlockReaderRemote2 reader = (BlockReaderRemote2)spy(
         util.getBlockReader(testBlock, 0, FILE_SIZE_K * 1024 / 2));
     // And read half the file
     util.readAndCheckEOS(reader, FILE_SIZE_K * 1024 / 2, true);
@@ -107,7 +108,7 @@ public class TestClientBlockVerification {
       for (int length : lengths) {
         DFSClient.LOG.info("Testing startOffset = " + startOffset + " and " +
                            " len=" + length);
-        RemoteBlockReader2 reader = (RemoteBlockReader2)spy(
+        BlockReaderRemote2 reader = (BlockReaderRemote2)spy(
             util.getBlockReader(testBlock, startOffset, length));
         util.readAndCheckEOS(reader, length, true);
         verify(reader).sendReadResult(Status.CHECKSUM_OK);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java

@@ -35,7 +35,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.BlockReader;
-import org.apache.hadoop.hdfs.BlockReaderFactory;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderFactory;
 import org.apache.hadoop.hdfs.ClientContext;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java

@@ -42,7 +42,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.BlockReader;
-import org.apache.hadoop.hdfs.BlockReaderFactory;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderFactory;
 import org.apache.hadoop.hdfs.ClientContext;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java

@@ -43,7 +43,7 @@ import org.apache.hadoop.fs.HdfsBlockLocation;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
-import org.apache.hadoop.hdfs.BlockReaderTestUtil;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -59,7 +59,6 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetCache;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetCache.PageRounder;
-import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCacheRevocation.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.ReadOption;
-import org.apache.hadoop.hdfs.BlockReaderTestUtil;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java

@@ -55,7 +55,7 @@ import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.BlockReaderTestUtil;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java

@@ -39,8 +39,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.BlockReaderFactory;
-import org.apache.hadoop.hdfs.BlockReaderTestUtil;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderFactory;
+import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
 import org.apache.hadoop.hdfs.DFSInputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;

+ 4 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitLocalRead.java

@@ -45,7 +45,7 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.TestBlockReaderLocal;
+import org.apache.hadoop.hdfs.client.impl.TestBlockReaderLocal;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
@@ -54,7 +54,6 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
@@ -593,7 +592,7 @@ public class TestShortCircuitLocalRead {
 
   /**
    * Test that file data can be read by reading the block
-   * through RemoteBlockReader
+   * through BlockReaderRemote
    * @throws IOException
   */
   public void doTestShortCircuitReadWithRemoteBlockReader(
@@ -623,9 +622,9 @@ public class TestShortCircuitLocalRead {
     try {
       checkFileContent(uri, file1, fileData, readOffset, shortCircuitUser, 
           conf, shortCircuitFails);
-      //RemoteBlockReader have unsupported method read(ByteBuffer bf)
+      //BlockReaderRemote have unsupported method read(ByteBuffer bf)
       assertTrue(
-          "RemoteBlockReader unsupported method read(ByteBuffer bf) error",
+          "BlockReaderRemote unsupported method read(ByteBuffer bf) error",
           checkUnsupportedMethod(fs, file1, fileData, readOffset));
     } catch(IOException e) {
       throw new IOException(