Bladeren bron

Revert "Replace use of Guava's Stopwatch with Hadoop's StopWatch. (ozawa)" because of missing JIRA's number.

This reverts commit 2eba7eb9aff5f7a1bf63ff1ebbe28d21fd37065b.
Tsuyoshi Ozawa 10 jaren geleden
bovenliggende
commit
9886396400
15 gewijzigde bestanden met toevoegingen van 49 en 234 verwijderingen
  1. 0 3
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 3 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java
  3. 0 108
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java
  4. 6 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestChunkedArrayList.java
  5. 6 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDataChecksum.java
  6. 0 62
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStopWatch.java
  7. 0 3
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  8. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java
  9. 6 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
  10. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMultiThreadedHflush.java
  11. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
  12. 0 3
      hadoop-mapreduce-project/CHANGES.txt
  13. 5 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java
  14. 5 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
  15. 3 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVJob.java

+ 0 - 3
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -146,9 +146,6 @@ Trunk (Unreleased)
     HADOOP-11058. Missing HADOOP_CONF_DIR generates strange results
     HADOOP-11058. Missing HADOOP_CONF_DIR generates strange results
     (Masatake Iwasaki via aw)
     (Masatake Iwasaki via aw)
 
 
-    HADOOP-11032. Replace use of Guava's Stopwatch with Hadoop's StopWatch
-    (ozawa)
-
   BUG FIXES
   BUG FIXES
 
 
     HADOOP-9451. Fault single-layer config if node group topology is enabled.
     HADOOP-9451. Fault single-layer config if node group topology is enabled.

+ 3 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java

