HDFS-8323. Bump GenerationStamp for write faliure in DFSStripedOutputStream. Contributed by Tsz Wo Nicholas Sze.

This commit is contained in:
Jing Zhao 2015-05-19 21:19:51 -07:00 committed by Zhe Zhang
parent 47ef869fa7
commit 45db1a0b72
12 changed files with 257 additions and 128 deletions

View File

@ -31,16 +31,22 @@ import java.util.Arrays;
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class LocatedStripedBlock extends LocatedBlock { public class LocatedStripedBlock extends LocatedBlock {
private static final int[] EMPTY_INDICES = {};
private int[] blockIndices; private int[] blockIndices;
public LocatedStripedBlock(ExtendedBlock b, DatanodeInfo[] locs, public LocatedStripedBlock(ExtendedBlock b, DatanodeInfo[] locs,
String[] storageIDs, StorageType[] storageTypes, int[] indices, String[] storageIDs, StorageType[] storageTypes, int[] indices,
long startOffset, boolean corrupt, DatanodeInfo[] cachedLocs) { long startOffset, boolean corrupt, DatanodeInfo[] cachedLocs) {
super(b, locs, storageIDs, storageTypes, startOffset, corrupt, cachedLocs); super(b, locs, storageIDs, storageTypes, startOffset, corrupt, cachedLocs);
assert indices != null && indices.length == locs.length;
if (indices == null) {
this.blockIndices = EMPTY_INDICES;
} else {
this.blockIndices = new int[indices.length]; this.blockIndices = new int[indices.length];
System.arraycopy(indices, 0, blockIndices, 0, indices.length); System.arraycopy(indices, 0, blockIndices, 0, indices.length);
} }
}
@Override @Override
public String toString() { public String toString() {

View File

@ -238,3 +238,6 @@
HDFS-8428. Erasure Coding: Fix the NullPointerException when deleting file. HDFS-8428. Erasure Coding: Fix the NullPointerException when deleting file.
(Yi Liu via zhz). (Yi Liu via zhz).
HDFS-8323. Bump GenerationStamp for write faliure in DFSStripedOutputStream.
(Tsz Wo Nicholas Sze via jing9)

View File

@ -33,7 +33,6 @@ import java.util.concurrent.TimeUnit;
import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf; import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@ -61,52 +60,72 @@ import com.google.common.base.Preconditions;
@InterfaceAudience.Private @InterfaceAudience.Private
public class DFSStripedOutputStream extends DFSOutputStream { public class DFSStripedOutputStream extends DFSOutputStream {
static class MultipleBlockingQueue<T> {
private final int pullTimeout;
private final List<BlockingQueue<T>> queues;
MultipleBlockingQueue(int numQueue, int queueSize, int pullTimeout) {
queues = new ArrayList<>(numQueue);
for (int i = 0; i < numQueue; i++) {
queues.add(new LinkedBlockingQueue<T>(queueSize));
}
this.pullTimeout = pullTimeout;
}
void offer(int i, T object) {
final boolean b = queues.get(i).offer(object);
Preconditions.checkState(b, "Failed to offer " + object
+ " to queue, i=" + i);
}
T poll(int i) throws InterruptedIOException {
try {
return queues.get(i).poll(pullTimeout, TimeUnit.SECONDS);
} catch (InterruptedException e) {
throw DFSUtil.toInterruptedIOException("poll interrupted, i=" + i, e);
}
}
T peek(int i) {
return queues.get(i).peek();
}
}
/** Coordinate the communication between the streamers. */ /** Coordinate the communication between the streamers. */
static class Coordinator { static class Coordinator {
private final DfsClientConf conf; private final MultipleBlockingQueue<LocatedBlock> stripedBlocks;
private final List<BlockingQueue<ExtendedBlock>> endBlocks; private final MultipleBlockingQueue<ExtendedBlock> endBlocks;
private final List<BlockingQueue<LocatedBlock>> stripedBlocks; private final MultipleBlockingQueue<ExtendedBlock> updateBlocks;
private volatile boolean shouldLocateFollowingBlock = false;
Coordinator(final DfsClientConf conf, final int numDataBlocks, Coordinator(final DfsClientConf conf, final int numDataBlocks,
final int numAllBlocks) { final int numAllBlocks) {
this.conf = conf; stripedBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1,
endBlocks = new ArrayList<>(numDataBlocks); conf.getStripedWriteMaxSecondsGetStripedBlock());
for (int i = 0; i < numDataBlocks; i++) { endBlocks = new MultipleBlockingQueue<>(numDataBlocks, 1,
endBlocks.add(new LinkedBlockingQueue<ExtendedBlock>(1)); conf.getStripedWriteMaxSecondsGetEndedBlock());
} updateBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1,
conf.getStripedWriteMaxSecondsGetStripedBlock());
stripedBlocks = new ArrayList<>(numAllBlocks);
for (int i = 0; i < numAllBlocks; i++) {
stripedBlocks.add(new LinkedBlockingQueue<LocatedBlock>(1));
}
}
boolean shouldLocateFollowingBlock() {
return shouldLocateFollowingBlock;
} }
void putEndBlock(int i, ExtendedBlock block) { void putEndBlock(int i, ExtendedBlock block) {
shouldLocateFollowingBlock = true; endBlocks.offer(i, block);
final boolean b = endBlocks.get(i).offer(block);
Preconditions.checkState(b, "Failed to add " + block
+ " to endBlocks queue, i=" + i);
} }
ExtendedBlock getEndBlock(int i) throws InterruptedIOException { ExtendedBlock getEndBlock(int i) throws InterruptedIOException {
try { return endBlocks.poll(i);
return endBlocks.get(i).poll(
conf.getStripedWriteMaxSecondsGetEndedBlock(),
TimeUnit.SECONDS);
} catch (InterruptedException e) {
throw DFSUtil.toInterruptedIOException(
"getEndBlock interrupted, i=" + i, e);
} }
void putUpdateBlock(int i, ExtendedBlock block) {
updateBlocks.offer(i, block);
}
ExtendedBlock getUpdateBlock(int i) throws InterruptedIOException {
return updateBlocks.poll(i);
} }
void setBytesEndBlock(int i, long newBytes, ExtendedBlock block) { void setBytesEndBlock(int i, long newBytes, ExtendedBlock block) {
ExtendedBlock b = endBlocks.get(i).peek(); ExtendedBlock b = endBlocks.peek(i);
if (b == null) { if (b == null) {
// streamer just has failed, put end block and continue // streamer just has failed, put end block and continue
b = block; b = block;
@ -119,22 +138,11 @@ public class DFSStripedOutputStream extends DFSOutputStream {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("putStripedBlock " + block + ", i=" + i); LOG.debug("putStripedBlock " + block + ", i=" + i);
} }
final boolean b = stripedBlocks.get(i).offer(block); stripedBlocks.offer(i, block);
if (!b) {
throw new IOException("Failed: " + block + ", i=" + i);
}
} }
LocatedBlock getStripedBlock(int i) throws IOException { LocatedBlock getStripedBlock(int i) throws IOException {
final LocatedBlock lb; final LocatedBlock lb = stripedBlocks.poll(i);
try {
lb = stripedBlocks.get(i).poll(
conf.getStripedWriteMaxSecondsGetStripedBlock(),
TimeUnit.SECONDS);
} catch (InterruptedException e) {
throw DFSUtil.toInterruptedIOException("getStripedBlock interrupted", e);
}
if (lb == null) { if (lb == null) {
throw new IOException("Failed: i=" + i); throw new IOException("Failed: i=" + i);
} }
@ -218,6 +226,11 @@ public class DFSStripedOutputStream extends DFSOutputStream {
return streamers.get(0); return streamers.get(0);
} }
@Override
ExtendedBlock getBlock() {
return getLeadingStreamer().getBlock();
}
/** Construct a new output stream for creating a file. */ /** Construct a new output stream for creating a file. */
DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat, DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
EnumSet<CreateFlag> flag, Progressable progress, EnumSet<CreateFlag> flag, Progressable progress,
@ -292,6 +305,7 @@ public class DFSStripedOutputStream extends DFSOutputStream {
int count = 0; int count = 0;
for(StripedDataStreamer s : streamers) { for(StripedDataStreamer s : streamers) {
if (!s.isFailed()) { if (!s.isFailed()) {
s.getErrorState().initExtenalError();
count++; count++;
} }
} }

View File

@ -44,7 +44,6 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf; import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
@ -210,6 +209,7 @@ class DataStreamer extends Daemon {
static class ErrorState { static class ErrorState {
private boolean error = false; private boolean error = false;
private boolean extenalError = false;
private int badNodeIndex = -1; private int badNodeIndex = -1;
private int restartingNodeIndex = -1; private int restartingNodeIndex = -1;
private long restartingNodeDeadline = 0; private long restartingNodeDeadline = 0;
@ -221,6 +221,7 @@ class DataStreamer extends Daemon {
synchronized void reset() { synchronized void reset() {
error = false; error = false;
extenalError = false;
badNodeIndex = -1; badNodeIndex = -1;
restartingNodeIndex = -1; restartingNodeIndex = -1;
restartingNodeDeadline = 0; restartingNodeDeadline = 0;
@ -231,13 +232,19 @@ class DataStreamer extends Daemon {
} }
synchronized boolean hasDatanodeError() { synchronized boolean hasDatanodeError() {
return error && isNodeMarked(); return error && (isNodeMarked() || extenalError);
} }
synchronized void setError(boolean err) { synchronized void setError(boolean err) {
this.error = err; this.error = err;
} }
synchronized void initExtenalError() {
setError(true);
this.extenalError = true;
}
synchronized void setBadNodeIndex(int index) { synchronized void setBadNodeIndex(int index) {
this.badNodeIndex = index; this.badNodeIndex = index;
} }
@ -1736,6 +1743,10 @@ class DataStreamer extends Daemon {
return accessToken; return accessToken;
} }
ErrorState getErrorState() {
return errorState;
}
/** /**
* Put a packet to the data queue * Put a packet to the data queue
* *

View File

@ -94,24 +94,34 @@ public class StripedDataStreamer extends DataStreamer {
protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
throws IOException { throws IOException {
if (isLeadingStreamer()) { if (isLeadingStreamer()) {
if (coordinator.shouldLocateFollowingBlock()) { if (block != null) {
// set numByte for the previous block group // set numByte for the previous block group
long bytes = 0; long bytes = 0;
for (int i = 0; i < NUM_DATA_BLOCKS; i++) { for (int i = 0; i < NUM_DATA_BLOCKS; i++) {
final ExtendedBlock b = coordinator.getEndBlock(i); final ExtendedBlock b = coordinator.getEndBlock(i);
bytes += b == null ? 0 : b.getNumBytes(); if (b != null) {
StripedBlockUtil.checkBlocks(block, i, b);
bytes += b.getNumBytes();
}
} }
block.setNumBytes(bytes); block.setNumBytes(bytes);
} }
final LocatedStripedBlock lsb putLoactedBlocks(super.locateFollowingBlock(excludedNodes));
= (LocatedStripedBlock)super.locateFollowingBlock(excludedNodes);
if (LOG.isDebugEnabled()) {
LOG.debug("Obtained block group " + lsb);
} }
LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(lsb,
return coordinator.getStripedBlock(index);
}
void putLoactedBlocks(LocatedBlock lb) throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Obtained block group " + lb);
}
LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
(LocatedStripedBlock)lb,
BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
// TODO allow write to continue if blocks.length >= NUM_DATA_BLOCKS
assert blocks.length == (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) : assert blocks.length == (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) :
"Fail to get block group from namenode: blockGroupSize: " + "Fail to get block group from namenode: blockGroupSize: " +
(NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) + ", blocks.length: " + (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS) + ", blocks.length: " +
@ -121,9 +131,32 @@ public class StripedDataStreamer extends DataStreamer {
} }
} }
@Override
LocatedBlock updateBlockForPipeline() throws IOException {
if (isLeadingStreamer()) {
final LocatedBlock updated = super.updateBlockForPipeline();
final ExtendedBlock block = updated.getBlock();
for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) {
final LocatedBlock lb = new LocatedBlock(block, null, null, null,
-1, updated.isCorrupt(), null);
lb.setBlockToken(updated.getBlockToken());
coordinator.putStripedBlock(i, lb);
}
}
return coordinator.getStripedBlock(index); return coordinator.getStripedBlock(index);
} }
@Override
ExtendedBlock updatePipeline(long newGS) throws IOException {
if (isLeadingStreamer()) {
final ExtendedBlock newBlock = super.updatePipeline(newGS);
for (int i = 0; i < NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; i++) {
coordinator.putUpdateBlock(i, new ExtendedBlock(newBlock));
}
}
return coordinator.getUpdateBlock(index);
}
@Override @Override
public String toString() { public String toString() {
return "#" + index + ": isFailed? " + Boolean.toString(isFailed).charAt(0) return "#" + index + ": isFailed? " + Boolean.toString(isFailed).charAt(0)

View File

@ -21,18 +21,15 @@ import java.util.Arrays;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import com.google.common.annotations.VisibleForTesting;
/** /**
* A Datanode has one or more storages. A storage in the Datanode is represented * A Datanode has one or more storages. A storage in the Datanode is represented
* by this class. * by this class.
@ -41,7 +38,7 @@ public class DatanodeStorageInfo {
public static final DatanodeStorageInfo[] EMPTY_ARRAY = {}; public static final DatanodeStorageInfo[] EMPTY_ARRAY = {};
public static DatanodeInfo[] toDatanodeInfos(DatanodeStorageInfo[] storages) { public static DatanodeInfo[] toDatanodeInfos(DatanodeStorageInfo[] storages) {
return toDatanodeInfos(Arrays.asList(storages)); return storages == null? null: toDatanodeInfos(Arrays.asList(storages));
} }
static DatanodeInfo[] toDatanodeInfos(List<DatanodeStorageInfo> storages) { static DatanodeInfo[] toDatanodeInfos(List<DatanodeStorageInfo> storages) {
final DatanodeInfo[] datanodes = new DatanodeInfo[storages.size()]; final DatanodeInfo[] datanodes = new DatanodeInfo[storages.size()];
@ -61,6 +58,9 @@ public class DatanodeStorageInfo {
} }
public static String[] toStorageIDs(DatanodeStorageInfo[] storages) { public static String[] toStorageIDs(DatanodeStorageInfo[] storages) {
if (storages == null) {
return null;
}
String[] storageIDs = new String[storages.length]; String[] storageIDs = new String[storages.length];
for(int i = 0; i < storageIDs.length; i++) { for(int i = 0; i < storageIDs.length; i++) {
storageIDs[i] = storages[i].getStorageID(); storageIDs[i] = storages[i].getStorageID();
@ -69,6 +69,9 @@ public class DatanodeStorageInfo {
} }
public static StorageType[] toStorageTypes(DatanodeStorageInfo[] storages) { public static StorageType[] toStorageTypes(DatanodeStorageInfo[] storages) {
if (storages == null) {
return null;
}
StorageType[] storageTypes = new StorageType[storages.length]; StorageType[] storageTypes = new StorageType[storages.length];
for(int i = 0; i < storageTypes.length; i++) { for(int i = 0; i < storageTypes.length; i++) {
storageTypes[i] = storages[i].getStorageType(); storageTypes[i] = storages[i].getStorageType();

View File

@ -5787,29 +5787,30 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
* Get a new generation stamp together with an access token for * Get a new generation stamp together with an access token for
* a block under construction * a block under construction
* *
* This method is called for recovering a failed pipeline or setting up * This method is called for recovering a failed write or setting up
* a pipeline to append to a block. * a block for appended.
* *
* @param block a block * @param block a block
* @param clientName the name of a client * @param clientName the name of a client
* @return a located block with a new generation stamp and an access token * @return a located block with a new generation stamp and an access token
* @throws IOException if any error occurs * @throws IOException if any error occurs
*/ */
LocatedBlock updateBlockForPipeline(ExtendedBlock block, LocatedBlock bumpBlockGenerationStamp(ExtendedBlock block,
String clientName) throws IOException { String clientName) throws IOException {
LocatedBlock locatedBlock; final LocatedBlock locatedBlock;
checkOperation(OperationCategory.WRITE); checkOperation(OperationCategory.WRITE);
writeLock(); writeLock();
try { try {
checkOperation(OperationCategory.WRITE); checkOperation(OperationCategory.WRITE);
// check vadility of parameters // check vadility of parameters
checkUCBlock(block, clientName); final INodeFile file = checkUCBlock(block, clientName);
// get a new generation stamp and an access token // get a new generation stamp and an access token
block.setGenerationStamp(nextGenerationStamp(blockIdManager.isLegacyBlock(block.getLocalBlock()))); block.setGenerationStamp(nextGenerationStamp(blockIdManager.isLegacyBlock(block.getLocalBlock())));
locatedBlock = new LocatedBlock(block, new DatanodeInfo[0]);
blockManager.setBlockToken(locatedBlock, BlockTokenIdentifier.AccessMode.WRITE); locatedBlock = BlockManager.newLocatedBlock(
block, file.getLastBlock(), null, -1);
} finally { } finally {
writeUnlock(); writeUnlock();
} }
@ -5864,23 +5865,26 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
// check the vadility of the block and lease holder name // check the vadility of the block and lease holder name
final INodeFile pendingFile = checkUCBlock(oldBlock, clientName); final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
final BlockInfo lastBlock = pendingFile.getLastBlock(); final BlockInfo lastBlock = pendingFile.getLastBlock();
// when updating pipeline, the last block must be contiguous block final BlockInfoUnderConstruction blockinfo = (BlockInfoUnderConstruction)lastBlock;
assert lastBlock instanceof BlockInfoContiguousUnderConstruction;
BlockInfoContiguousUnderConstruction blockinfo =
(BlockInfoContiguousUnderConstruction) lastBlock;
// check new GS & length: this is not expected // check new GS & length: this is not expected
if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() || if (newBlock.getGenerationStamp() <= lastBlock.getGenerationStamp()) {
newBlock.getNumBytes() < blockinfo.getNumBytes()) { final String msg = "Update " + oldBlock + " but the new block " + newBlock
String msg = "Update " + oldBlock + " (len = " + + " does not have a larger generation stamp than the last block "
blockinfo.getNumBytes() + ") to an older state: " + newBlock + + lastBlock;
" (len = " + newBlock.getNumBytes() +")"; LOG.warn(msg);
throw new IOException(msg);
}
if (newBlock.getNumBytes() < lastBlock.getNumBytes()) {
final String msg = "Update " + oldBlock + " (size="
+ oldBlock.getNumBytes() + ") to a smaller size block " + newBlock
+ " (size=" + newBlock.getNumBytes() + ")";
LOG.warn(msg); LOG.warn(msg);
throw new IOException(msg); throw new IOException(msg);
} }
// Update old block with the new generation stamp and new length // Update old block with the new generation stamp and new length
blockinfo.setNumBytes(newBlock.getNumBytes()); lastBlock.setNumBytes(newBlock.getNumBytes());
blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp()); blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
// find the DatanodeDescriptor objects // find the DatanodeDescriptor objects

View File

@ -788,7 +788,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
public LocatedBlock updateBlockForPipeline(ExtendedBlock block, String clientName) public LocatedBlock updateBlockForPipeline(ExtendedBlock block, String clientName)
throws IOException { throws IOException {
checkNNStartup(); checkNNStartup();
return namesystem.updateBlockForPipeline(block, clientName); return namesystem.bumpBlockGenerationStamp(block, clientName);
} }

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.io.erasurecode.ECSchema;
import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder; import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
import java.util.*; import java.util.*;
import java.io.IOException;
import java.util.concurrent.CancellationException; import java.util.concurrent.CancellationException;
import java.util.concurrent.CompletionService; import java.util.concurrent.CompletionService;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
@ -104,12 +105,17 @@ public class StripedBlockUtil {
final ExtendedBlock blk = constructInternalBlock( final ExtendedBlock blk = constructInternalBlock(
bg.getBlock(), cellSize, dataBlkNum, idxInBlockGroup); bg.getBlock(), cellSize, dataBlkNum, idxInBlockGroup);
final long offset = bg.getStartOffset() + idxInBlockGroup * cellSize;
if (idxInReturnedLocs < bg.getLocations().length) {
return new LocatedBlock(blk, return new LocatedBlock(blk,
new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]}, new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
new String[]{bg.getStorageIDs()[idxInReturnedLocs]}, new String[]{bg.getStorageIDs()[idxInReturnedLocs]},
new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]}, new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]},
bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(), offset, bg.isCorrupt(), null);
null); } else {
return new LocatedBlock(blk, null, null, null,
offset, bg.isCorrupt(), null);
}
} }
/** /**
@ -823,4 +829,26 @@ public class StripedBlockUtil {
return "(index=" + index + ", state =" + state + ")"; return "(index=" + index + ", state =" + state + ")";
} }
} }
/**
* Check if the information such as IDs and generation stamps in block-i
* match block-0.
*/
public static void checkBlocks(ExtendedBlock block0, int i,
ExtendedBlock blocki) throws IOException {
if (!blocki.getBlockPoolId().equals(block0.getBlockPoolId())) {
throw new IOException("Block pool IDs mismatched: block0="
+ block0 + ", block" + i + "=" + blocki);
}
if (blocki.getBlockId() - i != block0.getBlockId()) {
throw new IOException("Block IDs mismatched: block0="
+ block0 + ", block" + i + "=" + blocki);
}
if (blocki.getGenerationStamp() != block0.getGenerationStamp()) {
throw new IOException("Generation stamps mismatched: block0="
+ block0 + ", block" + i + "=" + blocki);
}
}
} }

View File

@ -1953,11 +1953,9 @@ public class DFSTestUtil {
* Because currently DFSStripedOutputStream does not support hflush/hsync, * Because currently DFSStripedOutputStream does not support hflush/hsync,
* tests can use this method to flush all the buffered data to DataNodes. * tests can use this method to flush all the buffered data to DataNodes.
*/ */
public static void writeAndFlushStripedOutputStream( public static ExtendedBlock flushInternal(DFSStripedOutputStream out)
DFSStripedOutputStream out, int chunkSize) throws IOException { throws IOException {
// FSOutputSummer.BUFFER_NUM_CHUNKS == 9
byte[] toWrite = new byte[chunkSize * 9 + 1];
out.write(toWrite);
out.flushInternal(); out.flushInternal();
return out.getBlock();
} }
} }

View File

@ -21,7 +21,6 @@ import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
@ -44,6 +43,8 @@ import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import com.google.common.base.Preconditions;
public class TestDFSStripedOutputStreamWithFailure { public class TestDFSStripedOutputStreamWithFailure {
public static final Log LOG = LogFactory.getLog( public static final Log LOG = LogFactory.getLog(
TestDFSStripedOutputStreamWithFailure.class); TestDFSStripedOutputStreamWithFailure.class);
@ -59,6 +60,9 @@ public class TestDFSStripedOutputStreamWithFailure {
private static final int BLOCK_SIZE = CELL_SIZE * STRIPES_PER_BLOCK; private static final int BLOCK_SIZE = CELL_SIZE * STRIPES_PER_BLOCK;
private static final int BLOCK_GROUP_SIZE = BLOCK_SIZE * NUM_DATA_BLOCKS; private static final int BLOCK_GROUP_SIZE = BLOCK_SIZE * NUM_DATA_BLOCKS;
private static final int FLUSH_POS
= 9*DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1;
private final HdfsConfiguration conf = new HdfsConfiguration(); private final HdfsConfiguration conf = new HdfsConfiguration();
private MiniDFSCluster cluster; private MiniDFSCluster cluster;
private DistributedFileSystem dfs; private DistributedFileSystem dfs;
@ -149,50 +153,53 @@ public class TestDFSStripedOutputStreamWithFailure {
cluster.startDataNodes(conf, 1, true, null, null); cluster.startDataNodes(conf, 1, true, null, null);
cluster.waitActive(); cluster.waitActive();
runTest(new Path(dir, src), length, dnIndex); runTest(new Path(dir, src), length, length/2, dnIndex);
} catch(Exception e) { } catch(Exception e) {
LOG.info("FAILED", e); LOG.info("FAILED", e);
Assert.fail(StringUtils.stringifyException(e)); Assert.fail(StringUtils.stringifyException(e));
} }
} }
private void runTest(final Path p, final int length, private void runTest(final Path p, final int length, final int killPos,
final int dnIndex) throws Exception { final int dnIndex) throws Exception {
LOG.info("p=" + p + ", length=" + length + ", dnIndex=" + dnIndex); LOG.info("p=" + p + ", length=" + length + ", killPos=" + killPos
+ ", dnIndex=" + dnIndex);
Preconditions.checkArgument(killPos < length);
Preconditions.checkArgument(killPos > FLUSH_POS);
final String fullPath = p.toString(); final String fullPath = p.toString();
final AtomicInteger pos = new AtomicInteger(); final AtomicInteger pos = new AtomicInteger();
final FSDataOutputStream out = dfs.create(p); final FSDataOutputStream out = dfs.create(p);
final AtomicBoolean killed = new AtomicBoolean(); final DFSStripedOutputStream stripedOut
final Thread killer = new Thread(new Runnable() { = (DFSStripedOutputStream)out.getWrappedStream();
@Override
public void run() {
killDatanode(cluster, (DFSStripedOutputStream)out.getWrappedStream(),
dnIndex, pos);
killed.set(true);
}
});
killer.start();
final int mask = (1 << 16) - 1; long oldGS = -1;
boolean killed = false;
for(; pos.get() < length; ) { for(; pos.get() < length; ) {
final int i = pos.getAndIncrement(); final int i = pos.getAndIncrement();
if (i == killPos) {
final long gs = getGenerationStamp(stripedOut);
Assert.assertTrue(oldGS != -1);
Assert.assertEquals(oldGS, gs);
killDatanode(cluster, stripedOut, dnIndex, pos);
killed = true;
}
write(out, i); write(out, i);
if ((i & mask) == 0) {
final long ms = 100; if (i == FLUSH_POS) {
LOG.info("i=" + i + " sleep " + ms); oldGS = getGenerationStamp(stripedOut);
Thread.sleep(ms);
} }
} }
killer.join(10000);
Assert.assertTrue(killed.get());
out.close(); out.close();
Assert.assertTrue(killed);
// check file length // check file length
final FileStatus status = dfs.getFileStatus(p); final FileStatus status = dfs.getFileStatus(p);
Assert.assertEquals(length, status.getLen()); Assert.assertEquals(length, status.getLen());
checkData(dfs, fullPath, length, dnIndex); checkData(dfs, fullPath, length, dnIndex, oldGS);
} }
static void write(FSDataOutputStream out, int i) throws IOException { static void write(FSDataOutputStream out, int i) throws IOException {
@ -203,6 +210,14 @@ public class TestDFSStripedOutputStreamWithFailure {
} }
} }
static long getGenerationStamp(DFSStripedOutputStream out)
throws IOException {
final long gs = DFSTestUtil.flushInternal(out).getGenerationStamp();
LOG.info("getGenerationStamp returns " + gs);
return gs;
}
static DatanodeInfo getDatanodes(StripedDataStreamer streamer) { static DatanodeInfo getDatanodes(StripedDataStreamer streamer) {
for(;;) { for(;;) {
final DatanodeInfo[] datanodes = streamer.getNodes(); final DatanodeInfo[] datanodes = streamer.getNodes();
@ -228,7 +243,7 @@ public class TestDFSStripedOutputStreamWithFailure {
} }
static void checkData(DistributedFileSystem dfs, String src, int length, static void checkData(DistributedFileSystem dfs, String src, int length,
int killedDnIndex) throws IOException { int killedDnIndex, long oldGS) throws IOException {
List<List<LocatedBlock>> blockGroupList = new ArrayList<>(); List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(src, 0L); LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(src, 0L);
final int expectedNumGroup = (length - 1)/BLOCK_GROUP_SIZE + 1; final int expectedNumGroup = (length - 1)/BLOCK_GROUP_SIZE + 1;
@ -236,6 +251,12 @@ public class TestDFSStripedOutputStreamWithFailure {
for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) { for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) {
Assert.assertTrue(firstBlock instanceof LocatedStripedBlock); Assert.assertTrue(firstBlock instanceof LocatedStripedBlock);
final long gs = firstBlock.getBlock().getGenerationStamp();
final String s = "gs=" + gs + ", oldGS=" + oldGS;
LOG.info(s);
Assert.assertTrue(s, gs > oldGS);
LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
(LocatedStripedBlock) firstBlock, (LocatedStripedBlock) firstBlock,
CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
@ -247,7 +268,7 @@ public class TestDFSStripedOutputStreamWithFailure {
final boolean isLastGroup = group == blockGroupList.size() - 1; final boolean isLastGroup = group == blockGroupList.size() - 1;
final int groupSize = !isLastGroup? BLOCK_GROUP_SIZE final int groupSize = !isLastGroup? BLOCK_GROUP_SIZE
: length - (blockGroupList.size() - 1)*BLOCK_GROUP_SIZE; : length - (blockGroupList.size() - 1)*BLOCK_GROUP_SIZE;
final int numCellInGroup = (int)((groupSize - 1)/CELL_SIZE + 1); final int numCellInGroup = (groupSize - 1)/CELL_SIZE + 1;
final int lastCellIndex = (numCellInGroup - 1) % NUM_DATA_BLOCKS; final int lastCellIndex = (numCellInGroup - 1) % NUM_DATA_BLOCKS;
final int lastCellSize = groupSize - (numCellInGroup - 1)*CELL_SIZE; final int lastCellSize = groupSize - (numCellInGroup - 1)*CELL_SIZE;

View File

@ -105,6 +105,14 @@ public class TestAddStripedBlocks {
Assert.assertEquals(firstId + HdfsServerConstants.MAX_BLOCKS_IN_GROUP, secondId); Assert.assertEquals(firstId + HdfsServerConstants.MAX_BLOCKS_IN_GROUP, secondId);
} }
private static void writeAndFlushStripedOutputStream(
DFSStripedOutputStream out, int chunkSize) throws IOException {
// FSOutputSummer.BUFFER_NUM_CHUNKS == 9
byte[] toWrite = new byte[chunkSize * 9 + 1];
out.write(toWrite);
DFSTestUtil.flushInternal(out);
}
@Test (timeout=60000) @Test (timeout=60000)
public void testAddStripedBlock() throws Exception { public void testAddStripedBlock() throws Exception {
final Path file = new Path("/file1"); final Path file = new Path("/file1");
@ -112,7 +120,7 @@ public class TestAddStripedBlocks {
FSDataOutputStream out = null; FSDataOutputStream out = null;
try { try {
out = dfs.create(file, (short) 1); out = dfs.create(file, (short) 1);
DFSTestUtil.writeAndFlushStripedOutputStream( writeAndFlushStripedOutputStream(
(DFSStripedOutputStream) out.getWrappedStream(), (DFSStripedOutputStream) out.getWrappedStream(),
DFS_BYTES_PER_CHECKSUM_DEFAULT); DFS_BYTES_PER_CHECKSUM_DEFAULT);
@ -190,7 +198,7 @@ public class TestAddStripedBlocks {
FSDataOutputStream out = null; FSDataOutputStream out = null;
try { try {
out = dfs.create(file, (short) 1); out = dfs.create(file, (short) 1);
DFSTestUtil.writeAndFlushStripedOutputStream( writeAndFlushStripedOutputStream(
(DFSStripedOutputStream) out.getWrappedStream(), (DFSStripedOutputStream) out.getWrappedStream(),
DFS_BYTES_PER_CHECKSUM_DEFAULT); DFS_BYTES_PER_CHECKSUM_DEFAULT);