org.apache.hadoop.fs
Class FSInputChecker

java.lang.Object
  extended by java.io.InputStream
      extended by org.apache.hadoop.fs.FSInputStream
          extended by org.apache.hadoop.fs.FSInputChecker
All Implemented Interfaces:
Closeable, PositionedReadable, Seekable

public abstract class FSInputChecker
extends FSInputStream

This is a generic input stream for verifying checksums for data before it is read by a user.


Field Summary
protected  Path file
          The file name from which data is read from
static org.apache.commons.logging.Log LOG
           
 
Constructor Summary
protected FSInputChecker(Path file, int numOfRetries)
          Constructor
protected FSInputChecker(Path file, int numOfRetries, boolean verifyChecksum, Checksum sum, int chunkSize, int checksumSize)
          Constructor
 
Method Summary
 int available()
           
static long checksum2long(byte[] checksum)
          Convert a checksum byte array to a long
protected abstract  long getChunkPosition(long pos)
          Return position of beginning of chunk containing pos.
 long getPos()
          Return the current offset from the start of the file
 void mark(int readlimit)
           
 boolean markSupported()
           
protected  boolean needChecksum()
          Return true if there is a need for checksum verification
 int read()
          Read one checksum-verified byte
 int read(byte[] b, int off, int len)
          Read checksum verified bytes from this byte-input stream into the specified byte array, starting at the given offset.
protected abstract  int readChunk(long pos, byte[] buf, int offset, int len, byte[] checksum)
          Reads in next checksum chunk data into buf at offset and checksum into checksum.
protected static int readFully(InputStream stm, byte[] buf, int offset, int len)
          A utility function that tries to read up to len bytes from stm
 void reset()
           
 void seek(long pos)
          Seek to the given position in the stream.
protected  void set(boolean verifyChecksum, Checksum sum, int maxChunkSize, int checksumSize)
          Set the checksum related parameters
 long skip(long n)
          Skips over and discards n bytes of data from the input stream.
 
Methods inherited from class org.apache.hadoop.fs.FSInputStream
read, readFully, readFully, seekToNewSource
 
Methods inherited from class java.io.InputStream
close, read
 
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
 

Field Detail

LOG

public static final org.apache.commons.logging.Log LOG

file

protected Path file
The file name from which data is read from

Constructor Detail

FSInputChecker

protected FSInputChecker(Path file,
                         int numOfRetries)
Constructor

Parameters:
file - The name of the file to be read
numOfRetries - Number of read retries when ChecksumError occurs

FSInputChecker

protected FSInputChecker(Path file,
                         int numOfRetries,
                         boolean verifyChecksum,
                         Checksum sum,
                         int chunkSize,
                         int checksumSize)
Constructor

Parameters:
file - The name of the file to be read
numOfRetries - Number of read retries when ChecksumError occurs
sum - the type of Checksum engine
chunkSize - maximun chunk size
checksumSize - the number byte of each checksum
Method Detail

readChunk

protected abstract int readChunk(long pos,
                                 byte[] buf,
                                 int offset,
                                 int len,
                                 byte[] checksum)
                          throws IOException
Reads in next checksum chunk data into buf at offset and checksum into checksum. The method is used for implementing read, therefore, it should be optimized for sequential reading

Parameters:
pos - chunkPos
buf - desitination buffer
offset - offset in buf at which to store data
len - maximun number of bytes to read
Returns:
number of bytes read
Throws:
IOException

getChunkPosition

protected abstract long getChunkPosition(long pos)
Return position of beginning of chunk containing pos.

Parameters:
pos - a postion in the file
Returns:
the starting position of the chunk which contains the byte

needChecksum

protected boolean needChecksum()
Return true if there is a need for checksum verification


read

public int read()
         throws IOException
Read one checksum-verified byte

Specified by:
read in class InputStream
Returns:
the next byte of data, or -1 if the end of the stream is reached.
Throws:
IOException - if an I/O error occurs.

read

public int read(byte[] b,
                int off,
                int len)
         throws IOException
Read checksum verified bytes from this byte-input stream into the specified byte array, starting at the given offset.

This method implements the general contract of the corresponding read method of the InputStream class. As an additional convenience, it attempts to read as many bytes as possible by repeatedly invoking the read method of the underlying stream. This iterated read continues until one of the following conditions becomes true:

If the first read on the underlying stream returns -1 to indicate end-of-file then this method returns -1. Otherwise this method returns the number of bytes actually read.

Overrides:
read in class InputStream
Parameters:
b - destination buffer.
off - offset at which to start storing bytes.
len - maximum number of bytes to read.
Returns:
the number of bytes read, or -1 if the end of the stream has been reached.
Throws:
IOException - if an I/O error occurs. ChecksumException if any checksum error occurs

checksum2long

public static long checksum2long(byte[] checksum)
Convert a checksum byte array to a long


getPos

public long getPos()
            throws IOException
Description copied from class: FSInputStream
Return the current offset from the start of the file

Specified by:
getPos in interface Seekable
Specified by:
getPos in class FSInputStream
Throws:
IOException

available

public int available()
              throws IOException
Overrides:
available in class InputStream
Throws:
IOException

skip

public long skip(long n)
          throws IOException
Skips over and discards n bytes of data from the input stream.

This method may skip more bytes than are remaining in the backing file. This produces no exception and the number of bytes skipped may include some number of bytes that were beyond the EOF of the backing file. Attempting to read from the stream after skipping past the end will result in -1 indicating the end of the file.

If n is negative, no bytes are skipped.

Overrides:
skip in class InputStream
Parameters:
n - the number of bytes to be skipped.
Returns:
the actual number of bytes skipped.
Throws:
IOException - if an I/O error occurs. ChecksumException if the chunk to skip to is corrupted

seek

public void seek(long pos)
          throws IOException
Seek to the given position in the stream. The next read() will be from that position.

This method may seek past the end of the file. This produces no exception and an attempt to read from the stream will result in -1 indicating the end of the file.

Specified by:
seek in interface Seekable
Specified by:
seek in class FSInputStream
Parameters:
pos - the postion to seek to.
Throws:
IOException - if an I/O error occurs. ChecksumException if the chunk to seek to is corrupted

readFully

protected static int readFully(InputStream stm,
                               byte[] buf,
                               int offset,
                               int len)
                        throws IOException
A utility function that tries to read up to len bytes from stm

Parameters:
stm - an input stream
buf - destiniation buffer
offset - offset at which to store data
len - number of bytes to read
Returns:
actual number of bytes read
Throws:
IOException - if there is any IO error

set

protected final void set(boolean verifyChecksum,
                         Checksum sum,
                         int maxChunkSize,
                         int checksumSize)
Set the checksum related parameters

Parameters:
verifyChecksum - whether to verify checksum
sum - which type of checksum to use
maxChunkSize - maximun chunk size
checksumSize - checksum size

markSupported

public final boolean markSupported()
Overrides:
markSupported in class InputStream

mark

public final void mark(int readlimit)
Overrides:
mark in class InputStream

reset

public final void reset()
                 throws IOException
Overrides:
reset in class InputStream
Throws:
IOException


Copyright © 2009 The Apache Software Foundation