HDFS-3179. Improve the exception message thrown by DataStreamer when it failed to add a datanode.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1324892 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2012-04-11 17:49:28 +00:00
parent cfafd8c29d
commit ed678e52ce
3 changed files with 64 additions and 4 deletions

View File

@ -367,6 +367,9 @@ Release 2.0.0 - UNRELEASED
HDFS-3249. Use ToolRunner.confirmPrompt in NameNode (todd)
HDFS-3179. Improve the exception message thrown by DataStreamer when
it failed to add a datanode. (szetszwo)
OPTIMIZATIONS
HDFS-3024. Improve performance of stringification in addStoredBlock (todd)

View File

@ -128,7 +128,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
private volatile boolean appendChunk = false; // appending to existing partial block
private long initialFileSize = 0; // at time of file open
private Progressable progress;
private short blockReplication; // replication factor of file
private final short blockReplication; // replication factor of file
private class Packet {
long seqno; // sequencenumber of buffer in block
@ -775,9 +775,13 @@ private void setHflush() {
private int findNewDatanode(final DatanodeInfo[] original
) throws IOException {
if (nodes.length != original.length + 1) {
throw new IOException("Failed to add a datanode:"
+ " nodes.length != original.length + 1, nodes="
+ Arrays.asList(nodes) + ", original=" + Arrays.asList(original));
throw new IOException("Failed to add a datanode. "
+ "User may turn off this feature by setting "
+ DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY
+ " in configuration, where the current policy is "
+ dfsClient.dtpReplaceDatanodeOnFailure
+ ". (Nodes: current=" + Arrays.asList(nodes)
+ ", original=" + Arrays.asList(original) + ")");
}
for(int i = 0; i < nodes.length; i++) {
int j = 0;

View File

@ -25,6 +25,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -234,4 +235,56 @@ void checkReplication() throws IOException {
Assert.assertEquals(REPLICATION, dfsout.getNumCurrentReplicas());
}
}
@Test
public void testAppend() throws Exception {
final Configuration conf = new HdfsConfiguration();
final short REPLICATION = (short)3;
Assert.assertEquals(ReplaceDatanodeOnFailure.DEFAULT, ReplaceDatanodeOnFailure.get(conf));
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf
).numDataNodes(1).build();
try {
final DistributedFileSystem fs = (DistributedFileSystem)cluster.getFileSystem();
final Path f = new Path(DIR, "testAppend");
{
LOG.info("create an empty file " + f);
fs.create(f, REPLICATION).close();
final FileStatus status = fs.getFileStatus(f);
Assert.assertEquals(REPLICATION, status.getReplication());
Assert.assertEquals(0L, status.getLen());
}
final byte[] bytes = new byte[1000];
{
LOG.info("append " + bytes.length + " bytes to " + f);
final FSDataOutputStream out = fs.append(f);
out.write(bytes);
out.close();
final FileStatus status = fs.getFileStatus(f);
Assert.assertEquals(REPLICATION, status.getReplication());
Assert.assertEquals(bytes.length, status.getLen());
}
{
LOG.info("append another " + bytes.length + " bytes to " + f);
try {
final FSDataOutputStream out = fs.append(f);
out.write(bytes);
out.close();
Assert.fail();
} catch(IOException ioe) {
LOG.info("This exception is expected", ioe);
}
}
} finally {
if (cluster != null) {cluster.shutdown();}
}
}
}