HDFS-11160. VolumeScanner reports write-in-progress replicas as corrupt incorrectly. Contributed by Wei-Chiu Chuang and Yongjun Zhang.

This commit is contained in:
Wei-Chiu Chuang 2016-12-15 16:32:50 -08:00
parent 6fce191549
commit aebb9127ba
10 changed files with 203 additions and 11 deletions

View File

@ -64,7 +64,15 @@ public class BlockScanner {
/**
* The scanner configuration.
*/
private final Conf conf;
private Conf conf;
@VisibleForTesting
void setConf(Conf conf) {
this.conf = conf;
for (Entry<String, VolumeScanner> entry : scanners.entrySet()) {
entry.getValue().setConf(conf);
}
}
/**
* The cached scanner configuration.

View File

@ -240,14 +240,23 @@ class BlockSender implements java.io.Closeable {
Preconditions.checkArgument(sendChecksum,
"If verifying checksum, currently must also send it.");
}
// if there is a append write happening right after the BlockSender
// is constructed, the last partial checksum maybe overwritten by the
// append, the BlockSender need to use the partial checksum before
// the append write.
ChunkChecksum chunkChecksum = null;
final long replicaVisibleLength;
try(AutoCloseableLock lock = datanode.data.acquireDatasetLock()) {
replica = getReplica(block, datanode);
replicaVisibleLength = replica.getVisibleLength();
if (replica instanceof FinalizedReplica) {
// Load last checksum in case the replica is being written
// concurrently
final FinalizedReplica frep = (FinalizedReplica) replica;
chunkChecksum = frep.getLastChecksumAndDataLen();
}
}
// if there is a write in progress
ChunkChecksum chunkChecksum = null;
if (replica.getState() == ReplicaState.RBW) {
final ReplicaInPipeline rbw = (ReplicaInPipeline) replica;
waitForMinLength(rbw, startOffset + length);
@ -473,7 +482,7 @@ private static void waitForMinLength(ReplicaInPipeline rbw, long len)
bytesOnDisk));
}
}
/**
* Converts an IOExcpetion (not subclasses) to SocketException.
* This is typically done to indicate to upper layers that the error
@ -547,7 +556,6 @@ private int sendPacket(ByteBuffer pkt, int maxChunks, OutputStream out,
if (lastDataPacket && lastChunkChecksum != null) {
int start = checksumOff + checksumDataLen - checksumSize;
byte[] updatedChecksum = lastChunkChecksum.getChecksum();
if (updatedChecksum != null) {
System.arraycopy(updatedChecksum, 0, buf, start, checksumSize);
}

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.hdfs.server.datanode;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
@ -113,4 +115,31 @@ public ReplicaRecoveryInfo createInfo() {
throw new UnsupportedOperationException("Replica of type " + getState() +
" does not support createInfo");
}
/**
* gets the last chunk checksum and the length of the block corresponding
* to that checksum.
* Note, need to be called with the FsDataset lock acquired. May improve to
* lock only the FsVolume in the future.
* @throws IOException
*/
public ChunkChecksum getLastChecksumAndDataLen() throws IOException {
ChunkChecksum chunkChecksum = null;
try {
byte[] lastChecksum = getVolume().loadLastPartialChunkChecksum(
getBlockFile(), getMetaFile());
if (lastChecksum != null) {
chunkChecksum =
new ChunkChecksum(getVisibleLength(), lastChecksum);
}
} catch (FileNotFoundException e) {
// meta file is lost. Try to continue anyway.
DataNode.LOG.warn("meta file " + getMetaFile() +
" is missing!");
} catch (IOException ioe) {
DataNode.LOG.warn("Unable to read checksum from meta file " +
getMetaFile(), ioe);
}
return chunkChecksum;
}
}

View File

