You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by wh...@apache.org on 2015/09/25 05:30:03 UTC

hadoop git commit: HDFS-9134. Move LEASE_{SOFTLIMIT, HARDLIMIT}_PERIOD constants from HdfsServerConstants to HdfsConstants. Contributed by Mingliang Liu.

Repository: hadoop
Updated Branches:
  refs/heads/branch-2 bf3f08cc6 -> 85c73d159


HDFS-9134. Move LEASE_{SOFTLIMIT,HARDLIMIT}_PERIOD constants from HdfsServerConstants to HdfsConstants. Contributed by Mingliang Liu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/85c73d15
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/85c73d15
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/85c73d15

Branch: refs/heads/branch-2
Commit: 85c73d159e60dcbb83f08d54c8cb0c694697c108
Parents: bf3f08c
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Sep 24 20:19:16 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Thu Sep 24 20:29:52 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/HdfsConstants.java     | 23 +++++++++++++++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  5 ++--
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   |  6 ++---
 .../hdfs/server/common/HdfsServerConstants.java | 27 +++++---------------
 .../hadoop/hdfs/server/namenode/BackupNode.java |  3 ++-
 .../hdfs/server/namenode/LeaseManager.java      |  5 ++--
 .../org/apache/hadoop/hdfs/TestFileAppend4.java |  4 +--
 .../java/org/apache/hadoop/hdfs/TestLease.java  |  8 +++---
 .../apache/hadoop/hdfs/TestLeaseRecovery2.java  |  5 ++--
 .../hdfs/server/namenode/TestFileTruncate.java  |  5 ++--
 ...tINodeFileUnderConstructionWithSnapshot.java |  8 +++---
 12 files changed, 58 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/85c73d15/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index 8b60496..1a64933 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -93,6 +93,29 @@ public class HdfsConstants {
   //for write pipeline
   public static final int WRITE_TIMEOUT_EXTENSION = 5 * 1000;
 
+  /**
+   * For a HDFS client to write to a file, a lease is granted; During the lease
+   * period, no other client can write to the file. The writing client can
+   * periodically renew the lease. When the file is closed, the lease is
+   * revoked. The lease duration is bound by this soft limit and a
+   * {@link HdfsConstants#LEASE_HARDLIMIT_PERIOD hard limit}. Until the
+   * soft limit expires, the writer has sole write access to the file. If the
+   * soft limit expires and the client fails to close the file or renew the
+   * lease, another client can preempt the lease.
+   */
+  public static final long LEASE_SOFTLIMIT_PERIOD = 60 * 1000;
+  /**
+   * For a HDFS client to write to a file, a lease is granted; During the lease
+   * period, no other client can write to the file. The writing client can
+   * periodically renew the lease. When the file is closed, the lease is
+   * revoked. The lease duration is bound by a
+   * {@link HdfsConstants#LEASE_SOFTLIMIT_PERIOD soft limit} and this hard
+   * limit. If after the hard limit expires and the client has failed to renew
+   * the lease, HDFS assumes that the client has quit and will automatically
+   * close the file on behalf of the writer, and recover the lease.
+   */
+  public static final long LEASE_HARDLIMIT_PERIOD = 60 * LEASE_SOFTLIMIT_PERIOD;
+
   // SafeMode actions
   public enum SafeModeAction {
     SAFEMODE_LEAVE, SAFEMODE_ENTER, SAFEMODE_GET

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85c73d15/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 354c3cc..840f1ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -611,6 +611,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7529. Consolidate encryption zone related implementation into a single
     class. (Rakesh R via wheat9)
 
+    HDFS-9134. Move LEASE_{SOFTLIMIT,HARDLIMIT}_PERIOD constants from
+    HdfsServerConstants to HdfsConstants. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85c73d15/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 496b079..3b09953 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -154,7 +154,6 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
@@ -547,10 +546,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       } catch (IOException e) {
         // Abort if the lease has already expired. 
         final long elapsed = Time.monotonicNow() - getLastLeaseRenewal();
-        if (elapsed > HdfsServerConstants.LEASE_HARDLIMIT_PERIOD) {
+        if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
           LOG.warn("Failed to renew lease for " + clientName + " for "
               + (elapsed/1000) + " seconds (>= hard-limit ="
-              + (HdfsServerConstants.LEASE_HARDLIMIT_PERIOD/1000) + " seconds.) "
+              + (HdfsConstants.LEASE_HARDLIMIT_PERIOD / 1000) + " seconds.) "
               + "Closing all files being written ...", e);
           closeAllFilesBeingWritten(true);
         } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85c73d15/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
index c689b73..b41e2c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSOutputStream;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+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;
@@ -165,7 +165,7 @@ public class LeaseRenewer {
   /** The time in milliseconds that the map became empty. */
   private long emptyTime = Long.MAX_VALUE;
   /** A fixed lease renewal time period in milliseconds */
-  private long renewal = HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD/2;
+  private long renewal = HdfsConstants.LEASE_SOFTLIMIT_PERIOD / 2;
 
   /** A daemon for renewing lease */
   private Daemon daemon = null;
@@ -378,7 +378,7 @@ public class LeaseRenewer {
 
     //update renewal time
     if (renewal == dfsc.getConf().getHdfsTimeout()/2) {
-      long min = HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD;
+      long min = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
       for(DFSClient c : dfsclients) {
         final int timeout = c.getConf().getHdfsTimeout();
         if (timeout > 0 && timeout < min) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85c73d15/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index d1cdc78..ef2027e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -25,6 +25,7 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@@ -42,28 +43,14 @@ import org.apache.hadoop.util.StringUtils;
 @InterfaceAudience.Private
 public interface HdfsServerConstants {
   int MIN_BLOCKS_FOR_WRITE = 1;
+
   /**
-   * For a HDFS client to write to a file, a lease is granted; During the lease
-   * period, no other client can write to the file. The writing client can
-   * periodically renew the lease. When the file is closed, the lease is
-   * revoked. The lease duration is bound by this soft limit and a
-   * {@link HdfsServerConstants#LEASE_HARDLIMIT_PERIOD hard limit}. Until the
-   * soft limit expires, the writer has sole write access to the file. If the
-   * soft limit expires and the client fails to close the file or renew the
-   * lease, another client can preempt the lease.
-   */
-  long LEASE_SOFTLIMIT_PERIOD = 60 * 1000;
-  /**
-   * For a HDFS client to write to a file, a lease is granted; During the lease
-   * period, no other client can write to the file. The writing client can
-   * periodically renew the lease. When the file is closed, the lease is
-   * revoked. The lease duration is bound by a
-   * {@link HdfsServerConstants#LEASE_SOFTLIMIT_PERIOD soft limit} and this hard
-   * limit. If after the hard limit expires and the client has failed to renew
-   * the lease, HDFS assumes that the client has quit and will automatically
-   * close the file on behalf of the writer, and recover the lease.
+   * Please see {@link HdfsConstants#LEASE_SOFTLIMIT_PERIOD} and
+   * {@link HdfsConstants#LEASE_HARDLIMIT_PERIOD} for more information.
    */
-  long LEASE_HARDLIMIT_PERIOD = 60 * LEASE_SOFTLIMIT_PERIOD;
+  long LEASE_SOFTLIMIT_PERIOD = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
+  long LEASE_HARDLIMIT_PERIOD = HdfsConstants.LEASE_HARDLIMIT_PERIOD;
+
   long LEASE_RECOVER_PERIOD = 10 * 1000; // in ms
   // We need to limit the length and depth of a path in the filesystem.
   // HADOOP-438

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85c73d15/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
index d9a4573..5da1f01 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalProtocolService;
@@ -157,7 +158,7 @@ public class BackupNode extends NameNode {
     // Backup node should never do lease recovery,
     // therefore lease hard limit should never expire.
     namesystem.leaseManager.setLeasePeriod(
-        HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD, Long.MAX_VALUE);
+        HdfsConstants.LEASE_SOFTLIMIT_PERIOD, Long.MAX_VALUE);
 
     // register with the active name-node 
     registerWith(nsInfo);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85c73d15/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
index 7cd6f3d..908af45 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
@@ -34,6 +34,7 @@ import java.util.TreeMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.Daemon;
@@ -69,8 +70,8 @@ public class LeaseManager {
 
   private final FSNamesystem fsnamesystem;
 
-  private long softLimit = HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD;
-  private long hardLimit = HdfsServerConstants.LEASE_HARDLIMIT_PERIOD;
+  private long softLimit = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
+  private long hardLimit = HdfsConstants.LEASE_HARDLIMIT_PERIOD;
 
   //
   // Used for handling lock-leases

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85c73d15/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
index 284ed75..1acab73 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java
@@ -40,8 +40,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@@ -111,7 +111,7 @@ public class TestFileAppend4 {
 
     // set the soft limit to be 1 second so that the
     // namenode triggers lease recovery upon append request
-    cluster.setLeasePeriod(1000, HdfsServerConstants.LEASE_HARDLIMIT_PERIOD);
+    cluster.setLeasePeriod(1000, HdfsConstants.LEASE_HARDLIMIT_PERIOD);
 
     // Trying recovery
     int tries = 60;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85c73d15/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
index 985f43e..bacdc99 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
@@ -43,8 +43,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.EnumSetWritable;
@@ -99,8 +99,8 @@ public class TestLease {
       // call renewLease() manually.
       // make it look like the soft limit has been exceeded.
       LeaseRenewer originalRenewer = dfs.getLeaseRenewer();
-      dfs.lastLeaseRenewal = Time.monotonicNow()
-      - HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD - 1000;
+      dfs.lastLeaseRenewal = Time.monotonicNow() -
+          HdfsConstants.LEASE_SOFTLIMIT_PERIOD - 1000;
       try {
         dfs.renewLease();
       } catch (IOException e) {}
@@ -116,7 +116,7 @@ public class TestLease {
 
       // make it look like the hard limit has been exceeded.
       dfs.lastLeaseRenewal = Time.monotonicNow()
-      - HdfsServerConstants.LEASE_HARDLIMIT_PERIOD - 1000;
+      - HdfsConstants.LEASE_HARDLIMIT_PERIOD - 1000;
       dfs.renewLease();
 
       // this should not work.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85c73d15/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
index c06f9a5..8c389ec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -332,8 +333,8 @@ public class TestLeaseRecovery2 {
     DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2g_map);
 
     // Reset default lease periods
-    cluster.setLeasePeriod(HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD,
-                           HdfsServerConstants.LEASE_HARDLIMIT_PERIOD);
+    cluster.setLeasePeriod(HdfsConstants.LEASE_SOFTLIMIT_PERIOD,
+                           HdfsConstants.LEASE_HARDLIMIT_PERIOD);
     //create a file
     // create a random file name
     String filestr = "/foo" + AppendTestUtil.nextInt();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85c73d15/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index 711db2a..34677ef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -649,8 +650,8 @@ public class TestFileTruncate {
     checkBlockRecovery(p);
 
     NameNodeAdapter.getLeaseManager(cluster.getNamesystem())
-        .setLeasePeriod(HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD,
-            HdfsServerConstants.LEASE_HARDLIMIT_PERIOD);
+        .setLeasePeriod(HdfsConstants.LEASE_SOFTLIMIT_PERIOD,
+            HdfsConstants.LEASE_HARDLIMIT_PERIOD);
 
     checkFullFile(p, newLength, contents);
     fs.delete(p, false);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/85c73d15/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java
index 0b9c254..16044c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java
@@ -37,9 +37,9 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -302,10 +302,8 @@ public class TestINodeFileUnderConstructionWithSnapshot {
         fsn.writeUnlock();
       }
     } finally {
-      NameNodeAdapter.setLeasePeriod(
-          fsn,
-          HdfsServerConstants.LEASE_SOFTLIMIT_PERIOD,
-          HdfsServerConstants.LEASE_HARDLIMIT_PERIOD);
+      NameNodeAdapter.setLeasePeriod(fsn, HdfsConstants.LEASE_SOFTLIMIT_PERIOD,
+          HdfsConstants.LEASE_HARDLIMIT_PERIOD);
     }
   }
 }
\ No newline at end of file