You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by sz...@apache.org on 2009/08/11 01:14:15 UTC

svn commit: r802951 - in /hadoop/hdfs/trunk: ./ src/java/org/apache/hadoop/hdfs/ src/java/org/apache/hadoop/hdfs/server/common/ src/java/org/apache/hadoop/hdfs/server/namenode/ src/test/hdfs/org/apache/hadoop/hdfs/server/common/

Author: szetszwo
Date: Mon Aug 10 23:14:15 2009
New Revision: 802951

URL: http://svn.apache.org/viewvc?rev=802951&view=rev
Log:
HDFS-525. The SimpleDateFormat object in ListPathsServlet is not thread safe.  Contributed by Suresh Srinivas

Added:
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/ThreadLocalDateFormat.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestThreadLocalDateFormat.java
Modified:
    hadoop/hdfs/trunk/CHANGES.txt
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java

Modified: hadoop/hdfs/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/CHANGES.txt?rev=802951&r1=802950&r2=802951&view=diff
==============================================================================
--- hadoop/hdfs/trunk/CHANGES.txt (original)
+++ hadoop/hdfs/trunk/CHANGES.txt Mon Aug 10 23:14:15 2009
@@ -146,6 +146,9 @@
     HDFS-167. Fix a bug in DFSClient that caused infinite retries on write.
     (Bill Zeller via szetszwo)
 
+    HDFS-525. The SimpleDateFormat object in ListPathsServlet is not thread
+    safe.  (Suresh Srinivas via szetszwo)
+
 Release 0.20.1 - Unreleased
 
   IMPROVEMENTS

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java?rev=802951&r1=802950&r2=802951&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/HftpFileSystem.java Mon Aug 10 23:14:15 2009
@@ -27,7 +27,6 @@
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.text.ParseException;
-import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.Random;
@@ -45,6 +44,7 @@
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.common.ThreadLocalDateFormat;
 import org.apache.hadoop.hdfs.server.namenode.ListPathsServlet;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
@@ -74,7 +74,7 @@
   protected UserGroupInformation ugi; 
   protected final Random ran = new Random();
 
-  protected static final SimpleDateFormat df = ListPathsServlet.df;
+  protected static final ThreadLocalDateFormat df = ListPathsServlet.df;
 
   @Override
   public void initialize(URI name, Configuration conf) throws IOException {

Added: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/ThreadLocalDateFormat.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/ThreadLocalDateFormat.java?rev=802951&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/ThreadLocalDateFormat.java (added)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/common/ThreadLocalDateFormat.java Mon Aug 10 23:14:15 2009
@@ -0,0 +1,87 @@
+/**
+ * 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.common;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.TimeZone;
+
+/**
+ * Thread safe implementation of {@link SimpleDateFormat} 
+ * TODO: This needs to be moved to hadoop common project.
+ */
+public class ThreadLocalDateFormat {
+  private final String format;
+
+  /**
+   * Constructs {@link ThreadLocalDateFormat} using given date format pattern
+   * @param format Date format pattern
+   */
+  public ThreadLocalDateFormat(String format) {
+    this.format = format;
+  }
+
+  /**
+   * ThreadLocal based {@link SimpleDateFormat}
+   */
+  private final ThreadLocal<SimpleDateFormat> dateFormat = 
+    new ThreadLocal<SimpleDateFormat>() {
+      @Override
+      protected SimpleDateFormat initialValue() {
+        SimpleDateFormat df = new SimpleDateFormat(format);
+        return df;
+      }
+    };
+
+  /**
+   * Format given <code>Date</code> into date/time string.
+   * @param date Date to be formatted.
+   * @return the formatted date-time string.
+   */
+  public String format(Date date) {
+    return dateFormat.get().format(date);
+  }
+
+  /**
+   * Parse the String to produce <code>Date</code>.
+   * @param source String to parse.
+   * @return Date parsed from the String.
+   * @throws ParseException
+   *           - if the beginning of the specified string cannot be parsed.
+   */
+  public Date parse(String source) throws ParseException {
+    return dateFormat.get().parse(source);
+  }
+
+  /**
+   * @param zone
+   */
+  public void setTimeZone(TimeZone zone) {
+    dateFormat.get().setTimeZone(zone);
+  }
+
+  /**
+   * Get access to underlying SimpleDateFormat.
+   * Note: Do not pass reference to this Date to other threads!
+   * @return the SimpleDateFormat for the thread.
+   */
+  SimpleDateFormat get() {
+    return dateFormat.get();
+  }
+}

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java?rev=802951&r1=802950&r2=802951&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java Mon Aug 10 23:14:15 2009
@@ -19,6 +19,7 @@
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.server.common.ThreadLocalDateFormat;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.util.VersionInfo;
@@ -27,14 +28,12 @@
 
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Stack;
 import java.util.TimeZone;
 import java.util.regex.Pattern;
-import java.util.regex.PatternSyntaxException;
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
@@ -46,9 +45,9 @@
 public class ListPathsServlet extends DfsServlet {
   /** For java.io.Serializable */
   private static final long serialVersionUID = 1L;
+  public static final ThreadLocalDateFormat df = 
+    new ThreadLocalDateFormat("yyyy-MM-dd'T'HH:mm:ssZ");
 
-  public static final SimpleDateFormat df =
-    new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ");
   static {
     df.setTimeZone(TimeZone.getTimeZone("UTC"));
   }
@@ -163,13 +162,10 @@
         }
         catch(RemoteException re) {re.writeXml(p, doc);}
       }