@ -69,7 +69,12 @@ public class VolumeScanner extends Thread {
/**
* The configuration.
*/
private final Conf conf;
private Conf conf;
@VisibleForTesting
void setConf(Conf conf) {
this.conf = conf;
}
/**
* The DataNode this VolumEscanner is associated with.
@ -429,6 +434,7 @@ private long scanBlock(ExtendedBlock cblock, long bytesPerSec) {
if (block == null) {
return -1; // block not found.
}
LOG.debug("start scanning block {}", block);
BlockSender blockSender = null;
try {
blockSender = new BlockSender(block, 0, -1,
@ -610,6 +616,7 @@ public void run() {
break;
}
if (timeout > 0) {
LOG.debug("{}: wait for {} milliseconds", this, timeout);
wait(timeout);
if (stopping) {
break;

View File

@ -401,6 +401,17 @@ public long getGenStamp() {
}
}
/**
* Load last partial chunk checksum from checksum file.
* Need to be called with FsDataset lock acquired.
* @param blockFile
* @param metaFile
* @return the last partial checksum
* @throws IOException
*/
byte[] loadLastPartialChunkChecksum(File blockFile, File metaFile)
throws IOException;
/**
* Compile a list of {@link ScanInfo} for the blocks in
* the block pool with id {@code bpid}.

View File

@ -1119,7 +1119,8 @@ DatanodeStorage toDatanodeStorage() {
}
private byte[] loadLastPartialChunkChecksum(
@Override
public byte[] loadLastPartialChunkChecksum(
File blockFile, File metaFile) throws IOException {
// readHeader closes the temporary FileInputStream.
DataChecksum dcs = BlockMetadataHeader
@ -1135,13 +1136,22 @@ private byte[] loadLastPartialChunkChecksum(
return null;
}
int offsetInChecksum = BlockMetadataHeader.getHeaderSize() +
(int)(onDiskLen / bytesPerChecksum * checksumSize);
long offsetInChecksum = BlockMetadataHeader.getHeaderSize() +
(onDiskLen / bytesPerChecksum) * checksumSize;
byte[] lastChecksum = new byte[checksumSize];
try (RandomAccessFile raf = fileIoProvider.getRandomAccessFile(
this, metaFile, "r")) {
raf.seek(offsetInChecksum);
raf.read(lastChecksum, 0, checksumSize);
int readBytes = raf.read(lastChecksum, 0, checksumSize);
if (readBytes == -1) {
throw new IOException("Expected to read " + checksumSize +
" bytes from offset " + offsetInChecksum +
" but reached end of file.");
} else if (readBytes != checksumSize) {
throw new IOException("Expected to read " + checksumSize +
" bytes from offset " + offsetInChecksum + " but read " +
readBytes + " bytes.");
}
}
return lastChecksum;
}

View File

@ -556,6 +556,12 @@ public DF getUsageStats(Configuration conf) {
return null;
}
@Override
public byte[] loadLastPartialChunkChecksum(
File blockFile, File metaFile) throws IOException {
return null;
}
@Override
public LinkedList<ScanInfo> compileReport(String bpid,
LinkedList<ScanInfo> report, ReportCompiler reportCompiler)

View File

@ -36,8 +36,12 @@
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeoutException;
import com.google.common.base.Supplier;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hdfs.AppendTestUtil;
import org.apache.hadoop.hdfs.MiniDFSNNTopology;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils.MaterializedReplica;
@ -870,4 +874,100 @@ public Boolean get() {
}
info.sem.release(1);
}
/**
* Test concurrent append and scan.
* @throws Exception
*/
@Test(timeout=120000)
public void testAppendWhileScanning() throws Exception {
GenericTestUtils.setLogLevel(DataNode.LOG, Level.ALL);
Configuration conf = new Configuration();
// throttle the block scanner: 1MB per second
conf.setLong(DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND, 1048576);
// Set a really long scan period.
conf.setLong(DFS_DATANODE_SCAN_PERIOD_HOURS_KEY, 100L);
conf.set(INTERNAL_VOLUME_SCANNER_SCAN_RESULT_HANDLER,
TestScanResultHandler.class.getName());
conf.setLong(INTERNAL_DFS_BLOCK_SCANNER_CURSOR_SAVE_INTERVAL_MS, 0L);
final int numExpectedFiles = 1;
final int numExpectedBlocks = 1;
final int numNameServices = 1;
// the initial file length can not be too small.
// Otherwise checksum file stream buffer will be pre-filled and
// BlockSender will not see the updated checksum.
final int initialFileLength = 2*1024*1024+100;
final TestContext ctx = new TestContext(conf, numNameServices);
// create one file, with one block.
ctx.createFiles(0, numExpectedFiles, initialFileLength);
final TestScanResultHandler.Info info =
TestScanResultHandler.getInfo(ctx.volumes.get(0));
String storageID = ctx.volumes.get(0).getStorageID();
synchronized (info) {
info.sem = new Semaphore(numExpectedBlocks*2);
info.shouldRun = true;
info.notify();
}
// VolumeScanner scans the first block when DN starts.
// Due to throttler, this should take approximately 2 seconds.
waitForRescan(info, numExpectedBlocks);
// update throttler to schedule rescan immediately.
// this number must be larger than initial file length, otherwise
// throttler prevents immediate rescan.
conf.setLong(DFS_BLOCK_SCANNER_VOLUME_BYTES_PER_SECOND,
initialFileLength+32*1024);
BlockScanner.Conf newConf = new BlockScanner.Conf(conf);
ctx.datanode.getBlockScanner().setConf(newConf);
// schedule the first block for scanning
ExtendedBlock first = ctx.getFileBlock(0, 0);
ctx.datanode.getBlockScanner().markSuspectBlock(storageID, first);
// append the file before VolumeScanner completes scanning the block,
// which takes approximately 2 seconds to complete.
FileSystem fs = ctx.cluster.getFileSystem();
FSDataOutputStream os = fs.append(ctx.getPath(0));
long seed = -1;
int size = 200;
final byte[] bytes = AppendTestUtil.randomBytes(seed, size);
os.write(bytes);
os.hflush();
os.close();
fs.close();
// verify that volume scanner does not find bad blocks after append.
waitForRescan(info, numExpectedBlocks);
GenericTestUtils.setLogLevel(DataNode.LOG, Level.INFO);
}
private void waitForRescan(final TestScanResultHandler.Info info,
final int numExpectedBlocks)
throws TimeoutException, InterruptedException {
LOG.info("Waiting for the first 1 blocks to be scanned.");
GenericTestUtils.waitFor(new Supplier<Boolean>() {
@Override
public Boolean get() {
synchronized (info) {
if (info.blocksScanned >= numExpectedBlocks) {
LOG.info("info = {}. blockScanned has now reached 1.", info);
return true;
} else {
LOG.info("info = {}. Waiting for blockScanned to reach 1.", info);
return false;
}
}
}
}, 1000, 30000);
synchronized (info) {
assertEquals("Expected 1 good block.",
numExpectedBlocks, info.goodBlocks.size());
info.goodBlocks.clear();
assertEquals("Expected 1 blocksScanned",
numExpectedBlocks, info.blocksScanned);
assertEquals("Did not expect bad blocks.", 0, info.badBlocks.size());
info.blocksScanned = 0;
}
}
}

View File

@ -898,6 +898,12 @@ public DF getUsageStats(Configuration conf) {
return null;
}
@Override
public byte[] loadLastPartialChunkChecksum(
File blockFile, File metaFile) throws IOException {
return null;
}
@Override
public LinkedList<ScanInfo> compileReport(String bpid,
LinkedList<ScanInfo> report, ReportCompiler reportCompiler)

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hdfs.server.datanode.extdataset;
import java.io.File;
import java.io.IOException;
import java.net.URI;
import java.nio.channels.ClosedChannelException;
@ -108,6 +109,12 @@ public DF getUsageStats(Configuration conf) {
return null;
}
@Override
public byte[] loadLastPartialChunkChecksum(
File blockFile, File metaFile) throws IOException {
return null;
}
@Override
public LinkedList<ScanInfo> compileReport(String bpid,
LinkedList<ScanInfo> report, ReportCompiler reportCompiler)