From 7751a058f3f33c46bb670d8bf28c1aa69f85767f Mon Sep 17 00:00:00 2001 From: Konstantin Shvachko Date: Thu, 15 Aug 2013 21:24:16 +0000 Subject: [PATCH] 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/branches/branch-2@1514501 13f79535-47bb-0310-9956-ffa450edef68 --- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 ++ .../hdfs/server/namenode/FSNamesystem.java | 9 +++- .../apache/hadoop/hdfs/TestFileAppend.java | 44 +++++++++++++++++++ 3 files changed, 54 insertions(+), 2 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 8303c3c8001..63a6c36bcc8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -116,6 +116,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 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 6a996df6571..7f774d28b73 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -2136,10 +2136,15 @@ public class FSNamesystem implements Namesystem, FSClusterStats, 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, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java index 54ff9036b91..e4015944692 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java @@ -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 class TestFileAppend{ 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(); + } + } + }