Browse Source

MAPREDUCE-6542. HistoryViewer uses SimpleDateFormat, but SimpleDateFormat is not threadsafe. Contributed by zhangyubiao.

Akira Ajisaka 9 years ago
parent
commit
a0f6109907

+ 45 - 9
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java

@@ -26,7 +26,6 @@ import java.text.DateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Date;
 import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.List;
@@ -38,6 +37,7 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import org.apache.commons.lang.SystemUtils;
+import org.apache.commons.lang.time.FastDateFormat;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.Path;
@@ -323,20 +323,56 @@ public class StringUtils {
   }
 
   /**
-   * Formats time in ms and appends difference (finishTime - startTime) 
+   * @param dateFormat date format to use
+   * @param finishTime finish time
+   * @param startTime  start time
+   * @return formatted value.
+   * Formats time in ms and appends difference (finishTime - startTime)
+   * as returned by formatTimeDiff().
+   * If finish time is 0, empty string is returned, if start time is 0
+   * then difference is not appended to return value.
+   * @deprecated Use
+   * {@link StringUtils#getFormattedTimeWithDiff(FastDateFormat, long, long)} or
+   * {@link StringUtils#getFormattedTimeWithDiff(String, long, long)} instead.
+   */
+  @Deprecated
+  public static String getFormattedTimeWithDiff(DateFormat dateFormat,
+      long finishTime, long startTime){
+    String formattedFinishTime = dateFormat.format(finishTime);
+    return getFormattedTimeWithDiff(formattedFinishTime, finishTime, startTime);
+  }
+
+  /**
+   * Formats time in ms and appends difference (finishTime - startTime)
    * as returned by formatTimeDiff().
-   * If finish time is 0, empty string is returned, if start time is 0 
-   * then difference is not appended to return value. 
+   * If finish time is 0, empty string is returned, if start time is 0
+   * then difference is not appended to return value.
+   *
    * @param dateFormat date format to use
-   * @param finishTime fnish time
+   * @param finishTime finish time
+   * @param startTime  start time
+   * @return formatted value.
+   */
+  public static String getFormattedTimeWithDiff(FastDateFormat dateFormat,
+      long finishTime, long startTime) {
+    String formattedFinishTime = dateFormat.format(finishTime);
+    return getFormattedTimeWithDiff(formattedFinishTime, finishTime, startTime);
+  }
+  /**
+   * Formats time in ms and appends difference (finishTime - startTime)
+   * as returned by formatTimeDiff().
+   * If finish time is 0, empty string is returned, if start time is 0
+   * then difference is not appended to return value.
+   * @param formattedFinishTime formattedFinishTime to use
+   * @param finishTime finish time
    * @param startTime start time
-   * @return formatted value. 
+   * @return formatted value.
    */
-  public static String getFormattedTimeWithDiff(DateFormat dateFormat, 
-                                                long finishTime, long startTime){
+  public static String getFormattedTimeWithDiff(String formattedFinishTime,
+      long finishTime, long startTime){
     StringBuilder buf = new StringBuilder();
     if (0 != finishTime) {
-      buf.append(dateFormat.format(new Date(finishTime)));
+      buf.append(formattedFinishTime);
       if (0 != startTime){
         buf.append(" (" + formatTimeDiff(finishTime , startTime) + ")");
       }

+ 41 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStringUtils.java

@@ -33,11 +33,16 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 
+import org.apache.commons.lang.time.FastDateFormat;
 import org.apache.hadoop.test.UnitTestcaseTimeLimit;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
-import org.junit.Assume;
 import org.junit.Test;
 
 public class TestStringUtils extends UnitTestcaseTimeLimit {
@@ -51,6 +56,9 @@ public class TestStringUtils extends UnitTestcaseTimeLimit {
   final private static String STR_WITH_BOTH2 = ",A\\,,B\\\\,";
   final private static String ESCAPED_STR_WITH_BOTH2 = 
     "\\,A\\\\\\,\\,B\\\\\\\\\\,";
+
+  final private static FastDateFormat FAST_DATE_FORMAT =
+      FastDateFormat.getInstance("d-MMM-yyyy HH:mm:ss");
   
   @Test (timeout = 30000)
   public void testEscapeString() throws Exception {
@@ -387,8 +395,6 @@ public class TestStringUtils extends UnitTestcaseTimeLimit {
       pattern, replacements));
     assertEquals("___", StringUtils.replaceTokens("$UNDER_SCORES", pattern,
       replacements));
-    assertEquals("//one//two//", StringUtils.replaceTokens("//$FOO/$BAR/$BAZ//",
-      pattern, replacements));
   }
 
   @Test (timeout = 5000)
@@ -438,6 +444,38 @@ public class TestStringUtils extends UnitTestcaseTimeLimit {
     }
   }
 
+  @Test
+  //Multithreaded Test GetFormattedTimeWithDiff()
+  public void testGetFormattedTimeWithDiff() throws InterruptedException {
+    ExecutorService executorService = Executors.newFixedThreadPool(16);
+    final CyclicBarrier cyclicBarrier = new CyclicBarrier(10);
+    for (int i = 0; i < 10; i++) {
+
+      executorService.execute(new Runnable() {
+        @Override
+        public void run() {
+          try {
+            cyclicBarrier.await();
+          } catch (InterruptedException | BrokenBarrierException e) {
+            //Ignored
+          }
+          final long end = System.currentTimeMillis();
+          final long start = end - 30000;
+          String formattedTime1 = StringUtils.getFormattedTimeWithDiff(
+              FAST_DATE_FORMAT, start, end);
+          String formattedTime2 = StringUtils.getFormattedTimeWithDiff(
+              FAST_DATE_FORMAT, start, end);
+          assertTrue("Method returned inconsistent results indicative of"
+              + " a race condition", formattedTime1.equals(formattedTime2));
+
+        }
+      });
+    }
+
+    executorService.shutdown();
+    executorService.awaitTermination(50, TimeUnit.SECONDS);
+  }
+
   // Benchmark for StringUtils split
   public static void main(String []args) {
     final String TO_SPLIT = "foo,bar,baz,blah,blah";

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryViewer.java

@@ -21,7 +21,6 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.text.DecimalFormat;
 import java.text.Format;
-import java.text.SimpleDateFormat;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -30,6 +29,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 
+import org.apache.commons.lang.time.FastDateFormat;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -59,8 +59,8 @@ import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 @InterfaceStability.Unstable
 public class HistoryViewer {
   private static final Log LOG = LogFactory.getLog(HistoryViewer.class);
-  private static final SimpleDateFormat dateFormat =
-    new SimpleDateFormat("d-MMM-yyyy HH:mm:ss");
+  private final FastDateFormat dateFormat =
+      FastDateFormat.getInstance("d-MMM-yyyy HH:mm:ss");
   private FileSystem fs;
   private JobInfo job;
   private String jobId;