瀏覽代碼

Revert HDFS-525

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20@808642 13f79535-47bb-0310-9956-ffa450edef68
Christopher Douglas 16 年之前
父節點
當前提交
f9cb2fca1e

+ 1 - 4
CHANGES.txt

@@ -202,9 +202,6 @@ Release 0.20.1 - Unreleased
     MAPREDUCE-796. Fixes a ClassCastException in an exception log in
     MultiThreadedMapRunner. (Amar Kamat via ddas)
 
-    HDFS-525. The SimpleDateFormat object in ListPathsServlet is not thread
-    safe.  (Suresh Srinivas via szetszwo)
-
     MAPREDUCE-838. Fixes a problem in the way commit of task outputs
     happens. The bug was that even if commit failed, the task would
     be declared as successful. (Amareshwari Sriramadasu via ddas)
@@ -220,7 +217,7 @@ Release 0.20.1 - Unreleased
 
     MAPREDUCE-832. Reduce number of warning messages printed when 
     deprecated memory variables are used. (Rahul Kumar Singh via yhemanth)
- 
+
     MAPREDUCE-745. Fixes a testcase problem to do with generation of JobTracker
     IDs. (Amar Kamat via ddas)
 

+ 2 - 2
src/hdfs/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -31,6 +31,7 @@ import java.net.URL;
 import java.net.UnknownHostException;
 
 import java.text.ParseException;
+import java.text.SimpleDateFormat;
 
 import java.util.ArrayList;
 import java.util.Random;
@@ -55,7 +56,6 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 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.security.*;
@@ -77,7 +77,7 @@ public class HftpFileSystem extends FileSystem {
   protected UserGroupInformation ugi; 
   protected final Random ran = new Random();
 
-  protected static final ThreadLocalDateFormat df = ListPathsServlet.df;
+  protected static final SimpleDateFormat df = ListPathsServlet.df;
 
   @Override
   public void initialize(URI name, Configuration conf) throws IOException {

+ 0 - 87
src/hdfs/org/apache/hadoop/hdfs/server/common/ThreadLocalDateFormat.java

@@ -1,87 +0,0 @@
-/**
- * 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();
-  }
-}

+ 8 - 4
src/hdfs/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 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;
@@ -28,12 +27,14 @@ import org.znerd.xmlenc.*;
 
 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;
@@ -45,9 +46,9 @@ import javax.servlet.http.HttpServletResponse;
 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"));
   }
@@ -162,10 +163,13 @@ public class ListPathsServlet extends DfsServlet {
         }
         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();
       }

+ 0 - 107
src/test/org/apache/hadoop/hdfs/server/common/TestThreadLocalDateFormat.java

@@ -1,107 +0,0 @@
-/**
- * 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 junit.framework.TestCase;
-
-/**
- * Test for {@link ThreadLocalDateFormat}
- */
-public class TestThreadLocalDateFormat extends TestCase {
-  private static final int TOTAL_THREADS = 3;
-  private static final Log LOG = LogFactory.getLog(TestThreadLocalDateFormat.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
-   */
-  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);
-      }
-    }
-    assertFalse(failed);
-  }
-}