You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by su...@apache.org on 2011/09/03 00:07:16 UTC

svn commit: r1164737 - in /hadoop/common/branches/branch-0.20-security: CHANGES.txt src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java src/test/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java

Author: suresh
Date: Fri Sep  2 22:07:15 2011
New Revision: 1164737

URL: http://svn.apache.org/viewvc?rev=1164737&view=rev
Log:
Port from 0.20-append - HDFS-1204. Lease expiration should recover single files, not entire lease holder. Contributed by Sam Rash.


Added:
    hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
Modified:
    hadoop/common/branches/branch-0.20-security/CHANGES.txt
    hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

Modified: hadoop/common/branches/branch-0.20-security/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/CHANGES.txt?rev=1164737&r1=1164736&r2=1164737&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.20-security/CHANGES.txt Fri Sep  2 22:07:15 2011
@@ -72,6 +72,9 @@ Release 0.20.205.0 - unreleased
     HDFS-1141. completeFile does not check lease ownership.
     (Todd Lipcon via dhruba)
 
+    HDFS-1204. Lease expiration should recover single files, 
+    not entire lease holder (Sam Rash via dhruba)
+
   IMPROVEMENTS
 
     MAPREDUCE-2187. Reporter sends progress during sort/merge. (Anupam Seth via

Modified: hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java?rev=1164737&r1=1164736&r2=1164737&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java (original)
+++ hadoop/common/branches/branch-0.20-security/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java Fri Sep  2 22:07:15 2011
@@ -374,7 +374,7 @@ public class LeaseManager {
   }
 
   /** Check the leases beginning from the oldest. */
-  private synchronized void checkLeases() {
+  synchronized void checkLeases() {
     for(; sortedLeases.size() > 0; ) {
       final Lease oldest = sortedLeases.first();
       if (!oldest.expiredHardLimit()) {
@@ -392,7 +392,7 @@ public class LeaseManager {
       oldest.getPaths().toArray(leasePaths);
       for(String p : leasePaths) {
         try {
-          fsnamesystem.internalReleaseLease(oldest, p);
+          fsnamesystem.internalReleaseLeaseOne(oldest, p);
         } catch (IOException e) {
           LOG.error("Cannot release the path "+p+" in the lease "+oldest, e);
           removing.add(p);

Added: hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java?rev=1164737&view=auto
==============================================================================
--- hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java (added)
+++ hadoop/common/branches/branch-0.20-security/src/test/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java Fri Sep  2 22:07:15 2011
@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import junit.framework.TestCase;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.util.Daemon;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+
+import static org.mockito.Matchers.anyObject;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
+
+public class TestLeaseManager extends TestCase {
+  public static final Log LOG = LogFactory.getLog(TestLeaseManager.class);
+
+  /*
+   * test case: two leases are added for a singler holder, should use
+   * the internalReleaseOne method
+   */
+  public void testMultiPathLeaseRecovery()
+    throws IOException, InterruptedException {
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, 3, true, null);
+    NameNode namenode = cluster.getNameNode();
+    FSNamesystem spyNamesystem = spy(namenode.getNamesystem());
+    LeaseManager leaseManager = new LeaseManager(spyNamesystem);
+    
+    spyNamesystem.leaseManager = leaseManager;
+    spyNamesystem.lmthread.interrupt();
+    
+    String holder = "client-1";
+    String path1 = "/file-1";
+    String path2 = "/file-2";
+    
+    CalledAnswer internalReleaseCalled = new CalledAnswer();
+    CalledAnswer internalReleaseOneCalled = new CalledAnswer();
+    doAnswer(internalReleaseCalled)
+        .when(spyNamesystem)
+        .internalReleaseLease((LeaseManager.Lease) anyObject(), anyString());
+    doAnswer(internalReleaseOneCalled)
+        .when(spyNamesystem)
+        .internalReleaseLeaseOne((LeaseManager.Lease) anyObject(), anyString());
+    
+    leaseManager.setLeasePeriod(1, 2);
+    leaseManager.addLease(holder, path1);
+    leaseManager.addLease(holder, path2);
+    Thread.sleep(1000);
+
+    synchronized (spyNamesystem) { // checkLeases is always called with FSN lock
+      leaseManager.checkLeases();
+    }
+    
+    assertTrue("internalReleaseOne not called", internalReleaseOneCalled.isCalled());
+    assertFalse("internalRelease called", internalReleaseCalled.isCalled());
+  }
+  
+  private static class CalledAnswer<T> implements Answer<T>{
+    private volatile boolean called = false;
+
+    @Override
+    public T answer(InvocationOnMock invocationOnMock) throws Throwable {
+      called = true;
+      
+      return (T)invocationOnMock.callRealMethod();
+    }
+
+    public boolean isCalled() {
+      return called;
+    }
+  }
+}
\ No newline at end of file