-    } catch (PatternSyntaxException e) {
-      out.println(e.toString());
-    } finally {
       if (doc != null) {
         doc.endDocument();
       }
-
+    } finally {
       if (out != null) {
         out.close();
       }

Added: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestThreadLocalDateFormat.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestThreadLocalDateFormat.java?rev=802951&view=auto
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestThreadLocalDateFormat.java (added)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/common/TestThreadLocalDateFormat.java Mon Aug 10 23:14:15 2009
@@ -0,0 +1,110 @@
+/**
+ * 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.common;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Random;
+import java.util.TimeZone;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hdfs.server.datanode.TestDirectoryScanner;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Test for {@link ThreadLocalDateFormat}
+ */
+public class TestThreadLocalDateFormat {
+  private static final int TOTAL_THREADS = 3;
+  private static final Log LOG = LogFactory.getLog(TestDirectoryScanner.class);
+  private static final ThreadLocalDateFormat TDF = new ThreadLocalDateFormat(
+      "dd-MM-yyyy HH:mm:ss:S Z");
+  private static volatile boolean failed = false;
+  private final static Random rand = new Random();
+
+  private static synchronized void setFailed() {
+    failed = true;
+  }
+
+  /**
+   * Run formatting and parsing test and look for multi threaded access related
+   * failures
+   */
+  private void runTest(final SimpleDateFormat df) {
+    while (!failed) {
+      try {
+        df.setTimeZone(TimeZone.getTimeZone("UTC"));
+        Date date = new Date(rand.nextInt(Integer.MAX_VALUE));
+        String s1 = df.format(date);
+        Date parsedDate = df.parse(s1);
+        String s2 = df.format(parsedDate);
+        if (!s1.equals(s2)) {
+          LOG.warn("Parse failed, actual /" + s2 + "/ expected /" + s1 + "/");
+          setFailed();
+        }
+      } catch (ArrayIndexOutOfBoundsException e) {
+        setFailed();
+        LOG.warn("exception ", e);
+      } catch (ParseException e) {
+        LOG.warn("Parsing failed ", e);
+        setFailed();
+      } catch (Exception e) {
+        LOG.warn("Unknown exception", e);
+        setFailed();
+      }
+    }
+  }
+
+  /**
+   * {@link SimpleDateFormat} when using with multiple threads has following
+   * issues:
+   * <ul>
+   * <li>format method throws {@link ArrayIndexOutOfBoundsException}
+   * <li>parse method throws {@link ParseException} or returns invalid parse
+   * </ul>
+   * This test shows ThreadLocal based implementation of
+   * {@link SimpleDateFormat} does not have these issues.
+   * 
+   * @throws InterruptedException
+   */
+  @Test
+  public void testDateFormat() throws InterruptedException {
+    for (int i = 0; i < TOTAL_THREADS; i++) {
+      Thread thread = new Thread() {
+        public void run() {
+          runTest(TDF.get());
+        }
+      };
+      thread.start();
+    }
+
+    // Wait up to 30 seconds for failure to occur
+    long endTime = System.currentTimeMillis() + 30 * 1000;
+    while (!failed && endTime > System.currentTimeMillis()) {
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException ie) {
+        LOG.debug("Exception", ie);
+      }
+    }
+    Assert.assertFalse(failed);
+  }
+}