HADOOP-7316. Add public javadocs to FSDataInputStream and FSDataOutputStream. Contributed by Eli Collins
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1131268 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
90a9262611
commit
01b83a1e32
|
@ -190,6 +190,9 @@ Trunk (unreleased changes)
|
|||
HADOOP-7331. Make hadoop-daemon.sh return exit code 1 if daemon processes
|
||||
did not get started. (Tanping Wang via todd)
|
||||
|
||||
HADOOP-7316. Add public javadocs to FSDataInputStream and
|
||||
FSDataOutputStream. (eli)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HADOOP-7333. Performance improvement in PureJavaCrc32. (Eric Caspole
|
||||
|
|
|
@ -38,34 +38,81 @@ public class FSDataInputStream extends DataInputStream
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Seek to the given offset.
|
||||
*
|
||||
* @param desired offset to seek to
|
||||
*/
|
||||
public synchronized void seek(long desired) throws IOException {
|
||||
((Seekable)in).seek(desired);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the current position in the input stream.
|
||||
*
|
||||
* @return current position in the input stream
|
||||
*/
|
||||
public long getPos() throws IOException {
|
||||
return ((Seekable)in).getPos();
|
||||
}
|
||||
|
||||
/**
|
||||
* Read bytes from the given position in the stream to the given buffer.
|
||||
*
|
||||
* @param position position in the input stream to seek
|
||||
* @param buffer buffer into which data is read
|
||||
* @param offset offset into the buffer in which data is written
|
||||
* @param length maximum number of bytes to read
|
||||
* @return total number of bytes read into the buffer, or <code>-1</code>
|
||||
* if there is no more data because the end of the stream has been
|
||||
* reached
|
||||
*/
|
||||
public int read(long position, byte[] buffer, int offset, int length)
|
||||
throws IOException {
|
||||
return ((PositionedReadable)in).read(position, buffer, offset, length);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Read bytes from the given position in the stream to the given buffer.
|
||||
* Continues to read until <code>length</code> bytes have been read.
|
||||
*
|
||||
* @param position position in the input stream to seek
|
||||
* @param buffer buffer into which data is read
|
||||
* @param offset offset into the buffer in which data is written
|
||||
* @param length the number of bytes to read
|
||||
* @throws EOFException If the end of stream is reached while reading.
|
||||
* If an exception is thrown an undetermined number
|
||||
* of bytes in the buffer may have been written.
|
||||
*/
|
||||
public void readFully(long position, byte[] buffer, int offset, int length)
|
||||
throws IOException {
|
||||
((PositionedReadable)in).readFully(position, buffer, offset, length);
|
||||
}
|
||||
|
||||
/**
|
||||
* See {@link #readFully(long, byte[], int, int)}.
|
||||
*/
|
||||
public void readFully(long position, byte[] buffer)
|
||||
throws IOException {
|
||||
((PositionedReadable)in).readFully(position, buffer, 0, buffer.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Seek to the given position on an alternate copy of the data.
|
||||
*
|
||||
* @param targetPos position to seek to
|
||||
* @return true if a new source is found, false otherwise
|
||||
*/
|
||||
public boolean seekToNewSource(long targetPos) throws IOException {
|
||||
return ((Seekable)in).seekToNewSource(targetPos);
|
||||
}
|
||||
|
||||
// Returns the underlying input stream. This is used by unit tests.
|
||||
/**
|
||||
* Get a reference to the wrapped input stream. Used by unit tests.
|
||||
*
|
||||
* @return the underlying input stream
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate({"HDFS"})
|
||||
public InputStream getWrappedStream() {
|
||||
return in;
|
||||
}
|
||||
|
|
|
@ -83,15 +83,28 @@ public class FSDataOutputStream extends DataOutputStream implements Syncable {
|
|||
wrappedStream = out;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the current position in the output stream.
|
||||
*
|
||||
* @return the current position in the output stream
|
||||
*/
|
||||
public long getPos() throws IOException {
|
||||
return ((PositionCache)out).getPos();
|
||||
}
|
||||
|
||||
/**
|
||||
* Close the underlying output stream.
|
||||
*/
|
||||
public void close() throws IOException {
|
||||
out.close(); // This invokes PositionCache.close()
|
||||
out.close(); // This invokes PositionCache.close()
|
||||
}
|
||||
|
||||
// Returns the underlying output stream. This is used by unit tests.
|
||||
/**
|
||||
* Get a reference to the wrapped output stream. Used by unit tests.
|
||||
*
|
||||
* @return the underlying output stream
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate({"HDFS"})
|
||||
public OutputStream getWrappedStream() {
|
||||
return wrappedStream;
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ import java.io.*;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.*;
|
||||
|
||||
/** Stream that permits positional reading. */
|
||||
@InterfaceAudience.Public
|
||||
|
|
|
@ -232,7 +232,7 @@ abstract public class Command extends Configured {
|
|||
* Processes a {@link PathData} item, calling
|
||||
* {@link #processPathArgument(PathData)} or
|
||||
* {@link #processNonexistentPath(PathData)} on each item.
|
||||
* @param arg {@link PathData} item to process
|
||||
* @param item {@link PathData} item to process
|
||||
* @throws IOException if anything goes wrong...
|
||||
*/
|
||||
protected void processArgument(PathData item) throws IOException {
|
||||
|
|
|
@ -75,7 +75,7 @@ abstract public class FsCommand extends Command {
|
|||
throw new RuntimeException("not supposed to get here");
|
||||
}
|
||||
|
||||
/** @deprecated use {@link #run(String...argv)} */
|
||||
/** @deprecated use {@link Command#run(String...argv)} */
|
||||
@Deprecated
|
||||
@Override
|
||||
public int runAll() {
|
||||
|
|
Loading…
Reference in New Issue