Kaynağa Gözat

HDFS-3641. Move server Util time methods to common and use now instead of System#currentTimeMillis. Contributed by Eli Collins

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1360896 13f79535-47bb-0310-9956-ffa450edef68
Eli Collins 13 yıl önce
ebeveyn
işleme
dce88a6dbc
100 değiştirilmiş dosya ile 466 ekleme ve 352 silme
  1. 3 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java
  2. 5 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java
  3. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java
  4. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
  5. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
  6. 4 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
  7. 5 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
  8. 3 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
  9. 9 8
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  10. 5 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
  11. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsRecordBuilderImpl.java
  12. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSinkAdapter.java
  13. 5 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSourceAdapter.java
  14. 5 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
  15. 7 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketIOWithTimeout.java
  16. 3 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
  17. 14 13
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java
  18. 8 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java
  19. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/AsyncDiskService.java
  20. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java
  21. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
  22. 3 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ThreadUtil.java
  23. 52 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java
  24. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestReconfiguration.java
  25. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java
  26. 14 13
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java
  27. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/s3native/InMemoryNativeFileSystemStore.java
  28. 6 5
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ActiveStandbyElectorTestUtil.java
  29. 9 8
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ClientBaseWithFixes.java
  30. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestHealthMonitor.java
  31. 4 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java
  32. 7 6
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverControllerStress.java
  33. 4 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java
  34. 5 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/Timer.java
  35. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
  36. 5 4
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java
  37. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLog4Json.java
  38. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java
  39. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSocketIOWithTimeout.java
  40. 2 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java
  41. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java
  42. 3 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MultithreadedTestUtil.java
  43. 9 8
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestMultithreadedTestUtil.java
  44. 1 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
  45. 5 4
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/instrumentation/InstrumentationService.java
  46. 3 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/scheduler/SchedulerService.java
  47. 17 16
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/instrumentation/TestInstrumentationService.java
  48. 10 8
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HTestCase.java
  49. 11 10
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHFSTestCase.java
  50. 12 10
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHTestCase.java
  51. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  52. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  53. 12 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  54. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java
  55. 7 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java
  56. 9 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
  57. 7 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  58. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  59. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  60. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
  61. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  62. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
  63. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
  64. 0 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Util.java
  65. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
  66. 11 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
  67. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  68. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  69. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  70. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  71. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
  72. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
  73. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  74. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  75. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  76. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  77. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  78. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  79. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  80. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  81. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  82. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
  83. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  84. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
  85. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
  86. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
  87. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/DataTransferThrottler.java
  88. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java
  89. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BenchmarkThroughput.java
  90. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DataNodeCluster.java
  91. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendDifferentChecksum.java
  92. 10 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
  93. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSMkdirs.java
  94. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java
  95. 9 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java
  96. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
  97. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
  98. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java
  99. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
  100. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java

+ 3 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java

@@ -27,6 +27,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.Time;
 
 /**
  * A daemon thread that waits for the next file system to renew.
@@ -62,7 +63,7 @@ public class DelegationTokenRenewer<T extends FileSystem & DelegationTokenRenewe
     /** Get the delay until this event should happen. */
     @Override
     public long getDelay(final TimeUnit unit) {
-      final long millisLeft = renewalTime - System.currentTimeMillis();
+      final long millisLeft = renewalTime - Time.now();
       return unit.convert(millisLeft, TimeUnit.MILLISECONDS);
     }
 
