HDFS-2994. If lease soft limit is recovered successfully the append can fail. Contributed by Tao Luo.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1514500 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Konstantin Shvachko 2013-08-15 21:21:10 +00:00
parent 2fc7e14e39
commit 10ec8a248e
3 changed files with 54 additions and 2 deletions

View File

@ -342,6 +342,9 @@ Release 2.1.1-beta - UNRELEASED
HDFS-5099. Namenode#copyEditLogSegmentsToSharedDir should close
EditLogInputStreams upon finishing. (Chuan Liu via cnauroth)
HDFS-2994. If lease soft limit is recovered successfully
the append can fail. (Tao Luo via shv)
Release 2.1.0-beta - 2013-08-22
INCOMPATIBLE CHANGES

View File

@ -2153,10 +2153,15 @@ private LocatedBlock appendFileInternal(FSPermissionChecker pc, String src,
throw new FileNotFoundException("failed to append to non-existent file "
+ src + " on client " + clientMachine);
}
final INodeFile myFile = INodeFile.valueOf(inode, src, true);
INodeFile myFile = INodeFile.valueOf(inode, src, true);
// Opening an existing file for write - may need to recover lease.
recoverLeaseInternal(myFile, src, holder, clientMachine, false);
// recoverLeaseInternal may create a new InodeFile via
// finalizeINodeFileUnderConstruction so we need to refresh
// the referenced file.
myFile = INodeFile.valueOf(dir.getINode(src), src, true);
final DatanodeDescriptor clientNode =
blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
return prepareFileForWrite(src, myFile, holder, clientMachine, clientNode,

View File

@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hdfs;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.File;
@ -337,4 +338,47 @@ public void testAppendTwice() throws Exception {
cluster.shutdown();
}
}
/** Tests appending after soft-limit expires. */
@Test
public void testAppendAfterSoftLimit()
throws IOException, InterruptedException {
Configuration conf = new HdfsConfiguration();
conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
conf.setBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY, true);
//Set small soft-limit for lease
final long softLimit = 1L;
final long hardLimit = 9999999L;
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1)
.build();
cluster.setLeasePeriod(softLimit, hardLimit);
cluster.waitActive();
FileSystem fs = cluster.getFileSystem();
FileSystem fs2 = new DistributedFileSystem();
fs2.initialize(fs.getUri(), conf);
final Path testPath = new Path("/testAppendAfterSoftLimit");
final byte[] fileContents = AppendTestUtil.initBuffer(32);
// create a new file without closing
FSDataOutputStream out = fs.create(testPath);
out.write(fileContents);
//Wait for > soft-limit
Thread.sleep(250);
try {
FSDataOutputStream appendStream2 = fs2.append(testPath);
appendStream2.write(fileContents);
appendStream2.close();
assertEquals(fileContents.length, fs.getFileStatus(testPath).getLen());
} finally {
fs.close();
fs2.close();
cluster.shutdown();
}
}
}