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