|
@@ -22,17 +22,25 @@ import java.io.EOFException;
|
|
import java.io.FileNotFoundException;
|
|
import java.io.FileNotFoundException;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.io.InputStream;
|
|
import java.io.InputStream;
|
|
|
|
+import java.nio.ByteBuffer;
|
|
|
|
+import java.nio.IntBuffer;
|
|
import java.nio.channels.ClosedChannelException;
|
|
import java.nio.channels.ClosedChannelException;
|
|
|
|
+import java.util.ArrayList;
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
import java.util.EnumSet;
|
|
import java.util.EnumSet;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
import java.util.concurrent.CompletableFuture;
|
|
import java.util.concurrent.CompletableFuture;
|
|
|
|
+import java.util.concurrent.CompletionException;
|
|
|
|
+import java.util.function.IntFunction;
|
|
|
|
+import java.util.zip.CRC32;
|
|
|
|
|
|
import org.apache.hadoop.util.Preconditions;
|
|
import org.apache.hadoop.util.Preconditions;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl;
|
|
import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl;
|
|
|
|
+import org.apache.hadoop.fs.impl.VectoredReadUtils;
|
|
|
|
+import org.apache.hadoop.fs.impl.CombinedFileRange;
|
|
import org.apache.hadoop.fs.impl.FutureDataInputStreamBuilderImpl;
|
|
import org.apache.hadoop.fs.impl.FutureDataInputStreamBuilderImpl;
|
|
import org.apache.hadoop.fs.impl.OpenFileParameters;
|
|
import org.apache.hadoop.fs.impl.OpenFileParameters;
|
|
import org.apache.hadoop.fs.permission.AclEntry;
|
|
import org.apache.hadoop.fs.permission.AclEntry;
|
|
@@ -66,7 +74,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
public static double getApproxChkSumLength(long size) {
|
|
public static double getApproxChkSumLength(long size) {
|
|
return ChecksumFSOutputSummer.CHKSUM_AS_FRACTION * size;
|
|
return ChecksumFSOutputSummer.CHKSUM_AS_FRACTION * size;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
public ChecksumFileSystem(FileSystem fs) {
|
|
public ChecksumFileSystem(FileSystem fs) {
|
|
super(fs);
|
|
super(fs);
|
|
}
|
|
}
|
|
@@ -82,7 +90,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
bytesPerChecksum);
|
|
bytesPerChecksum);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Set whether to verify checksum.
|
|
* Set whether to verify checksum.
|
|
*/
|
|
*/
|
|
@@ -95,7 +103,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
public void setWriteChecksum(boolean writeChecksum) {
|
|
public void setWriteChecksum(boolean writeChecksum) {
|
|
this.writeChecksum = writeChecksum;
|
|
this.writeChecksum = writeChecksum;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/** get the raw file system */
|
|
/** get the raw file system */
|
|
@Override
|
|
@Override
|
|
public FileSystem getRawFileSystem() {
|
|
public FileSystem getRawFileSystem() {
|
|
@@ -162,18 +170,18 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
private ChecksumFileSystem fs;
|
|
private ChecksumFileSystem fs;
|
|
private FSDataInputStream datas;
|
|
private FSDataInputStream datas;
|
|
private FSDataInputStream sums;
|
|
private FSDataInputStream sums;
|
|
-
|
|
|
|
|
|
+
|
|
private static final int HEADER_LENGTH = 8;
|
|
private static final int HEADER_LENGTH = 8;
|
|
-
|
|
|
|
|
|
+
|
|
private int bytesPerSum = 1;
|
|
private int bytesPerSum = 1;
|
|
-
|
|
|
|
|
|
+
|
|
public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file)
|
|
public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file)
|
|
throws IOException {
|
|
throws IOException {
|
|
this(fs, file, fs.getConf().getInt(
|
|
this(fs, file, fs.getConf().getInt(
|
|
- LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY,
|
|
|
|
|
|
+ LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY,
|
|
LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_DEFAULT));
|
|
LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_DEFAULT));
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file, int bufferSize)
|
|
public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file, int bufferSize)
|
|
throws IOException {
|
|
throws IOException {
|
|
super( file, fs.getFileStatus(file).getReplication() );
|
|
super( file, fs.getFileStatus(file).getReplication() );
|
|
@@ -189,7 +197,8 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
if (!Arrays.equals(version, CHECKSUM_VERSION))
|
|
if (!Arrays.equals(version, CHECKSUM_VERSION))
|
|
throw new IOException("Not a checksum file: "+sumFile);
|
|
throw new IOException("Not a checksum file: "+sumFile);
|
|
this.bytesPerSum = sums.readInt();
|
|
this.bytesPerSum = sums.readInt();
|
|
- set(fs.verifyChecksum, DataChecksum.newCrc32(), bytesPerSum, 4);
|
|
|
|
|
|
+ set(fs.verifyChecksum, DataChecksum.newCrc32(), bytesPerSum,
|
|
|
|
+ FSInputChecker.CHECKSUM_SIZE);
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
// mincing the message is terrible, but java throws permission
|
|
// mincing the message is terrible, but java throws permission
|
|
// exceptions as FNF because that's all the method signatures allow!
|
|
// exceptions as FNF because that's all the method signatures allow!
|
|
@@ -201,21 +210,21 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
set(fs.verifyChecksum, null, 1, 0);
|
|
set(fs.verifyChecksum, null, 1, 0);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private long getChecksumFilePos( long dataPos ) {
|
|
private long getChecksumFilePos( long dataPos ) {
|
|
- return HEADER_LENGTH + 4*(dataPos/bytesPerSum);
|
|
|
|
|
|
+ return HEADER_LENGTH + FSInputChecker.CHECKSUM_SIZE*(dataPos/bytesPerSum);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
protected long getChunkPosition( long dataPos ) {
|
|
protected long getChunkPosition( long dataPos ) {
|
|
return dataPos/bytesPerSum*bytesPerSum;
|
|
return dataPos/bytesPerSum*bytesPerSum;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public int available() throws IOException {
|
|
public int available() throws IOException {
|
|
return datas.available() + super.available();
|
|
return datas.available() + super.available();
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public int read(long position, byte[] b, int off, int len)
|
|
public int read(long position, byte[] b, int off, int len)
|
|
throws IOException {
|
|
throws IOException {
|
|
@@ -233,7 +242,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
}
|
|
}
|
|
return nread;
|
|
return nread;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void close() throws IOException {
|
|
public void close() throws IOException {
|
|
datas.close();
|
|
datas.close();
|
|
@@ -242,7 +251,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
}
|
|
}
|
|
set(fs.verifyChecksum, null, 1, 0);
|
|
set(fs.verifyChecksum, null, 1, 0);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public boolean seekToNewSource(long targetPos) throws IOException {
|
|
public boolean seekToNewSource(long targetPos) throws IOException {
|
|
@@ -265,7 +274,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
final int checksumsToRead = Math.min(
|
|
final int checksumsToRead = Math.min(
|
|
len/bytesPerSum, // number of checksums based on len to read
|
|
len/bytesPerSum, // number of checksums based on len to read
|
|
checksum.length / CHECKSUM_SIZE); // size of checksum buffer
|
|
checksum.length / CHECKSUM_SIZE); // size of checksum buffer
|
|
- long checksumPos = getChecksumFilePos(pos);
|
|
|
|
|
|
+ long checksumPos = getChecksumFilePos(pos);
|
|
if(checksumPos != sums.getPos()) {
|
|
if(checksumPos != sums.getPos()) {
|
|
sums.seek(checksumPos);
|
|
sums.seek(checksumPos);
|
|
}
|
|
}
|
|
@@ -305,8 +314,129 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
public IOStatistics getIOStatistics() {
|
|
public IOStatistics getIOStatistics() {
|
|
return IOStatisticsSupport.retrieveIOStatistics(datas);
|
|
return IOStatisticsSupport.retrieveIOStatistics(datas);
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ public static long findChecksumOffset(long dataOffset,
|
|
|
|
+ int bytesPerSum) {
|
|
|
|
+ return HEADER_LENGTH + (dataOffset/bytesPerSum) * FSInputChecker.CHECKSUM_SIZE;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Find the checksum ranges that correspond to the given data ranges.
|
|
|
|
+ * @param dataRanges the input data ranges, which are assumed to be sorted
|
|
|
|
+ * and non-overlapping
|
|
|
|
+ * @return a list of AsyncReaderUtils.CombinedFileRange that correspond to
|
|
|
|
+ * the checksum ranges
|
|
|
|
+ */
|
|
|
|
+ public static List<CombinedFileRange> findChecksumRanges(
|
|
|
|
+ List<? extends FileRange> dataRanges,
|
|
|
|
+ int bytesPerSum,
|
|
|
|
+ int minSeek,
|
|
|
|
+ int maxSize) {
|
|
|
|
+ List<CombinedFileRange> result = new ArrayList<>();
|
|
|
|
+ CombinedFileRange currentCrc = null;
|
|
|
|
+ for(FileRange range: dataRanges) {
|
|
|
|
+ long crcOffset = findChecksumOffset(range.getOffset(), bytesPerSum);
|
|
|
|
+ long crcEnd = findChecksumOffset(range.getOffset() + range.getLength() +
|
|
|
|
+ bytesPerSum - 1, bytesPerSum);
|
|
|
|
+ if (currentCrc == null ||
|
|
|
|
+ !currentCrc.merge(crcOffset, crcEnd, range, minSeek, maxSize)) {
|
|
|
|
+ currentCrc = new CombinedFileRange(crcOffset, crcEnd, range);
|
|
|
|
+ result.add(currentCrc);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return result;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Check the data against the checksums.
|
|
|
|
+ * @param sumsBytes the checksum data
|
|
|
|
+ * @param sumsOffset where from the checksum file this buffer started
|
|
|
|
+ * @param data the file data
|
|
|
|
+ * @param dataOffset where the file data started (must be a multiple of
|
|
|
|
+ * bytesPerSum)
|
|
|
|
+ * @param bytesPerSum how many bytes per a checksum
|
|
|
|
+ * @param file the path of the filename
|
|
|
|
+ * @return the data buffer
|
|
|
|
+ * @throws CompletionException if the checksums don't match
|
|
|
|
+ */
|
|
|
|
+ static ByteBuffer checkBytes(ByteBuffer sumsBytes,
|
|
|
|
+ long sumsOffset,
|
|
|
|
+ ByteBuffer data,
|
|
|
|
+ long dataOffset,
|
|
|
|
+ int bytesPerSum,
|
|
|
|
+ Path file) {
|
|
|
|
+ // determine how many bytes we need to skip at the start of the sums
|
|
|
|
+ int offset =
|
|
|
|
+ (int) (findChecksumOffset(dataOffset, bytesPerSum) - sumsOffset);
|
|
|
|
+ IntBuffer sums = sumsBytes.asIntBuffer();
|
|
|
|
+ sums.position(offset / FSInputChecker.CHECKSUM_SIZE);
|
|
|
|
+ ByteBuffer current = data.duplicate();
|
|
|
|
+ int numChunks = data.remaining() / bytesPerSum;
|
|
|
|
+ CRC32 crc = new CRC32();
|
|
|
|
+ // check each chunk to ensure they match
|
|
|
|
+ for(int c = 0; c < numChunks; ++c) {
|
|
|
|
+ // set the buffer position and the limit
|
|
|
|
+ current.limit((c + 1) * bytesPerSum);
|
|
|
|
+ current.position(c * bytesPerSum);
|
|
|
|
+ // compute the crc
|
|
|
|
+ crc.reset();
|
|
|
|
+ crc.update(current);
|
|
|
|
+ int expected = sums.get();
|
|
|
|
+ int calculated = (int) crc.getValue();
|
|
|
|
+
|
|
|
|
+ if (calculated != expected) {
|
|
|
|
+ // cast of c added to silence findbugs
|
|
|
|
+ long errPosn = dataOffset + (long) c * bytesPerSum;
|
|
|
|
+ throw new CompletionException(new ChecksumException(
|
|
|
|
+ "Checksum error: " + file + " at " + errPosn +
|
|
|
|
+ " exp: " + expected + " got: " + calculated, errPosn));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ // if everything matches, we return the data
|
|
|
|
+ return data;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void readVectored(List<? extends FileRange> ranges,
|
|
|
|
+ IntFunction<ByteBuffer> allocate) throws IOException {
|
|
|
|
+ // If the stream doesn't have checksums, just delegate.
|
|
|
|
+ VectoredReadUtils.validateVectoredReadRanges(ranges);
|
|
|
|
+ if (sums == null) {
|
|
|
|
+ datas.readVectored(ranges, allocate);
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+ int minSeek = minSeekForVectorReads();
|
|
|
|
+ int maxSize = maxReadSizeForVectorReads();
|
|
|
|
+ List<CombinedFileRange> dataRanges =
|
|
|
|
+ VectoredReadUtils.sortAndMergeRanges(ranges, bytesPerSum,
|
|
|
|
+ minSeek, maxReadSizeForVectorReads());
|
|
|
|
+ List<CombinedFileRange> checksumRanges = findChecksumRanges(dataRanges,
|
|
|
|
+ bytesPerSum, minSeek, maxSize);
|
|
|
|
+ sums.readVectored(checksumRanges, allocate);
|
|
|
|
+ datas.readVectored(dataRanges, allocate);
|
|
|
|
+ // Data read is correct. I have verified content of dataRanges.
|
|
|
|
+ // There is some bug below here as test (testVectoredReadMultipleRanges)
|
|
|
|
+ // is failing, should be
|
|
|
|
+ // somewhere while slicing the merged data into smaller user ranges.
|
|
|
|
+ // Spend some time figuring out but it is a complex code.
|
|
|
|
+ for(CombinedFileRange checksumRange: checksumRanges) {
|
|
|
|
+ for(FileRange dataRange: checksumRange.getUnderlying()) {
|
|
|
|
+ // when we have both the ranges, validate the checksum
|
|
|
|
+ CompletableFuture<ByteBuffer> result =
|
|
|
|
+ checksumRange.getData().thenCombineAsync(dataRange.getData(),
|
|
|
|
+ (sumBuffer, dataBuffer) ->
|
|
|
|
+ checkBytes(sumBuffer, checksumRange.getOffset(),
|
|
|
|
+ dataBuffer, dataRange.getOffset(), bytesPerSum, file));
|
|
|
|
+ // Now, slice the read data range to the user's ranges
|
|
|
|
+ for(FileRange original: ((CombinedFileRange) dataRange).getUnderlying()) {
|
|
|
|
+ original.setData(result.thenApply(
|
|
|
|
+ (b) -> VectoredReadUtils.sliceTo(b, dataRange.getOffset(), original)));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
private static class FSDataBoundedInputStream extends FSDataInputStream {
|
|
private static class FSDataBoundedInputStream extends FSDataInputStream {
|
|
private FileSystem fs;
|
|
private FileSystem fs;
|
|
private Path file;
|
|
private Path file;
|
|
@@ -317,12 +447,12 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
this.fs = fs;
|
|
this.fs = fs;
|
|
this.file = file;
|
|
this.file = file;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public boolean markSupported() {
|
|
public boolean markSupported() {
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/* Return the file length */
|
|
/* Return the file length */
|
|
private long getFileLength() throws IOException {
|
|
private long getFileLength() throws IOException {
|
|
if( fileLen==-1L ) {
|
|
if( fileLen==-1L ) {
|
|
@@ -330,7 +460,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
}
|
|
}
|
|
return fileLen;
|
|
return fileLen;
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Skips over and discards <code>n</code> bytes of data from the
|
|
* Skips over and discards <code>n</code> bytes of data from the
|
|
* input stream.
|
|
* input stream.
|
|
@@ -354,11 +484,11 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
}
|
|
}
|
|
return super.skip(n);
|
|
return super.skip(n);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Seek to the given position in the stream.
|
|
* Seek to the given position in the stream.
|
|
* The next read() will be from that position.
|
|
* The next read() will be from that position.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* <p>This method does not allow seek past the end of the file.
|
|
* <p>This method does not allow seek past the end of the file.
|
|
* This produces IOException.
|
|
* This produces IOException.
|
|
*
|
|
*
|
|
@@ -424,22 +554,22 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
*/
|
|
*/
|
|
public static long getChecksumLength(long size, int bytesPerSum) {
|
|
public static long getChecksumLength(long size, int bytesPerSum) {
|
|
//the checksum length is equal to size passed divided by bytesPerSum +
|
|
//the checksum length is equal to size passed divided by bytesPerSum +
|
|
- //bytes written in the beginning of the checksum file.
|
|
|
|
- return ((size + bytesPerSum - 1) / bytesPerSum) * 4 +
|
|
|
|
- CHECKSUM_VERSION.length + 4;
|
|
|
|
|
|
+ //bytes written in the beginning of the checksum file.
|
|
|
|
+ return ((size + bytesPerSum - 1) / bytesPerSum) * FSInputChecker.CHECKSUM_SIZE +
|
|
|
|
+ ChecksumFSInputChecker.HEADER_LENGTH;
|
|
}
|
|
}
|
|
|
|
|
|
/** This class provides an output stream for a checksummed file.
|
|
/** This class provides an output stream for a checksummed file.
|
|
* It generates checksums for data. */
|
|
* It generates checksums for data. */
|
|
private static class ChecksumFSOutputSummer extends FSOutputSummer
|
|
private static class ChecksumFSOutputSummer extends FSOutputSummer
|
|
implements IOStatisticsSource, StreamCapabilities {
|
|
implements IOStatisticsSource, StreamCapabilities {
|
|
- private FSDataOutputStream datas;
|
|
|
|
|
|
+ private FSDataOutputStream datas;
|
|
private FSDataOutputStream sums;
|
|
private FSDataOutputStream sums;
|
|
private static final float CHKSUM_AS_FRACTION = 0.01f;
|
|
private static final float CHKSUM_AS_FRACTION = 0.01f;
|
|
private boolean isClosed = false;
|
|
private boolean isClosed = false;
|
|
-
|
|
|
|
- public ChecksumFSOutputSummer(ChecksumFileSystem fs,
|
|
|
|
- Path file,
|
|
|
|
|
|
+
|
|
|
|
+ ChecksumFSOutputSummer(ChecksumFileSystem fs,
|
|
|
|
+ Path file,
|
|
boolean overwrite,
|
|
boolean overwrite,
|
|
int bufferSize,
|
|
int bufferSize,
|
|
short replication,
|
|
short replication,
|
|
@@ -460,7 +590,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
sums.write(CHECKSUM_VERSION, 0, CHECKSUM_VERSION.length);
|
|
sums.write(CHECKSUM_VERSION, 0, CHECKSUM_VERSION.length);
|
|
sums.writeInt(bytesPerSum);
|
|
sums.writeInt(bytesPerSum);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void close() throws IOException {
|
|
public void close() throws IOException {
|
|
try {
|
|
try {
|
|
@@ -471,7 +601,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
isClosed = true;
|
|
isClosed = true;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
protected void writeChunk(byte[] b, int offset, int len, byte[] checksum,
|
|
protected void writeChunk(byte[] b, int offset, int len, byte[] checksum,
|
|
int ckoff, int cklen)
|
|
int ckoff, int cklen)
|
|
@@ -727,7 +857,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
value = fs.rename(srcCheckFile, dstCheckFile);
|
|
value = fs.rename(srcCheckFile, dstCheckFile);
|
|
} else if (fs.exists(dstCheckFile)) {
|
|
} else if (fs.exists(dstCheckFile)) {
|
|
// no src checksum, so remove dst checksum
|
|
// no src checksum, so remove dst checksum
|
|
- value = fs.delete(dstCheckFile, true);
|
|
|
|
|
|
+ value = fs.delete(dstCheckFile, true);
|
|
}
|
|
}
|
|
|
|
|
|
return value;
|
|
return value;
|
|
@@ -759,7 +889,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
return fs.delete(f, true);
|
|
return fs.delete(f, true);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
final private static PathFilter DEFAULT_FILTER = new PathFilter() {
|
|
final private static PathFilter DEFAULT_FILTER = new PathFilter() {
|
|
@Override
|
|
@Override
|
|
public boolean accept(Path file) {
|
|
public boolean accept(Path file) {
|
|
@@ -770,7 +900,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
/**
|
|
/**
|
|
* List the statuses of the files/directories in the given path if the path is
|
|
* List the statuses of the files/directories in the given path if the path is
|
|
* a directory.
|
|
* a directory.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @param f
|
|
* @param f
|
|
* given path
|
|
* given path
|
|
* @return the statuses of the files/directories in the given path
|
|
* @return the statuses of the files/directories in the given path
|
|
@@ -791,7 +921,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
/**
|
|
/**
|
|
* List the statuses of the files/directories in the given path if the path is
|
|
* List the statuses of the files/directories in the given path if the path is
|
|
* a directory.
|
|
* a directory.
|
|
- *
|
|
|
|
|
|
+ *
|
|
* @param f
|
|
* @param f
|
|
* given path
|
|
* given path
|
|
* @return the statuses of the files/directories in the given patch
|
|
* @return the statuses of the files/directories in the given patch
|
|
@@ -802,7 +932,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
throws IOException {
|
|
throws IOException {
|
|
return fs.listLocatedStatus(f, DEFAULT_FILTER);
|
|
return fs.listLocatedStatus(f, DEFAULT_FILTER);
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public boolean mkdirs(Path f) throws IOException {
|
|
public boolean mkdirs(Path f) throws IOException {
|
|
return fs.mkdirs(f);
|
|
return fs.mkdirs(f);
|
|
@@ -856,7 +986,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
} else {
|
|
} else {
|
|
FileStatus[] srcs = listStatus(src);
|
|
FileStatus[] srcs = listStatus(src);
|
|
for (FileStatus srcFile : srcs) {
|
|
for (FileStatus srcFile : srcs) {
|
|
- copyToLocalFile(srcFile.getPath(),
|
|
|
|
|
|
+ copyToLocalFile(srcFile.getPath(),
|
|
new Path(dst, srcFile.getPath().getName()), copyCrc);
|
|
new Path(dst, srcFile.getPath().getName()), copyCrc);
|
|
}
|
|
}
|
|
}
|
|
}
|