Author: dhruba
Date: Tue May 20 16:09:07 2008
New Revision: 658483
URL: http://svn.apache.org/viewvc?rev=658483&view=rev
Log:
HADOOP-3375. Lease paths were sometimes not removed from
LeaseManager.sortedLeasesByPath. (Tsz Wo (Nicholas), SZE via dhruba)
Added:
hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestLease.java
Modified:
hadoop/core/trunk/CHANGES.txt
hadoop/core/trunk/src/java/org/apache/hadoop/dfs/LeaseManager.java
Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=658483&r1=658482&r2=658483&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Tue May 20 16:09:07 2008
@@ -294,6 +294,9 @@
HADOOP-3370. Ensure that the TaskTracker.runningJobs data-structure is
correctly cleaned-up on task completion. (Zheng Shao via acmurthy)
+ HADOOP-3375. Lease paths were sometimes not removed from
+ LeaseManager.sortedLeasesByPath. (Tsz Wo (Nicholas), SZE via dhruba)
+
Release 0.17.0 - 2008-05-18
INCOMPATIBLE CHANGES
Modified: hadoop/core/trunk/src/java/org/apache/hadoop/dfs/LeaseManager.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/java/org/apache/hadoop/dfs/LeaseManager.java?rev=658483&r1=658482&r2=658483&view=diff
==============================================================================
--- hadoop/core/trunk/src/java/org/apache/hadoop/dfs/LeaseManager.java (original)
+++ hadoop/core/trunk/src/java/org/apache/hadoop/dfs/LeaseManager.java Tue May 20 16:09:07
2008
@@ -54,6 +54,9 @@
SortedSet<Lease> getSortedLeases() {return sortedLeases;}
+ /** @return the lease containing src */
+ Lease getLeaseByPath(String src) {return sortedLeasesByPath.get(src);}
+
/** @return the number of leases currently in the system */
synchronized int countLease() {return sortedLeases.size();}
@@ -91,10 +94,11 @@
Lease lease = getLease(holder);
if (lease != null) {
lease.completedCreate(src);
+ sortedLeasesByPath.remove(src);
+
if (!lease.hasPath()) {
leases.remove(new StringBytesWritable(holder));
sortedLeases.remove(lease);
- sortedLeasesByPath.remove(src);
}
}
}
@@ -112,11 +116,17 @@
}
synchronized void removeExpiredLease(Lease lease) throws IOException {
- lease.releaseLocks();
+ String holder = lease.holder.getString();
+ for(StringBytesWritable sbw : lease.paths) {
+ String p = sbw.getString();
+ fsnamesystem.internalReleaseCreate(p, holder);
+ sortedLeasesByPath.remove(p);
+ }
+ lease.paths.clear();
+
leases.remove(lease.holder);
if (!sortedLeases.remove(lease)) {
- LOG.error("startFile: Unknown failure trying to remove " + lease +
- " from lease set.");
+ LOG.error("removeExpiredLease: " + lease + " not found in sortedLeases");
}
}
@@ -160,14 +170,6 @@
boolean hasPath() {return !paths.isEmpty();}
- void releaseLocks() throws IOException {
- String holderStr = holder.getString();
- for(StringBytesWritable s : paths) {
- fsnamesystem.internalReleaseCreate(s.getString(), holderStr);
- }
- paths.clear();
- }
-
/** {@inheritDoc} */
public String toString() {
return "[Lease. Holder: " + holder
Added: hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestLease.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestLease.java?rev=658483&view=auto
==============================================================================
--- hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestLease.java (added)
+++ hadoop/core/trunk/src/test/org/apache/hadoop/dfs/TestLease.java Tue May 20 16:09:07 2008
@@ -0,0 +1,66 @@
+/**
+ * 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.dfs;
+
+import java.io.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class TestLease extends junit.framework.TestCase {
+ static boolean hasLease(MiniDFSCluster cluster, Path src) {
+ return cluster.getNameNode().namesystem.leaseManager.getLeaseByPath(src.toString()) !=
null;
+ }
+
+ final Path dir = new Path("/test/lease/");
+
+ public void testLease() throws Exception {
+ Configuration conf = new Configuration();
+ MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
+ try {
+ FileSystem fs = cluster.getFileSystem();
+ assertTrue(fs.mkdirs(dir));
+
+ Path a = new Path(dir, "a");
+ Path b = new Path(dir, "b");
+
+ DataOutputStream a_out = fs.create(a);
+ a_out.writeBytes("something");
+
+ assertTrue(hasLease(cluster, a));
+ assertTrue(!hasLease(cluster, b));
+
+ DataOutputStream b_out = fs.create(b);
+ b_out.writeBytes("something");
+
+ assertTrue(hasLease(cluster, a));
+ assertTrue(hasLease(cluster, b));
+
+ a_out.close();
+ b_out.close();
+
+ assertTrue(!hasLease(cluster, a));
+ assertTrue(!hasLease(cluster, b));
+
+ fs.delete(dir, true);
+ } finally {
+ if (cluster != null) {cluster.shutdown();}
+ }
+ }
+}
|