|
@@ -21,7 +21,6 @@ package org.apache.hadoop.fs;
|
|
|
import java.io.*;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.zip.CRC32;
|
|
|
-import java.util.zip.Checksum;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -40,7 +39,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
|
private static final byte[] CHECKSUM_VERSION = new byte[] {'c', 'r', 'c', 0};
|
|
|
|
|
|
public static double getApproxChkSumLength(long size) {
|
|
|
- return FSOutputSummer.CHKSUM_AS_FRACTION * size;
|
|
|
+ return ChecksumFSOutputSummer.CHKSUM_AS_FRACTION * size;
|
|
|
}
|
|
|
|
|
|
public ChecksumFileSystem(FileSystem fs) {
|
|
@@ -67,7 +66,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
|
* actual file.
|
|
|
**/
|
|
|
public long getChecksumFileLength(Path file, long fileSize) {
|
|
|
- return FSOutputSummer.getChecksumLength(fileSize, getBytesPerSum());
|
|
|
+ return ChecksumFSOutputSummer.getChecksumLength(fileSize, getBytesPerSum());
|
|
|
}
|
|
|
|
|
|
/** Return the bytes Per Checksum */
|
|
@@ -86,31 +85,28 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
|
* For open()'s FSInputStream
|
|
|
* It verifies that data matches checksums.
|
|
|
*******************************************************/
|
|
|
- private static class FSInputChecker extends FSInputStream {
|
|
|
+ private static class ChecksumFSInputChecker extends FSInputChecker {
|
|
|
public static final Log LOG
|
|
|
= LogFactory.getLog("org.apache.hadoop.fs.FSInputChecker");
|
|
|
|
|
|
private ChecksumFileSystem fs;
|
|
|
- private Path file;
|
|
|
private FSDataInputStream datas;
|
|
|
private FSDataInputStream sums;
|
|
|
- private Checksum sum = new CRC32();
|
|
|
- private int inSum;
|
|
|
|
|
|
private static final int HEADER_LENGTH = 8;
|
|
|
|
|
|
private int bytesPerSum = 1;
|
|
|
|
|
|
- public FSInputChecker(ChecksumFileSystem fs, Path file)
|
|
|
+ public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file)
|
|
|
throws IOException {
|
|
|
this(fs, file, fs.getConf().getInt("io.file.buffer.size", 4096));
|
|
|
}
|
|
|
|
|
|
- public FSInputChecker(ChecksumFileSystem fs, Path file, int bufferSize)
|
|
|
+ public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file, int bufferSize)
|
|
|
throws IOException {
|
|
|
- datas = fs.getRawFileSystem().open(file, bufferSize);
|
|
|
+ super( file, fs.getFileStatus(file).getReplication() );
|
|
|
+ this.datas = fs.getRawFileSystem().open(file, bufferSize);
|
|
|
this.fs = fs;
|
|
|
- this.file = file;
|
|
|
Path sumFile = fs.getChecksumFile(file);
|
|
|
try {
|
|
|
int sumBufferSize = fs.getSumBufferSize(fs.getBytesPerSum(), bufferSize);
|
|
@@ -120,208 +116,92 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
|
sums.readFully(version);
|
|
|
if (!Arrays.equals(version, CHECKSUM_VERSION))
|
|
|
throw new IOException("Not a checksum file: "+sumFile);
|
|
|
- bytesPerSum = sums.readInt();
|
|
|
+ this.bytesPerSum = sums.readInt();
|
|
|
+ set(new CRC32(), bytesPerSum, 4);
|
|
|
} catch (FileNotFoundException e) { // quietly ignore
|
|
|
- stopSumming();
|
|
|
+ set(null, 1, 0);
|
|
|
} catch (IOException e) { // loudly ignore
|
|
|
LOG.warn("Problem opening checksum file: "+ file +
|
|
|
". Ignoring exception: " +
|
|
|
StringUtils.stringifyException(e));
|
|
|
- stopSumming();
|
|
|
+ set(null, 1, 0);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private long getChecksumFilePos( long dataPos ) {
|
|
|
return HEADER_LENGTH + 4*(dataPos/bytesPerSum);
|
|
|
}
|
|
|
|
|
|
- public void seek(long desired) throws IOException {
|
|
|
- // seek to a checksum boundary
|
|
|
- long checksumBoundary = desired/bytesPerSum*bytesPerSum;
|
|
|
- if (checksumBoundary != getPos()) {
|
|
|
- datas.seek(checksumBoundary);
|
|
|
- if (sums != null) {
|
|
|
- sums.seek(getChecksumFilePos(checksumBoundary));
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- if (sums != null) {
|
|
|
- sum.reset();
|
|
|
- inSum = 0;
|
|
|
- }
|
|
|
-
|
|
|
- // scan to desired position
|
|
|
- int delta = (int)(desired - checksumBoundary);
|
|
|
- readBuffer(new byte[delta], 0, delta);
|
|
|
- }
|
|
|
-
|
|
|
- public int read() throws IOException {
|
|
|
- byte[] b = new byte[1];
|
|
|
- readBuffer(b, 0, 1);
|
|
|
- return b[0] & 0xff;
|
|
|
- }
|
|
|
-
|
|
|
- public int read(byte b[]) throws IOException {
|
|
|
- return read(b, 0, b.length);
|
|
|
- }
|
|
|
-
|
|
|
- public int read(byte b[], int off, int len) throws IOException {
|
|
|
- // make sure that it ends at a checksum boundary
|
|
|
- long curPos = getPos();
|
|
|
- long endPos = len+curPos/bytesPerSum*bytesPerSum;
|
|
|
- return readBuffer(b, off, (int)(endPos-curPos));
|
|
|
+ protected long getChunkPosition( long dataPos ) {
|
|
|
+ return dataPos/bytesPerSum*bytesPerSum;
|
|
|
}
|
|
|
|
|
|
- private int readBuffer(byte b[], int off, int len) throws IOException {
|
|
|
- int read;
|
|
|
- boolean retry;
|
|
|
- int retriesLeft = 3;
|
|
|
- long oldPos = getPos();
|
|
|
- do {
|
|
|
- retriesLeft--;
|
|
|
- retry = false;
|
|
|
-
|
|
|
- read = 0;
|
|
|
- boolean endOfFile=false;
|
|
|
- while (read < len && !endOfFile) {
|
|
|
- int count = datas.read(b, off + read, len - read);
|
|
|
- if (count < 0)
|
|
|
- endOfFile = true;
|
|
|
- else
|
|
|
- read += count;
|
|
|
- }
|
|
|
-
|
|
|
- if (sums != null && read!=0) {
|
|
|
- long oldSumsPos = sums.getPos();
|
|
|
- try {
|
|
|
- int summed = 0;
|
|
|
- while (summed < read) {
|
|
|
- int goal = bytesPerSum - inSum;
|
|
|
- int inBuf = read - summed;
|
|
|
- int toSum = inBuf <= goal ? inBuf : goal;
|
|
|
-
|
|
|
- try {
|
|
|
- sum.update(b, off+summed, toSum);
|
|
|
- } catch (ArrayIndexOutOfBoundsException e) {
|
|
|
- throw new RuntimeException("Summer buffer overflow b.len=" +
|
|
|
- b.length + ", off=" + off +
|
|
|
- ", summed=" + summed + ", read=" +
|
|
|
- read + ", bytesPerSum=" + bytesPerSum +
|
|
|
- ", inSum=" + inSum, e);
|
|
|
- }
|
|
|
- summed += toSum;
|
|
|
-
|
|
|
- inSum += toSum;
|
|
|
- if (inSum == bytesPerSum) {
|
|
|
- verifySum(read-(summed-bytesPerSum));
|
|
|
- } else if (read == summed && endOfFile) {
|
|
|
- verifySum(read-read/bytesPerSum*bytesPerSum);
|
|
|
- }
|
|
|
- }
|
|
|
- } catch (ChecksumException ce) {
|
|
|
- LOG.info("Found checksum error: "+StringUtils.stringifyException(ce));
|
|
|
- long errPos = ce.getPos();
|
|
|
- boolean shouldRetry = fs.reportChecksumFailure(
|
|
|
- file, datas, errPos, sums, errPos/bytesPerSum);
|
|
|
- if (!shouldRetry || retriesLeft == 0) {
|
|
|
- throw ce;
|
|
|
- }
|
|
|
-
|
|
|
- if (seekToNewSource(oldPos)) {
|
|
|
- // Since at least one of the sources is different,
|
|
|
- // the read might succeed, so we'll retry.
|
|
|
- retry = true;
|
|
|
- seek(oldPos); //make sure Checksum sum's value gets restored
|
|
|
- } else {
|
|
|
- // Neither the data stream nor the checksum stream are being read
|
|
|
- // from different sources, meaning we'll still get a checksum error
|
|
|
- // if we try to do the read again. We throw an exception instead.
|
|
|
- throw ce;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- } while (retry);
|
|
|
- return read==0?-1:read;
|
|
|
+ public int available() throws IOException {
|
|
|
+ return datas.available() + super.available();
|
|
|
}
|
|
|
|
|
|
- private void verifySum(int delta) throws IOException {
|
|
|
- int crc;
|
|
|
- try {
|
|
|
- crc = sums.readInt();
|
|
|
- } catch (IOException e) {
|
|
|
- LOG.warn("Problem reading checksum file: "+e+". Ignoring.");
|
|
|
- stopSumming();
|
|
|
- return;
|
|
|
+ public int read(long position, byte[] b, int off, int len)
|
|
|
+ throws IOException {
|
|
|
+ // parameter check
|
|
|
+ if ((off | len | (off + len) | (b.length - (off + len))) < 0) {
|
|
|
+ throw new IndexOutOfBoundsException();
|
|
|
+ } else if (len == 0) {
|
|
|
+ return 0;
|
|
|
}
|
|
|
- int sumValue = (int)sum.getValue();
|
|
|
- sum.reset();
|
|
|
- inSum = 0;
|
|
|
- if (crc != sumValue) {
|
|
|
- long pos = getPos() - delta;
|
|
|
- throw new ChecksumException("Checksum error: "+file+" at "+pos, pos);
|
|
|
+ if( position<0 ) {
|
|
|
+ throw new IllegalArgumentException(
|
|
|
+ "Parameter position can not to be negative");
|
|
|
}
|
|
|
- }
|
|
|
-
|
|
|
- public long getPos() throws IOException {
|
|
|
- return datas.getPos();
|
|
|
- }
|
|
|
-
|
|
|
- public int read(long position, byte[] buffer, int offset, int length)
|
|
|
- throws IOException {
|
|
|
- return datas.read(position, buffer, offset, length);
|
|
|
- }
|
|
|
-
|
|
|
- public void readFully(long position, byte[] buffer, int offset, int length)
|
|
|
- throws IOException {
|
|
|
- datas.readFully(position, buffer, offset, length);
|
|
|
- }
|
|
|
-
|
|
|
- public void readFully(long position, byte[] buffer)
|
|
|
- throws IOException {
|
|
|
- datas.readFully(position, buffer);
|
|
|
+
|
|
|
+ ChecksumFSInputChecker checker = new ChecksumFSInputChecker(fs, file);
|
|
|
+ checker.seek(position);
|
|
|
+ int nread = checker.read(b, off, len);
|
|
|
+ checker.close();
|
|
|
+ return nread;
|
|
|
}
|
|
|
|
|
|
public void close() throws IOException {
|
|
|
datas.close();
|
|
|
- stopSumming();
|
|
|
- }
|
|
|
-
|
|
|
- private void stopSumming() {
|
|
|
- if (sums != null) {
|
|
|
- try {
|
|
|
- sums.close();
|
|
|
- } catch (IOException f) {}
|
|
|
- sums = null;
|
|
|
- bytesPerSum = 1;
|
|
|
+ if( sums != null ) {
|
|
|
+ sums.close();
|
|
|
}
|
|
|
+ set(null, 1, 0);
|
|
|
}
|
|
|
|
|
|
- public int available() throws IOException {
|
|
|
- return datas.available();
|
|
|
- }
|
|
|
-
|
|
|
- public boolean markSupported() {
|
|
|
- return datas.markSupported();
|
|
|
- }
|
|
|
-
|
|
|
- public synchronized void mark(int readlimit) {
|
|
|
- datas.mark(readlimit);
|
|
|
- }
|
|
|
-
|
|
|
- public synchronized void reset() throws IOException {
|
|
|
- datas.reset();
|
|
|
- }
|
|
|
-
|
|
|
- public long skip(long n) throws IOException {
|
|
|
- return datas.skip(n);
|
|
|
- }
|
|
|
|
|
|
@Override
|
|
|
public boolean seekToNewSource(long targetPos) throws IOException {
|
|
|
+ long sumsPos = getChecksumFilePos(targetPos);
|
|
|
+ fs.reportChecksumFailure(file, datas, targetPos, sums, sumsPos);
|
|
|
boolean newDataSource = datas.seekToNewSource(targetPos);
|
|
|
- return sums.seekToNewSource(getChecksumFilePos(targetPos)) || newDataSource;
|
|
|
+ return sums.seekToNewSource(sumsPos) || newDataSource;
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ protected int readChunk(long pos, byte[] buf, int offset, int len,
|
|
|
+ byte[] checksum) throws IOException {
|
|
|
+ boolean eof = false;
|
|
|
+ if(needChecksum()) {
|
|
|
+ try {
|
|
|
+ long checksumPos = getChecksumFilePos(pos);
|
|
|
+ if(checksumPos != sums.getPos()) {
|
|
|
+ sums.seek(checksumPos);
|
|
|
+ }
|
|
|
+ sums.readFully(checksum);
|
|
|
+ } catch (EOFException e) {
|
|
|
+ eof = true;
|
|
|
+ }
|
|
|
+ len = bytesPerSum;
|
|
|
+ }
|
|
|
+ if(pos != datas.getPos()) {
|
|
|
+ datas.seek(pos);
|
|
|
+ }
|
|
|
+ int nread = readFully(datas, buf, offset, len);
|
|
|
+ if( eof && nread > 0) {
|
|
|
+ throw new ChecksumException("Checksum error: "+file+" at "+pos, pos);
|
|
|
+ }
|
|
|
+ return nread;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -331,21 +211,18 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
|
*/
|
|
|
@Override
|
|
|
public FSDataInputStream open(Path f, int bufferSize) throws IOException {
|
|
|
- return new FSDataInputStream(new FSInputChecker(this, f, bufferSize),
|
|
|
- getBytesPerSum());
|
|
|
+ return new FSDataInputStream(
|
|
|
+ new ChecksumFSInputChecker(this, f, bufferSize) );
|
|
|
}
|
|
|
|
|
|
/** This class provides an output stream for a checksummed file.
|
|
|
* It generates checksums for data. */
|
|
|
- private static class FSOutputSummer extends FilterOutputStream {
|
|
|
-
|
|
|
+ private static class ChecksumFSOutputSummer extends FSOutputSummer {
|
|
|
+ private FSDataOutputStream datas;
|
|
|
private FSDataOutputStream sums;
|
|
|
- private Checksum sum = new CRC32();
|
|
|
- private int inSum;
|
|
|
- private int bytesPerSum;
|
|
|
private static final float CHKSUM_AS_FRACTION = 0.01f;
|
|
|
|
|
|
- public FSOutputSummer(ChecksumFileSystem fs,
|
|
|
+ public ChecksumFSOutputSummer(ChecksumFileSystem fs,
|
|
|
Path file,
|
|
|
boolean overwrite,
|
|
|
short replication,
|
|
@@ -357,7 +234,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
|
replication, blockSize, null);
|
|
|
}
|
|
|
|
|
|
- public FSOutputSummer(ChecksumFileSystem fs,
|
|
|
+ public ChecksumFSOutputSummer(ChecksumFileSystem fs,
|
|
|
Path file,
|
|
|
boolean overwrite,
|
|
|
int bufferSize,
|
|
@@ -365,51 +242,22 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
|
long blockSize,
|
|
|
Progressable progress)
|
|
|
throws IOException {
|
|
|
- super(fs.getRawFileSystem().create(file, overwrite, bufferSize,
|
|
|
- replication, blockSize, progress));
|
|
|
- this.bytesPerSum = fs.getBytesPerSum();
|
|
|
+ super(new CRC32(), fs.getBytesPerSum(), 4);
|
|
|
+ int bytesPerSum = fs.getBytesPerSum();
|
|
|
+ this.datas = fs.getRawFileSystem().create(file, overwrite, bufferSize,
|
|
|
+ replication, blockSize, progress);
|
|
|
int sumBufferSize = fs.getSumBufferSize(bytesPerSum, bufferSize);
|
|
|
this.sums = fs.getRawFileSystem().create(fs.getChecksumFile(file), true,
|
|
|
sumBufferSize, replication,
|
|
|
blockSize);
|
|
|
sums.write(CHECKSUM_VERSION, 0, CHECKSUM_VERSION.length);
|
|
|
- sums.writeInt(this.bytesPerSum);
|
|
|
- }
|
|
|
-
|
|
|
- public void write(byte b[], int off, int len) throws IOException {
|
|
|
- int summed = 0;
|
|
|
- while (summed < len) {
|
|
|
-
|
|
|
- int goal = this.bytesPerSum - inSum;
|
|
|
- int inBuf = len - summed;
|
|
|
- int toSum = inBuf <= goal ? inBuf : goal;
|
|
|
-
|
|
|
- sum.update(b, off+summed, toSum);
|
|
|
- summed += toSum;
|
|
|
-
|
|
|
- inSum += toSum;
|
|
|
- if (inSum == this.bytesPerSum) {
|
|
|
- writeSum();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- out.write(b, off, len);
|
|
|
- }
|
|
|
-
|
|
|
- private void writeSum() throws IOException {
|
|
|
- if (inSum != 0) {
|
|
|
- sums.writeInt((int)sum.getValue());
|
|
|
- sum.reset();
|
|
|
- inSum = 0;
|
|
|
- }
|
|
|
+ sums.writeInt(bytesPerSum);
|
|
|
}
|
|
|
|
|
|
public void close() throws IOException {
|
|
|
- writeSum();
|
|
|
- if (sums != null) {
|
|
|
- sums.close();
|
|
|
- }
|
|
|
- out.close();
|
|
|
+ flushBuffer();
|
|
|
+ sums.close();
|
|
|
+ datas.close();
|
|
|
}
|
|
|
|
|
|
public static long getChecksumLength(long size, int bytesPerSum) {
|
|
@@ -418,6 +266,13 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
|
return ((long)(Math.ceil((float)size/bytesPerSum)) + 1) * 4 +
|
|
|
CHECKSUM_VERSION.length;
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void writeChunk(byte[] b, int offset, int len, byte[] checksum)
|
|
|
+ throws IOException {
|
|
|
+ datas.write(b, offset, len);
|
|
|
+ sums.write(checksum);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -433,15 +288,12 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
|
public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize,
|
|
|
short replication, long blockSize, Progressable progress)
|
|
|
throws IOException {
|
|
|
- if (exists(f) && !overwrite) {
|
|
|
- throw new IOException("File already exists:" + f);
|
|
|
- }
|
|
|
Path parent = f.getParent();
|
|
|
if (parent != null && !mkdirs(parent)) {
|
|
|
throw new IOException("Mkdirs failed to create " + parent);
|
|
|
}
|
|
|
- return new FSDataOutputStream(new FSOutputSummer(this, f, overwrite,
|
|
|
- bufferSize, replication, blockSize, progress), getBytesPerSum());
|
|
|
+ return new FSDataOutputStream(new ChecksumFSOutputSummer(
|
|
|
+ this, f, overwrite, bufferSize, replication, blockSize, progress));
|
|
|
}
|
|
|
|
|
|
/**
|