@@ -92,7 +93,7 @@ public class DelegationTokenRenewer<T extends FileSystem & DelegationTokenRenewe
      * @param newTime the new time
      */
     private void updateRenewalTime() {
-      renewalTime = RENEW_CYCLE + System.currentTimeMillis();
+      renewalTime = RENEW_CYCLE + Time.now();
     }
 
     /**

+ 5 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Time;
 
 /** Provides a <i>trash</i> feature.  Files are moved to a user's trash
  * directory, a subdirectory of their home directory named ".Trash".  Files are
@@ -136,7 +137,7 @@ public class TrashPolicyDefault extends TrashPolicy {
         String orig = trashPath.toString();
         
         while(fs.exists(trashPath)) {
-          trashPath = new Path(orig + System.currentTimeMillis());
+          trashPath = new Path(orig + Time.now());
         }
         
         if (fs.rename(path, trashPath))           // move to current trash
@@ -187,7 +188,7 @@ public class TrashPolicyDefault extends TrashPolicy {
       return;
     }
 
-    long now = System.currentTimeMillis();
+    long now = Time.now();
     for (int i = 0; i < dirs.length; i++) {
       Path path = dirs[i].getPath();
       String dir = path.toUri().getPath();
@@ -248,7 +249,7 @@ public class TrashPolicyDefault extends TrashPolicy {
     public void run() {
       if (emptierInterval == 0)
         return;                                   // trash disabled
-      long now = System.currentTimeMillis();
+      long now = Time.now();
       long end;
       while (true) {
         end = ceiling(now, emptierInterval);
@@ -259,7 +260,7 @@ public class TrashPolicyDefault extends TrashPolicy {
         }
 
         try {
-          now = System.currentTimeMillis();
+          now = Time.now();
           if (now >= end) {
 
             FileStatus[] homes = null;

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Time;
 
 /**
  * ViewFileSystem (extends the FileSystem interface) implements a client-side
@@ -146,7 +147,7 @@ public class ViewFileSystem extends FileSystem {
    */
   public ViewFileSystem() throws IOException {
     ugi = UserGroupInformation.getCurrentUser();
-    creationTime = System.currentTimeMillis();
+    creationTime = Time.now();
   }
 
   /**

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Time;
 
 
 /**
@@ -192,7 +193,7 @@ public class ViewFs extends AbstractFileSystem {
   ViewFs(final URI theUri, final Configuration conf) throws IOException,
       URISyntaxException {
     super(theUri, FsConstants.VIEWFS_SCHEME, false, -1);
-    creationTime = System.currentTimeMillis();
+    creationTime = Time.now();
     ugi = UserGroupInformation.getCurrentUser();
     config = conf;
     // Now build  client side view (i.e. client side mount table) from config.

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.hadoop.util.MergeSort;
 import org.apache.hadoop.util.PriorityQueue;
+import org.apache.hadoop.util.Time;
 
 /** 
  * <code>SequenceFile</code>s are flat files consisting of binary key/value 
@@ -835,7 +836,7 @@ public class SequenceFile {
     {
       try {                                       
         MessageDigest digester = MessageDigest.getInstance("MD5");
-        long time = System.currentTimeMillis();
+        long time = Time.now();
         digester.update((new UID()+"@"+time).getBytes());
         sync = digester.digest();
       } catch (Exception e) {

+ 4 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java

@@ -75,6 +75,7 @@ import org.apache.hadoop.security.token.TokenInfo;
 import org.apache.hadoop.security.token.TokenSelector;
 import org.apache.hadoop.util.ProtoUtil;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Time;
 
 /** A client for an IPC service.  IPC calls take a single {@link Writable} as a
  * parameter, and return a {@link Writable} as their value.  A service runs on
@@ -316,7 +317,7 @@ public class Client {
 
     /** Update lastActivity with the current time. */
     private void touch() {
-      lastActivity.set(System.currentTimeMillis());
+      lastActivity.set(Time.now());
     }
 
     /**
@@ -762,7 +763,7 @@ public class Client {
     private synchronized boolean waitForWork() {
       if (calls.isEmpty() && !shouldCloseConnection.get()  && running.get())  {
         long timeout = maxIdleTime-
-              (System.currentTimeMillis()-lastActivity.get());
+              (Time.now()-lastActivity.get());
         if (timeout>0) {
           try {
             wait(timeout);
@@ -792,7 +793,7 @@ public class Client {
      * since last I/O activity is equal to or greater than the ping interval
      */
     private synchronized void sendPing() throws IOException {
-      long curTime = System.currentTimeMillis();
+      long curTime = Time.now();
       if ( curTime - lastActivity.get() >= pingInterval) {
         lastActivity.set(curTime);
         synchronized (out) {

+ 5 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.ProtoUtil;
+import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.BlockingService;
@@ -185,7 +186,7 @@ public class ProtobufRpcEngine implements RpcEngine {
         throws ServiceException {
       long startTime = 0;
       if (LOG.isDebugEnabled()) {
-        startTime = System.currentTimeMillis();
+        startTime = Time.now();
       }
 
       HadoopRpcRequestProto rpcRequest = constructRpcRequest(method, args);
@@ -198,7 +199,7 @@ public class ProtobufRpcEngine implements RpcEngine {
       }
 
       if (LOG.isDebugEnabled()) {
-        long callTime = System.currentTimeMillis() - startTime;
+        long callTime = Time.now() - startTime;
         LOG.debug("Call: " + method.getName() + " " + callTime);
       }
       
@@ -426,10 +427,10 @@ public class ProtobufRpcEngine implements RpcEngine {
             .mergeFrom(rpcRequest.getRequest()).build();
         Message result;
         try {
-          long startTime = System.currentTimeMillis();
+          long startTime = Time.now();
           server.rpcDetailedMetrics.init(protocolImpl.protocolClass);
           result = service.callBlockingMethod(methodDescriptor, null, param);
-          int processingTime = (int) (System.currentTimeMillis() - startTime);
+          int processingTime = (int) (Time.now() - startTime);
           int qTime = (int) (startTime - receiveTime);
           if (LOG.isDebugEnabled()) {
             LOG.info("Served: " + methodName + " queueTime= " + qTime +

+ 3 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Time;
 
 import com.google.protobuf.BlockingService;
 
@@ -369,7 +370,7 @@ public class RPC {
                                int rpcTimeout,
                                RetryPolicy connectionRetryPolicy,
                                long timeout) throws IOException { 
-    long startTime = System.currentTimeMillis();
+    long startTime = Time.now();
     IOException ioe;
     while (true) {
       try {
@@ -387,7 +388,7 @@ public class RPC {
         ioe = nrthe;
       }
       // check if timed out
-      if (System.currentTimeMillis()-timeout >= startTime) {
+      if (Time.now()-timeout >= startTime) {
         throw ioe;
       }
 

+ 9 - 8
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -95,6 +95,7 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.ProtoUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -411,7 +412,7 @@ public abstract class Server {
       this.callId = id;
       this.rpcRequest = param;
       this.connection = connection;
-      this.timestamp = System.currentTimeMillis();
+      this.timestamp = Time.now();
       this.rpcResponse = null;
       this.rpcKind = kind;
     }
@@ -561,7 +562,7 @@ public abstract class Server {
      */
     private void cleanupConnections(boolean force) {
       if (force || numConnections > thresholdIdleConnections) {
-        long currentTime = System.currentTimeMillis();
+        long currentTime = Time.now();
         if (!force && (currentTime - lastCleanupRunTime) < cleanupInterval) {
           return;
         }
@@ -597,7 +598,7 @@ public abstract class Server {
           }
           else i++;
         }
-        lastCleanupRunTime = System.currentTimeMillis();
+        lastCleanupRunTime = Time.now();
       }
     }
 
@@ -682,7 +683,7 @@ public abstract class Server {
         try {
           reader.startAdd();
           SelectionKey readKey = reader.registerChannel(channel);
-          c = new Connection(readKey, channel, System.currentTimeMillis());
+          c = new Connection(readKey, channel, Time.now());
           readKey.attach(c);
           synchronized (connectionList) {
             connectionList.add(numConnections, c);
@@ -704,7 +705,7 @@ public abstract class Server {
       if (c == null) {
         return;  
       }
-      c.setLastContact(System.currentTimeMillis());
+      c.setLastContact(Time.now());
       
       try {
         count = c.readAndProcess();
@@ -726,7 +727,7 @@ public abstract class Server {
         c = null;
       }
       else {
-        c.setLastContact(System.currentTimeMillis());
+        c.setLastContact(Time.now());
       }
     }   
 
@@ -805,7 +806,7 @@ public abstract class Server {
               LOG.info(getName() + ": doAsyncWrite threw exception " + e);
             }
           }
-          long now = System.currentTimeMillis();
+          long now = Time.now();
           if (now < lastPurgeTime + PURGE_INTERVAL) {
             continue;
           }
@@ -951,7 +952,7 @@ public abstract class Server {
             
             if (inHandler) {
               // set the serve time when the response has to be sent later
-              call.timestamp = System.currentTimeMillis();
+              call.timestamp = Time.now();
               
               incPending();
               try {

+ 5 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.*;
@@ -218,13 +219,13 @@ public class WritableRpcEngine implements RpcEngine {
       throws Throwable {
       long startTime = 0;
       if (LOG.isDebugEnabled()) {
-        startTime = System.currentTimeMillis();
+        startTime = Time.now();
       }
 
       ObjectWritable value = (ObjectWritable)
         client.call(RPC.RpcKind.RPC_WRITABLE, new Invocation(method, args), remoteId);
       if (LOG.isDebugEnabled()) {
-        long callTime = System.currentTimeMillis() - startTime;
+        long callTime = Time.now() - startTime;
         LOG.debug("Call: " + method.getName() + " " + callTime);
       }
       return value.get();
@@ -464,7 +465,7 @@ public class WritableRpcEngine implements RpcEngine {
 
           // Invoke the protocol method
 
-          long startTime = System.currentTimeMillis();
+          long startTime = Time.now();
           Method method = 
               protocolImpl.protocolClass.getMethod(call.getMethodName(),
               call.getParameterClasses());
@@ -472,7 +473,7 @@ public class WritableRpcEngine implements RpcEngine {
           server.rpcDetailedMetrics.init(protocolImpl.protocolClass);
           Object value = 
               method.invoke(protocolImpl.protocolImpl, call.getParameters());
-          int processingTime = (int) (System.currentTimeMillis() - startTime);
+          int processingTime = (int) (Time.now() - startTime);
           int qTime = (int) (startTime-receivedTime);
           if (LOG.isDebugEnabled()) {
             LOG.debug("Served: " + call.getMethodName() +

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsRecordBuilderImpl.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.metrics2.MetricsFilter;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsTag;
 import org.apache.hadoop.metrics2.lib.Interns;
+import org.apache.hadoop.util.Time;
 
 class MetricsRecordBuilderImpl extends MetricsRecordBuilder {
   private final MetricsCollector parent;
@@ -44,7 +45,7 @@ class MetricsRecordBuilderImpl extends MetricsRecordBuilder {
                            MetricsFilter rf, MetricsFilter mf,
                            boolean acceptable) {
     this.parent = parent;
-    timestamp = System.currentTimeMillis();
+    timestamp = Time.now();
     recInfo = info;
     metrics = Lists.newArrayList();
     tags = Lists.newArrayList();

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSinkAdapter.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import static org.apache.hadoop.metrics2.util.Contracts.*;
 import org.apache.hadoop.metrics2.MetricsFilter;
 import org.apache.hadoop.metrics2.MetricsSink;
+import org.apache.hadoop.util.Time;
 
 /**
  * An adapter class for metrics sink and associated filters
@@ -158,7 +159,7 @@ class MetricsSinkAdapter implements SinkQueue.Consumer<MetricsBuffer> {
     }
     if (ts > 0) {
       sink.flush();
-      latency.add(System.currentTimeMillis() - ts);
+      latency.add(Time.now() - ts);
     }
     LOG.debug("Done");
   }

+ 5 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSourceAdapter.java

@@ -40,6 +40,8 @@ import org.apache.hadoop.metrics2.MetricsSource;
 import org.apache.hadoop.metrics2.MetricsTag;
 import static org.apache.hadoop.metrics2.impl.MetricsConfig.*;
 import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.util.Time;
+
 import static org.apache.hadoop.metrics2.util.Contracts.*;
 
 /**
@@ -152,9 +154,9 @@ class MetricsSourceAdapter implements DynamicMBean {
   private void updateJmxCache() {
     boolean getAllMetrics = false;
     synchronized(this) {
-      if (System.currentTimeMillis() - jmxCacheTS >= jmxCacheTTL) {
+      if (Time.now() - jmxCacheTS >= jmxCacheTTL) {
         // temporarilly advance the expiry while updating the cache
-        jmxCacheTS = System.currentTimeMillis() + jmxCacheTTL;
+        jmxCacheTS = Time.now() + jmxCacheTTL;
         if (lastRecs == null) {
           getAllMetrics = true;
         }
@@ -175,7 +177,7 @@ class MetricsSourceAdapter implements DynamicMBean {
       if (oldCacheSize < newCacheSize) {
         updateInfoCache();
       }
-      jmxCacheTS = System.currentTimeMillis();
+      jmxCacheTS = Time.now();
       lastRecs = null;  // in case regular interval update is not running
     }
   }

+ 5 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java

@@ -60,6 +60,7 @@ import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MetricsSourceBuilder;
 import org.apache.hadoop.metrics2.lib.MutableStat;
 import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.util.Time;
 
 /**
  * A base class for metrics system singletons
@@ -372,10 +373,10 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
 
   private void snapshotMetrics(MetricsSourceAdapter sa,
                                MetricsBufferBuilder bufferBuilder) {
-    long startTime = System.currentTimeMillis();
+    long startTime = Time.now();
     bufferBuilder.add(sa.name(), sa.getMetrics(collector, false));
     collector.clear();
-    snapshotStat.add(System.currentTimeMillis() - startTime);
+    snapshotStat.add(Time.now() - startTime);
     LOG.debug("Snapshotted source "+ sa.name());
   }
 
@@ -386,9 +387,9 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
   synchronized void publishMetrics(MetricsBuffer buffer) {
     int dropped = 0;
     for (MetricsSinkAdapter sa : sinks.values()) {
-      long startTime = System.currentTimeMillis();
+      long startTime = Time.now();
       dropped += sa.putMetrics(buffer, logicalTime) ? 0 : 1;
-      publishStat.add(System.currentTimeMillis() - startTime);
+      publishStat.add(Time.now() - startTime);
     }
     droppedPubAll.incr(dropped);
   }

+ 7 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketIOWithTimeout.java

@@ -34,6 +34,7 @@ import java.util.LinkedList;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
 
 /**
  * This supports input and output streams for a socket channels. 
@@ -194,7 +195,7 @@ abstract class SocketIOWithTimeout {
       }
 
       long timeoutLeft = timeout;
-      long endTime = (timeout > 0) ? (System.currentTimeMillis() + timeout): 0;
+      long endTime = (timeout > 0) ? (Time.now() + timeout): 0;
       
       while (true) {
         // we might have to call finishConnect() more than once
@@ -209,7 +210,7 @@ abstract class SocketIOWithTimeout {
         
         if (ret == 0 ||
             (timeout > 0 &&  
-              (timeoutLeft = (endTime - System.currentTimeMillis())) <= 0)) {
+              (timeoutLeft = (endTime - Time.now())) <= 0)) {
           throw new SocketTimeoutException(
                     timeoutExceptionString(channel, timeout, 
                                            SelectionKey.OP_CONNECT));
@@ -329,7 +330,7 @@ abstract class SocketIOWithTimeout {
       
       try {
         while (true) {
-          long start = (timeout == 0) ? 0 : System.currentTimeMillis();
+          long start = (timeout == 0) ? 0 : Time.now();
 
           key = channel.register(info.selector, ops);
           ret = info.selector.select(timeout);
@@ -342,7 +343,7 @@ abstract class SocketIOWithTimeout {
            * unknown reasons. So select again if required.
            */
           if (timeout > 0) {
-            timeout -= System.currentTimeMillis() - start;
+            timeout -= Time.now() - start;
             if (timeout <= 0) {
               return 0;
             }
@@ -414,7 +415,7 @@ abstract class SocketIOWithTimeout {
         selInfo = queue.removeLast();
       }
       
-      trimIdleSelectors(System.currentTimeMillis());
+      trimIdleSelectors(Time.now());
       return selInfo;
     }
     
@@ -425,7 +426,7 @@ abstract class SocketIOWithTimeout {
      * @param info
      */
     private synchronized void release(SelectorInfo info) {
-      long now = System.currentTimeMillis();
+      long now = Time.now();
       trimIdleSelectors(now);
       info.lastActivityTime = now;
       info.queue.addLast(info);

+ 3 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Time;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -75,7 +76,7 @@ public class Groups {
   public List<String> getGroups(String user) throws IOException {
     // Return cached value if available
     CachedGroups groups = userToGroupsMap.get(user);
-    long now = System.currentTimeMillis();
+    long now = Time.now();
     // if cache has a value and it hasn't expired
     if (groups != null && (groups.getTimestamp() + cacheTimeout > now)) {
       if(LOG.isDebugEnabled()) {
@@ -134,7 +135,7 @@ public class Groups {
      */
     CachedGroups(List<String> groups) {
       this.groups = groups;
-      this.timestamp = System.currentTimeMillis();
+      this.timestamp = Time.now();
     }
 
     /**

+ 14 - 13
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java

@@ -64,6 +64,7 @@ import org.apache.hadoop.security.authentication.util.KerberosUtil;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.Shell;
+import org.apache.hadoop.util.Time;
 
 /**
  * User and group information for Hadoop.
@@ -710,7 +711,7 @@ public class UserGroupInformation {
             long nextRefresh = getRefreshTime(tgt);
             while (true) {
               try {
-                long now = System.currentTimeMillis();
+                long now = Time.now();
                 if(LOG.isDebugEnabled()) {
                   LOG.debug("Current time is " + now);
                   LOG.debug("Next refresh is " + nextRefresh);
@@ -772,15 +773,15 @@ public class UserGroupInformation {
     try {
       login = newLoginContext(HadoopConfiguration.KEYTAB_KERBEROS_CONFIG_NAME,
             subject, new HadoopConfiguration());
-      start = System.currentTimeMillis();
+      start = Time.now();
       login.login();
-      metrics.loginSuccess.add(System.currentTimeMillis() - start);
+      metrics.loginSuccess.add(Time.now() - start);
       loginUser = new UserGroupInformation(subject);
       loginUser.setLogin(login);
       loginUser.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
     } catch (LoginException le) {
       if (start > 0) {
-        metrics.loginFailure.add(System.currentTimeMillis() - start);
+        metrics.loginFailure.add(Time.now() - start);
       }
       throw new IOException("Login failure for " + user + " from keytab " + 
                             path, le);
@@ -800,7 +801,7 @@ public class UserGroupInformation {
         || !isKeytab)
       return;
     KerberosTicket tgt = getTGT();
-    if (tgt != null && System.currentTimeMillis() < getRefreshTime(tgt)) {
+    if (tgt != null && Time.now() < getRefreshTime(tgt)) {
       return;
     }
     reloginFromKeytab();
@@ -824,7 +825,7 @@ public class UserGroupInformation {
          !isKeytab)
       return;
     
-    long now = System.currentTimeMillis();
+    long now = Time.now();
     if (!hasSufficientTimeElapsed(now)) {
       return;
     }
@@ -856,14 +857,14 @@ public class UserGroupInformation {
             HadoopConfiguration.KEYTAB_KERBEROS_CONFIG_NAME, getSubject(),
             new HadoopConfiguration());
         LOG.info("Initiating re-login for " + keytabPrincipal);
-        start = System.currentTimeMillis();
+        start = Time.now();
         login.login();
-        metrics.loginSuccess.add(System.currentTimeMillis() - start);
+        metrics.loginSuccess.add(Time.now() - start);
         setLogin(login);
       }
     } catch (LoginException le) {
       if (start > 0) {
-        metrics.loginFailure.add(System.currentTimeMillis() - start);
+        metrics.loginFailure.add(Time.now() - start);
       }
       throw new IOException("Login failure for " + keytabPrincipal + 
           " from keytab " + keytabFile, le);
@@ -889,7 +890,7 @@ public class UserGroupInformation {
     if (login == null) {
       throw new IOException("login must be done first");
     }
-    long now = System.currentTimeMillis();
+    long now = Time.now();
     if (!hasSufficientTimeElapsed(now)) {
       return;
     }
@@ -944,9 +945,9 @@ public class UserGroupInformation {
           HadoopConfiguration.KEYTAB_KERBEROS_CONFIG_NAME, subject,
           new HadoopConfiguration());
        
-      start = System.currentTimeMillis();
+      start = Time.now();
       login.login();
-      metrics.loginSuccess.add(System.currentTimeMillis() - start);
+      metrics.loginSuccess.add(Time.now() - start);
       UserGroupInformation newLoginUser = new UserGroupInformation(subject);
       newLoginUser.setLogin(login);
       newLoginUser.setAuthenticationMethod(AuthenticationMethod.KERBEROS);
@@ -954,7 +955,7 @@ public class UserGroupInformation {
       return newLoginUser;
     } catch (LoginException le) {
       if (start > 0) {
-        metrics.loginFailure.add(System.currentTimeMillis() - start);
+        metrics.loginFailure.add(Time.now() - start);
       }
       throw new IOException("Login failure for " + user + " from keytab " + 
                             path, le);

+ 8 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.security.HadoopKerberosName;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.Time;
 
 import com.google.common.base.Preconditions;
 
@@ -165,7 +166,7 @@ extends AbstractDelegationTokenIdentifier>
     synchronized (this) {
       removeExpiredKeys();
       /* set final expiry date for retiring currentKey */
-      currentKey.setExpiryDate(System.currentTimeMillis() + tokenMaxLifetime);
+      currentKey.setExpiryDate(Time.now() + tokenMaxLifetime);
       /*
        * currentKey might have been removed by removeExpiredKeys(), if
        * updateMasterKey() isn't called at expected interval. Add it back to
@@ -177,7 +178,7 @@ extends AbstractDelegationTokenIdentifier>
   }
 
   private synchronized void removeExpiredKeys() {
-    long now = System.currentTimeMillis();
+    long now = Time.now();
     for (Iterator<Map.Entry<Integer, DelegationKey>> it = allKeys.entrySet()
         .iterator(); it.hasNext();) {
       Map.Entry<Integer, DelegationKey> e = it.next();
@@ -191,7 +192,7 @@ extends AbstractDelegationTokenIdentifier>
   protected synchronized byte[] createPassword(TokenIdent identifier) {
     LOG.info("Creating password for identifier: "+identifier);
     int sequenceNum;
-    long now = System.currentTimeMillis();
+    long now = Time.now();
     sequenceNum = ++delegationTokenSequenceNumber;
     identifier.setIssueDate(now);
     identifier.setMaxDate(now + tokenMaxLifetime);
@@ -211,7 +212,7 @@ extends AbstractDelegationTokenIdentifier>
       throw new InvalidToken("token (" + identifier.toString()
           + ") can't be found in cache");
     }
-    long now = System.currentTimeMillis();
+    long now = Time.now();
     if (info.getRenewDate() < now) {
       throw new InvalidToken("token (" + identifier.toString() + ") is expired");
     }
@@ -243,7 +244,7 @@ extends AbstractDelegationTokenIdentifier>
    */
   public synchronized long renewToken(Token<TokenIdent> token,
                          String renewer) throws InvalidToken, IOException {
-    long now = System.currentTimeMillis();
+    long now = Time.now();
     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
     DataInputStream in = new DataInputStream(buf);
     TokenIdent id = createIdentifier();
@@ -353,7 +354,7 @@ extends AbstractDelegationTokenIdentifier>
   
   /** Remove expired delegation tokens from cache */
   private synchronized void removeExpiredToken() {
-    long now = System.currentTimeMillis();
+    long now = Time.now();
     Iterator<DelegationTokenInformation> i = currentTokens.values().iterator();
     while (i.hasNext()) {
       long renewDate = i.next().getRenewDate();
@@ -399,7 +400,7 @@ extends AbstractDelegationTokenIdentifier>
           / (60 * 1000) + " min(s)");
       try {
         while (running) {
-          long now = System.currentTimeMillis();
+          long now = Time.now();
           if (lastMasterKeyUpdate + keyUpdateInterval < now) {
             try {
               rollMasterKey();

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/AsyncDiskService.java

@@ -126,12 +126,12 @@ public class AsyncDiskService {
   public synchronized boolean awaitTermination(long milliseconds) 
       throws InterruptedException {
 
-    long end = System.currentTimeMillis() + milliseconds;
+    long end = Time.now() + milliseconds;
     for (Map.Entry<String, ThreadPoolExecutor> e:
         executors.entrySet()) {
       ThreadPoolExecutor executor = e.getValue();
       if (!executor.awaitTermination(
-          Math.max(end - System.currentTimeMillis(), 0),
+          Math.max(end - Time.now(), 0),
           TimeUnit.MILLISECONDS)) {
         LOG.warn("AsyncDiskService awaitTermination timeout.");
         return false;

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java

@@ -205,7 +205,7 @@ public class ReflectionUtils {
     boolean dumpStack = false;
     if (log.isInfoEnabled()) {
       synchronized (ReflectionUtils.class) {
-        long now = System.currentTimeMillis();
+        long now = Time.now();
         if (now - previousLogTime >= minInterval * 1000) {
           previousLogTime = now;
           dumpStack = true;

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java

@@ -124,7 +124,7 @@ abstract public class Shell {
 
   /** check to see if a command needs to be executed and execute if needed */
   protected void run() throws IOException {
-    if (lastTime + interval > System.currentTimeMillis())
+    if (lastTime + interval > Time.now())
       return;
     exitCode = 0; // reset for next run
     runCommand();
@@ -223,7 +223,7 @@ abstract public class Shell {
         LOG.warn("Error while closing the error stream", ioe);
       }
       process.destroy();
-      lastTime = System.currentTimeMillis();
+      lastTime = Time.now();
     }
   }
 

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

@@ -35,10 +35,10 @@ public class ThreadUtil {
    * @param millis the number of milliseconds for the current thread to sleep
    */
   public static void sleepAtLeastIgnoreInterrupts(long millis) {
-    long start = System.currentTimeMillis();
-    while (System.currentTimeMillis() - start < millis) {
+    long start = Time.now();
+    while (Time.now() - start < millis) {
       long timeToSleep = millis -
-          (System.currentTimeMillis() - start);
+          (Time.now() - start);
       try {
         Thread.sleep(timeToSleep);
       } catch (InterruptedException ie) {

+ 52 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java

@@ -0,0 +1,52 @@
+/**
+ * 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.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Utility methods for getting the time and computing intervals.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Unstable
+public final class Time {
+
+  /**
+   * Current system time.  Do not use this to calculate a duration or interval
+   * to sleep, because it will be broken by settimeofday.  Instead, use
+   * monotonicNow.
+   * @return current time in msec.
+   */
+  public static long now() {
+    return System.currentTimeMillis();
+  }
+  
+  /**
+   * Current time from some arbitrary time base in the past, counting in
+   * milliseconds, and not affected by settimeofday or similar system clock
+   * changes.  This is appropriate to use when computing how much longer to
+   * wait for an interval to expire.
+   * @return a monotonic clock that counts in milliseconds.
+   */
+  public static long monotonicNow() {
+    final long NANOSECONDS_PER_MILLISECOND = 1000000;
+
+    return System.nanoTime() / NANOSECONDS_PER_MILLISECOND;
+  }
+}

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestReconfiguration.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.conf;
 
+import org.apache.hadoop.util.Time;
 import org.junit.Test;
 import org.junit.Before;
 import static org.junit.Assert.*;
@@ -295,8 +296,8 @@ public class TestReconfiguration {
     }
     dummy.reconfigureProperty(PROP1, VAL2);
 
-    long endWait = System.currentTimeMillis() + 2000;
-    while (dummyThread.isAlive() && System.currentTimeMillis() < endWait) {
+    long endWait = Time.now() + 2000;
+    while (dummyThread.isAlive() && Time.now() < endWait) {
       try {
         Thread.sleep(50);
       } catch (InterruptedException ignore) {

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java

@@ -32,6 +32,7 @@ import java.util.Set;
 import junit.framework.TestCase;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Time;
 
 /**
  * This class tests commands from Trash.
@@ -600,7 +601,7 @@ public class TestTrash extends TestCase {
       
       writeFile(fs, myFile, 10);
       
-      start = System.currentTimeMillis();
+      start = Time.now();
       
       try {
         retVal = shell.run(args);
@@ -612,7 +613,7 @@ public class TestTrash extends TestCase {
       
       assertTrue(retVal == 0);
       
-      long iterTime = System.currentTimeMillis() - start;
+      long iterTime = Time.now() - start;
       // take median of the first 10 runs
       if(i<10) {
         if(i==0) {

+ 14 - 13
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/loadGenerator/LoadGenerator.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options.CreateOpts;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -121,7 +122,7 @@ public class LoadGenerator extends Configured implements Tool {
   private double [] writeProbs = {0.3333};
   private volatile int currentIndex = 0;
   long totalTime = 0;
-  private long startTime = System.currentTimeMillis()+10000;
+  private long startTime = Time.now()+10000;
   final static private int BLOCK_SIZE = 10;
   private ArrayList<String> files = new ArrayList<String>();  // a table of file names
   private ArrayList<String> dirs = new ArrayList<String>(); // a table of directory names
@@ -232,9 +233,9 @@ public class LoadGenerator extends Configured implements Tool {
      * the entire file */
     private void read() throws IOException {
       String fileName = files.get(r.nextInt(files.size()));
-      long startTime = System.currentTimeMillis();
+      long startTime = Time.now();
       InputStream in = fc.open(new Path(fileName));
-      executionTime[OPEN] += (System.currentTimeMillis()-startTime);
+      executionTime[OPEN] += (Time.now()-startTime);
       totalNumOfOps[OPEN]++;
       while (in.read(buffer) != -1) {}
       in.close();
@@ -254,9 +255,9 @@ public class LoadGenerator extends Configured implements Tool {
       double fileSize = 0;
       while ((fileSize = r.nextGaussian()+2)<=0) {}
       genFile(file, (long)(fileSize*BLOCK_SIZE));
-      long startTime = System.currentTimeMillis();
+      long startTime = Time.now();
       fc.delete(file, true);
-      executionTime[DELETE] += (System.currentTimeMillis()-startTime);
+      executionTime[DELETE] += (Time.now()-startTime);
       totalNumOfOps[DELETE]++;
     }
     
@@ -265,9 +266,9 @@ public class LoadGenerator extends Configured implements Tool {
      */
     private void list() throws IOException {
       String dirName = dirs.get(r.nextInt(dirs.size()));
-      long startTime = System.currentTimeMillis();
+      long startTime = Time.now();
       fc.listStatus(new Path(dirName));
-      executionTime[LIST] += (System.currentTimeMillis()-startTime);
+      executionTime[LIST] += (Time.now()-startTime);
       totalNumOfOps[LIST]++;
     }
   }
@@ -435,7 +436,7 @@ public class LoadGenerator extends Configured implements Tool {
     }
     
     if (r==null) {
-      r = new Random(System.currentTimeMillis()+hostHashCode);
+      r = new Random(Time.now()+hostHashCode);
     }
     
     return initFileDirTables();
@@ -571,7 +572,7 @@ public class LoadGenerator extends Configured implements Tool {
    */
   private void barrier() {
     long sleepTime;
-    while ((sleepTime = startTime - System.currentTimeMillis()) > 0) {
+    while ((sleepTime = startTime - Time.now()) > 0) {
       try {
         Thread.sleep(sleepTime);
       } catch (InterruptedException ex) {
@@ -583,20 +584,20 @@ public class LoadGenerator extends Configured implements Tool {
    * The file is filled with 'a'.
    */
   private void genFile(Path file, long fileSize) throws IOException {
-    long startTime = System.currentTimeMillis();
+    long startTime = Time.now();
     FSDataOutputStream out = fc.create(file,
         EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
         CreateOpts.createParent(), CreateOpts.bufferSize(4096),
         CreateOpts.repFac((short) 3));
-    executionTime[CREATE] += (System.currentTimeMillis()-startTime);
+    executionTime[CREATE] += (Time.now()-startTime);
     totalNumOfOps[CREATE]++;
 
     for (long i=0; i<fileSize; i++) {
       out.writeByte('a');
     }
-    startTime = System.currentTimeMillis();
+    startTime = Time.now();
     out.close();
-    executionTime[WRITE_CLOSE] += (System.currentTimeMillis()-startTime);
+    executionTime[WRITE_CLOSE] += (Time.now()-startTime);
     totalNumOfOps[WRITE_CLOSE]++;
   }
   

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/s3native/InMemoryNativeFileSystemStore.java

@@ -39,6 +39,7 @@ import java.util.TreeSet;
 import java.util.Map.Entry;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Time;
 
 /**
  * <p>
@@ -59,7 +60,7 @@ class InMemoryNativeFileSystemStore implements NativeFileSystemStore {
   }
 
   public void storeEmptyFile(String key) throws IOException {
-    metadataMap.put(key, new FileMetadata(key, 0, System.currentTimeMillis()));
+    metadataMap.put(key, new FileMetadata(key, 0, Time.now()));
     dataMap.put(key, new byte[0]);
   }
 
@@ -81,7 +82,7 @@ class InMemoryNativeFileSystemStore implements NativeFileSystemStore {
       }
     }
     metadataMap.put(key,
-        new FileMetadata(key, file.length(), System.currentTimeMillis()));
+        new FileMetadata(key, file.length(), Time.now()));
     dataMap.put(key, out.toByteArray());
   }
 

+ 6 - 5
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ActiveStandbyElectorTestUtil.java

@@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.server.ZooKeeperServer;
@@ -36,7 +37,7 @@ public abstract class ActiveStandbyElectorTestUtil {
   public static void waitForActiveLockData(TestContext ctx,
       ZooKeeperServer zks, String parentDir, byte[] activeData)
       throws Exception {
-    long st = System.currentTimeMillis();
+    long st = Time.now();
     long lastPrint = st;
     while (true) {
       if (ctx != null) {
@@ -51,17 +52,17 @@ public abstract class ActiveStandbyElectorTestUtil {
             Arrays.equals(activeData, data)) {
           return;
         }
-        if (System.currentTimeMillis() > lastPrint + LOG_INTERVAL_MS) {
+        if (Time.now() > lastPrint + LOG_INTERVAL_MS) {
           LOG.info("Cur data: " + StringUtils.byteToHexString(data));
-          lastPrint = System.currentTimeMillis();
+          lastPrint = Time.now();
         }
       } catch (NoNodeException nne) {
         if (activeData == null) {
           return;
         }
-        if (System.currentTimeMillis() > lastPrint + LOG_INTERVAL_MS) {
+        if (Time.now() > lastPrint + LOG_INTERVAL_MS) {
           LOG.info("Cur data: no node");
-          lastPrint = System.currentTimeMillis();
+          lastPrint = Time.now();
         }
       }
       Thread.sleep(50);

+ 9 - 8
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/ClientBaseWithFixes.java

@@ -32,6 +32,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
+import org.apache.hadoop.util.Time;
 import org.apache.zookeeper.PortAssignment;
 import org.apache.zookeeper.TestableZooKeeper;
 import org.apache.zookeeper.WatchedEvent;
@@ -111,11 +112,11 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
             return connected;
         }
         synchronized void waitForConnected(long timeout) throws InterruptedException, TimeoutException {
-            long expire = System.currentTimeMillis() + timeout;
+            long expire = Time.now() + timeout;
             long left = timeout;
             while(!connected && left > 0) {
                 wait(left);
-                left = expire - System.currentTimeMillis();
+                left = expire - Time.now();
             }
             if (!connected) {
                 throw new TimeoutException("Did not connect");
@@ -123,11 +124,11 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
             }
         }
         synchronized void waitForDisconnected(long timeout) throws InterruptedException, TimeoutException {
-            long expire = System.currentTimeMillis() + timeout;
+            long expire = Time.now() + timeout;
             long left = timeout;
             while(connected && left > 0) {
                 wait(left);
-                left = expire - System.currentTimeMillis();
+                left = expire - Time.now();
             }
             if (connected) {
                 throw new TimeoutException("Did not disconnect");
@@ -248,7 +249,7 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
     }
 
     public static boolean waitForServerUp(String hp, long timeout) {
-        long start = System.currentTimeMillis();
+        long start = Time.now();
         while (true) {
             try {
                 // if there are multiple hostports, just take the first one
@@ -263,7 +264,7 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
                 LOG.info("server " + hp + " not up " + e);
             }
 
-            if (System.currentTimeMillis() > start + timeout) {
+            if (Time.now() > start + timeout) {
                 break;
             }
             try {
@@ -275,7 +276,7 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
         return false;
     }
     public static boolean waitForServerDown(String hp, long timeout) {
-        long start = System.currentTimeMillis();
+        long start = Time.now();
         while (true) {
             try {
                 HostPort hpobj = parseHostPortList(hp).get(0);
@@ -284,7 +285,7 @@ public abstract class ClientBaseWithFixes extends ZKTestCase {
                 return true;
             }
 
-            if (System.currentTimeMillis() > start + timeout) {
+            if (Time.now() > start + timeout) {
                 break;
             }
             try {

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestHealthMonitor.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HealthMonitor.Callback;
 import org.apache.hadoop.ha.HealthMonitor.State;
+import org.apache.hadoop.util.Time;
 
 import org.junit.Before;
 import org.junit.Test;
@@ -136,8 +137,8 @@ public class TestHealthMonitor {
 
   private void waitForState(HealthMonitor hm, State state)
       throws InterruptedException {
-    long st = System.currentTimeMillis();
-    while (System.currentTimeMillis() - st < 2000) {
+    long st = Time.now();
+    while (Time.now() - st < 2000) {
       if (hm.getHealthState() == state) {
         return;
       }

+ 4 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
 import org.apache.hadoop.ha.HealthMonitor.State;
 import org.apache.hadoop.ha.MiniZKFCCluster.DummyZKFC;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
@@ -394,9 +395,9 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
       // Ask it to cede active for 3 seconds. It should respond promptly
       // (i.e. the RPC itself should not take 3 seconds!)
       ZKFCProtocol proxy = zkfc.getLocalTarget().getZKFCProxy(conf, 5000);
-      long st = System.currentTimeMillis();
+      long st = Time.now();
       proxy.cedeActive(3000);
-      long et = System.currentTimeMillis();
+      long et = Time.now();
       assertTrue("RPC to cedeActive took " + (et - st) + " ms",
           et - st < 1000);
       
@@ -408,7 +409,7 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
       // After the prescribed 3 seconds, should go into STANDBY state,
       // since the other node in the cluster would have taken ACTIVE.
       cluster.waitForElectorState(0, ActiveStandbyElector.State.STANDBY);
-      long et2 = System.currentTimeMillis();
+      long et2 = Time.now();
       assertTrue("Should take ~3 seconds to rejoin. Only took " + (et2 - et) +
           "ms before rejoining.",
           et2 - et > 2800);      

+ 7 - 6
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverControllerStress.java

@@ -21,6 +21,7 @@ import java.util.Random;
 
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.Time;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -61,11 +62,11 @@ public class TestZKFailoverControllerStress extends ClientBaseWithFixes {
   @Test(timeout=(STRESS_RUNTIME_SECS + EXTRA_TIMEOUT_SECS) * 1000)
   public void testExpireBackAndForth() throws Exception {
     cluster.start();
-    long st = System.currentTimeMillis();
+    long st = Time.now();
     long runFor = STRESS_RUNTIME_SECS * 1000;
 
     int i = 0;
-    while (System.currentTimeMillis() - st < runFor) {
+    while (Time.now() - st < runFor) {
       // flip flop the services back and forth
       int from = i % 2;
       int to = (i + 1) % 2;
@@ -87,11 +88,11 @@ public class TestZKFailoverControllerStress extends ClientBaseWithFixes {
   @Test(timeout=(STRESS_RUNTIME_SECS + EXTRA_TIMEOUT_SECS) * 1000)
   public void testRandomExpirations() throws Exception {
     cluster.start();
-    long st = System.currentTimeMillis();
+    long st = Time.now();
     long runFor = STRESS_RUNTIME_SECS * 1000;
 
     Random r = new Random();
-    while (System.currentTimeMillis() - st < runFor) {
+    while (Time.now() - st < runFor) {
       cluster.getTestContext().checkException();
       int targetIdx = r.nextInt(2);
       ActiveStandbyElector target = cluster.getElector(targetIdx);
@@ -125,8 +126,8 @@ public class TestZKFailoverControllerStress extends ClientBaseWithFixes {
     // setting up the mock.
     cluster.start();
     
-    long st = System.currentTimeMillis();
-    while (System.currentTimeMillis() - st < runFor) {
+    long st = Time.now();
+    while (Time.now() - st < runFor) {
       cluster.getTestContext().checkException();
       serverFactory.closeAll();
       Thread.sleep(50);

+ 4 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.file.tfile.TFile.Reader.Scanner.Entry;
+import org.apache.hadoop.util.Time;
 
 public class TestTFileSeqFileComparison extends TestCase {
   MyOptions options;
@@ -86,12 +87,12 @@ public class TestTFileSeqFileComparison extends TestCase {
   }
 
   public void startTime() throws IOException {
-    startTimeEpoch = System.currentTimeMillis();
+    startTimeEpoch = Time.now();
     System.out.println(formatTime() + " Started timing.");
   }
 
   public void stopTime() throws IOException {
-    finishTimeEpoch = System.currentTimeMillis();
+    finishTimeEpoch = Time.now();
     System.out.println(formatTime() + " Stopped timing.");
   }
 
@@ -111,7 +112,7 @@ public class TestTFileSeqFileComparison extends TestCase {
   }
 
   public String formatTime() {
-    return formatTime(System.currentTimeMillis());
+    return formatTime(Time.now());
   }
 
   private interface KVAppendable {

+ 5 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/Timer.java

@@ -20,6 +20,8 @@ import java.io.IOException;
 import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 
+import org.apache.hadoop.util.Time;
+
 /**
  * this class is a time class to 
  * measure to measure the time 
@@ -31,11 +33,11 @@ public  class Timer {
   private DateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
   
   public void startTime() throws IOException {
-      startTimeEpoch = System.currentTimeMillis();
+      startTimeEpoch = Time.now();
     }
 
     public void stopTime() throws IOException {
-      finishTimeEpoch = System.currentTimeMillis();
+      finishTimeEpoch = Time.now();
     }
 
     public long getIntervalMillis() throws IOException {
@@ -56,7 +58,7 @@ public  class Timer {
     }
     
     public String formatCurrentTime() {
-      return formatTime(System.currentTimeMillis());
+      return formatTime(Time.now());
     }
 
 }

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.NativeCodeLoader;
+import org.apache.hadoop.util.Time;
 
 public class TestNativeIO {
   static final Log LOG = LogFactory.getLog(TestNativeIO.class);
@@ -88,8 +89,8 @@ public class TestNativeIO {
     for (int i = 0; i < 10; i++) {
       Thread statter = new Thread() {
         public void run() {
-          long et = System.currentTimeMillis() + 5000;
-          while (System.currentTimeMillis() < et) {
+          long et = Time.now() + 5000;
+          while (Time.now() < et) {
             try {
               NativeIO.Stat stat = NativeIO.fstat(fos.getFD());
               assertEquals(System.getProperty("user.name"), stat.getOwner());

+ 5 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.security.token.TokenInfo;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
 import org.apache.hadoop.security.token.delegation.TestDelegationToken.TestDelegationTokenIdentifier;
 import org.apache.hadoop.security.token.delegation.TestDelegationToken.TestDelegationTokenSecretManager;
+import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 
@@ -186,10 +187,10 @@ public class MiniRPCBenchmark {
   throws IOException {
     MiniProtocol client = null;
     try {
-      long start = System.currentTimeMillis();
+      long start = Time.now();
       client = (MiniProtocol) RPC.getProxy(MiniProtocol.class,
           MiniProtocol.versionID, addr, conf);
-      long end = System.currentTimeMillis();
+      long end = Time.now();
       return end - start;
     } finally {
       RPC.stopProxy(client);
@@ -231,7 +232,7 @@ public class MiniRPCBenchmark {
       final Configuration conf, final InetSocketAddress addr) throws IOException {
     MiniProtocol client = null;
     try {
-      long start = System.currentTimeMillis();
+      long start = Time.now();
       try {
         client = currentUgi.doAs(new PrivilegedExceptionAction<MiniProtocol>() {
           public MiniProtocol run() throws IOException {
@@ -242,7 +243,7 @@ public class MiniRPCBenchmark {
       } catch (InterruptedException e) {
         e.printStackTrace();
       }
-      long end = System.currentTimeMillis();
+      long end = Time.now();
       return end - start;
     } finally {
       RPC.stopProxy(client);

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/log/TestLog4Json.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.log;
 import junit.framework.TestCase;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.Time;
 import org.apache.log4j.Appender;
 import org.apache.log4j.Category;
 import org.apache.log4j.Level;
@@ -63,7 +64,7 @@ public class TestLog4Json extends TestCase {
         new NoRouteToHostException("that box caught fire 3 years ago");
     ThrowableInformation ti = new ThrowableInformation(e);
     Log4Json l4j = new Log4Json();
-    long timeStamp = System.currentTimeMillis();
+    long timeStamp = Time.now();
     String outcome = l4j.toJson(new StringWriter(),
         "testException",
         timeStamp,
@@ -82,7 +83,7 @@ public class TestLog4Json extends TestCase {
     Exception ioe = new IOException("Datacenter problems", e);
     ThrowableInformation ti = new ThrowableInformation(ioe);
     Log4Json l4j = new Log4Json();
-    long timeStamp = System.currentTimeMillis();
+    long timeStamp = Time.now();
     String outcome = l4j.toJson(new StringWriter(),
         "testNestedException",
         timeStamp,

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java

@@ -25,6 +25,7 @@ import javax.naming.NameNotFoundException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.Time;
 
 import org.junit.Test;
 import static org.junit.Assert.*;
@@ -57,9 +58,9 @@ public class TestDNS {
     String hostname1 = DNS.getDefaultHost(DEFAULT);
     assertNotNull(hostname1);
     String hostname2 = DNS.getDefaultHost(DEFAULT);
-    long t1 = System.currentTimeMillis();
+    long t1 = Time.now();
     String hostname3 = DNS.getDefaultHost(DEFAULT);
-    long t2 = System.currentTimeMillis();
+    long t2 = Time.now();
     assertEquals(hostname3, hostname2);
     assertEquals(hostname2, hostname1);
     long interval = t2 - t1;

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestSocketIOWithTimeout.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.MultithreadedTestUtil;
 import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
 import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread;
+import org.apache.hadoop.util.Time;
 
 import org.junit.Test;
 import static org.junit.Assert.*;
@@ -59,7 +60,7 @@ public class TestSocketIOWithTimeout {
     byte buf[] = new byte[4192];
     
     while (true) {
-      long start = System.currentTimeMillis();
+      long start = Time.now();
       try {
         if (in != null) {
           in.read(buf);
@@ -67,7 +68,7 @@ public class TestSocketIOWithTimeout {
           out.write(buf);
         }
       } catch (SocketTimeoutException e) {
-        long diff = System.currentTimeMillis() - start;
+        long diff = Time.now() - start;
         LOG.info("Got SocketTimeoutException as expected after " + 
                  diff + " millis : " + e.getMessage());
         assertTrue(Math.abs(expectedTimeout - diff) <=

+ 2 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/token/delegation/TestDelegationToken.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.DelegationTokenInformation;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
@@ -188,7 +189,7 @@ public class TestDelegationToken {
         }
       }, AccessControlException.class);
       long time = dtSecretManager.renewToken(token, "JobTracker");
-      assertTrue("renew time is in future", time > System.currentTimeMillis());
+      assertTrue("renew time is in future", time > Time.now());
       TestDelegationTokenIdentifier identifier = 
         new TestDelegationTokenIdentifier();
       byte[] tokenId = token.getIdentifier();

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java

@@ -31,6 +31,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
 import org.apache.log4j.Layout;
 import org.apache.log4j.Logger;
 import org.apache.log4j.WriterAppender;
@@ -94,7 +95,7 @@ public abstract class GenericTestUtils {
       int checkEveryMillis, int waitForMillis)
       throws TimeoutException, InterruptedException
   {
-    long st = System.currentTimeMillis();
+    long st = Time.now();
     do {
       boolean result = check.get();
       if (result) {
@@ -102,7 +103,7 @@ public abstract class GenericTestUtils {
       }
       
       Thread.sleep(checkEveryMillis);
-    } while (System.currentTimeMillis() - st < waitForMillis);
+    } while (Time.now() - st < waitForMillis);
     throw new TimeoutException("Timed out waiting for condition");
   }
   

+ 3 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MultithreadedTestUtil.java

@@ -22,6 +22,7 @@ import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.Time;
 
 /**
  * A utility to easily test threaded/synchronized code.
@@ -109,10 +110,10 @@ public abstract class MultithreadedTestUtil {
      * have thrown up an error.
      */
     public synchronized void waitFor(long millis) throws Exception {
-      long endTime = System.currentTimeMillis() + millis;
+      long endTime = Time.now() + millis;
       while (shouldRun() &&
              finishedThreads.size() < testThreads.size()) {
-        long left = endTime - System.currentTimeMillis();
+        long left = endTime - Time.now();
         if (left <= 0) break;
         checkException();
         wait(left);

+ 9 - 8
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestMultithreadedTestUtil.java

@@ -26,6 +26,7 @@ import org.junit.Test;
 import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
 import org.apache.hadoop.test.MultithreadedTestUtil.TestingThread;
 import org.apache.hadoop.test.MultithreadedTestUtil.RepeatingTestThread;
+import org.apache.hadoop.util.Time;
 
 public class TestMultithreadedTestUtil {
 
@@ -47,9 +48,9 @@ public class TestMultithreadedTestUtil {
     }
     assertEquals(0, threadsRun.get());
     ctx.startThreads();
-    long st = System.currentTimeMillis();
+    long st = Time.now();
     ctx.waitFor(30000);
-    long et = System.currentTimeMillis();
+    long et = Time.now();
 
     // All threads should have run
     assertEquals(3, threadsRun.get());
@@ -69,7 +70,7 @@ public class TestMultithreadedTestUtil {
       }
     });
     ctx.startThreads();
-    long st = System.currentTimeMillis();
+    long st = Time.now();
     try {
       ctx.waitFor(30000);
       fail("waitFor did not throw");
@@ -77,7 +78,7 @@ public class TestMultithreadedTestUtil {
       // expected
       assertEquals(FAIL_MSG, rte.getCause().getMessage());
     }
-    long et = System.currentTimeMillis();
+    long et = Time.now();
     // Test shouldn't have waited the full 30 seconds, since
     // the thread throws faster than that
     assertTrue("Test took " + (et - st) + "ms",
@@ -94,7 +95,7 @@ public class TestMultithreadedTestUtil {
       }
     });
     ctx.startThreads();
-    long st = System.currentTimeMillis();
+    long st = Time.now();
     try {
       ctx.waitFor(30000);
       fail("waitFor did not throw");
@@ -102,7 +103,7 @@ public class TestMultithreadedTestUtil {
       // expected
       assertEquals("my ioe", rte.getCause().getMessage());
     }
-    long et = System.currentTimeMillis();
+    long et = Time.now();
     // Test shouldn't have waited the full 30 seconds, since
     // the thread throws faster than that
     assertTrue("Test took " + (et - st) + "ms",
@@ -121,10 +122,10 @@ public class TestMultithreadedTestUtil {
       }
     });
     ctx.startThreads();
-    long st = System.currentTimeMillis();
+    long st = Time.now();
     ctx.waitFor(3000);
     ctx.stop();
-    long et = System.currentTimeMillis();
+    long et = Time.now();
     long elapsed = et - st;
 
     // Test should have waited just about 3 seconds

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java

@@ -55,7 +55,7 @@ public class TestShell extends TestCase {
     testInterval(Long.MIN_VALUE / 60000);  // test a negative interval
     testInterval(0L);  // test a zero interval
     testInterval(10L); // interval equal to 10mins
-    testInterval(System.currentTimeMillis() / 60000 + 60); // test a very big interval
+    testInterval(Time.now() / 60000 + 60); // test a very big interval
   }
 
   /**

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/instrumentation/InstrumentationService.java

@@ -22,6 +22,7 @@ import org.apache.hadoop.lib.server.BaseService;
 import org.apache.hadoop.lib.server.ServiceException;
 import org.apache.hadoop.lib.service.Instrumentation;
 import org.apache.hadoop.lib.service.Scheduler;
+import org.apache.hadoop.util.Time;
 import org.json.simple.JSONAware;
 import org.json.simple.JSONObject;
 import org.json.simple.JSONStreamAware;
@@ -164,10 +165,10 @@ public class InstrumentationService extends BaseService implements Instrumentati
         throw new IllegalStateException("Cron already used");
       }
       if (start == 0) {
-        start = System.currentTimeMillis();
+        start = Time.now();
         lapStart = start;
       } else if (lapStart == 0) {
-        lapStart = System.currentTimeMillis();
+        lapStart = Time.now();
       }
       return this;
     }
@@ -177,7 +178,7 @@ public class InstrumentationService extends BaseService implements Instrumentati
         throw new IllegalStateException("Cron already used");
       }
       if (lapStart > 0) {
-        own += System.currentTimeMillis() - lapStart;
+        own += Time.now() - lapStart;
         lapStart = 0;
       }
       return this;
@@ -185,7 +186,7 @@ public class InstrumentationService extends BaseService implements Instrumentati
 
     void end() {
       stop();
-      total = System.currentTimeMillis() - start;
+      total = Time.now() - start;
     }
 
   }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/scheduler/SchedulerService.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.lib.server.ServiceException;
 import org.apache.hadoop.lib.service.Instrumentation;
 import org.apache.hadoop.lib.service.Scheduler;
 import org.apache.hadoop.lib.util.Check;
+import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -59,11 +60,11 @@ public class SchedulerService extends BaseService implements Scheduler {
   @Override
   public void destroy() {
     try {
-      long limit = System.currentTimeMillis() + 30 * 1000;
+      long limit = Time.now() + 30 * 1000;
       scheduler.shutdownNow();
       while (!scheduler.awaitTermination(1000, TimeUnit.MILLISECONDS)) {
         LOG.debug("Waiting for scheduler to shutdown");
-        if (System.currentTimeMillis() > limit) {
+        if (Time.now() > limit) {
           LOG.warn("Gave up waiting for scheduler to shutdown");
           break;
         }

+ 17 - 16
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/service/instrumentation/TestInstrumentationService.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.test.HTestCase;
 import org.apache.hadoop.test.TestDir;
 import org.apache.hadoop.test.TestDirHelper;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
 import org.json.simple.JSONObject;
 import org.json.simple.parser.JSONParser;
 import org.junit.Test;
@@ -50,26 +51,26 @@ public class TestInstrumentationService extends HTestCase {
     Assert.assertEquals(cron.lapStart, 0);
     Assert.assertEquals(cron.own, 0);
     Assert.assertEquals(cron.total, 0);
-    long begin = System.currentTimeMillis();
+    long begin = Time.now();
     Assert.assertEquals(cron.start(), cron);
     Assert.assertEquals(cron.start(), cron);
     Assert.assertEquals(cron.start, begin, 20);
     Assert.assertEquals(cron.start, cron.lapStart);
     sleep(100);
     Assert.assertEquals(cron.stop(), cron);
-    long end = System.currentTimeMillis();
+    long end = Time.now();
     long delta = end - begin;
     Assert.assertEquals(cron.own, delta, 20);
     Assert.assertEquals(cron.total, 0);
     Assert.assertEquals(cron.lapStart, 0);
     sleep(100);
-    long reStart = System.currentTimeMillis();
+    long reStart = Time.now();
     cron.start();
     Assert.assertEquals(cron.start, begin, 20);
     Assert.assertEquals(cron.lapStart, reStart, 20);
     sleep(100);
     cron.stop();
-    long reEnd = System.currentTimeMillis();
+    long reEnd = Time.now();
     delta += reEnd - reStart;
     Assert.assertEquals(cron.own, delta, 20);
     Assert.assertEquals(cron.total, 0);
@@ -109,22 +110,22 @@ public class TestInstrumentationService extends HTestCase {
     long avgOwn;
 
     cron.start();
-    ownStart = System.currentTimeMillis();
+    ownStart = Time.now();
     totalStart = ownStart;
     ownDelta = 0;
     sleep(100);
 
     cron.stop();
-    ownEnd = System.currentTimeMillis();
+    ownEnd = Time.now();
     ownDelta += ownEnd - ownStart;
     sleep(100);
 
     cron.start();
-    ownStart = System.currentTimeMillis();
+    ownStart = Time.now();
     sleep(100);
 
     cron.stop();
-    ownEnd = System.currentTimeMillis();
+    ownEnd = Time.now();
     ownDelta += ownEnd - ownStart;
     totalEnd = ownEnd;
     totalDelta = totalEnd - totalStart;
@@ -142,22 +143,22 @@ public class TestInstrumentationService extends HTestCase {
     cron = new InstrumentationService.Cron();
 
     cron.start();
-    ownStart = System.currentTimeMillis();
+    ownStart = Time.now();
     totalStart = ownStart;
     ownDelta = 0;
     sleep(200);
 
     cron.stop();
-    ownEnd = System.currentTimeMillis();
+    ownEnd = Time.now();
     ownDelta += ownEnd - ownStart;
     sleep(200);
 
     cron.start();
-    ownStart = System.currentTimeMillis();
+    ownStart = Time.now();
     sleep(200);
 
     cron.stop();
-    ownEnd = System.currentTimeMillis();
+    ownEnd = Time.now();
     ownDelta += ownEnd - ownStart;
     totalEnd = ownEnd;
     totalDelta = totalEnd - totalStart;
@@ -178,22 +179,22 @@ public class TestInstrumentationService extends HTestCase {
     cron = new InstrumentationService.Cron();
 
     cron.start();
-    ownStart = System.currentTimeMillis();
+    ownStart = Time.now();
     totalStart = ownStart;
     ownDelta = 0;
     sleep(300);
 
     cron.stop();
-    ownEnd = System.currentTimeMillis();
+    ownEnd = Time.now();
     ownDelta += ownEnd - ownStart;
     sleep(300);
 
     cron.start();
-    ownStart = System.currentTimeMillis();
+    ownStart = Time.now();
     sleep(300);
 
     cron.stop();
-    ownEnd = System.currentTimeMillis();
+    ownEnd = Time.now();
     ownDelta += ownEnd - ownStart;
     totalEnd = ownEnd;
     totalDelta = totalEnd - totalStart;

+ 10 - 8
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/HTestCase.java

@@ -18,6 +18,8 @@
 package org.apache.hadoop.test;
 
 import junit.framework.Assert;
+
+import org.apache.hadoop.util.Time;
 import org.junit.Rule;
 import org.junit.rules.MethodRule;
 
@@ -142,18 +144,18 @@ public abstract class HTestCase {
    *         to <code>true</code>.
    */
   protected long waitFor(int timeout, boolean failIfTimeout, Predicate predicate) {
-    long started = System.currentTimeMillis();
-    long mustEnd = System.currentTimeMillis() + (long) (getWaitForRatio() * timeout);
+    long started = Time.now();
+    long mustEnd = Time.now() + (long) (getWaitForRatio() * timeout);
     long lastEcho = 0;
     try {
-      long waiting = mustEnd - System.currentTimeMillis();
+      long waiting = mustEnd - Time.now();
       System.out.println(MessageFormat.format("Waiting up to [{0}] msec", waiting));
       boolean eval;
-      while (!(eval = predicate.evaluate()) && System.currentTimeMillis() < mustEnd) {
-        if ((System.currentTimeMillis() - lastEcho) > 5000) {
-          waiting = mustEnd - System.currentTimeMillis();
+      while (!(eval = predicate.evaluate()) && Time.now() < mustEnd) {
+        if ((Time.now() - lastEcho) > 5000) {
+          waiting = mustEnd - Time.now();
           System.out.println(MessageFormat.format("Waiting up to [{0}] msec", waiting));
-          lastEcho = System.currentTimeMillis();
+          lastEcho = Time.now();
         }
         Thread.sleep(100);
       }
@@ -164,7 +166,7 @@ public abstract class HTestCase {
           System.out.println(MessageFormat.format("Waiting timed out after [{0}] msec", timeout));
         }
       }
-      return (eval) ? System.currentTimeMillis() - started : -1;
+      return (eval) ? Time.now() - started : -1;
     } catch (Exception ex) {
       throw new RuntimeException(ex);
     }

+ 11 - 10
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHFSTestCase.java

@@ -22,6 +22,7 @@ import junit.framework.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.Time;
 import org.junit.Test;
 import org.mortbay.jetty.Server;
 import org.mortbay.jetty.servlet.Context;
@@ -73,13 +74,13 @@ public class TestHFSTestCase extends HFSTestCase {
 
   @Test
   public void waitFor() {
-    long start = System.currentTimeMillis();
+    long start = Time.now();
     long waited = waitFor(1000, new Predicate() {
       public boolean evaluate() throws Exception {
         return true;
       }
     });
-    long end = System.currentTimeMillis();
+    long end = Time.now();
     Assert.assertEquals(waited, 0, 50);
     Assert.assertEquals(end - start - waited, 0, 50);
   }
@@ -87,13 +88,13 @@ public class TestHFSTestCase extends HFSTestCase {
   @Test
   public void waitForTimeOutRatio1() {
     setWaitForRatio(1);
-    long start = System.currentTimeMillis();
+    long start = Time.now();
     long waited = waitFor(200, new Predicate() {
       public boolean evaluate() throws Exception {
         return false;
       }
     });
-    long end = System.currentTimeMillis();
+    long end = Time.now();
     Assert.assertEquals(waited, -1);
     Assert.assertEquals(end - start, 200, 50);
   }
@@ -101,13 +102,13 @@ public class TestHFSTestCase extends HFSTestCase {
   @Test
   public void waitForTimeOutRatio2() {
     setWaitForRatio(2);
-    long start = System.currentTimeMillis();
+    long start = Time.now();
     long waited = waitFor(200, new Predicate() {
       public boolean evaluate() throws Exception {
         return false;
       }
     });
-    long end = System.currentTimeMillis();
+    long end = Time.now();
     Assert.assertEquals(waited, -1);
     Assert.assertEquals(end - start, 200 * getWaitForRatio(), 50 * getWaitForRatio());
   }
@@ -115,18 +116,18 @@ public class TestHFSTestCase extends HFSTestCase {
   @Test
   public void sleepRatio1() {
     setWaitForRatio(1);
-    long start = System.currentTimeMillis();
+    long start = Time.now();
     sleep(100);
-    long end = System.currentTimeMillis();
+    long end = Time.now();
     Assert.assertEquals(end - start, 100, 50);
   }
 
   @Test
   public void sleepRatio2() {
     setWaitForRatio(1);
-    long start = System.currentTimeMillis();
+    long start = Time.now();
     sleep(100);
-    long end = System.currentTimeMillis();
+    long end = Time.now();
     Assert.assertEquals(end - start, 100 * getWaitForRatio(), 50 * getWaitForRatio());
   }
 

+ 12 - 10
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHTestCase.java

@@ -19,6 +19,8 @@
 package org.apache.hadoop.test;
 
 import junit.framework.Assert;
+
+import org.apache.hadoop.util.Time;
 import org.junit.Test;
 import org.mortbay.jetty.Server;
 import org.mortbay.jetty.servlet.Context;
@@ -58,13 +60,13 @@ public class TestHTestCase extends HTestCase {
 
   @Test
   public void waitFor() {
-    long start = System.currentTimeMillis();
+    long start = Time.now();
     long waited = waitFor(1000, new Predicate() {
       public boolean evaluate() throws Exception {
         return true;
       }
     });
-    long end = System.currentTimeMillis();
+    long end = Time.now();
     Assert.assertEquals(waited, 0, 50);
     Assert.assertEquals(end - start - waited, 0, 50);
   }
@@ -72,13 +74,13 @@ public class TestHTestCase extends HTestCase {
   @Test
   public void waitForTimeOutRatio1() {
     setWaitForRatio(1);
-    long start = System.currentTimeMillis();
+    long start = Time.now();
     long waited = waitFor(200, new Predicate() {
       public boolean evaluate() throws Exception {
         return false;
       }
     });
-    long end = System.currentTimeMillis();
+    long end = Time.now();
     Assert.assertEquals(waited, -1);
     Assert.assertEquals(end - start, 200, 50);
   }
@@ -86,13 +88,13 @@ public class TestHTestCase extends HTestCase {
   @Test
   public void waitForTimeOutRatio2() {
     setWaitForRatio(2);
-    long start = System.currentTimeMillis();
+    long start = Time.now();
     long waited = waitFor(200, new Predicate() {
       public boolean evaluate() throws Exception {
         return false;
       }
     });
-    long end = System.currentTimeMillis();
+    long end = Time.now();
     Assert.assertEquals(waited, -1);
     Assert.assertEquals(end - start, 200 * getWaitForRatio(), 50 * getWaitForRatio());
   }
@@ -100,18 +102,18 @@ public class TestHTestCase extends HTestCase {
   @Test
   public void sleepRatio1() {
     setWaitForRatio(1);
-    long start = System.currentTimeMillis();
+    long start = Time.now();
     sleep(100);
-    long end = System.currentTimeMillis();
+    long end = Time.now();
     Assert.assertEquals(end - start, 100, 50);
   }
 
   @Test
   public void sleepRatio2() {
     setWaitForRatio(1);
-    long start = System.currentTimeMillis();
+    long start = Time.now();
     sleep(100);
-    long end = System.currentTimeMillis();
+    long end = Time.now();
     Assert.assertEquals(end - start, 100 * getWaitForRatio(), 50 * getWaitForRatio());
   }
 

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

@@ -125,6 +125,9 @@ Release 2.0.1-alpha - UNRELEASED
 
     HDFS-3582. Hook daemon process exit for testing. (eli)
 
+    HDFS-3641. Move server Util time methods to common and use now
+    instead of System#currentTimeMillis. (eli)
+
   OPTIMIZATIONS
 
     HDFS-2982. Startup performance suffers when there are many edit log

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -139,6 +139,7 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenRenewer;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
@@ -520,7 +521,7 @@ public class DFSClient implements java.io.Closeable {
       if (filesBeingWritten.isEmpty()) {
         return;
       }
-      lastLeaseRenewal = System.currentTimeMillis();
+      lastLeaseRenewal = Time.now();
     }
   }
 
@@ -537,7 +538,7 @@ public class DFSClient implements java.io.Closeable {
         return true;
       } catch (IOException e) {
         // Abort if the lease has already expired. 
-        final long elapsed = System.currentTimeMillis() - getLastLeaseRenewal();
+        final long elapsed = Time.now() - getLastLeaseRenewal();
         if (elapsed > HdfsConstants.LEASE_SOFTLIMIT_PERIOD) {
           LOG.warn("Failed to renew lease for " + clientName + " for "
               + (elapsed/1000) + " seconds (>= soft-limit ="
@@ -635,7 +636,7 @@ public class DFSClient implements java.io.Closeable {
    * @see ClientProtocol#getServerDefaults()
    */
   public FsServerDefaults getServerDefaults() throws IOException {
-    long now = System.currentTimeMillis();
+    long now = Time.now();
     if (now - serverDefaultsLastUpdate > SERVER_DEFAULTS_VALIDITY_PERIOD) {
       serverDefaults = namenode.getServerDefaults();
       serverDefaultsLastUpdate = now;

+ 12 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -74,6 +74,7 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Time;
 
 
 /****************************************************************
@@ -396,7 +397,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
      * and closes them. Any error recovery is also done by this thread.
      */
     public void run() {
-      long lastPacket = System.currentTimeMillis();
+      long lastPacket = Time.now();
       while (!streamerClosed && dfsClient.clientRunning) {
 
         // if the Responder encountered an error, shutdown Responder
@@ -420,7 +421,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
 
           synchronized (dataQueue) {
             // wait for a packet to be sent.
-            long now = System.currentTimeMillis();
+            long now = Time.now();
             while ((!streamerClosed && !hasError && dfsClient.clientRunning 
                 && dataQueue.size() == 0 && 
                 (stage != BlockConstructionStage.DATA_STREAMING || 
@@ -435,7 +436,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
               } catch (InterruptedException  e) {
               }
               doSleep = false;
-              now = System.currentTimeMillis();
+              now = Time.now();
             }
             if (streamerClosed || hasError || !dfsClient.clientRunning) {
               continue;
@@ -518,7 +519,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
             errorIndex = 0;
             throw e;
           }
-          lastPacket = System.currentTimeMillis();
+          lastPacket = Time.now();
           
           if (one.isHeartbeatPacket()) {  //heartbeat packet
           }
@@ -981,7 +982,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
         errorIndex = -1;
         success = false;
 
-        long startTime = System.currentTimeMillis();
+        long startTime = Time.now();
         DatanodeInfo[] excluded = excludedNodes.toArray(
             new DatanodeInfo[excludedNodes.size()]);
         block = oldBlock;
@@ -1107,7 +1108,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
       int retries = dfsClient.getConf().nBlockWriteLocateFollowingRetry;
       long sleeptime = 400;
       while (true) {
-        long localstart = System.currentTimeMillis();
+        long localstart = Time.now();
         while (true) {
           try {
             return dfsClient.namenode.addBlock(src, dfsClient.clientName, block, excludedNodes);
@@ -1130,9 +1131,9 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
               } else {
                 --retries;
                 DFSClient.LOG.info("Exception while adding a block", e);
-                if (System.currentTimeMillis() - localstart > 5000) {
+                if (Time.now() - localstart > 5000) {
                   DFSClient.LOG.info("Waiting for replication for "
-                      + (System.currentTimeMillis() - localstart) / 1000
+                      + (Time.now() - localstart) / 1000
                       + " seconds");
                 }
                 try {
@@ -1727,14 +1728,14 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
   // should be called holding (this) lock since setTestFilename() may 
   // be called during unit tests
   private void completeFile(ExtendedBlock last) throws IOException {
-    long localstart = System.currentTimeMillis();
+    long localstart = Time.now();
     boolean fileComplete = false;
     while (!fileComplete) {
       fileComplete = dfsClient.namenode.complete(src, dfsClient.clientName, last);
       if (!fileComplete) {
         if (!dfsClient.clientRunning ||
               (dfsClient.hdfsTimeout > 0 &&
-               localstart + dfsClient.hdfsTimeout < System.currentTimeMillis())) {
+               localstart + dfsClient.hdfsTimeout < Time.now())) {
             String msg = "Unable to close file because dfsclient " +
                           " was unable to contact the HDFS servers." +
                           " clientRunning " + dfsClient.clientRunning +
@@ -1744,7 +1745,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
         }
         try {
           Thread.sleep(400);
-          if (System.currentTimeMillis() - localstart > 5000) {
+          if (Time.now() - localstart > 5000) {
             DFSClient.LOG.info("Could not complete file " + src + " retrying...");
           }
         } catch (InterruptedException ie) {

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HsftpFileSystem.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.web.URLUtils;
+import org.apache.hadoop.util.Time;
 
 /**
  * An implementation of a protocol for accessing filesystems over HTTPS. The
@@ -164,8 +165,7 @@ public class HsftpFileSystem extends HftpFileSystem {
     final int warnDays = ExpWarnDays;
     if (warnDays > 0) { // make sure only check once
       ExpWarnDays = 0;
-      long expTimeThreshold = warnDays * MM_SECONDS_PER_DAY
-          + System.currentTimeMillis();
+      long expTimeThreshold = warnDays * MM_SECONDS_PER_DAY + Time.now();
       X509Certificate[] clientCerts = (X509Certificate[]) conn
           .getLocalCertificates();
       if (clientCerts != null) {
@@ -175,7 +175,7 @@ public class HsftpFileSystem extends HftpFileSystem {
             StringBuilder sb = new StringBuilder();
             sb.append("\n Client certificate "
                 + cert.getSubjectX500Principal().getName());
-            int dayOffSet = (int) ((expTime - System.currentTimeMillis()) / MM_SECONDS_PER_DAY);
+            int dayOffSet = (int) ((expTime - Time.now()) / MM_SECONDS_PER_DAY);
             sb.append(" have " + dayOffSet + " days to expire");
             LOG.warn(sb.toString());
           }

+ 7 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
 
 /**
  * <p>
@@ -279,7 +280,7 @@ class LeaseRenewer {
   /** Is the empty period longer than the grace period? */  
   private synchronized boolean isRenewerExpired() {
     return emptyTime != Long.MAX_VALUE
-        && System.currentTimeMillis() - emptyTime > gracePeriod;
+        && Time.now() - emptyTime > gracePeriod;
   }
 
   synchronized void put(final String src, final DFSOutputStream out,
@@ -339,7 +340,7 @@ class LeaseRenewer {
           }
         }
         //discover the first time that all file-being-written maps are empty.
-        emptyTime = System.currentTimeMillis();
+        emptyTime = Time.now();
       }
     }
   }
@@ -354,7 +355,7 @@ class LeaseRenewer {
       }
       if (emptyTime == Long.MAX_VALUE) {
         //discover the first time that the client list is empty.
-        emptyTime = System.currentTimeMillis();
+        emptyTime = Time.now();
       }
     }
 
@@ -427,10 +428,10 @@ class LeaseRenewer {
    * when the lease period is half over.
    */
   private void run(final int id) throws InterruptedException {
-    for(long lastRenewed = System.currentTimeMillis();
+    for(long lastRenewed = Time.now();
         clientsRunning() && !Thread.interrupted();
         Thread.sleep(getSleepPeriod())) {
-      final long elapsed = System.currentTimeMillis() - lastRenewed;
+      final long elapsed = Time.now() - lastRenewed;
       if (elapsed >= getRenewalTime()) {
         try {
           renew();
@@ -438,7 +439,7 @@ class LeaseRenewer {
             LOG.debug("Lease renewer daemon for " + clientsString()
                 + " with renew id " + id + " executed");
           }
-          lastRenewed = System.currentTimeMillis();
+          lastRenewed = Time.now();
         } catch (SocketTimeoutException ie) {
           LOG.warn("Failed to renew lease for " + clientsString() + " for "
               + (elapsed/1000) + " seconds.  Aborting ...", ie);

+ 9 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -137,10 +138,10 @@ public class BlockTokenSecretManager extends
      * more.
      */
     setSerialNo(serialNo + 1);
-    currentKey = new BlockKey(serialNo, System.currentTimeMillis() + 2
+    currentKey = new BlockKey(serialNo, Time.now() + 2
         * keyUpdateInterval + tokenLifetime, generateSecret());
     setSerialNo(serialNo + 1);
-    nextKey = new BlockKey(serialNo, System.currentTimeMillis() + 3
+    nextKey = new BlockKey(serialNo, Time.now() + 3
         * keyUpdateInterval + tokenLifetime, generateSecret());
     allKeys.put(currentKey.getKeyId(), currentKey);
     allKeys.put(nextKey.getKeyId(), nextKey);
@@ -157,7 +158,7 @@ public class BlockTokenSecretManager extends
   }
 
   private synchronized void removeExpiredKeys() {
-    long now = System.currentTimeMillis();
+    long now = Time.now();
     for (Iterator<Map.Entry<Integer, BlockKey>> it = allKeys.entrySet()
         .iterator(); it.hasNext();) {
       Map.Entry<Integer, BlockKey> e = it.next();
@@ -207,15 +208,15 @@ public class BlockTokenSecretManager extends
     removeExpiredKeys();
     // set final expiry date of retiring currentKey
     allKeys.put(currentKey.getKeyId(), new BlockKey(currentKey.getKeyId(),
-        System.currentTimeMillis() + keyUpdateInterval + tokenLifetime,
+        Time.now() + keyUpdateInterval + tokenLifetime,
         currentKey.getKey()));
     // update the estimated expiry date of new currentKey
-    currentKey = new BlockKey(nextKey.getKeyId(), System.currentTimeMillis()
+    currentKey = new BlockKey(nextKey.getKeyId(), Time.now()
         + 2 * keyUpdateInterval + tokenLifetime, nextKey.getKey());
     allKeys.put(currentKey.getKeyId(), currentKey);
     // generate a new nextKey
     setSerialNo(serialNo + 1);
-    nextKey = new BlockKey(serialNo, System.currentTimeMillis() + 3
+    nextKey = new BlockKey(serialNo, Time.now() + 3
         * keyUpdateInterval + tokenLifetime, generateSecret());
     allKeys.put(nextKey.getKeyId(), nextKey);
     return true;
@@ -290,7 +291,7 @@ public class BlockTokenSecretManager extends
   }
 
   private static boolean isExpired(long expiryDate) {
-    return System.currentTimeMillis() > expiryDate;
+    return Time.now() > expiryDate;
   }
 
   /**
@@ -335,7 +336,7 @@ public class BlockTokenSecretManager extends
     }
     if (key == null)
       throw new IllegalStateException("currentKey hasn't been initialized.");
-    identifier.setExpiryDate(System.currentTimeMillis() + tokenLifetime);
+    identifier.setExpiryDate(Time.now() + tokenLifetime);
     identifier.setKeyId(key.getKeyId());
     if (LOG.isDebugEnabled()) {
       LOG.debug("Generating block token for " + identifier.toString());

+ 7 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -72,6 +72,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -709,7 +710,7 @@ public class Balancer {
      */ 
     private static final long MAX_ITERATION_TIME = 20*60*1000L; //20 mins
     private void dispatchBlocks() {
-      long startTime = Util.now();
+      long startTime = Time.now();
       this.blocksToReceive = 2*scheduledSize;
       boolean isTimeUp = false;
       while(!isTimeUp && scheduledSize>0 &&
@@ -738,7 +739,7 @@ public class Balancer {
         } 
         
         // check if time is up or not
-        if (Util.now()-startTime > MAX_ITERATION_TIME) {
+        if (Time.now()-startTime > MAX_ITERATION_TIME) {
           isTimeUp = true;
           continue;
         }
@@ -1143,7 +1144,7 @@ public class Balancer {
    * move blocks in current window to old window.
    */ 
   private static class MovedBlocks {
-    private long lastCleanupTime = System.currentTimeMillis();
+    private long lastCleanupTime = Time.now();
     final private static int CUR_WIN = 0;
     final private static int OLD_WIN = 1;
     final private static int NUM_WINS = 2;
@@ -1174,7 +1175,7 @@ public class Balancer {
 
     /* remove old blocks */
     synchronized private void cleanup() {
-      long curTime = System.currentTimeMillis();
+      long curTime = Time.now();
       // check if old win is older than winWidth
       if (lastCleanupTime + WIN_WIDTH <= curTime) {
         // purge the old window
@@ -1471,7 +1472,7 @@ public class Balancer {
     /** Parse arguments and then run Balancer */
     @Override
     public int run(String[] args) {
-      final long startTime = Util.now();
+      final long startTime = Time.now();
       final Configuration conf = getConf();
       WIN_WIDTH = conf.getLong(
           DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 
@@ -1489,7 +1490,7 @@ public class Balancer {
         System.out.println(e + ".  Exiting ...");
         return ReturnStatus.INTERRUPTED.code;
       } finally {
-        System.out.println("Balancing took " + time2Str(Util.now()-startTime));
+        System.out.println("Balancing took " + time2Str(Time.now()-startTime));
       }
     }
 

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -72,6 +72,7 @@ import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -1457,7 +1458,7 @@ public class BlockManager {
   public void processReport(final DatanodeID nodeID, final String poolId,
       final BlockListAsLongs newReport) throws IOException {
     namesystem.writeLock();
-    final long startTime = Util.now(); //after acquiring write lock
+    final long startTime = Time.now(); //after acquiring write lock
     final long endTime;
     try {
       final DatanodeDescriptor node = datanodeManager.getDatanode(nodeID);
@@ -1496,7 +1497,7 @@ public class BlockManager {
       }
       
     } finally {
-      endTime = Util.now();
+      endTime = Time.now();
       namesystem.writeUnlock();
     }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 
 import java.util.ArrayList;
 import java.util.Collection;

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.util.LightWeightHashSet;
+import org.apache.hadoop.util.Time;
 
 /**
  * This class extends the DatanodeInfo class with ephemeral information (eg
@@ -310,7 +311,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
     this.dfsUsed = dfsUsed;
     this.remaining = remaining;
     this.blockPoolUsed = blockPoolUsed;
-    this.lastUpdate = System.currentTimeMillis();
+    this.lastUpdate = Time.now();
     this.xceiverCount = xceiverCount;
     this.volumeFailures = volFailures;
     this.heartbeatedSinceFailover = true;

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 
 import java.io.IOException;
 import java.io.PrintWriter;
@@ -70,6 +70,7 @@ import org.apache.hadoop.net.ScriptBasedMapping;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.HostsFileReader;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.net.InetAddresses;
@@ -320,7 +321,7 @@ public class DatanodeManager {
   /** Is the datanode dead? */
   boolean isDatanodeDead(DatanodeDescriptor node) {
     return (node.getLastUpdate() <
-            (Util.now() - heartbeatExpireInterval));
+            (Time.now() - heartbeatExpireInterval));
   }
 
   /** Add a datanode. */

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.Time;
 
 /**
  * Manage the heartbeats received from datanodes.
@@ -247,7 +248,7 @@ class HeartbeatManager implements DatanodeStatistics {
     public void run() {
       while(namesystem.isRunning()) {
         try {
-          final long now = Util.now();
+          final long now = Time.now();
           if (lastHeartbeatCheck + heartbeatRecheckInterval < now) {
             heartbeatCheck();
             lastHeartbeatCheck = now;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 
 import java.io.PrintWriter;
 import java.sql.Time;

+ 0 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Util.java

@@ -33,29 +33,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 public final class Util {
   private final static Log LOG = LogFactory.getLog(Util.class.getName());
 
-  /**
-   * Current system time.  Do not use this to calculate a duration or interval
-   * to sleep, because it will be broken by settimeofday.  Instead, use
-   * monotonicNow.
-   * @return current time in msec.
-   */
-  public static long now() {
-    return System.currentTimeMillis();
-  }
-  
-  /**
-   * Current time from some arbitrary time base in the past, counting in
-   * milliseconds, and not affected by settimeofday or similar system clock
-   * changes.  This is appropriate to use when computing how much longer to
-   * wait for an interval to expire.
-   * @return a monotonic clock that counts in milliseconds.
-   */
-  public static long monotonicNow() {
-    final long NANOSECONDS_PER_MILLISECOND = 1000000;
-
-    return System.nanoTime() / NANOSECONDS_PER_MILLISECOND;
-  }
-
   /**
    * Interprets the passed string as a URI. In case of error it 
    * assumes the specified string is a file.

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionUtil;
 
@@ -226,7 +227,7 @@ class BPServiceActor implements Runnable {
    */
   void scheduleBlockReport(long delay) {
     if (delay > 0) { // send BR after random delay
-      lastBlockReport = System.currentTimeMillis()
+      lastBlockReport = Time.now()
       - ( dnConf.blockReportInterval - DFSUtil.getRandom().nextInt((int)(delay)));
     } else { // send at next heartbeat
       lastBlockReport = lastHeartbeat - dnConf.blockReportInterval;
@@ -561,7 +562,7 @@ class BPServiceActor implements Runnable {
         // or work arrives, and then iterate again.
         //
         long waitTime = dnConf.heartBeatInterval - 
-        (System.currentTimeMillis() - lastHeartbeat);
+        (Time.now() - lastHeartbeat);
         synchronized(pendingIncrementalBR) {
           if (waitTime > 0 && pendingReceivedRequests == 0) {
             try {

+ 11 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java

@@ -49,6 +49,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.Time;
 
 /**
  * Scans the block files under a block pool and verifies that the
@@ -90,7 +91,7 @@ class BlockPoolSliceScanner {
   private long totalTransientErrors = 0;
   private final AtomicInteger totalBlocksScannedInLastRun = new AtomicInteger(); // Used for test only
   
-  private long currentPeriodStart = System.currentTimeMillis();
+  private long currentPeriodStart = Time.now();
   private long bytesLeft = 0; // Bytes to scan in this period
   private long totalBytesToScan = 0;
   
@@ -224,7 +225,7 @@ class BlockPoolSliceScanner {
     long period = Math.min(scanPeriod, 
                            Math.max(blockMap.size(),1) * 600 * 1000L);
     int periodInt = Math.abs((int)period);
-    return System.currentTimeMillis() - scanPeriod + 
+    return Time.now() - scanPeriod + 
         DFSUtil.getRandom().nextInt(periodInt);
   }
 
@@ -281,7 +282,7 @@ class BlockPoolSliceScanner {
       info = new BlockScanInfo(block);
     }
     
-    long now = System.currentTimeMillis();
+    long now = Time.now();
     info.lastScanType = type;
     info.lastScanTime = now;
     info.lastScanOk = scanOk;
@@ -358,7 +359,7 @@ class BlockPoolSliceScanner {
   }
   
   private synchronized void adjustThrottler() {
-    long timeLeft = currentPeriodStart+scanPeriod - System.currentTimeMillis();
+    long timeLeft = currentPeriodStart+scanPeriod - Time.now();
     long bw = Math.max(bytesLeft*1000/timeLeft, MIN_SCAN_RATE);
     throttler.setBandwidth(Math.min(bw, MAX_SCAN_RATE));
   }
@@ -481,7 +482,7 @@ class BlockPoolSliceScanner {
   private boolean assignInitialVerificationTimes() {
     //First updates the last verification times from the log file.
     if (verificationLog != null) {
-      long now = System.currentTimeMillis();
+      long now = Time.now();
       RollingLogs.LineIterator logIterator = null;
       try {
         logIterator = verificationLog.logs.iterator(false);
@@ -529,7 +530,7 @@ class BlockPoolSliceScanner {
       // Initially spread the block reads over half of scan period
       // so that we don't keep scanning the blocks too quickly when restarted.
       long verifyInterval = Math.min(scanPeriod/(2L * numBlocks), 10*60*1000L);
-      long lastScanTime = System.currentTimeMillis() - scanPeriod;
+      long lastScanTime = Time.now() - scanPeriod;
 
       if (!blockInfoSet.isEmpty()) {
         BlockScanInfo info;
@@ -556,7 +557,7 @@ class BlockPoolSliceScanner {
 
     // reset the byte counts :
     bytesLeft = totalBytesToScan;
-    currentPeriodStart = System.currentTimeMillis();
+    currentPeriodStart = Time.now();
   }
   
   void scanBlockPoolSlice() {
@@ -571,7 +572,7 @@ class BlockPoolSliceScanner {
       scan();
     } finally {
       totalBlocksScannedInLastRun.set(processedBlocks.size());
-      lastScanTime.set(System.currentTimeMillis());
+      lastScanTime.set(Time.now());
     }
   }
   
@@ -584,7 +585,7 @@ class BlockPoolSliceScanner {
         
       while (datanode.shouldRun && !Thread.interrupted()
           && datanode.isBPServiceAlive(blockPoolId)) {
-        long now = System.currentTimeMillis();
+        long now = Time.now();
         synchronized (this) {
           if ( now >= (currentPeriodStart + scanPeriod)) {
             startNewPeriod();
@@ -642,7 +643,7 @@ class BlockPoolSliceScanner {
     
     int total = blockInfoSet.size();
     
-    long now = System.currentTimeMillis();
+    long now = Time.now();
     
     Date date = new Date();
     

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -165,6 +165,7 @@ import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.VersionInfo;
 import org.mortbay.util.ajax.JSON;
 
@@ -972,7 +973,7 @@ public class DataNode extends Configured
     
     int rand = DFSUtil.getSecureRandom().nextInt(Integer.MAX_VALUE);
     return "DS-" + rand + "-" + ip + "-" + port + "-"
-        + System.currentTimeMillis();
+        + Time.now();
   }
   
   /** Ensure the authentication method is kerberos */

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.ERROR_ACCESS_TOKEN;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
 
 import java.io.BufferedInputStream;

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.Time;
 
 /**
  * Periodically scans the data directories for block and block metadata files.
@@ -239,7 +240,7 @@ public class DirectoryScanner implements Runnable {
   void start() {
     shouldRun = true;
     long offset = DFSUtil.getRandom().nextInt((int) (scanPeriodMsecs/1000L)) * 1000L; //msec
-    long firstScanTime = System.currentTimeMillis() + offset;
+    long firstScanTime = Time.now() + offset;
     LOG.info("Periodic Directory Tree Verification scan starting at " 
         + firstScanTime + " with interval " + scanPeriodMsecs);
     masterThread.scheduleAtFixedRate(this, offset, scanPeriodMsecs, 

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -78,6 +78,7 @@ import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.Time;
 
 /**************************************************
  * FSDataset manages a set of data blocks.  Each block
@@ -1142,7 +1143,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     }
     
     // Otherwise remove blocks for the failed volumes
-    long mlsec = System.currentTimeMillis();
+    long mlsec = Time.now();
     synchronized (this) {
       for (FsVolumeImpl fv: failedVols) {
         for (String bpid : fv.getBlockPoolList()) {
@@ -1161,7 +1162,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         }
       }
     } // end of sync
-    mlsec = System.currentTimeMillis() - mlsec;
+    mlsec = Time.now() - mlsec;
     LOG.warn("Removed " + removedBlocks + " out of " + totalBlocks +
         "(took " + mlsec + " millisecs)");
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java

@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY;
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 
 import java.io.File;
 import java.io.IOException;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java

@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.IOException;
 import java.io.Closeable;
 
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 
 import java.io.Closeable;
 import java.io.FileNotFoundException;

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -17,7 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
+
 import java.net.URI;
 import java.io.IOException;
 import java.util.ArrayList;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 
 import java.io.File;
 import java.io.FilterInputStream;

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -43,7 +43,7 @@ import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageState;
 import org.apache.hadoop.hdfs.server.common.Util;
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 
@@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.util.Canceler;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -660,7 +661,7 @@ public class FSImage implements Closeable {
     final long checkpointTxnCount = conf.getLong(
         DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY, 
         DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT);
-    long checkpointAge = System.currentTimeMillis() - imageFile.lastModified();
+    long checkpointAge = Time.now() - imageFile.lastModified();
 
     return (checkpointAge > checkpointPeriod * 1000) ||
            (numEditsLoaded > checkpointTxnCount);
@@ -757,7 +758,7 @@ public class FSImage implements Closeable {
     saver.save(newFile, compression);
     
     MD5FileUtils.saveMD5File(dstFile, saver.getSavedDigest());
-    storage.setMostRecentCheckpointInfo(txid, Util.now());
+    storage.setMostRecentCheckpointInfo(txid, Time.now());
   }
 
   /**
@@ -1072,7 +1073,7 @@ public class FSImage implements Closeable {
     // advertise it as such to other checkpointers
     // from now on
     if (txid > storage.getMostRecentCheckpointTxId()) {
-      storage.setMostRecentCheckpointInfo(txid, Util.now());
+      storage.setMostRecentCheckpointInfo(txid, Time.now());
     }
   }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 
 import java.io.DataInputStream;
 import java.io.DataOutputStream;

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -62,7 +62,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SUPPORT_APPEND_KEY;
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 
 import java.io.BufferedWriter;
 import java.io.ByteArrayInputStream;
@@ -190,6 +190,7 @@ import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.VersionInfo;
 import org.mortbay.util.ajax.JSON;
 
@@ -5396,7 +5397,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
 
   private long getLastContact(DatanodeDescriptor alivenode) {
-    return (System.currentTimeMillis() - alivenode.getLastUpdate())/1000;
+    return (Time.now() - alivenode.getLastUpdate())/1000;
   }
 
   private long getDfsUsed(DatanodeDescriptor alivenode) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

@@ -39,7 +39,7 @@ import org.apache.hadoop.util.Daemon;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 
 /**
  * LeaseManager does the lease housekeeping for writing on files.   

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.util.AtomicFileOutputStream;
 
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.DNS;
+import org.apache.hadoop.util.Time;
 
 import com.google.common.base.Preconditions;
 import com.google.common.annotations.VisibleForTesting;
@@ -1023,7 +1024,7 @@ public class NNStorage extends Storage implements Closeable {
     }
     
     int rand = DFSUtil.getSecureRandom().nextInt(Integer.MAX_VALUE);
-    String bpid = "BP-" + rand + "-"+ ip + "-" + System.currentTimeMillis();
+    String bpid = "BP-" + rand + "-"+ ip + "-" + Time.now();
     return bpid;
   }
 

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -182,7 +183,7 @@ public class NamenodeFsck {
    * Check files on DFS, starting from the indicated path.
    */
   public void fsck() {
-    final long startTime = System.currentTimeMillis();
+    final long startTime = Time.now();
     try {
       String msg = "FSCK started by " + UserGroupInformation.getCurrentUser()
           + " from " + remoteAddress + " for path " + path + " at " + new Date();
@@ -207,7 +208,7 @@ public class NamenodeFsck {
         out.println(" Number of racks:\t\t" + networktopology.getNumOfRacks());
 
         out.println("FSCK ended at " + new Date() + " in "
-            + (System.currentTimeMillis() - startTime + " milliseconds"));
+            + (Time.now() - startTime + " milliseconds"));
 
         // If there were internal errors during the fsck operation, we want to
         // return FAILURE_STATUS, even if those errors were not immediately
@@ -233,7 +234,7 @@ public class NamenodeFsck {
       String errMsg = "Fsck on path '" + path + "' " + FAILURE_STATUS;
       LOG.warn(errMsg, e);
       out.println("FSCK ended at " + new Date() + " in "
-          + (System.currentTimeMillis() - startTime + " milliseconds"));
+          + (Time.now() - startTime + " milliseconds"));
       out.println(e.getMessage());
       out.print("\n\n" + errMsg);
     } finally {

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java

@@ -58,6 +58,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.ServletUtil;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.VersionInfo;
 import org.znerd.xmlenc.XMLOutputter;
 
@@ -487,7 +488,7 @@ class NamenodeJspHelper {
 
       long decommRequestTime = d.decommissioningStatus.getStartTime();
       long timestamp = d.getLastUpdate();
-      long currentTime = System.currentTimeMillis();
+      long currentTime = Time.now();
       long hoursSinceDecommStarted = (currentTime - decommRequestTime)/3600000;
       long remainderMinutes = ((currentTime - decommRequestTime)/60000) % 60;
       out.print("<td class=\"lastcontact\"> "
@@ -534,7 +535,7 @@ class NamenodeJspHelper {
       String adminState = d.getAdminState().toString();
 
       long timestamp = d.getLastUpdate();
-      long currentTime = System.currentTimeMillis();
+      long currentTime = Time.now();
       
       long bpUsed = d.getBlockPoolUsed();
       String percentBpUsed = StringUtils.limitDecimalTo2(d

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -73,6 +73,7 @@ import org.apache.hadoop.security.authorize.AccessControlList;
 
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
@@ -99,7 +100,7 @@ public class SecondaryNameNode implements Runnable {
   public static final Log LOG = 
     LogFactory.getLog(SecondaryNameNode.class.getName());
 
-  private final long starttime = System.currentTimeMillis();
+  private final long starttime = Time.now();
   private volatile long lastCheckpointTime = 0;
 
   private String fsName;
@@ -315,7 +316,7 @@ public class SecondaryNameNode implements Runnable {
         if(UserGroupInformation.isSecurityEnabled())
           UserGroupInformation.getCurrentUser().reloginFromKeytab();
         
-        long now = System.currentTimeMillis();
+        long now = Time.now();
 
         if (shouldCheckpointBasedOnCount() ||
             now >= lastCheckpointTime + 1000 * checkpointConf.getPeriod()) {

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java

@@ -33,6 +33,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -209,7 +210,7 @@ public class TransferFsImage {
     //
     // open connection to remote server
     //
-    long startTime = Util.monotonicNow();
+    long startTime = Time.monotonicNow();
     URL url = new URL(str);
 
     HttpURLConnection connection = (HttpURLConnection)
@@ -316,7 +317,7 @@ public class TransferFsImage {
       }
     }
     double xferSec = Math.max(
-        ((float)(Util.monotonicNow() - startTime)) / 1000.0, 0.001);
+        ((float)(Time.monotonicNow() - startTime)) / 1000.0, 0.001);
     long xferKb = received / 1024;
     LOG.info(String.format("Transfer took %.2fs at %.2f KB/s",
         xferSec, xferKb / xferSec));

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java

@@ -43,7 +43,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.SecurityUtil;
 
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import com.google.common.annotations.VisibleForTesting;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java

@@ -38,7 +38,7 @@ import org.apache.hadoop.hdfs.util.Canceler;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
-import static org.apache.hadoop.hdfs.server.common.Util.now;
+import static org.apache.hadoop.util.Time.now;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/DataTransferThrottler.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.util;
 
-import static org.apache.hadoop.hdfs.server.common.Util.monotonicNow;
+import static org.apache.hadoop.util.Time.monotonicNow;
 
 /** 
  * a class to throttle the data transfers.

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/loadGenerator/TestLoadGenerator.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 
 import static org.junit.Assert.*;
 
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
@@ -171,7 +172,7 @@ public class TestLoadGenerator extends Configured implements Tool {
       args = new String[] {"-readProbability", "0.3", "-writeProbability", "0.3",
           "-root", TEST_SPACE_ROOT, "-maxDelayBetweenOps", "0",
           "-numOfThreads", "1", "-startTime", 
-          Long.toString(System.currentTimeMillis()), "-elapsedTime", "10"};
+          Long.toString(Time.now()), "-elapsedTime", "10"};
       
       assertEquals(0, lg.run(args));
 
@@ -227,7 +228,7 @@ public class TestLoadGenerator extends Configured implements Tool {
       String[] scriptArgs = new String[] {
           "-root", TEST_SPACE_ROOT, "-maxDelayBetweenOps", "0",
           "-numOfThreads", "10", "-startTime", 
-          Long.toString(System.currentTimeMillis()), "-scriptFile", script};
+          Long.toString(Time.now()), "-scriptFile", script};
       
       assertEquals(0, lg.run(scriptArgs));
       

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BenchmarkThroughput.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.fs.ChecksumFileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
@@ -52,12 +53,12 @@ public class BenchmarkThroughput extends Configured implements Tool {
   private int BUFFER_SIZE;
 
   private void resetMeasurements() {
-    startTime = System.currentTimeMillis();
+    startTime = Time.now();
   }
 
   private void printMeasurements() {
     System.out.println(" time: " +
-                       ((System.currentTimeMillis() - startTime)/1000));
+                       ((Time.now() - startTime)/1000));
   }
 
   private Path writeLocalFile(String name, Configuration conf,

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DataNodeCluster.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.namenode.CreateEditsLog;
 import org.apache.hadoop.net.DNS;
+import org.apache.hadoop.util.Time;
 
 
 /**
@@ -231,7 +232,6 @@ public class DataNodeCluster {
     }
     
     int rand = DFSUtil.getSecureRandom().nextInt(Integer.MAX_VALUE);
-    return "/Rack-" + rand + "-"+ ip  + "-" + 
-                      System.currentTimeMillis();
+    return "/Rack-" + rand + "-"+ ip  + "-" + Time.now(); 
   }
 }

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAppendDifferentChecksum.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.Time;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
@@ -105,16 +106,16 @@ public class TestAppendDifferentChecksum {
     FileSystem fsWithCrc32C = createFsWithChecksum("CRC32C", 512);
 
     Path p = new Path("/testAlgoSwitchRandomized");
-    long seed = System.currentTimeMillis();
+    long seed = Time.now();
     System.out.println("seed: " + seed);
     Random r = new Random(seed);
     
     // Create empty to start
     IOUtils.closeStream(fsWithCrc32.create(p));
     
-    long st = System.currentTimeMillis();
+    long st = Time.now();
     int len = 0;
-    while (System.currentTimeMillis() - st < RANDOM_TEST_RUNTIME) {
+    while (Time.now() - st < RANDOM_TEST_RUNTIME) {
       int thisLen = r.nextInt(500);
       FileSystem fs = (r.nextBoolean() ? fsWithCrc32 : fsWithCrc32C);
       FSDataOutputStream stm = fs.append(p);

+ 10 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java

@@ -73,6 +73,7 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
 import org.mockito.Mockito;
 import org.mockito.internal.stubbing.answers.ThrowsException;
@@ -483,9 +484,9 @@ public class TestDFSClientRetries extends TestCase {
     //
     // Test 1: might fail
     // 
-    long timestamp = System.currentTimeMillis();
+    long timestamp = Time.now();
     boolean pass = busyTest(xcievers, threads, fileLen, timeWin, retries);
-    long timestamp2 = System.currentTimeMillis();
+    long timestamp2 = Time.now();
     if ( pass ) {
       LOG.info("Test 1 succeeded! Time spent: " + (timestamp2-timestamp)/1000.0 + " sec.");
     } else {
@@ -496,9 +497,9 @@ public class TestDFSClientRetries extends TestCase {
     // Test 2: should never fail
     // 
     retries = 50;
-    timestamp = System.currentTimeMillis();
+    timestamp = Time.now();
     pass = busyTest(xcievers, threads, fileLen, timeWin, retries);
-    timestamp2 = System.currentTimeMillis();
+    timestamp2 = Time.now();
     assertTrue("Something wrong! Test 2 got Exception with maxmum retries!", pass);
     LOG.info("Test 2 succeeded! Time spent: "  + (timestamp2-timestamp)/1000.0 + " sec.");
     
@@ -507,9 +508,9 @@ public class TestDFSClientRetries extends TestCase {
     // 
     retries = 3;
     timeWin = 1000;
-    timestamp = System.currentTimeMillis();
+    timestamp = Time.now();
     pass = busyTest(xcievers, threads, fileLen, timeWin, retries);
-    timestamp2 = System.currentTimeMillis();
+    timestamp2 = Time.now();
     if ( pass ) {
       LOG.info("Test 3 succeeded! Time spent: " + (timestamp2-timestamp)/1000.0 + " sec.");
     } else {
@@ -521,9 +522,9 @@ public class TestDFSClientRetries extends TestCase {
     //
     retries = 50;
     timeWin = 1000;
-    timestamp = System.currentTimeMillis();
+    timestamp = Time.now();
     pass = busyTest(xcievers, threads, fileLen, timeWin, retries);
-    timestamp2 = System.currentTimeMillis();
+    timestamp2 = Time.now();
     assertTrue("Something wrong! Test 4 got Exception with maxmum retries!", pass);
     LOG.info("Test 4 succeeded! Time spent: "  + (timestamp2-timestamp)/1000.0 + " sec.");
   }
@@ -564,7 +565,7 @@ public class TestDFSClientRetries extends TestCase {
       LOG.info("Path : \"" + file1 + "\"");
 
       // write 1 block to file
-      byte[] buffer = AppendTestUtil.randomBytes(System.currentTimeMillis(), fileLen);
+      byte[] buffer = AppendTestUtil.randomBytes(Time.now(), fileLen);
       stm.write(buffer, 0, fileLen);
       stm.close();
 

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSMkdirs.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Time;
 
 
 /**
@@ -85,11 +86,11 @@ public class TestDFSMkdirs extends TestCase {
     DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
     try {
       // Create a dir in root dir, should succeed
-      assertTrue(dfs.mkdir(new Path("/mkdir-" + System.currentTimeMillis()),
+      assertTrue(dfs.mkdir(new Path("/mkdir-" + Time.now()),
           FsPermission.getDefault()));
       // Create a dir when parent dir exists as a file, should fail
       IOException expectedException = null;
-      String filePath = "/mkdir-file-" + System.currentTimeMillis();
+      String filePath = "/mkdir-file-" + Time.now();
       writeFile(dfs, new Path(filePath));
       try {
         dfs.mkdir(new Path(filePath + "/mkdir"), FsPermission.getDefault());
@@ -103,7 +104,7 @@ public class TestDFSMkdirs extends TestCase {
       // Create a dir in a non-exist directory, should fail
       expectedException = null;
       try {
-        dfs.mkdir(new Path("/non-exist/mkdir-" + System.currentTimeMillis()),
+        dfs.mkdir(new Path("/non-exist/mkdir-" + Time.now()),
             FsPermission.getDefault());
       } catch (IOException e) {
         expectedException = e;

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSPermission.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
 
 /** Unit tests for permission */
 public class TestDFSPermission extends TestCase {
@@ -77,7 +78,7 @@ public class TestDFSPermission extends TestCase {
   static {
     try {
       // Initiate the random number generator and logging the seed
-      long seed = Util.now();
+      long seed = Time.now();
       r = new Random(seed);
       LOG.info("Random number generator uses seed " + seed);
       LOG.info("NUM_TEST_PERMISSIONS=" + NUM_TEST_PERMISSIONS);

+ 9 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeBlockScanner.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.Time;
 
 /**
  * This test verifies that block verification occurs on the datanode
@@ -72,15 +73,15 @@ public class TestDatanodeBlockScanner extends TestCase {
   throws IOException, TimeoutException {
     URL url = new URL("http://localhost:" + infoPort +
                       "/blockScannerReport?listblocks");
-    long lastWarnTime = System.currentTimeMillis();
+    long lastWarnTime = Time.now();
     if (newTime <= 0) newTime = 1L;
     long verificationTime = 0;
     
     String block = DFSTestUtil.getFirstBlock(fs, file).getBlockName();
     long failtime = (timeout <= 0) ? Long.MAX_VALUE 
-        : System.currentTimeMillis() + timeout;
+        : Time.now() + timeout;
     while (verificationTime < newTime) {
-      if (failtime < System.currentTimeMillis()) {
+      if (failtime < Time.now()) {
         throw new TimeoutException("failed to achieve block verification after "
             + timeout + " msec.  Current verification timestamp = "
             + verificationTime + ", requested verification time > " 
@@ -103,7 +104,7 @@ public class TestDatanodeBlockScanner extends TestCase {
       }
       
       if (verificationTime < newTime) {
-        long now = System.currentTimeMillis();
+        long now = Time.now();
         if ((now - lastWarnTime) >= 5*1000) {
           LOG.info("Waiting for verification of " + block);
           lastWarnTime = now; 
@@ -118,7 +119,7 @@ public class TestDatanodeBlockScanner extends TestCase {
   }
 
   public void testDatanodeBlockScanner() throws IOException, TimeoutException {
-    long startTime = System.currentTimeMillis();
+    long startTime = Time.now();
     
     Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
@@ -311,7 +312,7 @@ public class TestDatanodeBlockScanner extends TestCase {
     conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 3L);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
 
-    long startTime = System.currentTimeMillis();
+    long startTime = Time.now();
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
                                                .numDataNodes(REPLICATION_FACTOR)
                                                .build();
@@ -395,10 +396,10 @@ public class TestDatanodeBlockScanner extends TestCase {
   private static void waitForBlockDeleted(ExtendedBlock blk, int dnIndex,
       long timeout) throws TimeoutException, InterruptedException {
     File blockFile = MiniDFSCluster.getBlockFile(dnIndex, blk);
-    long failtime = System.currentTimeMillis() 
+    long failtime = Time.now() 
                     + ((timeout > 0) ? timeout : Long.MAX_VALUE);
     while (blockFile != null && blockFile.exists()) {
-      if (failtime < System.currentTimeMillis()) {
+      if (failtime < Time.now()) {
         throw new TimeoutException("waited too long for blocks to be deleted: "
             + blockFile.getPath() + (blockFile.exists() ? " still exists; " : " is absent; "));
       }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
 import org.junit.Test;
 
@@ -169,7 +170,7 @@ public class TestDistributedFileSystem {
       for(int i = 0; i < filepaths.length; i++) {
         filepaths[i] = new Path(filepathstring + i);
       }
-      final long millis = System.currentTimeMillis();
+      final long millis = Time.now();
 
       {
         DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -68,6 +68,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.Time;
 import org.apache.log4j.Level;
 
 /**
@@ -668,13 +669,13 @@ public class TestFileCreation extends junit.framework.TestCase {
     }
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     FileSystem fs = cluster.getFileSystem();
-    final Path path = new Path("/" + System.currentTimeMillis()
+    final Path path = new Path("/" + Time.now()
         + "-testFileCreationNonRecursive");
     FSDataOutputStream out = null;
 
     try {
       IOException expectedException = null;
-      final String nonExistDir = "/non-exist-" + System.currentTimeMillis();
+      final String nonExistDir = "/non-exist-" + Time.now();
 
       fs.delete(new Path(nonExistDir), true);
       EnumSet<CreateFlag> createFlag = EnumSet.of(CreateFlag.CREATE);

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.util.Time;
 
 
 /**
@@ -72,7 +73,7 @@ public class TestInjectionForSimulatedStorage extends TestCase {
                                        ClientProtocol namenode,
                                        int expected, long maxWaitSec) 
                                        throws IOException {
-    long start = System.currentTimeMillis();
+    long start = Time.now();
     
     //wait for all the blocks to be replicated;
     LOG.info("Checking for block replication for " + filename);
@@ -97,7 +98,7 @@ public class TestInjectionForSimulatedStorage extends TestCase {
                                actual + ".");
       
         if (maxWaitSec > 0 && 
-            (System.currentTimeMillis() - start) > (maxWaitSec * 1000)) {
+            (Time.now() - start) > (maxWaitSec * 1000)) {
           throw new IOException("Timedout while waiting for all blocks to " +
                                 " be replicated for " + filename);
         }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -82,7 +83,7 @@ public class TestLease {
       // We don't need to wait the lease renewer thread to act.
       // call renewLease() manually.
       // make it look like lease has already expired.
-      dfs.lastLeaseRenewal = System.currentTimeMillis() - 300000;
+      dfs.lastLeaseRenewal = Time.now() - 300000;
       dfs.renewLease();
 
       // this should not work.

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java

@@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Time;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -118,8 +119,8 @@ public class TestLeaseRenewer {
     renewer.put(filePath, mockStream, MOCK_DFSCLIENT);
 
     // Wait for lease to get renewed
-    long failTime = System.currentTimeMillis() + 5000;
-    while (System.currentTimeMillis() < failTime &&
+    long failTime = Time.now() + 5000;
+    while (Time.now() < failTime &&
         leaseRenewalCount.get() == 0) {
       Thread.sleep(50);
     }
@@ -206,8 +207,8 @@ public class TestLeaseRenewer {
     renewer.closeFile(filePath, MOCK_DFSCLIENT);
     
     // Should stop the renewer running within a few seconds
-    long failTime = System.currentTimeMillis() + 5000;
-    while (renewer.isRunning() && System.currentTimeMillis() < failTime) {
+    long failTime = Time.now() + 5000;
+    while (renewer.isRunning() && Time.now() < failTime) {
       Thread.sleep(50);
     }
     Assert.assertFalse(renewer.isRunning());

Bu fark içinde çok fazla dosya değişikliği olduğu için bazı dosyalar gösterilmiyor