|
@@ -24,6 +24,8 @@ import java.util.zip.Checksum;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
+import java.nio.ByteBuffer;
|
|
|
+import java.nio.IntBuffer;
|
|
|
|
|
|
/**
|
|
|
* This is a generic input stream for verifying checksums for
|
|
@@ -38,16 +40,26 @@ abstract public class FSInputChecker extends FSInputStream {
|
|
|
protected Path file;
|
|
|
private Checksum sum;
|
|
|
private boolean verifyChecksum = true;
|
|
|
- private byte[] buf;
|
|
|
+ private int maxChunkSize; // data bytes for checksum (eg 512)
|
|
|
+ private byte[] buf; // buffer for non-chunk-aligned reading
|
|
|
private byte[] checksum;
|
|
|
- private int pos;
|
|
|
- private int count;
|
|
|
+ private IntBuffer checksumInts; // wrapper on checksum buffer
|
|
|
+ private int pos; // the position of the reader inside buf
|
|
|
+ private int count; // the number of bytes currently in buf
|
|
|
|
|
|
private int numOfRetries;
|
|
|
|
|
|
// cached file position
|
|
|
+ // this should always be a multiple of maxChunkSize
|
|
|
private long chunkPos = 0;
|
|
|
-
|
|
|
+
|
|
|
+ // Number of checksum chunks that can be read at once into a user
|
|
|
+ // buffer. Chosen by benchmarks - higher values do not reduce
|
|
|
+ // CPU usage. The size of the data reads made to the underlying stream
|
|
|
+ // will be CHUNKS_PER_READ * maxChunkSize.
|
|
|
+ private static final int CHUNKS_PER_READ = 32;
|
|
|
+ protected static final int CHECKSUM_SIZE = 4; // 32-bit checksum
|
|
|
+
|
|
|
/** Constructor
|
|
|
*
|
|
|
* @param file The name of the file to be read
|
|
@@ -72,14 +84,34 @@ abstract public class FSInputChecker extends FSInputStream {
|
|
|
set(verifyChecksum, sum, chunkSize, checksumSize);
|
|
|
}
|
|
|
|
|
|
- /** Reads in next checksum chunk data into <code>buf</code> at <code>offset</code>
|
|
|
+ /**
|
|
|
+ * Reads in checksum chunks into <code>buf</code> at <code>offset</code>
|
|
|
* and checksum into <code>checksum</code>.
|
|
|
+ * Since checksums can be disabled, there are two cases implementors need
|
|
|
+ * to worry about:
|
|
|
+ *
|
|
|
+ * (a) needChecksum() will return false:
|
|
|
+ * - len can be any positive value
|
|
|
+ * - checksum will be null
|
|
|
+ * Implementors should simply pass through to the underlying data stream.
|
|
|
+ * or
|
|
|
+ * (b) needChecksum() will return true:
|
|
|
+ * - len >= maxChunkSize
|
|
|
+ * - checksum.length is a multiple of CHECKSUM_SIZE
|
|
|
+ * Implementors should read an integer number of data chunks into
|
|
|
+ * buf. The amount read should be bounded by len or by
|
|
|
+ * checksum.length / CHECKSUM_SIZE * maxChunkSize. Note that len may
|
|
|
+ * be a value that is not a multiple of maxChunkSize, in which case
|
|
|
+ * the implementation may return less than len.
|
|
|
+ *
|
|
|
* The method is used for implementing read, therefore, it should be optimized
|
|
|
- * for sequential reading
|
|
|
+ * for sequential reading.
|
|
|
+ *
|
|
|
* @param pos chunkPos
|
|
|
* @param buf desitination buffer
|
|
|
* @param offset offset in buf at which to store data
|
|
|
- * @param len maximun number of bytes to read
|
|
|
+ * @param len maximum number of bytes to read
|
|
|
+ * @param checksum the data buffer into which to write checksums
|
|
|
* @return number of bytes read
|
|
|
*/
|
|
|
abstract protected int readChunk(long pos, byte[] buf, int offset, int len,
|
|
@@ -96,7 +128,7 @@ abstract public class FSInputChecker extends FSInputStream {
|
|
|
protected synchronized boolean needChecksum() {
|
|
|
return verifyChecksum && sum != null;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/**
|
|
|
* Read one checksum-verified byte
|
|
|
*
|
|
@@ -173,7 +205,7 @@ abstract public class FSInputChecker extends FSInputStream {
|
|
|
private void fill( ) throws IOException {
|
|
|
assert(pos>=count);
|
|
|
// fill internal buffer
|
|
|
- count = readChecksumChunk(buf, 0, buf.length);
|
|
|
+ count = readChecksumChunk(buf, 0, maxChunkSize);
|
|
|
if (count < 0) count = 0;
|
|
|
}
|
|
|
|
|
@@ -185,13 +217,13 @@ abstract public class FSInputChecker extends FSInputStream {
|
|
|
throws IOException {
|
|
|
int avail = count-pos;
|
|
|
if( avail <= 0 ) {
|
|
|
- if(len>=buf.length) {
|
|
|
+ if(len >= maxChunkSize) {
|
|
|
// read a chunk to user buffer directly; avoid one copy
|
|
|
int nread = readChecksumChunk(b, off, len);
|
|
|
return nread;
|
|
|
} else {
|
|
|
// read a chunk into the local buffer
|
|
|
- fill();
|
|
|
+ fill();
|
|
|
if( count <= 0 ) {
|
|
|
return -1;
|
|
|
} else {
|
|
@@ -207,10 +239,10 @@ abstract public class FSInputChecker extends FSInputStream {
|
|
|
return cnt;
|
|
|
}
|
|
|
|
|
|
- /* Read up one checksum chunk to array <i>b</i> at pos <i>off</i>
|
|
|
- * It requires a checksum chunk boundary
|
|
|
+ /* Read up one or more checksum chunk to array <i>b</i> at pos <i>off</i>
|
|
|
+ * It requires at least one checksum chunk boundary
|
|
|
* in between <cur_pos, cur_pos+len>
|
|
|
- * and it stops reading at the boundary or at the end of the stream;
|
|
|
+ * and it stops reading at the last boundary or at the end of the stream;
|
|
|
* Otherwise an IllegalArgumentException is thrown.
|
|
|
* This makes sure that all data read are checksum verified.
|
|
|
*
|
|
@@ -223,7 +255,7 @@ abstract public class FSInputChecker extends FSInputStream {
|
|
|
* the stream has been reached.
|
|
|
* @throws IOException if an I/O error occurs.
|
|
|
*/
|
|
|
- private int readChecksumChunk(byte b[], int off, int len)
|
|
|
+ private int readChecksumChunk(byte b[], final int off, final int len)
|
|
|
throws IOException {
|
|
|
// invalidate buffer
|
|
|
count = pos = 0;
|
|
@@ -236,13 +268,12 @@ abstract public class FSInputChecker extends FSInputStream {
|
|
|
|
|
|
try {
|
|
|
read = readChunk(chunkPos, b, off, len, checksum);
|
|
|
- if( read > 0 ) {
|
|
|
+ if( read > 0) {
|
|
|
if( needChecksum() ) {
|
|
|
- sum.update(b, off, read);
|
|
|
- verifySum(chunkPos);
|
|
|
+ verifySums(b, off, read);
|
|
|
}
|
|
|
chunkPos += read;
|
|
|
- }
|
|
|
+ }
|
|
|
retry = false;
|
|
|
} catch (ChecksumException ce) {
|
|
|
LOG.info("Found checksum error: b[" + off + ", " + (off+read) + "]="
|
|
@@ -266,26 +297,38 @@ abstract public class FSInputChecker extends FSInputStream {
|
|
|
} while (retry);
|
|
|
return read;
|
|
|
}
|
|
|
-
|
|
|
- /* verify checksum for the chunk.
|
|
|
- * @throws ChecksumException if there is a mismatch
|
|
|
- */
|
|
|
- private void verifySum(long errPos) throws ChecksumException {
|
|
|
- long crc = getChecksum();
|
|
|
- long sumValue = sum.getValue();
|
|
|
- sum.reset();
|
|
|
- if (crc != sumValue) {
|
|
|
- throw new ChecksumException(
|
|
|
- "Checksum error: "+file+" at "+errPos, errPos);
|
|
|
+
|
|
|
+ private void verifySums(final byte b[], final int off, int read)
|
|
|
+ throws ChecksumException
|
|
|
+ {
|
|
|
+ int leftToVerify = read;
|
|
|
+ int verifyOff = 0;
|
|
|
+ checksumInts.rewind();
|
|
|
+ checksumInts.limit((read - 1)/maxChunkSize + 1);
|
|
|
+
|
|
|
+ while (leftToVerify > 0) {
|
|
|
+ sum.update(b, off + verifyOff, Math.min(leftToVerify, maxChunkSize));
|
|
|
+ int expected = checksumInts.get();
|
|
|
+ int calculated = (int)sum.getValue();
|
|
|
+ sum.reset();
|
|
|
+
|
|
|
+ if (expected != calculated) {
|
|
|
+ long errPos = chunkPos + verifyOff;
|
|
|
+ throw new ChecksumException(
|
|
|
+ "Checksum error: "+file+" at "+ errPos +
|
|
|
+ " exp: " + expected + " got: " + calculated, errPos);
|
|
|
+ }
|
|
|
+ leftToVerify -= maxChunkSize;
|
|
|
+ verifyOff += maxChunkSize;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- /* calculate checksum value */
|
|
|
- private long getChecksum() {
|
|
|
- return checksum2long(checksum);
|
|
|
- }
|
|
|
|
|
|
- /** Convert a checksum byte array to a long */
|
|
|
+ /**
|
|
|
+ * Convert a checksum byte array to a long
|
|
|
+ * This is deprecated since 0.22 since it is no longer in use
|
|
|
+ * by this class.
|
|
|
+ */
|
|
|
+ @Deprecated
|
|
|
static public long checksum2long(byte[] checksum) {
|
|
|
long crc = 0L;
|
|
|
for(int i=0; i<checksum.length; i++) {
|
|
@@ -293,7 +336,7 @@ abstract public class FSInputChecker extends FSInputStream {
|
|
|
}
|
|
|
return crc;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
@Override
|
|
|
public synchronized long getPos() throws IOException {
|
|
|
return chunkPos-Math.max(0L, count - pos);
|
|
@@ -399,11 +442,19 @@ abstract public class FSInputChecker extends FSInputStream {
|
|
|
* @param checksumSize checksum size
|
|
|
*/
|
|
|
final protected synchronized void set(boolean verifyChecksum,
|
|
|
- Checksum sum, int maxChunkSize, int checksumSize ) {
|
|
|
+ Checksum sum, int maxChunkSize, int checksumSize) {
|
|
|
+
|
|
|
+ // The code makes assumptions that checksums are always 32-bit.
|
|
|
+ assert !verifyChecksum || sum == null || checksumSize == CHECKSUM_SIZE;
|
|
|
+
|
|
|
+ this.maxChunkSize = maxChunkSize;
|
|
|
this.verifyChecksum = verifyChecksum;
|
|
|
this.sum = sum;
|
|
|
this.buf = new byte[maxChunkSize];
|
|
|
- this.checksum = new byte[checksumSize];
|
|
|
+ // The size of the checksum array here determines how much we can
|
|
|
+ // read in a single call to readChunk
|
|
|
+ this.checksum = new byte[CHUNKS_PER_READ * checksumSize];
|
|
|
+ this.checksumInts = ByteBuffer.wrap(checksum).asIntBuffer();
|
|
|
this.count = 0;
|
|
|
this.pos = 0;
|
|
|
}
|