HDFS-9040. Erasure coding: coordinate data streamers in DFSStripedOutputStream. Contributed by Jing Zhao and Walter Su.

This commit is contained in:
Jing Zhao 2015-09-28 14:40:27 -07:00
parent c09dc258a8
commit 6419900ac2
18 changed files with 1066 additions and 936 deletions

View File

@ -38,6 +38,8 @@ import com.google.common.annotations.VisibleForTesting;
@InterfaceStability.Evolving
public class DatanodeID implements Comparable<DatanodeID> {
public static final DatanodeID[] EMPTY_ARRAY = {};
public static final DatanodeID EMPTY_DATANODE_ID = new DatanodeID("null",
"null", "null", 0, 0, 0, 0);
private String ipAddr; // IP address
private String hostName; // hostname claimed by datanode

View File

@ -450,3 +450,6 @@
HDFS-8882. Erasure Coding: Use datablocks, parityblocks and cell size from
ErasureCodingPolicy (Vinayakumar B via zhz)
HDFS-9040. Erasure coding: coordinate data streamers in
DFSStripedOutputStream. (jing9 and Walter Su)

View File

@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
import org.apache.hadoop.hdfs.server.namenode.RetryStartFileException;
import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
import org.apache.hadoop.hdfs.util.ByteArrayManager;
@ -212,14 +213,17 @@ public class DFSOutputStream extends FSOutputSummer
/** Construct a new output stream for creating a file. */
protected DFSOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
EnumSet<CreateFlag> flag, Progressable progress,
DataChecksum checksum, String[] favoredNodes) throws IOException {
DataChecksum checksum, String[] favoredNodes, boolean createStreamer)
throws IOException {
this(dfsClient, src, progress, stat, checksum);
this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
cachingStrategy, byteArrayManager, favoredNodes);
if (createStreamer) {
streamer = new DataStreamer(stat, null, dfsClient, src, progress,
checksum, cachingStrategy, byteArrayManager, favoredNodes);
}
}
static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
@ -276,7 +280,7 @@ public class DFSOutputStream extends FSOutputSummer
flag, progress, checksum, favoredNodes);
} else {
out = new DFSOutputStream(dfsClient, src, stat,
flag, progress, checksum, favoredNodes);
flag, progress, checksum, favoredNodes, true);
}
out.start();
return out;
@ -476,7 +480,7 @@ public class DFSOutputStream extends FSOutputSummer
*
* @throws IOException
*/
protected void endBlock() throws IOException {
void endBlock() throws IOException {
if (getStreamer().getBytesCurBlock() == blockSize) {
setCurrentPacketToEmpty();
enqueueCurrentPacket();
@ -921,4 +925,52 @@ public class DFSOutputStream extends FSOutputSummer
public String toString() {
return getClass().getSimpleName() + ":" + streamer;
}
static LocatedBlock addBlock(DatanodeInfo[] excludedNodes, DFSClient dfsClient,
String src, ExtendedBlock prevBlock, long fileId, String[] favoredNodes)
throws IOException {
final DfsClientConf conf = dfsClient.getConf();
int retries = conf.getNumBlockWriteLocateFollowingRetry();
long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
long localstart = Time.monotonicNow();
while (true) {
try {
return dfsClient.namenode.addBlock(src, dfsClient.clientName, prevBlock,
excludedNodes, fileId, favoredNodes);
} catch (RemoteException e) {
IOException ue = e.unwrapRemoteException(FileNotFoundException.class,
AccessControlException.class,
NSQuotaExceededException.class,
DSQuotaExceededException.class,
QuotaByStorageTypeExceededException.class,
UnresolvedPathException.class);
if (ue != e) {
throw ue; // no need to retry these exceptions
}
if (NotReplicatedYetException.class.getName().equals(e.getClassName())) {
if (retries == 0) {
throw e;
} else {
--retries;
LOG.info("Exception while adding a block", e);
long elapsed = Time.monotonicNow() - localstart;
if (elapsed > 5000) {
LOG.info("Waiting for replication for " + (elapsed / 1000)
+ " seconds");
}
try {
LOG.warn("NotReplicatedYetException sleeping " + src
+ " retries left " + retries);
Thread.sleep(sleeptime);
sleeptime *= 2;
} catch (InterruptedException ie) {
LOG.warn("Caught exception", ie);
}
}
} else {
throw e;
}
}
}
}
}

View File

@ -25,23 +25,34 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
import org.apache.hadoop.io.MultipleIOException;
import org.apache.hadoop.io.erasurecode.CodecUtil;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.Time;
import org.apache.htrace.Sampler;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
@ -59,23 +70,11 @@ public class DFSStripedOutputStream extends DFSOutputStream {
private final List<BlockingQueue<T>> queues;
MultipleBlockingQueue(int numQueue, int queueSize) {
queues = new ArrayList<>(numQueue);
List<BlockingQueue<T>> list = new ArrayList<>(numQueue);
for (int i = 0; i < numQueue; i++) {
queues.add(new LinkedBlockingQueue<T>(queueSize));
list.add(new LinkedBlockingQueue<T>(queueSize));
}
}
boolean isEmpty() {
for(int i = 0; i < queues.size(); i++) {
if (!queues.get(i).isEmpty()) {
return false;
}
}
return true;
}
int numQueues() {
return queues.size();
queues = Collections.synchronizedList(list);
}
void offer(int i, T object) {
@ -92,6 +91,14 @@ public class DFSStripedOutputStream extends DFSOutputStream {
}
}
T takeWithTimeout(int i) throws InterruptedIOException {
try {
return queues.get(i).poll(100, TimeUnit.MILLISECONDS);
} catch (InterruptedException e) {
throw DFSUtil.toInterruptedIOException("take interrupted, i=" + i, e);
}
}
T poll(int i) {
return queues.get(i).poll();
}
@ -99,23 +106,44 @@ public class DFSStripedOutputStream extends DFSOutputStream {
T peek(int i) {
return queues.get(i).peek();
}
void clear() {
for (BlockingQueue<T> q : queues) {
q.clear();
}
}
}
/** Coordinate the communication between the streamers. */
class Coordinator {
static class Coordinator {
/**
* The next internal block to write to for each streamers. The
* DFSStripedOutputStream makes the {@link ClientProtocol#addBlock} RPC to
* get a new block group. The block group is split to internal blocks, which
* are then distributed into the queue for streamers to retrieve.
*/
private final MultipleBlockingQueue<LocatedBlock> followingBlocks;
/**
* Used to sync among all the streamers before allocating a new block. The
* DFSStripedOutputStream uses this to make sure every streamer has finished
* writing the previous block.
*/
private final MultipleBlockingQueue<ExtendedBlock> endBlocks;
/**
* The following data structures are used for syncing while handling errors
*/
private final MultipleBlockingQueue<LocatedBlock> newBlocks;
private final MultipleBlockingQueue<ExtendedBlock> updateBlocks;
private final Map<StripedDataStreamer, Boolean> updateStreamerMap;
private final MultipleBlockingQueue<Boolean> streamerUpdateResult;
Coordinator(final DfsClientConf conf, final int numDataBlocks,
final int numAllBlocks) {
Coordinator(final int numAllBlocks) {
followingBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
endBlocks = new MultipleBlockingQueue<>(numDataBlocks, 1);
endBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
newBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
updateBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1);
updateStreamerMap = Collections.synchronizedMap(
new HashMap<StripedDataStreamer, Boolean>(numAllBlocks));
streamerUpdateResult = new MultipleBlockingQueue<>(numAllBlocks, 1);
}
MultipleBlockingQueue<LocatedBlock> getFollowingBlocks() {
@ -126,68 +154,28 @@ public class DFSStripedOutputStream extends DFSOutputStream {
return newBlocks;
}
MultipleBlockingQueue<ExtendedBlock> getUpdateBlocks() {
return updateBlocks;
}
StripedDataStreamer getStripedDataStreamer(int i) {
return DFSStripedOutputStream.this.getStripedDataStreamer(i);
}
void offerEndBlock(int i, ExtendedBlock block) {
endBlocks.offer(i, block);
}
ExtendedBlock takeEndBlock(int i) throws InterruptedIOException {
return endBlocks.take(i);
void offerStreamerUpdateResult(int i, boolean success) {
streamerUpdateResult.offer(i, success);
}
boolean hasAllEndBlocks() {
for(int i = 0; i < endBlocks.numQueues(); i++) {
if (endBlocks.peek(i) == null) {
return false;
}
}
return true;
boolean takeStreamerUpdateResult(int i) throws InterruptedIOException {
return streamerUpdateResult.take(i);
}
void setBytesEndBlock(int i, long newBytes, ExtendedBlock block) {
ExtendedBlock b = endBlocks.peek(i);
if (b == null) {
// streamer just has failed, put end block and continue
b = block;
offerEndBlock(i, b);
}
b.setNumBytes(newBytes);
void updateStreamer(StripedDataStreamer streamer,
boolean success) {
assert !updateStreamerMap.containsKey(streamer);
updateStreamerMap.put(streamer, success);
}
/** @return a block representing the entire block group. */
ExtendedBlock getBlockGroup() {
final StripedDataStreamer s0 = getStripedDataStreamer(0);
final ExtendedBlock b0 = s0.getBlock();
if (b0 == null) {
return null;
}
final boolean atBlockGroupBoundary = s0.getBytesCurBlock() == 0 && b0.getNumBytes() > 0;
final ExtendedBlock block = new ExtendedBlock(b0);
long numBytes = atBlockGroupBoundary? b0.getNumBytes(): s0.getBytesCurBlock();
for (int i = 1; i < numAllBlocks; i++) {
final StripedDataStreamer si = getStripedDataStreamer(i);
final ExtendedBlock bi = si.getBlock();
if (bi != null && bi.getGenerationStamp() > block.getGenerationStamp()) {
block.setGenerationStamp(bi.getGenerationStamp());
}
if (i < numDataBlocks) {
numBytes += atBlockGroupBoundary? bi.getNumBytes(): si.getBytesCurBlock();
}
}
block.setNumBytes(numBytes);
if (LOG.isDebugEnabled()) {
LOG.debug("getBlockGroup: " + block + ", numBytes=" + block.getNumBytes());
}
return block;
void clearFailureStates() {
newBlocks.clear();
updateStreamerMap.clear();
streamerUpdateResult.clear();
}
}
@ -263,18 +251,16 @@ public class DFSStripedOutputStream extends DFSOutputStream {
private final int cellSize;
private final int numAllBlocks;
private final int numDataBlocks;
@Override
ExtendedBlock getBlock() {
return coordinator.getBlockGroup();
}
private ExtendedBlock currentBlockGroup;
private final String[] favoredNodes;
private final List<StripedDataStreamer> failedStreamers;
/** Construct a new output stream for creating a file. */
DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
EnumSet<CreateFlag> flag, Progressable progress,
DataChecksum checksum, String[] favoredNodes)
throws IOException {
super(dfsClient, src, stat, flag, progress, checksum, favoredNodes);
super(dfsClient, src, stat, flag, progress, checksum, favoredNodes, false);
if (LOG.isDebugEnabled()) {
LOG.debug("Creating DFSStripedOutputStream for " + src);
}
@ -284,12 +270,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
cellSize = ecPolicy.getCellSize();
numDataBlocks = ecPolicy.getNumDataUnits();
numAllBlocks = numDataBlocks + numParityBlocks;
this.favoredNodes = favoredNodes;
failedStreamers = new ArrayList<>();
encoder = CodecUtil.createRSRawEncoder(dfsClient.getConfiguration(),
numDataBlocks, numParityBlocks);
coordinator = new Coordinator(dfsClient.getConf(),
numDataBlocks, numAllBlocks);
coordinator = new Coordinator(numAllBlocks);
try {
cellBuffers = new CellBuffers(numParityBlocks);
} catch (InterruptedException ie) {
@ -297,14 +284,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
"Failed to create cell buffers", ie);
}
List<StripedDataStreamer> s = new ArrayList<>(numAllBlocks);
streamers = new ArrayList<>(numAllBlocks);
for (short i = 0; i < numAllBlocks; i++) {
StripedDataStreamer streamer = new StripedDataStreamer(stat,
dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
favoredNodes, i, coordinator);
s.add(streamer);
streamers.add(streamer);
}
streamers = Collections.unmodifiableList(s);
currentPackets = new DFSPacket[streamers.size()];
setCurrentStreamer(0);
}
@ -318,17 +304,19 @@ public class DFSStripedOutputStream extends DFSOutputStream {
}
private synchronized StripedDataStreamer getCurrentStreamer() {
return (StripedDataStreamer)streamer;
return (StripedDataStreamer) streamer;
}
private synchronized StripedDataStreamer setCurrentStreamer(int newIdx) {
// backup currentPacket for current streamer
int oldIdx = streamers.indexOf(streamer);
if (oldIdx >= 0) {
currentPackets[oldIdx] = currentPacket;
if (streamer != null) {
int oldIdx = streamers.indexOf(getCurrentStreamer());
if (oldIdx >= 0) {
currentPackets[oldIdx] = currentPacket;
}
}
streamer = streamers.get(newIdx);
streamer = getStripedDataStreamer(newIdx);
currentPacket = currentPackets[newIdx];
adjustChunkBoundary();
@ -350,38 +338,125 @@ public class DFSStripedOutputStream extends DFSOutputStream {
encoder.encode(dataBuffers, parityBuffers);
}
private void checkStreamers(boolean setExternalError) throws IOException {
int count = 0;
/**
* check all the existing StripedDataStreamer and find newly failed streamers.
* @return The newly failed streamers.
* @throws IOException if less than {@link #numDataBlocks} streamers are still
* healthy.
*/
private Set<StripedDataStreamer> checkStreamers() throws IOException {
Set<StripedDataStreamer> newFailed = new HashSet<>();
for(StripedDataStreamer s : streamers) {
if (!s.isFailed()) {
if (setExternalError && s.getBlock() != null) {
s.getErrorState().initExternalError();
}
count++;
if (!s.isHealthy() && !failedStreamers.contains(s)) {
newFailed.add(s);
}
}
final int failCount = failedStreamers.size() + newFailed.size();
if (LOG.isDebugEnabled()) {
LOG.debug("checkStreamers: " + streamers);
LOG.debug("count=" + count);
LOG.debug("healthy streamer count=" + (numAllBlocks - failCount));
LOG.debug("original failed streamers: " + failedStreamers);
LOG.debug("newly failed streamers: " + newFailed);
}
if (count < numDataBlocks) {
throw new IOException("Failed: the number of remaining blocks = "
+ count + " < the number of data blocks = " + numDataBlocks);
if (failCount > (numAllBlocks - numDataBlocks)) {
throw new IOException("Failed: the number of failed blocks = "
+ failCount + " > the number of data blocks = "
+ (numAllBlocks - numDataBlocks));
}
return newFailed;
}
private void handleStreamerFailure(String err, Exception e)
throws IOException {
handleStreamerFailure(err, e, true);
LOG.warn("Failed: " + err + ", " + this, e);
getCurrentStreamer().getErrorState().setInternalError();
getCurrentStreamer().close(true);
checkStreamers();
currentPacket = null;
}
private void handleStreamerFailure(String err, Exception e,
boolean setExternalError) throws IOException {
LOG.warn("Failed: " + err + ", " + this, e);
getCurrentStreamer().setFailed(true);
checkStreamers(setExternalError);
currentPacket = null;
private void replaceFailedStreamers() {
assert streamers.size() == numAllBlocks;
for (short i = 0; i < numAllBlocks; i++) {
final StripedDataStreamer oldStreamer = getStripedDataStreamer(i);
if (!oldStreamer.isHealthy()) {
StripedDataStreamer streamer = new StripedDataStreamer(oldStreamer.stat,
dfsClient, src, oldStreamer.progress,
oldStreamer.checksum4WriteBlock, cachingStrategy, byteArrayManager,
favoredNodes, i, coordinator);
streamers.set(i, streamer);
currentPackets[i] = null;
if (i == 0) {
this.streamer = streamer;
}
streamer.start();
}
}
}
private void waitEndBlocks(int i) throws IOException {
while (getStripedDataStreamer(i).isHealthy()) {
final ExtendedBlock b = coordinator.endBlocks.takeWithTimeout(i);
if (b != null) {
StripedBlockUtil.checkBlocks(currentBlockGroup, i, b);
return;
}
}
}
private void allocateNewBlock() throws IOException {
if (currentBlockGroup != null) {
for (int i = 0; i < numAllBlocks; i++) {
// sync all the healthy streamers before writing to the new block
waitEndBlocks(i);
}
}
failedStreamers.clear();
// replace failed streamers
replaceFailedStreamers();
if (LOG.isDebugEnabled()) {
LOG.debug("Allocating new block group. The previous block group: "
+ currentBlockGroup);
}
// TODO collect excludedNodes from all the data streamers
final LocatedBlock lb = addBlock(null, dfsClient, src, currentBlockGroup,
fileId, favoredNodes);
assert lb.isStriped();
if (lb.getLocations().length < numDataBlocks) {
throw new IOException("Failed to get " + numDataBlocks
+ " nodes from namenode: blockGroupSize= " + numAllBlocks
+ ", blocks.length= " + lb.getLocations().length);
}
// assign the new block to the current block group
currentBlockGroup = lb.getBlock();
final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup(
(LocatedStripedBlock) lb, cellSize, numDataBlocks,
numAllBlocks - numDataBlocks);
for (int i = 0; i < blocks.length; i++) {
StripedDataStreamer si = getStripedDataStreamer(i);
if (si.isHealthy()) { // skipping failed data streamer
if (blocks[i] == null) {
// Set exception and close streamer as there is no block locations
// found for the parity block.
LOG.warn("Failed to get block location for parity block, index=" + i);
si.getLastException().set(
new IOException("Failed to get following block, i=" + i));
si.getErrorState().setInternalError();
si.close(true);
} else {
coordinator.getFollowingBlocks().offer(i, blocks[i]);
}
}
}
}
private boolean shouldEndBlockGroup() {
return currentBlockGroup != null &&
currentBlockGroup.getNumBytes() == blockSize * numDataBlocks;
}
@Override
@ -392,8 +467,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
final int pos = cellBuffers.addTo(index, bytes, offset, len);
final boolean cellFull = pos == cellSize;
final long oldBytes = current.getBytesCurBlock();
if (!current.isFailed()) {
if (currentBlockGroup == null || shouldEndBlockGroup()) {
// the incoming data should belong to a new block. Allocate a new block.
allocateNewBlock();
}
currentBlockGroup.setNumBytes(currentBlockGroup.getNumBytes() + len);
if (current.isHealthy()) {
try {
super.writeChunk(bytes, offset, len, checksum, ckoff, cklen);
} catch(Exception e) {
@ -401,12 +481,6 @@ public class DFSStripedOutputStream extends DFSOutputStream {
}
}
if (current.isFailed()) {
final long newBytes = oldBytes + len;
coordinator.setBytesEndBlock(index, newBytes, current.getBlock());
current.setBytesCurBlock(newBytes);
}
// Two extra steps are needed when a striping cell is full:
// 1. Forward the current index pointer
// 2. Generate parity packets if a full stripe of data cells are present
@ -419,11 +493,209 @@ public class DFSStripedOutputStream extends DFSOutputStream {
cellBuffers.flipDataBuffers();
writeParityCells();
next = 0;
// check failure state for all the streamers. Bump GS if necessary
checkStreamerFailures();
// if this is the end of the block group, end each internal block
if (shouldEndBlockGroup()) {
for (int i = 0; i < numAllBlocks; i++) {
final StripedDataStreamer s = setCurrentStreamer(i);
if (s.isHealthy()) {
try {
endBlock();
} catch (IOException ignored) {}
}
}
}
}
setCurrentStreamer(next);
}
}
@Override
void enqueueCurrentPacketFull() throws IOException {
LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={},"
+ " appendChunk={}, {}", currentPacket, src, getStreamer()
.getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
getStreamer());
enqueueCurrentPacket();
adjustChunkBoundary();
// no need to end block here
}
private Set<StripedDataStreamer> markExternalErrorOnStreamers() {
Set<StripedDataStreamer> healthySet = new HashSet<>();
for (StripedDataStreamer streamer : streamers) {
if (streamer.isHealthy() &&
streamer.getStage() == BlockConstructionStage.DATA_STREAMING) {
streamer.setExternalError();
healthySet.add(streamer);
}
}
return healthySet;
}
/**
* Check and handle data streamer failures. This is called only when we have
* written a full stripe (i.e., enqueue all packets for a full stripe), or
* when we're closing the outputstream.
*/
private void checkStreamerFailures() throws IOException {
Set<StripedDataStreamer> newFailed = checkStreamers();
if (newFailed.size() > 0) {
// for healthy streamers, wait till all of them have fetched the new block
// and flushed out all the enqueued packets.
flushAllInternals();
}
// get all the current failed streamers after the flush
newFailed = checkStreamers();
while (newFailed.size() > 0) {
failedStreamers.addAll(newFailed);
coordinator.clearFailureStates();
// mark all the healthy streamers as external error
Set<StripedDataStreamer> healthySet = markExternalErrorOnStreamers();
// we have newly failed streamers, update block for pipeline
final ExtendedBlock newBG = updateBlockForPipeline(healthySet);
// wait till all the healthy streamers to
// 1) get the updated block info
// 2) create new block outputstream
newFailed = waitCreatingNewStreams(healthySet);
if (newFailed.size() + failedStreamers.size() >
numAllBlocks - numDataBlocks) {
throw new IOException(
"Data streamers failed while creating new block streams: "
+ newFailed + ". There are not enough healthy streamers.");
}
for (StripedDataStreamer failedStreamer : newFailed) {
assert !failedStreamer.isHealthy();
}
// TODO we can also succeed if all the failed streamers have not taken
// the updated block
if (newFailed.size() == 0) {
// reset external error state of all the streamers
for (StripedDataStreamer streamer : healthySet) {
assert streamer.isHealthy();
streamer.getErrorState().reset();
}
updatePipeline(newBG);
}
for (int i = 0; i < numAllBlocks; i++) {
coordinator.offerStreamerUpdateResult(i, newFailed.size() == 0);
}
}
}
private int checkStreamerUpdates(Set<StripedDataStreamer> failed,
Set<StripedDataStreamer> streamers) {
for (StripedDataStreamer streamer : streamers) {
if (!coordinator.updateStreamerMap.containsKey(streamer)) {
if (!streamer.isHealthy() &&
coordinator.getNewBlocks().peek(streamer.getIndex()) != null) {
// this streamer had internal error before getting updated block
failed.add(streamer);
}
}
}
return coordinator.updateStreamerMap.size() + failed.size();
}
private Set<StripedDataStreamer> waitCreatingNewStreams(
Set<StripedDataStreamer> healthyStreamers) throws IOException {
Set<StripedDataStreamer> failed = new HashSet<>();
final int expectedNum = healthyStreamers.size();
final long socketTimeout = dfsClient.getConf().getSocketTimeout();
// the total wait time should be less than the socket timeout, otherwise
// a slow streamer may cause other streamers to timeout. here we wait for
// half of the socket timeout
long remaingTime = socketTimeout > 0 ? socketTimeout/2 : Long.MAX_VALUE;
final long waitInterval = 1000;
synchronized (coordinator) {
while (checkStreamerUpdates(failed, healthyStreamers) < expectedNum
&& remaingTime > 0) {
try {
long start = Time.monotonicNow();
coordinator.wait(waitInterval);
remaingTime -= Time.monotonicNow() - start;
} catch (InterruptedException e) {
throw DFSUtil.toInterruptedIOException("Interrupted when waiting" +
" for results of updating striped streamers", e);
}
}
}
synchronized (coordinator) {
for (StripedDataStreamer streamer : healthyStreamers) {
if (!coordinator.updateStreamerMap.containsKey(streamer)) {
// close the streamer if it is too slow to create new connection
streamer.setStreamerAsClosed();
failed.add(streamer);
}
}
}
for (Map.Entry<StripedDataStreamer, Boolean> entry :
coordinator.updateStreamerMap.entrySet()) {
if (!entry.getValue()) {
failed.add(entry.getKey());
}
}
for (StripedDataStreamer failedStreamer : failed) {
healthyStreamers.remove(failedStreamer);
}
return failed;
}
/**
* Call {@link ClientProtocol#updateBlockForPipeline} and assign updated block
* to healthy streamers.
* @param healthyStreamers The healthy data streamers. These streamers join
* the failure handling.
*/
private ExtendedBlock updateBlockForPipeline(
Set<StripedDataStreamer> healthyStreamers) throws IOException {
final LocatedBlock updated = dfsClient.namenode.updateBlockForPipeline(
currentBlockGroup, dfsClient.clientName);
final long newGS = updated.getBlock().getGenerationStamp();
ExtendedBlock newBlock = new ExtendedBlock(currentBlockGroup);
newBlock.setGenerationStamp(newGS);
final LocatedBlock[] updatedBlks = StripedBlockUtil.parseStripedBlockGroup(
(LocatedStripedBlock) updated, cellSize, numDataBlocks,
numAllBlocks - numDataBlocks);
for (int i = 0; i < numAllBlocks; i++) {
StripedDataStreamer si = getStripedDataStreamer(i);
if (healthyStreamers.contains(si)) {
final LocatedBlock lb = new LocatedBlock(new ExtendedBlock(newBlock),
null, null, null, -1, updated.isCorrupt(), null);
lb.setBlockToken(updatedBlks[i].getBlockToken());
coordinator.getNewBlocks().offer(i, lb);
}
}
return newBlock;
}
private void updatePipeline(ExtendedBlock newBG) throws IOException {
final DatanodeInfo[] newNodes = new DatanodeInfo[numAllBlocks];
final String[] newStorageIDs = new String[numAllBlocks];
for (int i = 0; i < numAllBlocks; i++) {
final StripedDataStreamer streamer = getStripedDataStreamer(i);
final DatanodeInfo[] nodes = streamer.getNodes();
final String[] storageIDs = streamer.getStorageIDs();
if (streamer.isHealthy() && nodes != null && storageIDs != null) {
newNodes[i] = nodes[0];
newStorageIDs[i] = storageIDs[0];
} else {
newNodes[i] = new DatanodeInfo(DatanodeID.EMPTY_DATANODE_ID);
newStorageIDs[i] = "";
}
}
dfsClient.namenode.updatePipeline(dfsClient.clientName, currentBlockGroup,
newBG, newNodes, newStorageIDs);
currentBlockGroup = newBG;
}
private int stripeDataSize() {
return numDataBlocks * cellSize;
}
@ -500,28 +772,16 @@ public class DFSStripedOutputStream extends DFSOutputStream {
}
}
/**
* Simply add bytesCurBlock together. Note that this result is not accurately
* the size of the block group.
*/
private long getCurrentSumBytes() {
long sum = 0;
for (int i = 0; i < numDataBlocks; i++) {
sum += streamers.get(i).getBytesCurBlock();
}
return sum;
}
private boolean generateParityCellsForLastStripe() {
final long currentBlockGroupBytes = getCurrentSumBytes();
if (currentBlockGroupBytes % stripeDataSize() == 0) {
final long currentBlockGroupBytes = currentBlockGroup == null ?
0 : currentBlockGroup.getNumBytes();
final long lastStripeSize = currentBlockGroupBytes % stripeDataSize();
if (lastStripeSize == 0) {
return false;
}
final int firstCellSize =
(int)(getStripedDataStreamer(0).getBytesCurBlock() % cellSize);
final int parityCellSize = firstCellSize > 0 && firstCellSize < cellSize?
firstCellSize : cellSize;
final long parityCellSize = lastStripeSize < cellSize?
lastStripeSize : cellSize;
final ByteBuffer[] buffers = cellBuffers.getBuffers();
for (int i = 0; i < numAllBlocks; i++) {
@ -550,13 +810,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
cellBuffers.clear();
}
void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf
) throws IOException {
void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf)
throws IOException {
final StripedDataStreamer current = setCurrentStreamer(index);
final int len = buffer.limit();
final long oldBytes = current.getBytesCurBlock();
if (!current.isFailed()) {
if (current.isHealthy()) {
try {
DataChecksum sum = getDataChecksum();
sum.calculateChunkedSums(buffer.array(), 0, len, checksumBuf, 0);
@ -570,18 +830,13 @@ public class DFSStripedOutputStream extends DFSOutputStream {
handleStreamerFailure("oldBytes=" + oldBytes + ", len=" + len, e);
}
}
if (current.isFailed()) {
final long newBytes = oldBytes + len;
current.setBytesCurBlock(newBytes);
}
}
@Override
void setClosed() {
super.setClosed();
for (int i = 0; i < numAllBlocks; i++) {
streamers.get(i).release();
getStripedDataStreamer(i).release();
}
cellBuffers.release();
}
@ -607,37 +862,40 @@ public class DFSStripedOutputStream extends DFSOutputStream {
try {
// flush from all upper layers
try {
flushBuffer();
} catch(Exception e) {
handleStreamerFailure("flushBuffer " + getCurrentStreamer(), e);
}
flushBuffer();
// if the last stripe is incomplete, generate and write parity cells
if (generateParityCellsForLastStripe()) {
writeParityCells();
}
enqueueAllCurrentPackets();
// flush all the data packets
flushAllInternals();
// check failures
checkStreamerFailures();
for (int i = 0; i < numAllBlocks; i++) {
final StripedDataStreamer s = setCurrentStreamer(i);
if (!s.isFailed()) {
if (s.isHealthy()) {
try {
if (s.getBytesCurBlock() > 0) {
setCurrentPacketToEmpty();
}
// flush all data to Datanode
// flush the last "close" packet to Datanode
flushInternal();
} catch(Exception e) {
handleStreamerFailure("flushInternal " + s, e, false);
// TODO for both close and endBlock, we currently do not handle
// failures when sending the last packet. We actually do not need to
// bump GS for this kind of failure. Thus counting the total number
// of failures may be good enough.
}
}
}
closeThreads(false);
final ExtendedBlock lastBlock = coordinator.getBlockGroup();
TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER);
try {
completeFile(lastBlock);
completeFile(currentBlockGroup);
} finally {
scope.close();
}
@ -652,14 +910,45 @@ public class DFSStripedOutputStream extends DFSOutputStream {
int idx = streamers.indexOf(getCurrentStreamer());
for(int i = 0; i < streamers.size(); i++) {
final StripedDataStreamer si = setCurrentStreamer(i);
if (!si.isFailed() && currentPacket != null) {
if (si.isHealthy() && currentPacket != null) {
try {
enqueueCurrentPacket();
} catch (IOException e) {
handleStreamerFailure("enqueueAllCurrentPackets, i=" + i, e, false);
handleStreamerFailure("enqueueAllCurrentPackets, i=" + i, e);
}
}
}
setCurrentStreamer(idx);
}
void flushAllInternals() throws IOException {
int current = getCurrentIndex();
for (int i = 0; i < numAllBlocks; i++) {
final StripedDataStreamer s = setCurrentStreamer(i);
if (s.isHealthy()) {
try {
// flush all data to Datanode
flushInternal();
} catch(Exception e) {
handleStreamerFailure("flushInternal " + s, e);
}
}
}
setCurrentStreamer(current);
}
static void sleep(long ms, String op) throws InterruptedIOException {
try {
Thread.sleep(ms);
} catch(InterruptedException ie) {
throw DFSUtil.toInterruptedIOException(
"Sleep interrupted during " + op, ie);
}
}
@Override
ExtendedBlock getBlock() {
return currentBlockGroup;
}
}

View File

@ -22,7 +22,6 @@ import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SU
import java.io.BufferedOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.io.InterruptedIOException;
@ -46,16 +45,12 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
@ -69,13 +64,10 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
import org.apache.hadoop.hdfs.util.ByteArrayManager;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.MultipleIOException;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DataChecksum;
@ -204,9 +196,12 @@ class DataStreamer extends Daemon {
}
}
enum ErrorType {
NONE, INTERNAL, EXTERNAL
}
static class ErrorState {
private boolean error = false;
private boolean externalError = false;
ErrorType error = ErrorType.NONE;
private int badNodeIndex = -1;
private int restartingNodeIndex = -1;
private long restartingNodeDeadline = 0;
@ -216,36 +211,48 @@ class DataStreamer extends Daemon {
this.datanodeRestartTimeout = datanodeRestartTimeout;
}
synchronized void reset() {
error = false;
externalError = false;
synchronized void resetInternalError() {
if (hasInternalError()) {
error = ErrorType.NONE;
}
badNodeIndex = -1;
restartingNodeIndex = -1;
restartingNodeDeadline = 0;
}
synchronized boolean hasError() {
return error;
synchronized void reset() {
error = ErrorType.NONE;
badNodeIndex = -1;
restartingNodeIndex = -1;
restartingNodeDeadline = 0;
}
synchronized boolean hasExternalErrorOnly() {
return error && externalError && !isNodeMarked();
synchronized boolean hasInternalError() {
return error == ErrorType.INTERNAL;
}
synchronized boolean hasExternalError() {
return error == ErrorType.EXTERNAL;
}
synchronized boolean hasError() {
return error != ErrorType.NONE;
}
synchronized boolean hasDatanodeError() {
return error && (isNodeMarked() || externalError);
return error == ErrorType.INTERNAL && isNodeMarked();
}
synchronized void setError(boolean err) {
this.error = err;
synchronized void setInternalError() {
this.error = ErrorType.INTERNAL;
}
synchronized void initExternalError() {
setError(true);
this.externalError = true;
synchronized void setExternalError() {
if (!hasInternalError()) {
this.error = ErrorType.EXTERNAL;
}
}
synchronized void setBadNodeIndex(int index) {
this.badNodeIndex = index;
}
@ -306,14 +313,14 @@ class DataStreamer extends Daemon {
}
if (!isRestartingNode()) {
error = false;
error = ErrorType.NONE;
}
badNodeIndex = -1;
}
synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) {
if (restartingNodeIndex >= 0) {
if (!error) {
if (error == ErrorType.NONE) {
throw new IllegalStateException("error=false while checking" +
" restarting node deadline");
}
@ -345,7 +352,7 @@ class DataStreamer extends Daemon {
private volatile boolean streamerClosed = false;
protected ExtendedBlock block; // its length is number of bytes acked
private Token<BlockTokenIdentifier> accessToken;
protected Token<BlockTokenIdentifier> accessToken;
private DataOutputStream blockStream;
private DataInputStream blockReplyStream;
private ResponseProcessor response = null;
@ -355,7 +362,7 @@ class DataStreamer extends Daemon {
private final ErrorState errorState;
private BlockConstructionStage stage; // block construction stage
private long bytesSent = 0; // number of bytes that've been sent
protected long bytesSent = 0; // number of bytes that've been sent
private final boolean isLazyPersistFile;
/** Nodes have been used in the pipeline before and have failed. */
@ -378,13 +385,13 @@ class DataStreamer extends Daemon {
protected final DFSClient dfsClient;
protected final String src;
/** Only for DataTransferProtocol.writeBlock(..) */
private final DataChecksum checksum4WriteBlock;
private final Progressable progress;
final DataChecksum checksum4WriteBlock;
final Progressable progress;
protected final HdfsFileStatus stat;
// appending to existing partial block
private volatile boolean appendChunk = false;
// both dataQueue and ackQueue are protected by dataQueue lock
private final LinkedList<DFSPacket> dataQueue = new LinkedList<>();
protected final LinkedList<DFSPacket> dataQueue = new LinkedList<>();
private final LinkedList<DFSPacket> ackQueue = new LinkedList<>();
private final AtomicReference<CachingStrategy> cachingStrategy;
private final ByteArrayManager byteArrayManager;
@ -401,7 +408,7 @@ class DataStreamer extends Daemon {
CONGESTION_BACKOFF_MEAN_TIME_IN_MS * 10;
private int lastCongestionBackoffTime;
private final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
protected final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
private final String[] favoredNodes;
private DataStreamer(HdfsFileStatus stat, ExtendedBlock block,
@ -473,6 +480,10 @@ class DataStreamer extends Daemon {
}
}
void setAccessToken(Token<BlockTokenIdentifier> t) {
this.accessToken = t;
}
private void setPipeline(LocatedBlock lb) {
setPipeline(lb.getLocations(), lb.getStorageTypes(), lb.getStorageIDs());
}
@ -533,7 +544,7 @@ class DataStreamer extends Daemon {
DFSPacket one;
try {
// process datanode IO errors if any
boolean doSleep = processDatanodeError();
boolean doSleep = processDatanodeOrExternalError();
final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2;
synchronized (dataQueue) {
@ -696,7 +707,7 @@ class DataStreamer extends Daemon {
}
lastException.set(e);
assert !(e instanceof NullPointerException);
errorState.setError(true);
errorState.setInternalError();
if (!errorState.isNodeMarked()) {
// Not a datanode issue
streamerClosed = true;
@ -837,6 +848,9 @@ class DataStreamer extends Daemon {
}
}
void setStreamerAsClosed() {
streamerClosed = true;
}
private void checkClosed() throws IOException {
if (streamerClosed) {
@ -857,7 +871,7 @@ class DataStreamer extends Daemon {
}
}
private void closeStream() {
void closeStream() {
final MultipleIOException.Builder b = new MultipleIOException.Builder();
if (blockStream != null) {
@ -1037,7 +1051,7 @@ class DataStreamer extends Daemon {
} catch (Exception e) {
if (!responderClosed) {
lastException.set(e);
errorState.setError(true);
errorState.setInternalError();
errorState.markFirstNodeIfNotMarked();
synchronized (dataQueue) {
dataQueue.notifyAll();
@ -1059,18 +1073,18 @@ class DataStreamer extends Daemon {
}
}
private boolean shouldHandleExternalError(){
return errorState.hasExternalError() && blockStream != null;
}
/**
* If this stream has encountered any errors, shutdown threads
* and mark the stream as closed.
*
* @return true if it should sleep for a while after returning.
*/
private boolean processDatanodeError() throws IOException {
if (!errorState.hasDatanodeError()) {
return false;
}
if (errorState.hasExternalErrorOnly() && block == null) {
// block is not yet initialized, handle external error later.
private boolean processDatanodeOrExternalError() throws IOException {
if (!errorState.hasDatanodeError() && !shouldHandleExternalError()) {
return false;
}
if (response != null) {
@ -1103,7 +1117,8 @@ class DataStreamer extends Daemon {
return false;
}
}
boolean doSleep = setupPipelineForAppendOrRecovery();
setupPipelineForAppendOrRecovery();
if (!streamerClosed && dfsClient.clientRunning) {
if (stage == BlockConstructionStage.PIPELINE_CLOSE) {
@ -1135,7 +1150,7 @@ class DataStreamer extends Daemon {
}
}
return doSleep;
return false;
}
void setHflush() {
@ -1266,7 +1281,7 @@ class DataStreamer extends Daemon {
* This happens when a file is appended or data streaming fails
* It keeps on trying until a pipeline is setup
*/
private boolean setupPipelineForAppendOrRecovery() throws IOException {
private void setupPipelineForAppendOrRecovery() throws IOException {
// check number of datanodes
if (nodes == null || nodes.length == 0) {
String msg = "Could not get block locations. " + "Source file \""
@ -1274,19 +1289,23 @@ class DataStreamer extends Daemon {
LOG.warn(msg);
lastException.set(new IOException(msg));
streamerClosed = true;
return false;
return;
}
setupPipelineInternal(nodes, storageTypes);
}
protected void setupPipelineInternal(DatanodeInfo[] datanodes,
StorageType[] nodeStorageTypes) throws IOException {
boolean success = false;
long newGS = 0L;
while (!success && !streamerClosed && dfsClient.clientRunning) {
if (!handleRestartingDatanode()) {
return false;
return;
}
final boolean isRecovery = errorState.hasError();
final boolean isRecovery = errorState.hasInternalError();
if (!handleBadDatanode()) {
return false;
return;
}
handleDatanodeReplacement();
@ -1307,7 +1326,6 @@ class DataStreamer extends Daemon {
if (success) {
block = updatePipeline(newGS);
}
return false; // do not sleep, continue processing
}
/**
@ -1315,7 +1333,7 @@ class DataStreamer extends Daemon {
* This process is repeated until the deadline or the node starts back up.
* @return true if it should continue.
*/
private boolean handleRestartingDatanode() {
boolean handleRestartingDatanode() {
if (errorState.isRestartingNode()) {
// 4 seconds or the configured deadline period, whichever is shorter.
// This is the retry interval and recovery will be retried in this
@ -1338,7 +1356,7 @@ class DataStreamer extends Daemon {
* Remove bad node from list of nodes if badNodeIndex was set.
* @return true if it should continue.
*/
private boolean handleBadDatanode() {
boolean handleBadDatanode() {
final int badNodeIndex = errorState.getBadNodeIndex();
if (badNodeIndex >= 0) {
if (nodes.length <= 1) {
@ -1388,7 +1406,7 @@ class DataStreamer extends Daemon {
}
}
private void failPacket4Testing() {
void failPacket4Testing() {
if (failPacket) { // for testing
failPacket = false;
try {
@ -1400,13 +1418,8 @@ class DataStreamer extends Daemon {
}
}
LocatedBlock updateBlockForPipeline() throws IOException {
return callUpdateBlockForPipeline(block);
}
LocatedBlock callUpdateBlockForPipeline(ExtendedBlock newBlock) throws IOException {
return dfsClient.namenode.updateBlockForPipeline(
newBlock, dfsClient.clientName);
private LocatedBlock updateBlockForPipeline() throws IOException {
return dfsClient.namenode.updateBlockForPipeline(block, dfsClient.clientName);
}
static ExtendedBlock newBlock(ExtendedBlock b, final long newGS) {
@ -1417,18 +1430,12 @@ class DataStreamer extends Daemon {
/** update pipeline at the namenode */
ExtendedBlock updatePipeline(long newGS) throws IOException {
final ExtendedBlock newBlock = newBlock(block, newGS);
return callUpdatePipeline(block, newBlock, nodes, storageIDs);
}
ExtendedBlock callUpdatePipeline(ExtendedBlock oldBlock, ExtendedBlock newBlock,
DatanodeInfo[] newNodes, String[] newStorageIDs)
throws IOException {
dfsClient.namenode.updatePipeline(dfsClient.clientName, oldBlock, newBlock,
newNodes, newStorageIDs);
dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
nodes, storageIDs);
return newBlock;
}
int getNumBlockWriteRetry() {
private int getNumBlockWriteRetry() {
return dfsClient.getConf().getNumBlockWriteRetry();
}
@ -1438,7 +1445,7 @@ class DataStreamer extends Daemon {
* Must get block ID and the IDs of the destinations from the namenode.
* Returns the list of target datanodes.
*/
private LocatedBlock nextBlockOutputStream() throws IOException {
protected LocatedBlock nextBlockOutputStream() throws IOException {
LocatedBlock lb = null;
DatanodeInfo[] nodes = null;
StorageType[] storageTypes = null;
@ -1446,9 +1453,8 @@ class DataStreamer extends Daemon {
boolean success = false;
ExtendedBlock oldBlock = block;
do {
errorState.reset();
errorState.resetInternalError();
lastException.clear();
success = false;
DatanodeInfo[] excluded =
excludedNodes.getAllPresent(excludedNodes.asMap().keySet())
@ -1488,7 +1494,7 @@ class DataStreamer extends Daemon {
// connects to the first datanode in the pipeline
// Returns true if success, otherwise return failure.
//
private boolean createBlockOutputStream(DatanodeInfo[] nodes,
boolean createBlockOutputStream(DatanodeInfo[] nodes,
StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) {
if (nodes.length == 0) {
LOG.info("nodes are empty for write pipeline of " + block);
@ -1567,7 +1573,7 @@ class DataStreamer extends Daemon {
assert null == blockStream : "Previous blockStream unclosed";
blockStream = out;
result = true; // success
errorState.reset();
errorState.resetInternalError();
} catch (IOException ie) {
if (!errorState.isRestartingNode()) {
LOG.info("Exception in createBlockOutputStream " + this, ie);
@ -1603,7 +1609,7 @@ class DataStreamer extends Daemon {
if (checkRestart && shouldWaitForRestart(i)) {
errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]);
}
errorState.setError(true);
errorState.setInternalError();
lastException.set(ie);
result = false; // error
} finally {
@ -1645,58 +1651,10 @@ class DataStreamer extends Daemon {
}
}
LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
private LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
throws IOException {
final DfsClientConf conf = dfsClient.getConf();
int retries = conf.getNumBlockWriteLocateFollowingRetry();
long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
while (true) {
long localstart = Time.monotonicNow();
while (true) {
try {
return dfsClient.namenode.addBlock(src, dfsClient.clientName,
block, excludedNodes, stat.getFileId(), favoredNodes);
} catch (RemoteException e) {
IOException ue =
e.unwrapRemoteException(FileNotFoundException.class,
AccessControlException.class,
NSQuotaExceededException.class,
DSQuotaExceededException.class,
QuotaByStorageTypeExceededException.class,
UnresolvedPathException.class);
if (ue != e) {
throw ue; // no need to retry these exceptions
}
if (NotReplicatedYetException.class.getName().
equals(e.getClassName())) {
if (retries == 0) {
throw e;
} else {
--retries;
LOG.info("Exception while adding a block", e);
long elapsed = Time.monotonicNow() - localstart;
if (elapsed > 5000) {
LOG.info("Waiting for replication for "
+ (elapsed / 1000) + " seconds");
}
try {
LOG.warn("NotReplicatedYetException sleeping " + src
+ " retries left " + retries);
Thread.sleep(sleeptime);
sleeptime *= 2;
} catch (InterruptedException ie) {
LOG.warn("Caught exception", ie);
}
}
} else {
throw e;
}
}
}
}
return DFSOutputStream.addBlock(excludedNodes, dfsClient, src, block,
stat.getFileId(), favoredNodes);
}
/**
@ -1755,6 +1713,10 @@ class DataStreamer extends Daemon {
return storageIDs;
}
BlockConstructionStage getStage() {
return stage;
}
/**
* return the token of the block
*

View File

@ -19,18 +19,15 @@
package org.apache.hadoop.hdfs;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator;
import org.apache.hadoop.hdfs.DFSStripedOutputStream.MultipleBlockingQueue;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
import org.apache.hadoop.hdfs.util.ByteArrayManager;
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
import org.apache.hadoop.io.erasurecode.ECSchema;
import org.apache.hadoop.util.DataChecksum;
import org.apache.hadoop.util.Progressable;
@ -46,66 +43,8 @@ import com.google.common.annotations.VisibleForTesting;
* other streamers.
*/
public class StripedDataStreamer extends DataStreamer {
/**
* This class is designed for multiple threads to share a
* {@link MultipleBlockingQueue}. Initially, the queue is empty. The earliest
* thread calling poll populates entries to the queue and the other threads
* will wait for it. Once the entries are populated, all the threads can poll
* their entries.
*
* @param <T> the queue entry type.
*/
static abstract class ConcurrentPoll<T> {
final MultipleBlockingQueue<T> queue;
ConcurrentPoll(MultipleBlockingQueue<T> queue) {
this.queue = queue;
}
T poll(final int i) throws IOException {
for(;;) {
synchronized(queue) {
final T polled = queue.poll(i);
if (polled != null) { // already populated; return polled item.
return polled;
}
if (isReady2Populate()) {
try {
populate();
return queue.poll(i);
} catch(IOException ioe) {
LOG.warn("Failed to populate, " + this, ioe);
throw ioe;
}
}
}
// sleep and then retry.
sleep(100, "poll");
}
}
boolean isReady2Populate() {
return queue.isEmpty();
}
abstract void populate() throws IOException;
}
private static void sleep(long ms, String op) throws InterruptedIOException {
try {
Thread.sleep(ms);
} catch(InterruptedException ie) {
throw DFSUtil.toInterruptedIOException(
"Sleep interrupted during " + op, ie);
}
}
private final Coordinator coordinator;
private final int index;
private volatile boolean failed;
private final ECSchema schema;
private final int cellSize;
StripedDataStreamer(HdfsFileStatus stat,
DFSClient dfsClient, String src,
@ -117,102 +56,59 @@ public class StripedDataStreamer extends DataStreamer {
byteArrayManage, favoredNodes);
this.index = index;
this.coordinator = coordinator;
this.schema = stat.getErasureCodingPolicy().getSchema();
this.cellSize = stat.getErasureCodingPolicy().getCellSize();
}
int getIndex() {
return index;
}
void setFailed(boolean failed) {
this.failed = failed;
}
boolean isFailed() {
return failed;
}
private boolean isParityStreamer() {
return index >= schema.getNumDataUnits();
boolean isHealthy() {
return !streamerClosed() && !getErrorState().hasInternalError();
}
@Override
protected void endBlock() {
if (!isParityStreamer()) {
coordinator.offerEndBlock(index, block);
}
coordinator.offerEndBlock(index, block);
super.endBlock();
}
@Override
int getNumBlockWriteRetry() {
return 0;
/**
* The upper level DFSStripedOutputStream will allocate the new block group.
* All the striped data streamer only needs to fetch from the queue, which
* should be already be ready.
*/
private LocatedBlock getFollowingBlock() throws IOException {
if (!this.isHealthy()) {
// No internal block for this streamer, maybe no enough healthy DN.
// Throw the exception which has been set by the StripedOutputStream.
this.getLastException().check(false);
}
return coordinator.getFollowingBlocks().poll(index);
}
@Override
LocatedBlock locateFollowingBlock(final DatanodeInfo[] excludedNodes)
throws IOException {
return new ConcurrentPoll<LocatedBlock>(coordinator.getFollowingBlocks()) {
@Override
boolean isReady2Populate() {
return super.isReady2Populate()
&& (block == null || coordinator.hasAllEndBlocks());
}
protected LocatedBlock nextBlockOutputStream() throws IOException {
boolean success;
LocatedBlock lb = getFollowingBlock();
block = lb.getBlock();
block.setNumBytes(0);
bytesSent = 0;
accessToken = lb.getBlockToken();
@Override
void populate() throws IOException {
getLastException().check(false);
DatanodeInfo[] nodes = lb.getLocations();
StorageType[] storageTypes = lb.getStorageTypes();
if (block != null) {
// set numByte for the previous block group
long bytes = 0;
for (int i = 0; i < schema.getNumDataUnits(); i++) {
final ExtendedBlock b = coordinator.takeEndBlock(i);
StripedBlockUtil.checkBlocks(index, block, i, b);
bytes += b.getNumBytes();
}
block.setNumBytes(bytes);
block.setBlockId(block.getBlockId() - index);
}
// Connect to the DataNode. If fail the internal error state will be set.
success = createBlockOutputStream(nodes, storageTypes, 0L, false);
if (LOG.isDebugEnabled()) {
LOG.debug("locateFollowingBlock: index=" + index + ", block=" + block);
}
final LocatedBlock lb = StripedDataStreamer.super.locateFollowingBlock(
excludedNodes);
if (lb.getLocations().length < schema.getNumDataUnits()) {
throw new IOException(
"Failed to get datablocks number of nodes from namenode: blockGroupSize= "
+ (schema.getNumDataUnits() + schema.getNumParityUnits())
+ ", blocks.length= " + lb.getLocations().length);
}
final LocatedBlock[] blocks =
StripedBlockUtil.parseStripedBlockGroup((LocatedStripedBlock) lb,
cellSize, schema.getNumDataUnits(), schema.getNumParityUnits());
for (int i = 0; i < blocks.length; i++) {
StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
if (si.isFailed()) {
continue; // skipping failed data streamer
}
if (blocks[i] == null) {
// Set exception and close streamer as there is no block locations
// found for the parity block.
LOG.warn("Failed to get block location for parity block, index="
+ i);
si.getLastException().set(
new IOException("Failed to get following block, i=" + i));
si.setFailed(true);
si.endBlock();
si.close(true);
} else {
queue.offer(i, blocks[i]);
}
}
}
}.poll(index);
if (!success) {
block = null;
final DatanodeInfo badNode = nodes[getErrorState().getBadNodeIndex()];
LOG.info("Excluding datanode " + badNode);
excludedNodes.put(badNode, badNode);
throw new IOException("Unable to create new block.");
}
return lb;
}
@VisibleForTesting
@ -221,119 +117,71 @@ public class StripedDataStreamer extends DataStreamer {
}
@Override
LocatedBlock updateBlockForPipeline() throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("updateBlockForPipeline(), " + this);
}
return new ConcurrentPoll<LocatedBlock>(coordinator.getNewBlocks()) {
@Override
void populate() throws IOException {
final ExtendedBlock bg = coordinator.getBlockGroup();
final LocatedBlock updated = callUpdateBlockForPipeline(bg);
final long newGS = updated.getBlock().getGenerationStamp();
final LocatedBlock[] updatedBlks = StripedBlockUtil
.parseStripedBlockGroup((LocatedStripedBlock) updated, cellSize,
schema.getNumDataUnits(), schema.getNumParityUnits());
for (int i = 0; i < schema.getNumDataUnits()
+ schema.getNumParityUnits(); i++) {
StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
if (si.isFailed()) {
continue; // skipping failed data streamer
}
final ExtendedBlock bi = si.getBlock();
if (bi != null) {
final LocatedBlock lb = new LocatedBlock(newBlock(bi, newGS),
null, null, null, -1, updated.isCorrupt(), null);
lb.setBlockToken(updatedBlks[i].getBlockToken());
queue.offer(i, lb);
} else {
final MultipleBlockingQueue<LocatedBlock> followingBlocks
= coordinator.getFollowingBlocks();
synchronized(followingBlocks) {
final LocatedBlock lb = followingBlocks.peek(i);
if (lb != null) {
lb.getBlock().setGenerationStamp(newGS);
si.getErrorState().reset();
continue;
}
}
protected void setupPipelineInternal(DatanodeInfo[] nodes,
StorageType[] nodeStorageTypes) throws IOException {
boolean success = false;
while (!success && !streamerClosed() && dfsClient.clientRunning) {
if (!handleRestartingDatanode()) {
return;
}
if (!handleBadDatanode()) {
// for striped streamer if it is datanode error then close the stream
// and return. no need to replace datanode
return;
}
//streamer i just have polled the block, sleep and retry.
sleep(100, "updateBlockForPipeline, " + this);
i--;
}
// get a new generation stamp and an access token
final LocatedBlock lb = coordinator.getNewBlocks().take(index);
long newGS = lb.getBlock().getGenerationStamp();
setAccessToken(lb.getBlockToken());
// set up the pipeline again with the remaining nodes. when a striped
// data streamer comes here, it must be in external error state.
assert getErrorState().hasExternalError();
success = createBlockOutputStream(nodes, nodeStorageTypes, newGS, true);
failPacket4Testing();
getErrorState().checkRestartingNodeDeadline(nodes);
// notify coordinator the result of createBlockOutputStream
synchronized (coordinator) {
if (!streamerClosed()) {
coordinator.updateStreamer(this, success);
coordinator.notify();
} else {
success = false;
}
}
}.poll(index);
if (success) {
// wait for results of other streamers
success = coordinator.takeStreamerUpdateResult(index);
if (success) {
// if all succeeded, update its block using the new GS
block = newBlock(block, newGS);
} else {
// otherwise close the block stream and restart the recovery process
closeStream();
}
} else {
// if fail, close the stream. The internal error state and last
// exception have already been set in createBlockOutputStream
// TODO: wait for restarting DataNodes during RollingUpgrade
closeStream();
setStreamerAsClosed();
}
} // while
}
@Override
ExtendedBlock updatePipeline(final long newGS) throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("updatePipeline(newGS=" + newGS + "), " + this);
void setExternalError() {
getErrorState().setExternalError();
synchronized (dataQueue) {
dataQueue.notifyAll();
}
return new ConcurrentPoll<ExtendedBlock>(coordinator.getUpdateBlocks()) {
@Override
void populate() throws IOException {
final MultipleBlockingQueue<LocatedBlock> followingBlocks
= coordinator.getFollowingBlocks();
final ExtendedBlock bg = coordinator.getBlockGroup();
final ExtendedBlock newBG = newBlock(bg, newGS);
final int n = schema.getNumDataUnits() + schema.getNumParityUnits();
final DatanodeInfo[] newNodes = new DatanodeInfo[n];
final String[] newStorageIDs = new String[n];
for (int i = 0; i < n; i++) {
final StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
DatanodeInfo[] nodes = si.getNodes();
String[] storageIDs = si.getStorageIDs();
if (nodes == null || storageIDs == null) {
synchronized(followingBlocks) {
final LocatedBlock lb = followingBlocks.peek(i);
if (lb != null) {
nodes = lb.getLocations();
storageIDs = lb.getStorageIDs();
}
}
}
if (nodes != null && storageIDs != null) {
newNodes[i] = nodes[0];
newStorageIDs[i] = storageIDs[0];
} else {
//streamer i just have polled the block, sleep and retry.
sleep(100, "updatePipeline, " + this);
i--;
}
}
final ExtendedBlock updated = callUpdatePipeline(bg, newBG, newNodes,
newStorageIDs);
for (int i = 0; i < n; i++) {
final StripedDataStreamer si = coordinator.getStripedDataStreamer(i);
final ExtendedBlock bi = si.getBlock();
if (bi != null) {
queue.offer(i, newBlock(bi, updated.getGenerationStamp()));
} else if (!si.isFailed()) {
synchronized(followingBlocks) {
final LocatedBlock lb = followingBlocks.peek(i);
if (lb != null) {
lb.getBlock().setGenerationStamp(newGS);
si.getErrorState().reset();
continue;
}
}
//streamer i just have polled the block, sleep and retry.
sleep(100, "updatePipeline, " + this);
i--;
}
}
}
}.poll(index);
}
@Override
public String toString() {
return "#" + index + ": " + (failed? "failed, ": "") + super.toString();
return "#" + index + ": " + (!isHealthy() ? "failed, ": "") + super.toString();
}
}

View File

@ -68,16 +68,28 @@ public class BlockUnderConstructionFeature {
/** Set expected locations */
public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets,
boolean isStriped) {
int numLocations = targets == null ? 0 : targets.length;
if (targets == null) {
return;
}
int numLocations = 0;
for (DatanodeStorageInfo target : targets) {
if (target != null) {
numLocations++;
}
}
this.replicas = new ReplicaUnderConstruction[numLocations];
for(int i = 0; i < numLocations; i++) {
// when creating a new striped block we simply sequentially assign block
// index to each storage
Block replicaBlock = isStriped ?
new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) :
block;
replicas[i] = new ReplicaUnderConstruction(replicaBlock, targets[i],
ReplicaState.RBW);
int offset = 0;
for(int i = 0; i < targets.length; i++) {
if (targets[i] != null) {
// when creating a new striped block we simply sequentially assign block
// index to each storage
Block replicaBlock = isStriped ?
new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) :
block;
replicas[offset++] = new ReplicaUnderConstruction(replicaBlock,
targets[i], ReplicaState.RBW);
}
}
}

View File

@ -513,6 +513,10 @@ public class DatanodeManager {
}
final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[datanodeID.length];
for(int i = 0; i < datanodeID.length; i++) {
if (datanodeID[i].equals(DatanodeID.EMPTY_DATANODE_ID)) {
storages[i] = null;
continue;
}
final DatanodeDescriptor dd = getDatanode(datanodeID[i]);
storages[i] = dd.getStorageInfo(storageIDs[i]);
}

View File

@ -925,22 +925,21 @@ public class StripedBlockUtil {
/**
* Check if the information such as IDs and generation stamps in block-i
* match block-j, where block-i and block-j are in the same group.
* match the block group.
*/
public static void checkBlocks(int j, ExtendedBlock blockj,
public static void checkBlocks(ExtendedBlock blockGroup,
int i, ExtendedBlock blocki) throws IOException {
if (!blocki.getBlockPoolId().equals(blockj.getBlockPoolId())) {
throw new IOException("Block pool IDs mismatched: block" + j + "="
+ blockj + ", block" + i + "=" + blocki);
if (!blocki.getBlockPoolId().equals(blockGroup.getBlockPoolId())) {
throw new IOException("Block pool IDs mismatched: block" + i + "="
+ blocki + ", expected block group=" + blockGroup);
}
if (blocki.getBlockId() - i != blockj.getBlockId() - j) {
throw new IOException("Block IDs mismatched: block" + j + "="
+ blockj + ", block" + i + "=" + blocki);
if (blocki.getBlockId() - i != blockGroup.getBlockId()) {
throw new IOException("Block IDs mismatched: block" + i + "="
+ blocki + ", expected block group=" + blockGroup);
}
if (blocki.getGenerationStamp() != blockj.getGenerationStamp()) {
throw new IOException("Generation stamps mismatched: block" + j + "="
+ blockj + ", block" + i + "=" + blocki);
if (blocki.getGenerationStamp() != blockGroup.getGenerationStamp()) {
throw new IOException("Generation stamps mismatched: block" + i + "="
+ blocki + ", expected block group=" + blockGroup);
}
}

View File

@ -1988,35 +1988,14 @@ public class DFSTestUtil {
*/
public static ExtendedBlock flushInternal(DFSStripedOutputStream out)
throws IOException {
out.flushInternal();
out.flushAllInternals();
return out.getBlock();
}
/**
* Verify that blocks in striped block group are on different nodes, and every
* internal blocks exists.
*/
public static void verifyLocatedStripedBlocks(LocatedBlocks lbs,
int groupSize) {
for (LocatedBlock lb : lbs.getLocatedBlocks()) {
assert lb instanceof LocatedStripedBlock;
HashSet<DatanodeInfo> locs = new HashSet<>();
for (DatanodeInfo datanodeInfo : lb.getLocations()) {
locs.add(datanodeInfo);
}
assertEquals(groupSize, lb.getLocations().length);
assertEquals(groupSize, locs.size());
// verify that every internal blocks exists
int[] blockIndices = ((LocatedStripedBlock) lb).getBlockIndices();
assertEquals(groupSize, blockIndices.length);
HashSet<Integer> found = new HashSet<>();
for (int index : blockIndices) {
assert index >=0;
found.add(index);
}
assertEquals(groupSize, found.size());
}
public static ExtendedBlock flushBuffer(DFSStripedOutputStream out)
throws IOException {
out.flush();
return out.getBlock();
}
public static void waitForMetric(final JMXGet jmx, final String metricName, final int expectedValue)

View File

@ -20,23 +20,35 @@ package org.apache.hadoop.hdfs;
import com.google.common.base.Joiner;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
import org.apache.hadoop.hdfs.web.ByteRangeInputStream;
import org.apache.hadoop.io.erasurecode.CodecUtil;
import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
import org.junit.Assert;
import java.io.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Random;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import static org.junit.Assert.assertEquals;
public class StripedFileTestUtil {
public static final Log LOG = LogFactory.getLog(StripedFileTestUtil.class);
/*
@ -50,8 +62,8 @@ public class StripedFileTestUtil {
static final int stripesPerBlock = 4;
static final int blockSize = BLOCK_STRIPED_CELL_SIZE * stripesPerBlock;
static final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS + 2;
static final int BLOCK_GROUP_SIZE = blockSize * NUM_DATA_BLOCKS;
static final Random random = new Random();
static byte[] generateBytes(int cnt) {
byte[] bytes = new byte[cnt];
@ -61,6 +73,11 @@ public class StripedFileTestUtil {
return bytes;
}
static byte getByte(long pos) {
final int mod = 29;
return (byte) (pos % mod + 1);
}
static int readAll(FSDataInputStream in, byte[] buf) throws IOException {
int readLen = 0;
int ret;
@ -71,15 +88,10 @@ public class StripedFileTestUtil {
return readLen;
}
static byte getByte(long pos) {
final int mod = 29;
return (byte) (pos % mod + 1);
}
static void verifyLength(FileSystem fs, Path srcPath, int fileLength)
throws IOException {
FileStatus status = fs.getFileStatus(srcPath);
Assert.assertEquals("File length should be the same", fileLength, status.getLen());
assertEquals("File length should be the same", fileLength, status.getLen());
}
static void verifyPread(FileSystem fs, Path srcPath, int fileLength,
@ -101,9 +113,7 @@ public class StripedFileTestUtil {
offset += target;
}
for (int i = 0; i < fileLength - startOffset; i++) {
Assert.assertEquals("Byte at " + (startOffset + i) + " is different, "
+ "the startOffset is " + startOffset,
expected[startOffset + i], result[i]);
assertEquals("Byte at " + (startOffset + i) + " is different, " + "the startOffset is " + startOffset, expected[startOffset + i], result[i]);
}
}
}
@ -119,8 +129,7 @@ public class StripedFileTestUtil {
System.arraycopy(buf, 0, result, readLen, ret);
readLen += ret;
}
Assert.assertEquals("The length of file should be the same to write size",
fileLength, readLen);
assertEquals("The length of file should be the same to write size", fileLength, readLen);
Assert.assertArrayEquals(expected, result);
}
}
@ -137,8 +146,7 @@ public class StripedFileTestUtil {
result.put(buf);
buf.clear();
}
Assert.assertEquals("The length of file should be the same to write size",
fileLength, readLen);
assertEquals("The length of file should be the same to write size", fileLength, readLen);
Assert.assertArrayEquals(expected, result.array());
}
}
@ -199,10 +207,9 @@ public class StripedFileTestUtil {
fsdis.seek(pos);
byte[] buf = new byte[writeBytes];
int readLen = StripedFileTestUtil.readAll(fsdis, buf);
Assert.assertEquals(readLen, writeBytes - pos);
assertEquals(readLen, writeBytes - pos);
for (int i = 0; i < readLen; i++) {
Assert.assertEquals("Byte at " + i + " should be the same",
StripedFileTestUtil.getByte(pos + i), buf[i]);
assertEquals("Byte at " + i + " should be the same", StripedFileTestUtil.getByte(pos + i), buf[i]);
}
}
@ -210,6 +217,7 @@ public class StripedFileTestUtil {
final int dnIndex, final AtomicInteger pos) {
final StripedDataStreamer s = out.getStripedDataStreamer(dnIndex);
final DatanodeInfo datanode = getDatanodes(s);
assert datanode != null;
LOG.info("killDatanode " + dnIndex + ": " + datanode + ", pos=" + pos);
cluster.stopDataNode(datanode.getXferAddr());
}
@ -218,7 +226,7 @@ public class StripedFileTestUtil {
for(;;) {
final DatanodeInfo[] datanodes = streamer.getNodes();
if (datanodes != null) {
Assert.assertEquals(1, datanodes.length);
assertEquals(1, datanodes.length);
Assert.assertNotNull(datanodes[0]);
return datanodes[0];
}
@ -287,7 +295,6 @@ public class StripedFileTestUtil {
* @param min minimum of the range
* @param max maximum of the range
* @param n number to be generated
* @return
*/
public static int[] randomArray(int min, int max, int n){
if (n > (max - min + 1) || max < min || min < 0 || max < 0) {
@ -315,4 +322,170 @@ public class StripedFileTestUtil {
}
return result;
}
/**
* Verify that blocks in striped block group are on different nodes, and every
* internal blocks exists.
*/
public static void verifyLocatedStripedBlocks(LocatedBlocks lbs, int groupSize) {
for (LocatedBlock lb : lbs.getLocatedBlocks()) {
assert lb instanceof LocatedStripedBlock;
HashSet<DatanodeInfo> locs = new HashSet<>();
Collections.addAll(locs, lb.getLocations());
assertEquals(groupSize, lb.getLocations().length);
assertEquals(groupSize, locs.size());
// verify that every internal blocks exists
int[] blockIndices = ((LocatedStripedBlock) lb).getBlockIndices();
assertEquals(groupSize, blockIndices.length);
HashSet<Integer> found = new HashSet<>();
for (int index : blockIndices) {
assert index >=0;
found.add(index);
}
assertEquals(groupSize, found.size());
}
}
static void checkData(DistributedFileSystem dfs, Path srcPath, int length,
int[] killedDnIndex, long oldGS) throws IOException {
StripedFileTestUtil.verifyLength(dfs, srcPath, length);
Arrays.sort(killedDnIndex);
List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(srcPath.toString(), 0L,
Long.MAX_VALUE);
int expectedNumGroup = 0;
if (length > 0) {
expectedNumGroup = (length - 1) / BLOCK_GROUP_SIZE + 1;
}
assertEquals(expectedNumGroup, lbs.getLocatedBlocks().size());
for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) {
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(
(LocatedStripedBlock) firstBlock, BLOCK_STRIPED_CELL_SIZE,
NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
blockGroupList.add(Arrays.asList(blocks));
}
// test each block group
for (int group = 0; group < blockGroupList.size(); group++) {
final boolean isLastGroup = group == blockGroupList.size() - 1;
final int groupSize = !isLastGroup? BLOCK_GROUP_SIZE
: length - (blockGroupList.size() - 1)*BLOCK_GROUP_SIZE;
final int numCellInGroup = (groupSize - 1)/BLOCK_STRIPED_CELL_SIZE + 1;
final int lastCellIndex = (numCellInGroup - 1) % NUM_DATA_BLOCKS;
final int lastCellSize = groupSize - (numCellInGroup - 1)*BLOCK_STRIPED_CELL_SIZE;
//get the data of this block
List<LocatedBlock> blockList = blockGroupList.get(group);
byte[][] dataBlockBytes = new byte[NUM_DATA_BLOCKS][];
byte[][] parityBlockBytes = new byte[NUM_PARITY_BLOCKS][];
// for each block, use BlockReader to read data
for (int i = 0; i < blockList.size(); i++) {
final int j = i >= NUM_DATA_BLOCKS? 0: i;
final int numCellInBlock = (numCellInGroup - 1)/NUM_DATA_BLOCKS
+ (j <= lastCellIndex? 1: 0);
final int blockSize = numCellInBlock*BLOCK_STRIPED_CELL_SIZE
+ (isLastGroup && j == lastCellIndex? lastCellSize - BLOCK_STRIPED_CELL_SIZE: 0);
final byte[] blockBytes = new byte[blockSize];
if (i < NUM_DATA_BLOCKS) {
dataBlockBytes[i] = blockBytes;
} else {
parityBlockBytes[i - NUM_DATA_BLOCKS] = blockBytes;
}
final LocatedBlock lb = blockList.get(i);
LOG.info("i,j=" + i + ", " + j + ", numCellInBlock=" + numCellInBlock
+ ", blockSize=" + blockSize + ", lb=" + lb);
if (lb == null) {
continue;
}
final ExtendedBlock block = lb.getBlock();
assertEquals(blockSize, block.getNumBytes());
if (block.getNumBytes() == 0) {
continue;
}
if (Arrays.binarySearch(killedDnIndex, i) < 0) {
final BlockReader blockReader = BlockReaderTestUtil.getBlockReader(
dfs, lb, 0, block.getNumBytes());
blockReader.readAll(blockBytes, 0, (int) block.getNumBytes());
blockReader.close();
}
}
// check data
final int groupPosInFile = group*BLOCK_GROUP_SIZE;
for (int i = 0; i < dataBlockBytes.length; i++) {
boolean killed = false;
if (Arrays.binarySearch(killedDnIndex, i) >= 0){
killed = true;
}
final byte[] actual = dataBlockBytes[i];
for (int posInBlk = 0; posInBlk < actual.length; posInBlk++) {
final long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(
BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, posInBlk, i) + groupPosInFile;
Assert.assertTrue(posInFile < length);
final byte expected = getByte(posInFile);
if (killed) {
actual[posInBlk] = expected;
} else {
if(expected != actual[posInBlk]){
String s = "expected=" + expected + " but actual=" + actual[posInBlk]
+ ", posInFile=" + posInFile + ", posInBlk=" + posInBlk
+ ". group=" + group + ", i=" + i;
Assert.fail(s);
}
}
}
}
// check parity
verifyParityBlocks(dfs.getConf(), lbs.getLocatedBlocks().get(group)
.getBlockSize(),
BLOCK_STRIPED_CELL_SIZE, dataBlockBytes, parityBlockBytes, killedDnIndex);
}
}
static void verifyParityBlocks(Configuration conf, final long size, final int cellSize,
byte[][] dataBytes, byte[][] parityBytes, int[] killedDnIndex) {
Arrays.sort(killedDnIndex);
// verify the parity blocks
int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength(
size, cellSize, dataBytes.length, dataBytes.length);
final byte[][] expectedParityBytes = new byte[parityBytes.length][];
for (int i = 0; i < parityBytes.length; i++) {
expectedParityBytes[i] = new byte[parityBlkSize];
}
for (int i = 0; i < dataBytes.length; i++) {
if (dataBytes[i] == null) {
dataBytes[i] = new byte[dataBytes[0].length];
} else if (dataBytes[i].length < dataBytes[0].length) {
final byte[] tmp = dataBytes[i];
dataBytes[i] = new byte[dataBytes[0].length];
System.arraycopy(tmp, 0, dataBytes[i], 0, tmp.length);
}
}
final RawErasureEncoder encoder =
CodecUtil.createRSRawEncoder(conf, dataBytes.length, parityBytes.length);
encoder.encode(dataBytes, expectedParityBytes);
for (int i = 0; i < parityBytes.length; i++) {
if (Arrays.binarySearch(killedDnIndex, dataBytes.length + i) < 0){
Assert.assertArrayEquals("i=" + i + ", killedDnIndex=" + Arrays.toString(killedDnIndex),
expectedParityBytes[i], parityBytes[i]);
}
}
}
}

View File

@ -18,26 +18,14 @@
package org.apache.hadoop.hdfs;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
import org.apache.hadoop.io.erasurecode.CodecUtil;
import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Level;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@ -154,141 +142,15 @@ public class TestDFSStripedOutputStream {
+ cellSize + 123);
}
private byte[] generateBytes(int cnt) {
byte[] bytes = new byte[cnt];
for (int i = 0; i < cnt; i++) {
bytes[i] = getByte(i);
}
return bytes;
}
private byte getByte(long pos) {
int mod = 29;
return (byte) (pos % mod + 1);
}
private void testOneFile(String src, int writeBytes) throws Exception {
src += "_" + writeBytes;
Path testPath = new Path(src);
byte[] bytes = generateBytes(writeBytes);
byte[] bytes = StripedFileTestUtil.generateBytes(writeBytes);
DFSTestUtil.writeFile(fs, testPath, new String(bytes));
StripedFileTestUtil.waitBlockGroupsReported(fs, src);
// check file length
FileStatus status = fs.getFileStatus(testPath);
Assert.assertEquals(writeBytes, status.getLen());
checkData(src, writeBytes);
}
void checkData(String src, int writeBytes) throws IOException {
List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0L);
for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) {
Assert.assertTrue(firstBlock instanceof LocatedStripedBlock);
LocatedBlock[] blocks = StripedBlockUtil.
parseStripedBlockGroup((LocatedStripedBlock) firstBlock,
cellSize, dataBlocks, parityBlocks);
List<LocatedBlock> oneGroup = Arrays.asList(blocks);
blockGroupList.add(oneGroup);
}
// test each block group
for (int group = 0; group < blockGroupList.size(); group++) {
//get the data of this block
List<LocatedBlock> blockList = blockGroupList.get(group);
byte[][] dataBlockBytes = new byte[dataBlocks][];
byte[][] parityBlockBytes = new byte[parityBlocks][];
// for each block, use BlockReader to read data
for (int i = 0; i < blockList.size(); i++) {
LocatedBlock lblock = blockList.get(i);
if (lblock == null) {
continue;
}
ExtendedBlock block = lblock.getBlock();
byte[] blockBytes = new byte[(int)block.getNumBytes()];
if (i < dataBlocks) {
dataBlockBytes[i] = blockBytes;
} else {
parityBlockBytes[i - dataBlocks] = blockBytes;
}
if (block.getNumBytes() == 0) {
continue;
}
final BlockReader blockReader = BlockReaderTestUtil.getBlockReader(
fs, lblock, 0, block.getNumBytes());
blockReader.readAll(blockBytes, 0, (int) block.getNumBytes());
blockReader.close();
}
// check if we write the data correctly
for (int blkIdxInGroup = 0; blkIdxInGroup < dataBlockBytes.length;
blkIdxInGroup++) {
final byte[] actualBlkBytes = dataBlockBytes[blkIdxInGroup];
if (actualBlkBytes == null) {
continue;
}
for (int posInBlk = 0; posInBlk < actualBlkBytes.length; posInBlk++) {
// calculate the position of this byte in the file
long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(cellSize,
dataBlocks, posInBlk, blkIdxInGroup) +
group * blockSize * dataBlocks;
Assert.assertTrue(posInFile < writeBytes);
final byte expected = getByte(posInFile);
String s = "Unexpected byte " + actualBlkBytes[posInBlk]
+ ", expect " + expected
+ ". Block group index is " + group
+ ", stripe index is " + posInBlk / cellSize
+ ", cell index is " + blkIdxInGroup
+ ", byte index is " + posInBlk % cellSize;
Assert.assertEquals(s, expected, actualBlkBytes[posInBlk]);
}
}
verifyParity(lbs.getLocatedBlocks().get(group).getBlockSize(),
cellSize, dataBlockBytes, parityBlockBytes);
}
}
void verifyParity(final long size, final int cellSize,
byte[][] dataBytes, byte[][] parityBytes) {
verifyParity(conf, size, cellSize, dataBytes, parityBytes, -1);
}
static void verifyParity(Configuration conf, final long size,
final int cellSize, byte[][] dataBytes,
byte[][] parityBytes, int killedDnIndex) {
// verify the parity blocks
int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength(
size, cellSize, dataBytes.length, dataBytes.length);
final byte[][] expectedParityBytes = new byte[parityBytes.length][];
for (int i = 0; i < parityBytes.length; i++) {
expectedParityBytes[i] = new byte[parityBlkSize];
}
for (int i = 0; i < dataBytes.length; i++) {
if (dataBytes[i] == null) {
dataBytes[i] = new byte[dataBytes[0].length];
} else if (dataBytes[i].length < dataBytes[0].length) {
final byte[] tmp = dataBytes[i];
dataBytes[i] = new byte[dataBytes[0].length];
System.arraycopy(tmp, 0, dataBytes[i], 0, tmp.length);
}
}
final RawErasureEncoder encoder =
CodecUtil.createRSRawEncoder(conf,
dataBytes.length, parityBytes.length);
encoder.encode(dataBytes, expectedParityBytes);
for (int i = 0; i < parityBytes.length; i++) {
if (i != killedDnIndex) {
Assert.assertArrayEquals("i=" + i + ", killedDnIndex=" + killedDnIndex,
expectedParityBytes[i], parityBytes[i]);
}
}
StripedFileTestUtil.checkData(fs, testPath, writeBytes,
new int[]{}, 0);
}
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hdfs;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
@ -30,23 +31,18 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.SecurityTestUtil;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.util.StripedBlockUtil;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.StringUtils;
@ -74,6 +70,7 @@ public class TestDFSStripedOutputStreamWithFailure {
private static final int FLUSH_POS
= 9*DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1;
static {
System.out.println("NUM_DATA_BLOCKS = " + NUM_DATA_BLOCKS);
System.out.println("NUM_PARITY_BLOCKS= " + NUM_PARITY_BLOCKS);
@ -101,6 +98,32 @@ public class TestDFSStripedOutputStreamWithFailure {
return lengths;
}
private static final int[][] dnIndexSuite = {
{0, 1},
{0, 5},
{0, 6},
{0, 8},
{1, 5},
{1, 6},
{6, 8},
{0, 1, 2},
{3, 4, 5},
{0, 1, 6},
{0, 5, 6},
{0, 5, 8},
{0, 6, 7},
{5, 6, 7},
{6, 7, 8},
};
private int[] getKillPositions(int fileLen, int num) {
int[] positions = new int[num];
for (int i = 0; i < num; i++) {
positions[i] = fileLen * (i + 1) / (num + 1);
}
return positions;
}
private static final List<Integer> LENGTHS = newLengths();
static int getLength(int i) {
@ -127,41 +150,25 @@ public class TestDFSStripedOutputStreamWithFailure {
}
}
private static byte getByte(long pos) {
return (byte)pos;
}
private HdfsConfiguration newHdfsConfiguration() {
final HdfsConfiguration conf = new HdfsConfiguration();
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
conf.setLong(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L);
conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L);
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
return conf;
}
void runTest(final int length) {
final HdfsConfiguration conf = newHdfsConfiguration();
for (int dn = 0; dn < 9; dn++) {
try {
setup(conf);
runTest(length, dn, false, conf);
} catch (Exception e) {
final String err = "failed, dn=" + dn + ", length=" + length
+ StringUtils.stringifyException(e);
LOG.error(err);
Assert.fail(err);
} finally {
tearDown();
}
}
}
@Test(timeout=240000)
public void testDatanodeFailure56() throws Exception {
runTest(getLength(56));
}
@Test(timeout=240000)
public void testMultipleDatanodeFailure56() throws Exception {
runTestWithMultipleFailure(getLength(56));
}
@Test(timeout=240000)
public void testBlockTokenExpired() throws Exception {
final int length = NUM_DATA_BLOCKS * (BLOCK_SIZE - CELL_SIZE);
@ -174,7 +181,7 @@ public class TestDFSStripedOutputStreamWithFailure {
for (int dn = 0; dn < 9; dn += 2) {
try {
setup(conf);
runTest(length, dn, true, conf);
runTest(length, new int[]{length/2}, new int[]{dn}, true);
} catch (Exception e) {
LOG.error("failed, dn=" + dn + ", length=" + length);
throw e;
@ -214,22 +221,8 @@ public class TestDFSStripedOutputStreamWithFailure {
Assert.fail("Failed to validate available dns against blkGroupSize");
} catch (IOException ioe) {
// expected
GenericTestUtils.assertExceptionContains("Failed: the number of "
+ "remaining blocks = 5 < the number of data blocks = 6", ioe);
DFSStripedOutputStream dfsout = (DFSStripedOutputStream) out
.getWrappedStream();
// get leading streamer and verify the last exception
StripedDataStreamer datastreamer = dfsout.getStripedDataStreamer(0);
try {
datastreamer.getLastException().check(true);
Assert.fail("Failed to validate available dns against blkGroupSize");
} catch (IOException le) {
GenericTestUtils.assertExceptionContains(
"Failed to get datablocks number of nodes from"
+ " namenode: blockGroupSize= 9, blocks.length= "
+ numDatanodes, le);
}
GenericTestUtils.assertExceptionContains("Failed to get 6 nodes from" +
" namenode: blockGroupSize= 9, blocks.length= 5", ioe);
}
} finally {
tearDown();
@ -258,42 +251,73 @@ public class TestDFSStripedOutputStreamWithFailure {
int fileLength = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE - 1000;
final byte[] expected = StripedFileTestUtil.generateBytes(fileLength);
DFSTestUtil.writeFile(dfs, srcPath, new String(expected));
LOG.info("writing finished. Seek and read the file to verify.");
StripedFileTestUtil.verifySeek(dfs, srcPath, fileLength);
} finally {
tearDown();
}
}
private void runTest(final int length, final int dnIndex,
final boolean tokenExpire, final HdfsConfiguration conf) {
try {
runTest(length, length/2, dnIndex, tokenExpire, conf);
} catch(Exception e) {
LOG.info("FAILED", e);
Assert.fail(StringUtils.stringifyException(e));
void runTest(final int length) {
final HdfsConfiguration conf = newHdfsConfiguration();
for (int dn = 0; dn < 9; dn++) {
try {
setup(conf);
runTest(length, new int[]{length/2}, new int[]{dn}, false);
} catch (Throwable e) {
final String err = "failed, dn=" + dn + ", length=" + length
+ StringUtils.stringifyException(e);
LOG.error(err);
Assert.fail(err);
} finally {
tearDown();
}
}
}
private void runTest(final int length, final int killPos,
final int dnIndex, final boolean tokenExpire,
final HdfsConfiguration conf) throws Exception {
if (killPos <= FLUSH_POS) {
LOG.warn("killPos=" + killPos + " <= FLUSH_POS=" + FLUSH_POS
+ ", length=" + length + ", dnIndex=" + dnIndex);
void runTestWithMultipleFailure(final int length) throws Exception {
final HdfsConfiguration conf = newHdfsConfiguration();
for(int i=0;i<dnIndexSuite.length;i++){
int[] dnIndex = dnIndexSuite[i];
int[] killPos = getKillPositions(length, dnIndex.length);
try {
setup(conf);
runTest(length, killPos, dnIndex, false);
} catch (Throwable e) {
final String err = "failed, killPos=" + Arrays.toString(killPos)
+ ", dnIndex=" + Arrays.toString(dnIndex) + ", length=" + length;
LOG.error(err);
throw e;
} finally {
tearDown();
}
}
}
/**
* runTest implementation
* @param length file length
* @param killPos killing positions in ascending order
* @param dnIndex DN index to kill when meets killing positions
* @param tokenExpire wait token to expire when kill a DN
* @throws Exception
*/
private void runTest(final int length, final int[] killPos,
final int[] dnIndex, final boolean tokenExpire) throws Exception {
if (killPos[0] <= FLUSH_POS) {
LOG.warn("killPos=" + Arrays.toString(killPos) + " <= FLUSH_POS=" + FLUSH_POS
+ ", length=" + length + ", dnIndex=" + Arrays.toString(dnIndex));
return; //skip test
}
Preconditions.checkArgument(length > killPos,
"length=%s <= killPos=%s", length, killPos);
Preconditions.checkArgument(length > killPos[0], "length=%s <= killPos=%s",
length, killPos);
Preconditions.checkArgument(killPos.length == dnIndex.length);
// start a datanode now, will kill one later
cluster.startDataNodes(conf, 1, true, null, null);
cluster.waitActive();
final Path p = new Path(dir, "dn" + dnIndex + "len" + length + "kill" + killPos);
final Path p = new Path(dir, "dn" + Arrays.toString(dnIndex)
+ "len" + length + "kill" + Arrays.toString(killPos));
final String fullPath = p.toString();
LOG.info("fullPath=" + fullPath);
if (tokenExpire) {
final NameNode nn = cluster.getNameNode();
final BlockManager bm = nn.getNamesystem().getBlockManager();
@ -308,50 +332,56 @@ public class TestDFSStripedOutputStreamWithFailure {
final DFSStripedOutputStream stripedOut
= (DFSStripedOutputStream)out.getWrappedStream();
long oldGS = -1;
boolean killed = false;
long firstGS = -1; // first GS of this block group which never proceeds blockRecovery
long oldGS = -1; // the old GS before bumping
int numKilled=0;
for(; pos.get() < length; ) {
final int i = pos.getAndIncrement();
if (i == killPos) {
if (numKilled < killPos.length && i == killPos[numKilled]) {
assertTrue(firstGS != -1);
final long gs = getGenerationStamp(stripedOut);
Assert.assertTrue(oldGS != -1);
Assert.assertEquals(oldGS, gs);
if (numKilled == 0) {
assertEquals(firstGS, gs);
} else {
//TODO: implement hflush/hsync and verify gs strict greater than oldGS
assertTrue(gs >= oldGS);
}
oldGS = gs;
if (tokenExpire) {
DFSTestUtil.flushInternal(stripedOut);
waitTokenExpires(out);
}
killDatanode(cluster, stripedOut, dnIndex, pos);
killed = true;
killDatanode(cluster, stripedOut, dnIndex[numKilled], pos);
numKilled++;
}
write(out, i);
if (i == FLUSH_POS) {
oldGS = getGenerationStamp(stripedOut);
if (i % BLOCK_GROUP_SIZE == FLUSH_POS) {
firstGS = getGenerationStamp(stripedOut);
oldGS = firstGS;
}
}
out.close();
assertEquals(dnIndex.length, numKilled);
short expectedReported = StripedFileTestUtil.getRealTotalBlockNum(length);
if (length > dnIndex * CELL_SIZE || dnIndex >= NUM_DATA_BLOCKS) {
expectedReported--;
for(int idx :dnIndex) {
if (length > idx * CELL_SIZE || idx >= NUM_DATA_BLOCKS) {
expectedReported--;
}
}
DFSTestUtil.waitReplication(dfs, p, expectedReported);
Assert.assertTrue(killed);
// check file length
final FileStatus status = dfs.getFileStatus(p);
Assert.assertEquals(length, status.getLen());
checkData(dfs, fullPath, length, dnIndex, oldGS);
cluster.triggerBlockReports();
StripedFileTestUtil.checkData(dfs, p, length, dnIndex, oldGS);
}
static void write(FSDataOutputStream out, int i) throws IOException {
try {
out.write(getByte(i));
out.write(StripedFileTestUtil.getByte(i));
} catch(IOException ioe) {
throw new IOException("Failed at i=" + i, ioe);
}
@ -359,10 +389,10 @@ public class TestDFSStripedOutputStreamWithFailure {
static long getGenerationStamp(DFSStripedOutputStream out)
throws IOException {
DFSTestUtil.flushBuffer(out);
final long gs = DFSTestUtil.flushInternal(out).getGenerationStamp();
LOG.info("getGenerationStamp returns " + gs);
return gs;
}
static DatanodeInfo getDatanodes(StripedDataStreamer streamer) {
@ -399,106 +429,6 @@ public class TestDFSStripedOutputStreamWithFailure {
cluster.stopDataNode(datanode.getXferAddr());
}
static void checkData(DistributedFileSystem dfs, String src, int length,
int killedDnIndex, long oldGS) throws IOException {
List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(src, 0L);
final int expectedNumGroup = (length - 1)/BLOCK_GROUP_SIZE + 1;
Assert.assertEquals(expectedNumGroup, lbs.getLocatedBlocks().size());
for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) {
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(
(LocatedStripedBlock) firstBlock,
CELL_SIZE, NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
blockGroupList.add(Arrays.asList(blocks));
}
// test each block group
for (int group = 0; group < blockGroupList.size(); group++) {
final boolean isLastGroup = group == blockGroupList.size() - 1;
final int groupSize = !isLastGroup? BLOCK_GROUP_SIZE
: length - (blockGroupList.size() - 1)*BLOCK_GROUP_SIZE;
final int numCellInGroup = (groupSize - 1)/CELL_SIZE + 1;
final int lastCellIndex = (numCellInGroup - 1) % NUM_DATA_BLOCKS;
final int lastCellSize = groupSize - (numCellInGroup - 1)*CELL_SIZE;
//get the data of this block
List<LocatedBlock> blockList = blockGroupList.get(group);
byte[][] dataBlockBytes = new byte[NUM_DATA_BLOCKS][];
byte[][] parityBlockBytes = new byte[NUM_PARITY_BLOCKS][];
// for each block, use BlockReader to read data
for (int i = 0; i < blockList.size(); i++) {
final int j = i >= NUM_DATA_BLOCKS? 0: i;
final int numCellInBlock = (numCellInGroup - 1)/NUM_DATA_BLOCKS
+ (j <= lastCellIndex? 1: 0);
final int blockSize = numCellInBlock*CELL_SIZE
+ (isLastGroup && j == lastCellIndex? lastCellSize - CELL_SIZE: 0);
final byte[] blockBytes = new byte[blockSize];
if (i < NUM_DATA_BLOCKS) {
dataBlockBytes[i] = blockBytes;
} else {
parityBlockBytes[i - NUM_DATA_BLOCKS] = blockBytes;
}
final LocatedBlock lb = blockList.get(i);
LOG.info("i,j=" + i + ", " + j + ", numCellInBlock=" + numCellInBlock
+ ", blockSize=" + blockSize + ", lb=" + lb);
if (lb == null) {
continue;
}
final ExtendedBlock block = lb.getBlock();
Assert.assertEquals(blockSize, block.getNumBytes());
if (block.getNumBytes() == 0) {
continue;
}
if (i != killedDnIndex) {
final BlockReader blockReader = BlockReaderTestUtil.getBlockReader(
dfs, lb, 0, block.getNumBytes());
blockReader.readAll(blockBytes, 0, (int) block.getNumBytes());
blockReader.close();
}
}
// check data
final int groupPosInFile = group*BLOCK_GROUP_SIZE;
for (int i = 0; i < dataBlockBytes.length; i++) {
final byte[] actual = dataBlockBytes[i];
for (int posInBlk = 0; posInBlk < actual.length; posInBlk++) {
final long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG(
CELL_SIZE, NUM_DATA_BLOCKS, posInBlk, i) + groupPosInFile;
Assert.assertTrue(posInFile < length);
final byte expected = getByte(posInFile);
if (i == killedDnIndex) {
actual[posInBlk] = expected;
} else {
String s = "expected=" + expected + " but actual=" + actual[posInBlk]
+ ", posInFile=" + posInFile + ", posInBlk=" + posInBlk
+ ". group=" + group + ", i=" + i;
Assert.assertEquals(s, expected, actual[posInBlk]);
}
}
}
// check parity
TestDFSStripedOutputStream.verifyParity(dfs.getConf(),
lbs.getLocatedBlocks().get(group).getBlockSize(),
CELL_SIZE, dataBlockBytes, parityBlockBytes,
killedDnIndex - dataBlockBytes.length);
}
}
private void waitTokenExpires(FSDataOutputStream out) throws IOException {
Token<BlockTokenIdentifier> token = DFSTestUtil.getBlockToken(out);

View File

@ -23,6 +23,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Level;
import org.junit.Assert;
import org.junit.Ignore;
import org.junit.Test;
@ -39,6 +41,12 @@ public class TestWriteStripedFileWithFailure {
private static MiniDFSCluster cluster;
private static FileSystem fs;
private static Configuration conf = new HdfsConfiguration();
static {
GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL);
GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL);
}
private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS;
private final short parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS;
private final int smallFileLength = blockSize * dataBlocks - 123;

View File

@ -1745,7 +1745,7 @@ public class TestBalancer {
// verify locations of striped blocks
LocatedBlocks locatedBlocks = client.getBlockLocations(fileName, 0, fileLen);
DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
// add one datanode
String newRack = "/rack" + (++numOfRacks);
@ -1761,7 +1761,7 @@ public class TestBalancer {
// verify locations of striped blocks
locatedBlocks = client.getBlockLocations(fileName, 0, fileLen);
DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize);
} finally {
cluster.shutdown();
}

View File

@ -488,7 +488,7 @@ public class TestMover {
Assert.assertEquals(StorageType.DISK, type);
}
}
DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
dataBlocks + parityBlocks);
// start 5 more datanodes
@ -523,7 +523,7 @@ public class TestMover {
Assert.assertEquals(StorageType.ARCHIVE, type);
}
}
DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
dataBlocks + parityBlocks);
}finally{

View File

@ -26,7 +26,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.StripedFileTestUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
@ -42,7 +41,6 @@ import java.util.Arrays;
import java.util.List;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
public class TestAddOverReplicatedStripedBlocks {
@ -64,6 +62,7 @@ public class TestAddOverReplicatedStripedBlocks {
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
// disable block recovery
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0);
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1);
SimulatedFSDataset.setFactory(conf);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
cluster.waitActive();
@ -118,7 +117,7 @@ public class TestAddOverReplicatedStripedBlocks {
// verify that all internal blocks exists
lbs = cluster.getNameNodeRpc().getBlockLocations(
filePath.toString(), 0, fileLen);
DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE);
StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE);
}
@Test
@ -162,7 +161,7 @@ public class TestAddOverReplicatedStripedBlocks {
// verify that all internal blocks exists
lbs = cluster.getNameNodeRpc().getBlockLocations(
filePath.toString(), 0, fileLen);
DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1);
StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1);
}
@Test
@ -216,7 +215,7 @@ public class TestAddOverReplicatedStripedBlocks {
// verify that all internal blocks exists
lbs = cluster.getNameNodeRpc().getBlockLocations(
filePath.toString(), 0, fileLen);
DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE);
StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE);
}
@Test
@ -248,6 +247,7 @@ public class TestAddOverReplicatedStripedBlocks {
// update blocksMap
cluster.triggerBlockReports();
Thread.sleep(2000);
// add to invalidates
cluster.triggerHeartbeats();
// datanode delete block
@ -259,7 +259,7 @@ public class TestAddOverReplicatedStripedBlocks {
// we are left GROUP_SIZE - 1 blocks.
lbs = cluster.getNameNodeRpc().getBlockLocations(
filePath.toString(), 0, fileLen);
DFSTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1);
StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1);
}
}

View File

@ -73,6 +73,7 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
@ -736,7 +737,13 @@ public class TestRetryCacheWithHA {
DatanodeInfo[] newNodes = new DatanodeInfo[2];
newNodes[0] = nodes[0];
newNodes[1] = nodes[1];
String[] storageIDs = {"s0", "s1"};
final DatanodeManager dm = cluster.getNamesystem(0).getBlockManager()
.getDatanodeManager();
final String storageID1 = dm.getDatanode(newNodes[0]).getStorageInfos()[0]
.getStorageID();
final String storageID2 = dm.getDatanode(newNodes[1]).getStorageInfos()[0]
.getStorageID();
String[] storageIDs = {storageID1, storageID2};
client.getNamenode().updatePipeline(client.getClientName(), oldBlock,
newBlock, newNodes, storageIDs);