HDFS-8908. TestAppendSnapshotTruncate may fail with IOException: Failed to replace a bad datanode. (Tsz Wo Nicholas Sze via yliu)

This commit is contained in:
yliu 2015-08-19 21:55:43 +08:00
parent 22dc5fc209
commit 2da5aaab33
2 changed files with 11 additions and 5 deletions

View File

@ -1174,6 +1174,9 @@ Release 2.8.0 - UNRELEASED
HDFS-8891. HDFS concat should keep srcs order. (Yong Zhang via jing9)
HDFS-8908. TestAppendSnapshotTruncate may fail with IOException: Failed to
replace a bad datanode. (Tsz Wo Nicholas Sze via yliu)
Release 2.7.2 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -27,7 +27,6 @@ import java.io.RandomAccessFile;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.Callable;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean;
@ -42,6 +41,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.TestFileTruncate;
import org.apache.hadoop.test.GenericTestUtils;
@ -64,10 +64,10 @@ public class TestAppendSnapshotTruncate {
}
private static final Log LOG = LogFactory.getLog(TestAppendSnapshotTruncate.class);
private static final int BLOCK_SIZE = 1024;
private static final int DATANODE_NUM = 3;
private static final int DATANODE_NUM = 4;
private static final short REPLICATION = 3;
private static final int FILE_WORKER_NUM = 3;
private static final long TEST_TIME_SECOND = 10;
private static final int FILE_WORKER_NUM = 10;
private static final long TEST_TIME_SECOND = 20;
private static final long TEST_TIMEOUT_SECOND = TEST_TIME_SECOND + 60;
static final int SHORT_HEARTBEAT = 1;
@ -85,6 +85,7 @@ public class TestAppendSnapshotTruncate {
conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, SHORT_HEARTBEAT);
conf.setLong(
DFSConfigKeys.DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY, 1);
conf.setBoolean(ReplaceDatanodeOnFailure.BEST_EFFORT_KEY, true);
cluster = new MiniDFSCluster.Builder(conf)
.format(true)
.numDataNodes(DATANODE_NUM)
@ -476,7 +477,9 @@ public class TestAppendSnapshotTruncate {
}
void pause() {
Preconditions.checkState(state.compareAndSet(State.RUNNING, State.IDLE));
checkErrorState();
Preconditions.checkState(state.compareAndSet(State.RUNNING, State.IDLE),
"%s: state=%s != %s", name, state.get(), State.RUNNING);
}
void stop() throws InterruptedException {