Browse Source

svn merge -c 1417238 from branch-1 for HADOOP-9115. Backport HADOOP-7082 and HDFS-1542: Configuration.writeXML should not hold lock while outputting and add a test for a deadlock writing Configuration to HDFS.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1.1@1417245 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 years ago
parent
commit
58e62a6df4

+ 4 - 0
CHANGES.txt

@@ -31,6 +31,10 @@ Release 1.1.2 - Unreleased
     HADOOP-9111. Change some JUnit 3 tests to JUnit 4 so that @Ignore tests can
     be run with ant 1.8.x.  (Jing Zhao via szetszwo)
 
+    HADOOP-9115. Backport HADOOP-7082 and HDFS-1542: Configuration.writeXML
+    should not hold lock while outputting and add a test for a deadlock writing
+    Configuration to HDFS.  (Jing Zhao via szetszwo)
+
 Release 1.1.1 - 2012.11.18
 
   INCOMPATIBLE CHANGES

+ 48 - 34
src/core/org/apache/hadoop/conf/Configuration.java

@@ -1261,51 +1261,65 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
    * 
    * @param out the writer to write to.
    */
-  public synchronized void writeXml(Writer out) throws IOException {
-    Properties properties = getProps();
+  public void writeXml(Writer out) throws IOException {
+    Document doc = asXmlDocument();
     try {
-      Document doc =
-        DocumentBuilderFactory.newInstance().newDocumentBuilder().newDocument();
-      Element conf = doc.createElement("configuration");
-      doc.appendChild(conf);
-      conf.appendChild(doc.createTextNode("\n"));
-      for (Enumeration e = properties.keys(); e.hasMoreElements();) {
-        String name = (String)e.nextElement();
-        Object object = properties.get(name);
-        String value = null;
-        if (object instanceof String) {
-          value = (String) object;
-        }else {
-          continue;
-        }
-        Element propNode = doc.createElement("property");
-        conf.appendChild(propNode);
-        if (updatingResource != null) {
-          Comment commentNode = doc.createComment("Loaded from "
-              + updatingResource.get(name));
-          propNode.appendChild(commentNode);
-        }
-        Element nameNode = doc.createElement("name");
-        nameNode.appendChild(doc.createTextNode(name));
-        propNode.appendChild(nameNode);
-      
-        Element valueNode = doc.createElement("value");
-        valueNode.appendChild(doc.createTextNode(value));
-        propNode.appendChild(valueNode);
-
-        conf.appendChild(doc.createTextNode("\n"));
-      }
-    
       DOMSource source = new DOMSource(doc);
       StreamResult result = new StreamResult(out);
       TransformerFactory transFactory = TransformerFactory.newInstance();
       Transformer transformer = transFactory.newTransformer();
+      
+      // Important to not hold Configuration log while writing result, since
+      // 'out' may be an HDFS stream which needs to lock this configuration
+      // from another thread.
       transformer.transform(source, result);
     } catch (TransformerException te) {
       throw new IOException(te);
+    }
+  }
+  
+  /**
+   * Return the XML DOM corresponding to this Configuration.
+   */
+  private synchronized Document asXmlDocument() throws IOException {
+    Document doc;
+    Properties properties = getProps();
+    try {
+      doc =
+        DocumentBuilderFactory.newInstance().newDocumentBuilder().newDocument();
     } catch (ParserConfigurationException pe) {
       throw new IOException(pe);
     }
+    Element conf = doc.createElement("configuration");
+    doc.appendChild(conf);
+    conf.appendChild(doc.createTextNode("\n"));
+    for (Enumeration<Object> e = properties.keys(); e.hasMoreElements();) {
+      String name = (String) e.nextElement();
+      Object object = properties.get(name);
+      String value = null;
+      if (object instanceof String) {
+        value = (String) object;
+      } else {
+        continue;
+      }
+      Element propNode = doc.createElement("property");
+      conf.appendChild(propNode);
+      if (updatingResource != null) {
+        Comment commentNode = doc.createComment("Loaded from "
+            + updatingResource.get(name));
+        propNode.appendChild(commentNode);
+      }
+      Element nameNode = doc.createElement("name");
+      nameNode.appendChild(doc.createTextNode(name));
+      propNode.appendChild(nameNode);
+    
+      Element valueNode = doc.createElement("value");
+      valueNode.appendChild(doc.createTextNode(value));
+      propNode.appendChild(valueNode);
+
+      conf.appendChild(doc.createTextNode("\n"));
+    }
+    return doc;
   }
 
   /**

+ 55 - 0
src/test/org/apache/hadoop/hdfs/TestWriteConfigurationToDFS.java

@@ -0,0 +1,55 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import java.io.OutputStream;
+import org.junit.Test;
+
+/**
+ * Regression test for HDFS-1542, a deadlock between the main thread
+ * and the DFSOutputStream.DataStreamer thread caused because
+ * Configuration.writeXML holds a lock on itself while writing to DFS.
+ */
+public class TestWriteConfigurationToDFS {
+  @Test(timeout=60000)
+  public void testWriteConf() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 4096);
+    System.out.println("Setting conf in: " + System.identityHashCode(conf));
+    
+    MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null, null);
+    cluster.waitActive();
+    try {
+      FileSystem fs = cluster.getFileSystem();
+      Path filePath = new Path("/testWriteConf.xml");
+      OutputStream os = fs.create(filePath);
+      StringBuilder longString = new StringBuilder();
+      for (int i = 0; i < 100000; i++) {
+        longString.append("hello");
+      } // 500KB
+      conf.set("foobar", longString.toString());
+      conf.writeXml(os);
+      os.close();
+    } finally {
+      cluster.shutdown();
+    }
+  }
+}