Bläddra i källkod

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 år sedan
förälder
incheckning
f9cb2fca1e

+ 1 - 4
CHANGES.txt

@@ -202,9 +202,6 @@ Release 0.20.1 - Unreleased
     MAPREDUCE-796. Fixes a ClassCastException in an exception log in
     MAPREDUCE-796. Fixes a ClassCastException in an exception log in
     MultiThreadedMapRunner. (Amar Kamat via ddas)
     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
     MAPREDUCE-838. Fixes a problem in the way commit of task outputs
     happens. The bug was that even if commit failed, the task would
     happens. The bug was that even if commit failed, the task would
     be declared as successful. (Amareshwari Sriramadasu via ddas)
     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 
     MAPREDUCE-832. Reduce number of warning messages printed when 
     deprecated memory variables are used. (Rahul Kumar Singh via yhemanth)
     deprecated memory variables are used. (Rahul Kumar Singh via yhemanth)
- 
+
     MAPREDUCE-745. Fixes a testcase problem to do with generation of JobTracker
     MAPREDUCE-745. Fixes a testcase problem to do with generation of JobTracker
     IDs. (Amar Kamat via ddas)
     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.net.UnknownHostException;
 
 
 import java.text.ParseException;
 import java.text.ParseException;
+import java.text.SimpleDateFormat;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Random;
 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.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 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.hdfs.server.namenode.ListPathsServlet;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.*;
 import org.apache.hadoop.security.*;
@@ -77,7 +77,7 @@ public class HftpFileSystem extends FileSystem {
   protected UserGroupInformation ugi; 
   protected UserGroupInformation ugi; 
   protected final Random ran = new Random();
   protected final Random ran = new Random();
 
 
-  protected static final ThreadLocalDateFormat df = ListPathsServlet.df;
+  protected static final SimpleDateFormat df = ListPathsServlet.df;
 
 
   @Override
   @Override
   public void initialize(URI name, Configuration conf) throws IOException {
   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.fs.FileStatus;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 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.ipc.RemoteException;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.security.UnixUserGroupInformation;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionInfo;
@@ -28,12 +27,14 @@ import org.znerd.xmlenc.*;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
+import java.text.SimpleDateFormat;
 import java.util.Date;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map;
 import java.util.Stack;
 import java.util.Stack;
 import java.util.TimeZone;
 import java.util.TimeZone;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
 import javax.servlet.ServletException;
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.servlet.http.HttpServletResponse;
@@ -45,9 +46,9 @@ import javax.servlet.http.HttpServletResponse;
 public class ListPathsServlet extends DfsServlet {
 public class ListPathsServlet extends DfsServlet {
   /** For java.io.Serializable */
   /** For java.io.Serializable */
   private static final long serialVersionUID = 1L;
   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 {
   static {
     df.setTimeZone(TimeZone.getTimeZone("UTC"));
     df.setTimeZone(TimeZone.getTimeZone("UTC"));
   }
   }
@@ -162,10 +163,13 @@ public class ListPathsServlet extends DfsServlet {
         }
         }
         catch(RemoteException re) {re.writeXml(p, doc);}
         catch(RemoteException re) {re.writeXml(p, doc);}
       }
       }
+    } catch (PatternSyntaxException e) {
+      out.println(e.toString());
+    } finally {
       if (doc != null) {
       if (doc != null) {
         doc.endDocument();
         doc.endDocument();
       }
       }
-    } finally {
+
       if (out != null) {
       if (out != null) {
         out.close();
         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);
-  }
-}