|
@@ -78,7 +78,6 @@ import org.apache.hadoop.fs.BlockLocation;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.fs.ContentSummary;
|
|
|
import org.apache.hadoop.fs.CreateFlag;
|
|
|
-import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
@@ -91,6 +90,7 @@ import org.apache.hadoop.fs.ParentNotDirectoryException;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
+import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
|
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
|
|
|
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
|
|
@@ -1809,41 +1809,13 @@ public class DFSClient implements java.io.Closeable {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * The Hdfs implementation of {@link FSDataInputStream}
|
|
|
+ * @deprecated use {@link HdfsDataInputStream} instead.
|
|
|
*/
|
|
|
- @InterfaceAudience.Private
|
|
|
- public static class DFSDataInputStream extends FSDataInputStream {
|
|
|
- public DFSDataInputStream(DFSInputStream in)
|
|
|
- throws IOException {
|
|
|
- super(in);
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Returns the datanode from which the stream is currently reading.
|
|
|
- */
|
|
|
- public DatanodeInfo getCurrentDatanode() {
|
|
|
- return ((DFSInputStream)in).getCurrentDatanode();
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Returns the block containing the target position.
|
|
|
- */
|
|
|
- public ExtendedBlock getCurrentBlock() {
|
|
|
- return ((DFSInputStream)in).getCurrentBlock();
|
|
|
- }
|
|
|
+ @Deprecated
|
|
|
+ public static class DFSDataInputStream extends HdfsDataInputStream {
|
|
|
|
|
|
- /**
|
|
|
- * Return collection of blocks that has already been located.
|
|
|
- */
|
|
|
- synchronized List<LocatedBlock> getAllBlocks() throws IOException {
|
|
|
- return ((DFSInputStream)in).getAllBlocks();
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * @return The visible length of the file.
|
|
|
- */
|
|
|
- public long getVisibleLength() throws IOException {
|
|
|
- return ((DFSInputStream)in).getFileLength();
|
|
|
+ public DFSDataInputStream(DFSInputStream in) throws IOException {
|
|
|
+ super(in);
|
|
|
}
|
|
|
}
|
|
|
|