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 sh...@apache.org on 2008/12/31 20:24:21 UTC

svn commit: r730449 - in /hadoop/core/branches/branch-0.19: CHANGES.txt src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java src/test/org/apache/hadoop/hdfs/TestFileCreationEmpty.java

Author: shv
Date: Wed Dec 31 11:24:21 2008
New Revision: 730449

URL: http://svn.apache.org/viewvc?rev=730449&view=rev
Log:
HADOOP-4961. Merge -r 730438:730439 from trunk to branch 0.19.

Added:
    hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileCreationEmpty.java
Modified:
    hadoop/core/branches/branch-0.19/CHANGES.txt
    hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

Modified: hadoop/core/branches/branch-0.19/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/CHANGES.txt?rev=730449&r1=730448&r2=730449&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/CHANGES.txt (original)
+++ hadoop/core/branches/branch-0.19/CHANGES.txt Wed Dec 31 11:24:21 2008
@@ -1117,6 +1117,9 @@
     HADOOP-4935. processMisReplicatedBlocks() should not clear 
     excessReplicateMap. (shv)
 
+    HADOOP-4961. Fix ConcurrentModificationException in lease recovery 
+    of empty files. (shv)
+
 Release 0.18.2 - 2008-11-03
 
   BUG FIXES

Modified: hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java?rev=730449&r1=730448&r2=730449&view=diff
==============================================================================
--- hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java (original)
+++ hadoop/core/branches/branch-0.19/src/hdfs/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java Wed Dec 31 11:24:21 2008
@@ -371,7 +371,13 @@
       LOG.info("Lease " + oldest + " has expired hard limit");
 
       final List<StringBytesWritable> removing = new ArrayList<StringBytesWritable>();
-      for(StringBytesWritable p : oldest.getPaths()) {
+      // need to create a copy of the oldest lease paths, becuase 
+      // internalReleaseLease() removes paths corresponding to empty files,
+      // i.e. it needs to modify the collection being iterated over
+      // causing ConcurrentModificationException
+      StringBytesWritable[] leasePaths = new StringBytesWritable[oldest.getPaths().size()];
+      oldest.getPaths().toArray(leasePaths);
+      for(StringBytesWritable p : leasePaths) {
         try {
           fsnamesystem.internalReleaseLease(oldest, p.getString());
         } catch (IOException e) {

Added: hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileCreationEmpty.java
URL: http://svn.apache.org/viewvc/hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileCreationEmpty.java?rev=730449&view=auto
==============================================================================
--- hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileCreationEmpty.java (added)
+++ hadoop/core/branches/branch-0.19/src/test/org/apache/hadoop/hdfs/TestFileCreationEmpty.java Wed Dec 31 11:24:21 2008
@@ -0,0 +1,80 @@
+/**
+ * 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;
+
+import java.util.ConcurrentModificationException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+
+/**
+ * Test empty file creation.
+ */
+public class TestFileCreationEmpty extends junit.framework.TestCase {
+  private boolean isConcurrentModificationException = false;
+
+  /**
+   * This test creates three empty files and lets their leases expire.
+   * This triggers release of the leases. 
+   * The empty files are supposed to be closed by that 
+   * without causing ConcurrentModificationException.
+   */
+  public void testLeaseExpireEmptyFiles() throws Exception {
+    final Thread.UncaughtExceptionHandler oldUEH = Thread.getDefaultUncaughtExceptionHandler();
+    Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
+      public void uncaughtException(Thread t, Throwable e) {
+        if (e instanceof ConcurrentModificationException) {
+          FSNamesystem.LOG.error("t=" + t, e);
+          isConcurrentModificationException = true;
+        }
+      }
+    });
+
+    System.out.println("testLeaseExpireEmptyFiles start");
+    final long leasePeriod = 1000;
+    final int DATANODE_NUM = 3;
+
+    final Configuration conf = new Configuration();
+    conf.setInt("heartbeat.recheck.interval", 1000);
+    conf.setInt("dfs.heartbeat.interval", 1);
+
+    // create cluster
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, DATANODE_NUM, true, null);
+    try {
+      cluster.waitActive();
+      DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
+
+      // create a new file.
+      TestFileCreation.createFile(dfs, new Path("/foo"), DATANODE_NUM);
+      TestFileCreation.createFile(dfs, new Path("/foo2"), DATANODE_NUM);
+      TestFileCreation.createFile(dfs, new Path("/foo3"), DATANODE_NUM);
+
+      // set the soft and hard limit to be 1 second so that the
+      // namenode triggers lease recovery
+      cluster.setLeasePeriod(leasePeriod, leasePeriod);
+      // wait for the lease to expire
+      try {Thread.sleep(5 * leasePeriod);} catch (InterruptedException e) {}
+
+      assertFalse(isConcurrentModificationException);
+    } finally {
+      Thread.setDefaultUncaughtExceptionHandler(oldUEH);
+      cluster.shutdown();
+    }
+  }
+}