@@ -22,7 +22,6 @@ import java.lang.management.ManagementFactory;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Set;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -31,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
 
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Stopwatch;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.collect.Sets;
@@ -172,7 +172,7 @@ public class JvmPauseMonitor {
   private class Monitor implements Runnable {
   private class Monitor implements Runnable {
     @Override
     @Override
     public void run() {
     public void run() {
-      StopWatch sw = new StopWatch();
+      Stopwatch sw = new Stopwatch();
       Map<String, GcTimes> gcTimesBeforeSleep = getGcTimes();
       Map<String, GcTimes> gcTimesBeforeSleep = getGcTimes();
       while (shouldRun) {
       while (shouldRun) {
         sw.reset().start();
         sw.reset().start();
@@ -181,7 +181,7 @@ public class JvmPauseMonitor {
         } catch (InterruptedException ie) {
         } catch (InterruptedException ie) {
           return;
           return;
         }
         }
-        long extraSleepTime = sw.now(TimeUnit.MILLISECONDS) - SLEEP_INTERVAL_MS;
+        long extraSleepTime = sw.elapsedMillis() - SLEEP_INTERVAL_MS;
         Map<String, GcTimes> gcTimesAfterSleep = getGcTimes();
         Map<String, GcTimes> gcTimesAfterSleep = getGcTimes();
 
 
         if (extraSleepTime > warnThresholdMs) {
         if (extraSleepTime > warnThresholdMs) {

+ 0 - 108
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java

@@ -1,108 +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.util;
-
-import java.io.Closeable;
-import java.util.concurrent.TimeUnit;
-
-/**
- * A simplified StopWatch implementation which can measure times in nanoseconds.
- */
-public class StopWatch implements Closeable {
-  private boolean isStarted;
-  private long startNanos;
-  private long currentElapsedNanos;
-
-  public StopWatch() {
-  }
-
-  /**
-   * The method is used to find out if the StopWatch is started.
-   * @return boolean If the StopWatch is started.
-   */
-  public boolean isRunning() {
-    return isStarted;
-  }
-
-  /**
-   * Start to measure times and make the state of stopwatch running.
-   * @return this instance of StopWatch.
-   */
-  public StopWatch start() {
-    if (isStarted) {
-      throw new IllegalStateException("StopWatch is already running");
-    }
-    isStarted = true;
-    startNanos = System.nanoTime();
-    return this;
-  }
-
-  /**
-   * Stop elapsed time and make the state of stopwatch stop.
-   * @return this instance of StopWatch.
-   */
-  public StopWatch stop() {
-    if (!isStarted) {
-      throw new IllegalStateException("StopWatch is already stopped");
-    }
-    long now = System.nanoTime();
-    isStarted = false;
-    currentElapsedNanos += now - startNanos;
-    return this;
-  }
-
-  /**
-   * Reset elapsed time to zero and make the state of stopwatch stop.
-   * @return this instance of StopWatch.
-   */
-  public StopWatch reset() {
-    currentElapsedNanos = 0;
-    isStarted = false;
-    return this;
-  }
-
-  /**
-   * @return current elapsed time in specified timeunit.
-   */
-  public long now(TimeUnit timeUnit) {
-    return timeUnit.convert(now(), TimeUnit.NANOSECONDS);
-
-  }
-
-  /**
-   * @return current elapsed time in nanosecond.
-   */
-  public long now() {
-    return isStarted ?
-        System.nanoTime() - startNanos + currentElapsedNanos :
-        currentElapsedNanos;
-  }
-
-  @Override
-  public String toString() {
-    return String.valueOf(now());
-  }
-
-  @Override
-  public void close() {
-    if (isStarted) {
-      stop();
-    }
-  }
-}

+ 6 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestChunkedArrayList.java

@@ -21,11 +21,12 @@ import static org.junit.Assert.*;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Iterator;
-import java.util.concurrent.TimeUnit;
 
 
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 
 
+import com.google.common.base.Stopwatch;
+
 public class TestChunkedArrayList {
 public class TestChunkedArrayList {
 
 
   @Test
   @Test
@@ -70,24 +71,24 @@ public class TestChunkedArrayList {
       System.gc();
       System.gc();
       {
       {
         ArrayList<String> arrayList = new ArrayList<String>();
         ArrayList<String> arrayList = new ArrayList<String>();
-        StopWatch sw = new StopWatch();
+        Stopwatch sw = new Stopwatch();
         sw.start();
         sw.start();
         for (int i = 0; i < numElems; i++) {
         for (int i = 0; i < numElems; i++) {
           arrayList.add(obj);
           arrayList.add(obj);
         }
         }
-        System.out.println("       ArrayList " + sw.now(TimeUnit.MILLISECONDS));
+        System.out.println("       ArrayList " + sw.elapsedMillis());
       }
       }
       
       
       // test ChunkedArrayList
       // test ChunkedArrayList
       System.gc();
       System.gc();
       {
       {
         ChunkedArrayList<String> chunkedList = new ChunkedArrayList<String>();
         ChunkedArrayList<String> chunkedList = new ChunkedArrayList<String>();
-        StopWatch sw = new StopWatch();
+        Stopwatch sw = new Stopwatch();
         sw.start();
         sw.start();
         for (int i = 0; i < numElems; i++) {
         for (int i = 0; i < numElems; i++) {
           chunkedList.add(obj);
           chunkedList.add(obj);
         }
         }
-        System.out.println("ChunkedArrayList " + sw.now(TimeUnit.MILLISECONDS));
+        System.out.println("ChunkedArrayList " + sw.elapsedMillis());
       }
       }
     }
     }
   }
   }

+ 6 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDataChecksum.java

@@ -21,6 +21,8 @@ import java.nio.ByteBuffer;
 import java.util.Random;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
 
 
+import com.google.common.base.Stopwatch;
+
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.ChecksumException;
 import org.junit.Test;
 import org.junit.Test;
 
 
@@ -145,19 +147,19 @@ public class TestDataChecksum {
     Harness h = new Harness(checksum, dataLength, true);
     Harness h = new Harness(checksum, dataLength, true);
 
 
     for (int i = 0; i < NUM_RUNS; i++) {
     for (int i = 0; i < NUM_RUNS; i++) {
-      StopWatch s = new StopWatch().start();
+      Stopwatch s = new Stopwatch().start();
       // calculate real checksum, make sure it passes
       // calculate real checksum, make sure it passes
       checksum.calculateChunkedSums(h.dataBuf, h.checksumBuf);
       checksum.calculateChunkedSums(h.dataBuf, h.checksumBuf);
       s.stop();
       s.stop();
       System.err.println("Calculate run #" + i + ": " +
       System.err.println("Calculate run #" + i + ": " +
-                         s.now(TimeUnit.MICROSECONDS) + "us");
+                         s.elapsedTime(TimeUnit.MICROSECONDS) + "us");
 
 
-      s = new StopWatch().start();
+      s = new Stopwatch().start();
       // calculate real checksum, make sure it passes
       // calculate real checksum, make sure it passes
       checksum.verifyChunkedSums(h.dataBuf, h.checksumBuf, "fake file", 0);
       checksum.verifyChunkedSums(h.dataBuf, h.checksumBuf, "fake file", 0);
       s.stop();
       s.stop();
       System.err.println("Verify run #" + i + ": " +
       System.err.println("Verify run #" + i + ": " +
-                         s.now(TimeUnit.MICROSECONDS) + "us");
+                         s.elapsedTime(TimeUnit.MICROSECONDS) + "us");
     }
     }
   }
   }
 
 

+ 0 - 62
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestStopWatch.java

@@ -1,62 +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.util;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestStopWatch {
-
-  @Test
-  public void testStartAndStop() throws Exception {
-    try (StopWatch sw = new StopWatch()) {
-      Assert.assertFalse(sw.isRunning());
-      sw.start();
-      Assert.assertTrue(sw.isRunning());
-      sw.stop();
-      Assert.assertFalse(sw.isRunning());
-    }
-  }
-
-  @Test
-  public void testStopInTryWithResource() throws Exception {
-    try (StopWatch sw = new StopWatch()) {
-      // make sure that no exception is thrown.
-    }
-  }
-
-  @Test
-  public void testExceptions() throws Exception {
-    StopWatch sw = new StopWatch();
-    try {
-      sw.stop();
-    } catch (Exception e) {
-      Assert.assertTrue("IllegalStateException is expected",
-          e instanceof IllegalStateException);
-    }
-    sw.reset();
-    sw.start();
-    try {
-      sw.start();
-    } catch (Exception e) {
-      Assert.assertTrue("IllegalStateException is expected",
-          e instanceof IllegalStateException);
-    }
-  }
-
-}

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -479,9 +479,6 @@ Release 2.7.0 - UNRELEASED
     HDFS-7484. Make FSDirectory#addINode take existing INodes as its parameter.
     HDFS-7484. Make FSDirectory#addINode take existing INodes as its parameter.
     (jing9)
     (jing9)
 
 
-    HADOOP-11032. Replace use of Guava's Stopwatch with Hadoop's StopWatch
-    (ozawa)
-
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java

@@ -52,10 +52,10 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.util.StopWatch;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Stopwatch;
 import com.google.common.net.InetAddresses;
 import com.google.common.net.InetAddresses;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.Futures;
@@ -143,7 +143,7 @@ public class IPCLoggerChannel implements AsyncLogger {
   /**
   /**
    * Stopwatch which starts counting on each heartbeat that is sent
    * Stopwatch which starts counting on each heartbeat that is sent
    */
    */
-  private final StopWatch lastHeartbeatStopwatch = new StopWatch();
+  private final Stopwatch lastHeartbeatStopwatch = new Stopwatch();
   
   
   private static final long HEARTBEAT_INTERVAL_MILLIS = 1000;
   private static final long HEARTBEAT_INTERVAL_MILLIS = 1000;
 
 
@@ -463,8 +463,8 @@ public class IPCLoggerChannel implements AsyncLogger {
    * written.
    * written.
    */
    */
   private void heartbeatIfNecessary() throws IOException {
   private void heartbeatIfNecessary() throws IOException {
-    if (lastHeartbeatStopwatch.now(TimeUnit.MILLISECONDS)
-        > HEARTBEAT_INTERVAL_MILLIS || !lastHeartbeatStopwatch.isRunning()) {
+    if (lastHeartbeatStopwatch.elapsedMillis() > HEARTBEAT_INTERVAL_MILLIS ||
+        !lastHeartbeatStopwatch.isRunning()) {
       try {
       try {
         getProxy().heartbeat(createReqInfo());
         getProxy().heartbeat(createReqInfo());
       } finally {
       } finally {

+ 6 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java

@@ -65,11 +65,11 @@ import org.apache.hadoop.security.UserGroupInformation;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Charsets;
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Stopwatch;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Range;
 import com.google.common.collect.Range;
 import com.google.common.collect.Ranges;
 import com.google.common.collect.Ranges;
 import com.google.protobuf.TextFormat;
 import com.google.protobuf.TextFormat;
-import org.apache.hadoop.util.StopWatch;
 
 
 /**
 /**
  * A JournalNode can manage journals for several clusters at once.
  * A JournalNode can manage journals for several clusters at once.
@@ -374,20 +374,15 @@ public class Journal implements Closeable {
     
     
     curSegment.writeRaw(records, 0, records.length);
     curSegment.writeRaw(records, 0, records.length);
     curSegment.setReadyToFlush();
     curSegment.setReadyToFlush();
-    StopWatch sw = new StopWatch();
+    Stopwatch sw = new Stopwatch();
     sw.start();
     sw.start();
     curSegment.flush(shouldFsync);
     curSegment.flush(shouldFsync);
     sw.stop();
     sw.stop();
-
-    long nanoSeconds = sw.now();
-    metrics.addSync(
-        TimeUnit.MICROSECONDS.convert(nanoSeconds, TimeUnit.NANOSECONDS));
-    long milliSeconds = TimeUnit.MILLISECONDS.convert(
-        nanoSeconds, TimeUnit.NANOSECONDS);
-
-    if (milliSeconds > WARN_SYNC_MILLIS_THRESHOLD) {
+    
+    metrics.addSync(sw.elapsedTime(TimeUnit.MICROSECONDS));
+    if (sw.elapsedTime(TimeUnit.MILLISECONDS) > WARN_SYNC_MILLIS_THRESHOLD) {
       LOG.warn("Sync of transaction range " + firstTxnId + "-" + lastTxnId +
       LOG.warn("Sync of transaction range " + firstTxnId + "-" + lastTxnId +
-               " took " + milliSeconds + "ms");
+               " took " + sw.elapsedTime(TimeUnit.MILLISECONDS) + "ms");
     }
     }
 
 
     if (isLagging) {
     if (isLagging) {

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMultiThreadedHflush.java

@@ -32,11 +32,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.metrics2.util.Quantile;
 import org.apache.hadoop.metrics2.util.Quantile;
 import org.apache.hadoop.metrics2.util.SampleQuantiles;
 import org.apache.hadoop.metrics2.util.SampleQuantiles;
-import org.apache.hadoop.util.StopWatch;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
 import org.junit.Test;
 
 
+import com.google.common.base.Stopwatch;
+
 /**
 /**
  * This class tests hflushing concurrently from many threads.
  * This class tests hflushing concurrently from many threads.
  */
  */
@@ -99,10 +100,10 @@ public class TestMultiThreadedHflush {
     }
     }
 
 
     private void doAWrite() throws IOException {
     private void doAWrite() throws IOException {
-      StopWatch sw = new StopWatch().start();
+      Stopwatch sw = new Stopwatch().start();
       stm.write(toWrite);
       stm.write(toWrite);
       stm.hflush();
       stm.hflush();
-      long micros = sw.now(TimeUnit.MICROSECONDS);
+      long micros = sw.elapsedTime(TimeUnit.MICROSECONDS);
       quantiles.insert(micros);
       quantiles.insert(micros);
     }
     }
   }
   }
@@ -275,12 +276,12 @@ public class TestMultiThreadedHflush {
       int replication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
       int replication = conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
           DFSConfigKeys.DFS_REPLICATION_DEFAULT);
           DFSConfigKeys.DFS_REPLICATION_DEFAULT);
       
       
-      StopWatch sw = new StopWatch().start();
+      Stopwatch sw = new Stopwatch().start();
       test.doMultithreadedWrites(conf, p, numThreads, writeSize, numWrites,
       test.doMultithreadedWrites(conf, p, numThreads, writeSize, numWrites,
           replication);
           replication);
       sw.stop();
       sw.stop();
   
   
-      System.out.println("Finished in " + sw.now(TimeUnit.MILLISECONDS) + "ms");
+      System.out.println("Finished in " + sw.elapsedMillis() + "ms");
       System.out.println("Latency quantiles (in microseconds):\n" +
       System.out.println("Latency quantiles (in microseconds):\n" +
           test.quantiles);
           test.quantiles);
       return 0;
       return 0;

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java

@@ -27,7 +27,6 @@ import java.io.File;
 import java.net.HttpURLConnection;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.net.URL;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
@@ -39,6 +38,8 @@ import org.apache.hadoop.hdfs.qjournal.QJMTestUtil;
 import org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel;
 import org.apache.hadoop.hdfs.qjournal.client.IPCLoggerChannel;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
+import org.apache.hadoop.hdfs.qjournal.server.Journal;
+import org.apache.hadoop.hdfs.qjournal.server.JournalNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
@@ -47,12 +48,12 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.MetricsAsserts;
 import org.apache.hadoop.test.MetricsAsserts;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell;
-import org.apache.hadoop.util.StopWatch;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import com.google.common.base.Charsets;
 import com.google.common.base.Charsets;
+import com.google.common.base.Stopwatch;
 import com.google.common.primitives.Bytes;
 import com.google.common.primitives.Bytes;
 import com.google.common.primitives.Ints;
 import com.google.common.primitives.Ints;
 
 
@@ -319,11 +320,11 @@ public class TestJournalNode {
     ch.setEpoch(1);
     ch.setEpoch(1);
     ch.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION).get();
     ch.startLogSegment(1, NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION).get();
     
     
-    StopWatch sw = new StopWatch().start();
+    Stopwatch sw = new Stopwatch().start();
     for (int i = 1; i < numEdits; i++) {
     for (int i = 1; i < numEdits; i++) {
       ch.sendEdits(1L, i, 1, data).get();
       ch.sendEdits(1L, i, 1, data).get();
     }
     }
-    long time = sw.now(TimeUnit.MILLISECONDS);
+    long time = sw.elapsedMillis();
     
     
     System.err.println("Wrote " + numEdits + " batches of " + editsSize +
     System.err.println("Wrote " + numEdits + " batches of " + editsSize +
         " bytes in " + time + "ms");
         " bytes in " + time + "ms");

+ 0 - 3
hadoop-mapreduce-project/CHANGES.txt

@@ -92,9 +92,6 @@ Trunk (Unreleased)
     MAPREDUCE-6013. [post-HADOOP-9902] mapred version is missing (Akira AJISAKA
     MAPREDUCE-6013. [post-HADOOP-9902] mapred version is missing (Akira AJISAKA
     via aw)
     via aw)
 
 
-    HADOOP-11032. Replace use of Guava's Stopwatch with Hadoop's StopWatch
-    (ozawa)
-
   BUG FIXES
   BUG FIXES
 
 
     MAPREDUCE-6191. Improve clearing stale state of Java serialization
     MAPREDUCE-6191. Improve clearing stale state of Java serialization

+ 5 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java

@@ -28,7 +28,6 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Set;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -46,9 +45,9 @@ import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.util.StopWatch;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 
 
+import com.google.common.base.Stopwatch;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterables;
 
 
 /** 
 /** 
@@ -224,7 +223,7 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
             org.apache.hadoop.mapreduce.lib.input.FileInputFormat.LIST_STATUS_NUM_THREADS,
             org.apache.hadoop.mapreduce.lib.input.FileInputFormat.LIST_STATUS_NUM_THREADS,
             org.apache.hadoop.mapreduce.lib.input.FileInputFormat.DEFAULT_LIST_STATUS_NUM_THREADS);
             org.apache.hadoop.mapreduce.lib.input.FileInputFormat.DEFAULT_LIST_STATUS_NUM_THREADS);
     
     
-    StopWatch sw = new StopWatch().start();
+    Stopwatch sw = new Stopwatch().start();
     if (numThreads == 1) {
     if (numThreads == 1) {
       List<FileStatus> locatedFiles = singleThreadedListStatus(job, dirs, inputFilter, recursive); 
       List<FileStatus> locatedFiles = singleThreadedListStatus(job, dirs, inputFilter, recursive); 
       result = locatedFiles.toArray(new FileStatus[locatedFiles.size()]);
       result = locatedFiles.toArray(new FileStatus[locatedFiles.size()]);
@@ -243,8 +242,7 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
 
 
     sw.stop();
     sw.stop();
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Time taken to get FileStatuses: "
-          + sw.now(TimeUnit.MILLISECONDS));
+      LOG.debug("Time taken to get FileStatuses: " + sw.elapsedMillis());
     }
     }
     LOG.info("Total input paths to process : " + result.length);
     LOG.info("Total input paths to process : " + result.length);
     return result;
     return result;
@@ -311,7 +309,7 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
    * they're too big.*/ 
    * they're too big.*/ 
   public InputSplit[] getSplits(JobConf job, int numSplits)
   public InputSplit[] getSplits(JobConf job, int numSplits)
     throws IOException {
     throws IOException {
-    StopWatch sw = new StopWatch().start();
+    Stopwatch sw = new Stopwatch().start();
     FileStatus[] files = listStatus(job);
     FileStatus[] files = listStatus(job);
     
     
     // Save the number of input files for metrics/loadgen
     // Save the number of input files for metrics/loadgen
@@ -373,7 +371,7 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
     sw.stop();
     sw.stop();
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Total # of splits generated by getSplits: " + splits.size()
       LOG.debug("Total # of splits generated by getSplits: " + splits.size()
-          + ", TimeTaken: " + sw.now(TimeUnit.MILLISECONDS));
+          + ", TimeTaken: " + sw.elapsedMillis());
     }
     }
     return splits.toArray(new FileSplit[splits.size()]);
     return splits.toArray(new FileSplit[splits.size()]);
   }
   }

+ 5 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java

@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
 
 
-import java.util.concurrent.TimeUnit;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -44,9 +43,9 @@ import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.util.StopWatch;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 
 
+import com.google.common.base.Stopwatch;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
 
 
 /** 
 /** 
@@ -260,7 +259,7 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
 
 
     int numThreads = job.getConfiguration().getInt(LIST_STATUS_NUM_THREADS,
     int numThreads = job.getConfiguration().getInt(LIST_STATUS_NUM_THREADS,
         DEFAULT_LIST_STATUS_NUM_THREADS);
         DEFAULT_LIST_STATUS_NUM_THREADS);
-    StopWatch sw = new StopWatch().start();
+    Stopwatch sw = new Stopwatch().start();
     if (numThreads == 1) {
     if (numThreads == 1) {
       result = singleThreadedListStatus(job, dirs, inputFilter, recursive);
       result = singleThreadedListStatus(job, dirs, inputFilter, recursive);
     } else {
     } else {
@@ -277,8 +276,7 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
     
     
     sw.stop();
     sw.stop();
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Time taken to get FileStatuses: "
-          + sw.now(TimeUnit.MILLISECONDS));
+      LOG.debug("Time taken to get FileStatuses: " + sw.elapsedMillis());
     }
     }
     LOG.info("Total input paths to process : " + result.size()); 
     LOG.info("Total input paths to process : " + result.size()); 
     return result;
     return result;
@@ -378,7 +376,7 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
    * @throws IOException
    * @throws IOException
    */
    */
   public List<InputSplit> getSplits(JobContext job) throws IOException {
   public List<InputSplit> getSplits(JobContext job) throws IOException {
-    StopWatch sw = new StopWatch().start();
+    Stopwatch sw = new Stopwatch().start();
     long minSize = Math.max(getFormatMinSplitSize(), getMinSplitSize(job));
     long minSize = Math.max(getFormatMinSplitSize(), getMinSplitSize(job));
     long maxSize = getMaxSplitSize(job);
     long maxSize = getMaxSplitSize(job);
 
 
@@ -429,7 +427,7 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
     sw.stop();
     sw.stop();
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Total # of splits generated by getSplits: " + splits.size()
       LOG.debug("Total # of splits generated by getSplits: " + splits.size()
-          + ", TimeTaken: " + sw.now(TimeUnit.MILLISECONDS));
+          + ", TimeTaken: " + sw.elapsedMillis());
     }
     }
     return splits;
     return splits;
   }
   }

+ 3 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/KVJob.java

@@ -18,9 +18,9 @@
 package org.apache.hadoop.mapred.nativetask.kvtest;
 package org.apache.hadoop.mapred.nativetask.kvtest;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.util.concurrent.TimeUnit;
 import java.util.zip.CRC32;
 import java.util.zip.CRC32;
 
 
+import com.google.common.base.Stopwatch;
 import com.google.common.primitives.Longs;
 import com.google.common.primitives.Longs;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -36,7 +36,6 @@ import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.StopWatch;
 
 
 public class KVJob {
 public class KVJob {
   public static final String INPUTPATH = "nativetask.kvtest.inputfile.path";
   public static final String INPUTPATH = "nativetask.kvtest.inputfile.path";
@@ -94,10 +93,9 @@ public class KVJob {
       final TestInputFile testfile = new TestInputFile(Integer.valueOf(conf.get(
       final TestInputFile testfile = new TestInputFile(Integer.valueOf(conf.get(
           TestConstants.FILESIZE_KEY, "1000")),
           TestConstants.FILESIZE_KEY, "1000")),
           keyclass.getName(), valueclass.getName(), conf);
           keyclass.getName(), valueclass.getName(), conf);
-      StopWatch sw = new StopWatch().start();
+      Stopwatch sw = new Stopwatch().start();
       testfile.createSequenceTestFile(inputpath);
       testfile.createSequenceTestFile(inputpath);
-      LOG.info("Created test file " + inputpath + " in "
-          + sw.now(TimeUnit.MILLISECONDS) + "ms");
+      LOG.info("Created test file " + inputpath + " in " + sw.elapsedMillis() + "ms");
     }
     }
     job.setInputFormatClass(SequenceFileInputFormat.class);
     job.setInputFormatClass(SequenceFileInputFormat.class);
     FileInputFormat.addInputPath(job, new Path(inputpath));
     FileInputFormat.addInputPath(job, new Path(inputpath));