Jelajahi Sumber

HDFS-8169. Move LocatedBlocks and related classes to hdfs-client. Contributed by Haohui Mai.

Haohui Mai 10 tahun lalu
induk
melakukan
5d3a4d51bd
43 mengubah file dengan 299 tambahan dan 235 penghapusan
  1. 8 0
      hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
  2. 65 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
  3. 7 7
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
  4. 10 10
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
  5. 49 51
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  6. 0 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfoWithStorage.java
  7. 8 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ExtendedBlock.java
  8. 0 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/FsPermissionExtension.java
  9. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstantsClient.java
  10. 19 19
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
  11. 5 16
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
  12. 11 11
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
  13. 4 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenIdentifier.java
  14. 7 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenIdentifier.java
  15. 8 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsConstants.java
  16. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  17. 0 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  18. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
  19. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  20. 15 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  21. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
  22. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
  23. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
  24. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
  25. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
  26. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  27. 3 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  28. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  29. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  30. 7 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  31. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java
  32. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  33. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeMap.java
  34. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
  35. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
  36. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java
  37. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java
  38. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/SWebHdfsFileSystem.java
  39. 4 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
  40. 1 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  41. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
  42. 11 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
  43. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml

@@ -1,2 +1,10 @@
 <FindBugsFilter>
 <FindBugsFilter>
+  <Match>
+    <Or>
+      <Class name="org.apache.hadoop.hdfs.protocol.HdfsFileStatus"/>
+      <Class name="org.apache.hadoop.hdfs.protocol.LocatedBlock"/>
+      <Class name="org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier"/>
+    </Or>
+    <Bug pattern="EI_EXPOSE_REP,EI_EXPOSE_REP2" />
+  </Match>
 </FindBugsFilter>
 </FindBugsFilter>

+ 65 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java

@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.util.StringUtils;
+
+import java.io.UnsupportedEncodingException;
+
+public class DFSUtilClient {
+  /**
+   * Converts a byte array to a string using UTF8 encoding.
+   */
+  public static String bytes2String(byte[] bytes) {
+    return bytes2String(bytes, 0, bytes.length);
+  }
+
+  /** Return used as percentage of capacity */
+  public static float getPercentUsed(long used, long capacity) {
+    return capacity <= 0 ? 100 : (used * 100.0f)/capacity;
+  }
+
+  /** Return remaining as percentage of capacity */
+  public static float getPercentRemaining(long remaining, long capacity) {
+    return capacity <= 0 ? 0 : (remaining * 100.0f)/capacity;
+  }
+
+  /** Convert percentage to a string. */
+  public static String percent2String(double percentage) {
+    return StringUtils.format("%.2f%%", percentage);
+  }
+
+  /**
+   * Decode a specific range of bytes of the given byte array to a string
+   * using UTF8.
+   *
+   * @param bytes The bytes to be decoded into characters
+   * @param offset The index of the first byte to decode
+   * @param length The number of bytes to decode
+   * @return The decoded string
+   */
+  private static String bytes2String(byte[] bytes, int offset, int length) {
+    try {
+      return new String(bytes, offset, length, "UTF8");
+    } catch(UnsupportedEncodingException e) {
+      assert false : "UTF8 encoding is not supported ";
+    }
+    return null;
+  }
+
+}

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java

@@ -26,7 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.io.*;
 
 
 /**************************************************
 /**************************************************
- * A Block is a Hadoop FS primitive, identified by a 
+ * A Block is a Hadoop FS primitive, identified by a
  * long.
  * long.
  *
  *
  **************************************************/
  **************************************************/
@@ -124,7 +124,7 @@ public class Block implements Writable, Comparable<Block> {
   public long getBlockId() {
   public long getBlockId() {
     return blockId;
     return blockId;
   }
   }
-  
+
   public void setBlockId(long bid) {
   public void setBlockId(long bid) {
     blockId = bid;
     blockId = bid;
   }
   }
@@ -147,7 +147,7 @@ public class Block implements Writable, Comparable<Block> {
   public long getGenerationStamp() {
   public long getGenerationStamp() {
     return generationStamp;
     return generationStamp;
   }
   }
-  
+
   public void setGenerationStamp(long stamp) {
   public void setGenerationStamp(long stamp) {
     generationStamp = stamp;
     generationStamp = stamp;
   }
   }
@@ -179,13 +179,13 @@ public class Block implements Writable, Comparable<Block> {
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     readHelper(in);
     readHelper(in);
   }
   }
-  
+
   final void writeHelper(DataOutput out) throws IOException {
   final void writeHelper(DataOutput out) throws IOException {
     out.writeLong(blockId);
     out.writeLong(blockId);
     out.writeLong(numBytes);
     out.writeLong(numBytes);
     out.writeLong(generationStamp);
     out.writeLong(generationStamp);
   }
   }
-  
+
   final void readHelper(DataInput in) throws IOException {
   final void readHelper(DataInput in) throws IOException {
     this.blockId = in.readLong();
     this.blockId = in.readLong();
     this.numBytes = in.readLong();
     this.numBytes = in.readLong();
@@ -194,7 +194,7 @@ public class Block implements Writable, Comparable<Block> {
       throw new IOException("Unexpected block size: " + numBytes);
       throw new IOException("Unexpected block size: " + numBytes);
     }
     }
   }
   }
-  
+
   // write only the identifier part of the block
   // write only the identifier part of the block
   public void writeId(DataOutput out) throws IOException {
   public void writeId(DataOutput out) throws IOException {
     out.writeLong(blockId);
     out.writeLong(blockId);
@@ -223,7 +223,7 @@ public class Block implements Writable, Comparable<Block> {
     }
     }
     return compareTo((Block)o) == 0;
     return compareTo((Block)o) == 0;
   }
   }
-  
+
   /**
   /**
    * @return true if the two blocks have the same block ID and the same
    * @return true if the two blocks have the same block ID and the same
    * generation stamp, or if both blocks are null.
    * generation stamp, or if both blocks are null.

+ 10 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java

@@ -50,7 +50,7 @@ public class DatanodeID implements Comparable<DatanodeID> {
 
 
   /**
   /**
    * UUID identifying a given datanode. For upgraded Datanodes this is the
    * UUID identifying a given datanode. For upgraded Datanodes this is the
-   * same as the StorageID that was previously used by this Datanode. 
+   * same as the StorageID that was previously used by this Datanode.
    * For newly formatted Datanodes it is a UUID.
    * For newly formatted Datanodes it is a UUID.
    */
    */
   private final String datanodeUuid;
   private final String datanodeUuid;
@@ -80,7 +80,7 @@ public class DatanodeID implements Comparable<DatanodeID> {
    *                     e.g. if this is a new datanode. A new UUID will
    *                     e.g. if this is a new datanode. A new UUID will
    *                     be assigned by the namenode.
    *                     be assigned by the namenode.
    * @param xferPort data transfer port
    * @param xferPort data transfer port
-   * @param infoPort info server port 
+   * @param infoPort info server port
    * @param ipcPort ipc server port
    * @param ipcPort ipc server port
    */
    */
   public DatanodeID(String ipAddr, String hostName, String datanodeUuid,
   public DatanodeID(String ipAddr, String hostName, String datanodeUuid,
@@ -92,7 +92,7 @@ public class DatanodeID implements Comparable<DatanodeID> {
     this.infoSecurePort = infoSecurePort;
     this.infoSecurePort = infoSecurePort;
     this.ipcPort = ipcPort;
     this.ipcPort = ipcPort;
   }
   }
-  
+
   public void setIpAddr(String ipAddr) {
   public void setIpAddr(String ipAddr) {
     //updated during registration, preserve former xferPort
     //updated during registration, preserve former xferPort
     setIpAndXferPort(ipAddr, xferPort);
     setIpAndXferPort(ipAddr, xferPort);
@@ -108,7 +108,7 @@ public class DatanodeID implements Comparable<DatanodeID> {
   public void setPeerHostName(String peerHostName) {
   public void setPeerHostName(String peerHostName) {
     this.peerHostName = peerHostName;
     this.peerHostName = peerHostName;
   }
   }
-  
+
   /**
   /**
    * @return data node ID.
    * @return data node ID.
    */
    */
@@ -139,12 +139,12 @@ public class DatanodeID implements Comparable<DatanodeID> {
   }
   }
 
 
   /**
   /**
-   * @return hostname from the actual connection 
+   * @return hostname from the actual connection
    */
    */
   public String getPeerHostName() {
   public String getPeerHostName() {
     return peerHostName;
     return peerHostName;
   }
   }
-  
+
   /**
   /**
    * @return IP:xferPort string
    * @return IP:xferPort string
    */
    */
@@ -242,17 +242,17 @@ public class DatanodeID implements Comparable<DatanodeID> {
     return (getXferAddr().equals(((DatanodeID)to).getXferAddr()) &&
     return (getXferAddr().equals(((DatanodeID)to).getXferAddr()) &&
         datanodeUuid.equals(((DatanodeID)to).getDatanodeUuid()));
         datanodeUuid.equals(((DatanodeID)to).getDatanodeUuid()));
   }
   }
-  
+
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
     return datanodeUuid.hashCode();
     return datanodeUuid.hashCode();
   }
   }
-  
+
   @Override
   @Override
   public String toString() {
   public String toString() {
     return getXferAddr();
     return getXferAddr();
   }
   }
-  
+
   /**
   /**
    * Update fields when a new registration request comes in.
    * Update fields when a new registration request comes in.
    * Note that this does not update storageID.
    * Note that this does not update storageID.
@@ -265,7 +265,7 @@ public class DatanodeID implements Comparable<DatanodeID> {
     infoSecurePort = nodeReg.getInfoSecurePort();
     infoSecurePort = nodeReg.getInfoSecurePort();
     ipcPort = nodeReg.getIpcPort();
     ipcPort = nodeReg.getIpcPort();
   }
   }
-    
+
   /**
   /**
    * Compare based on data transfer address.
    * Compare based on data transfer address.
    *
    *

+ 49 - 51
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -19,8 +19,7 @@ package org.apache.hadoop.hdfs.protocol;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.net.Node;
@@ -32,9 +31,9 @@ import java.util.Date;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 
 
-import static org.apache.hadoop.hdfs.DFSUtil.percent2String;
+import static org.apache.hadoop.hdfs.DFSUtilClient.percent2String;
 
 
-/** 
+/**
  * This class extends the primary identifier of a Datanode with ephemeral
  * This class extends the primary identifier of a Datanode with ephemeral
  * state, eg usage information, current administrative state, and the
  * state, eg usage information, current administrative state, and the
  * network location that is communicated to clients.
  * network location that is communicated to clients.
@@ -54,12 +53,12 @@ public class DatanodeInfo extends DatanodeID implements Node {
   private String location = NetworkTopology.DEFAULT_RACK;
   private String location = NetworkTopology.DEFAULT_RACK;
   private String softwareVersion;
   private String softwareVersion;
   private List<String> dependentHostNames = new LinkedList<String>();
   private List<String> dependentHostNames = new LinkedList<String>();
-  
-  
+
+
   // Datanode administrative states
   // Datanode administrative states
   public enum AdminStates {
   public enum AdminStates {
-    NORMAL("In Service"), 
-    DECOMMISSION_INPROGRESS("Decommission In Progress"), 
+    NORMAL("In Service"),
+    DECOMMISSION_INPROGRESS("Decommission In Progress"),
     DECOMMISSIONED("Decommissioned");
     DECOMMISSIONED("Decommissioned");
 
 
     final String value;
     final String value;
@@ -72,7 +71,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     public String toString() {
     public String toString() {
       return value;
       return value;
     }
     }
-    
+
     public static AdminStates fromValue(final String value) {
     public static AdminStates fromValue(final String value) {
       for (AdminStates as : AdminStates.values()) {
       for (AdminStates as : AdminStates.values()) {
         if (as.value.equals(value)) return as;
         if (as.value.equals(value)) return as;
@@ -109,14 +108,14 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.lastUpdate = 0L;
     this.lastUpdate = 0L;
     this.lastUpdateMonotonic = 0L;
     this.lastUpdateMonotonic = 0L;
     this.xceiverCount = 0;
     this.xceiverCount = 0;
-    this.adminState = null;    
+    this.adminState = null;
   }
   }
-  
+
   public DatanodeInfo(DatanodeID nodeID, String location) {
   public DatanodeInfo(DatanodeID nodeID, String location) {
     this(nodeID);
     this(nodeID);
     this.location = location;
     this.location = location;
   }
   }
-  
+
   public DatanodeInfo(DatanodeID nodeID, String location,
   public DatanodeInfo(DatanodeID nodeID, String location,
       final long capacity, final long dfsUsed, final long remaining,
       final long capacity, final long dfsUsed, final long remaining,
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
@@ -152,16 +151,16 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.location = networkLocation;
     this.location = networkLocation;
     this.adminState = adminState;
     this.adminState = adminState;
   }
   }
-  
+
   /** Network location name */
   /** Network location name */
   @Override
   @Override
   public String getName() {
   public String getName() {
     return getXferAddr();
     return getXferAddr();
   }
   }
-  
+
   /** The raw capacity. */
   /** The raw capacity. */
   public long getCapacity() { return capacity; }
   public long getCapacity() { return capacity; }
-  
+
   /** The used space by the data node. */
   /** The used space by the data node. */
   public long getDfsUsed() { return dfsUsed; }
   public long getDfsUsed() { return dfsUsed; }
 
 
@@ -169,14 +168,14 @@ public class DatanodeInfo extends DatanodeID implements Node {
   public long getBlockPoolUsed() { return blockPoolUsed; }
   public long getBlockPoolUsed() { return blockPoolUsed; }
 
 
   /** The used space by the data node. */
   /** The used space by the data node. */
-  public long getNonDfsUsed() { 
+  public long getNonDfsUsed() {
     long nonDFSUsed = capacity - dfsUsed - remaining;
     long nonDFSUsed = capacity - dfsUsed - remaining;
     return nonDFSUsed < 0 ? 0 : nonDFSUsed;
     return nonDFSUsed < 0 ? 0 : nonDFSUsed;
   }
   }
 
 
   /** The used space by the data node as percentage of present capacity */
   /** The used space by the data node as percentage of present capacity */
-  public float getDfsUsedPercent() { 
-    return DFSUtil.getPercentUsed(dfsUsed, capacity);
+  public float getDfsUsedPercent() {
+    return DFSUtilClient.getPercentUsed(dfsUsed, capacity);
   }
   }
 
 
   /** The raw free space. */
   /** The raw free space. */
@@ -184,12 +183,12 @@ public class DatanodeInfo extends DatanodeID implements Node {
 
 
   /** Used space by the block pool as percentage of present capacity */
   /** Used space by the block pool as percentage of present capacity */
   public float getBlockPoolUsedPercent() {
   public float getBlockPoolUsedPercent() {
-    return DFSUtil.getPercentUsed(blockPoolUsed, capacity);
+    return DFSUtilClient.getPercentUsed(blockPoolUsed, capacity);
   }
   }
-  
+
   /** The remaining space as percentage of configured capacity. */
   /** The remaining space as percentage of configured capacity. */
-  public float getRemainingPercent() { 
-    return DFSUtil.getPercentRemaining(remaining, capacity);
+  public float getRemainingPercent() {
+    return DFSUtilClient.getPercentRemaining(remaining, capacity);
   }
   }
 
 
   /**
   /**
@@ -210,7 +209,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
    * @return Cache used as a percentage of the datanode's total cache capacity
    * @return Cache used as a percentage of the datanode's total cache capacity
    */
    */
   public float getCacheUsedPercent() {
   public float getCacheUsedPercent() {
-    return DFSUtil.getPercentUsed(cacheUsed, cacheCapacity);
+    return DFSUtilClient.getPercentUsed(cacheUsed, cacheCapacity);
   }
   }
 
 
   /**
   /**
@@ -225,7 +224,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
    * capacity
    * capacity
    */
    */
   public float getCacheRemainingPercent() {
   public float getCacheRemainingPercent() {
-    return DFSUtil.getPercentRemaining(getCacheRemaining(), cacheCapacity);
+    return DFSUtilClient.getPercentRemaining(getCacheRemaining(), cacheCapacity);
   }
   }
 
 
   /**
   /**
@@ -234,10 +233,10 @@ public class DatanodeInfo extends DatanodeID implements Node {
    */
    */
   public long getLastUpdate() { return lastUpdate; }
   public long getLastUpdate() { return lastUpdate; }
 
 
-  /** 
+  /**
    * The time when this information was accurate. <br>
    * The time when this information was accurate. <br>
    * Ps: So return value is ideal for calculation of time differences.
    * Ps: So return value is ideal for calculation of time differences.
-   * Should not be used to convert to Date.  
+   * Should not be used to convert to Date.
    */
    */
   public long getLastUpdateMonotonic() { return lastUpdateMonotonic;}
   public long getLastUpdateMonotonic() { return lastUpdateMonotonic;}
 
 
@@ -252,23 +251,23 @@ public class DatanodeInfo extends DatanodeID implements Node {
   public int getXceiverCount() { return xceiverCount; }
   public int getXceiverCount() { return xceiverCount; }
 
 
   /** Sets raw capacity. */
   /** Sets raw capacity. */
-  public void setCapacity(long capacity) { 
-    this.capacity = capacity; 
+  public void setCapacity(long capacity) {
+    this.capacity = capacity;
   }
   }
-  
+
   /** Sets the used space for the datanode. */
   /** Sets the used space for the datanode. */
   public void setDfsUsed(long dfsUsed) {
   public void setDfsUsed(long dfsUsed) {
     this.dfsUsed = dfsUsed;
     this.dfsUsed = dfsUsed;
   }
   }
 
 
   /** Sets raw free space. */
   /** Sets raw free space. */
-  public void setRemaining(long remaining) { 
-    this.remaining = remaining; 
+  public void setRemaining(long remaining) {
+    this.remaining = remaining;
   }
   }
 
 
   /** Sets block pool used space */
   /** Sets block pool used space */
-  public void setBlockPoolUsed(long bpUsed) { 
-    this.blockPoolUsed = bpUsed; 
+  public void setBlockPoolUsed(long bpUsed) {
+    this.blockPoolUsed = bpUsed;
   }
   }
 
 
   /** Sets cache capacity. */
   /** Sets cache capacity. */
@@ -282,38 +281,38 @@ public class DatanodeInfo extends DatanodeID implements Node {
   }
   }
 
 
   /** Sets time when this information was accurate. */
   /** Sets time when this information was accurate. */
-  public void setLastUpdate(long lastUpdate) { 
-    this.lastUpdate = lastUpdate; 
+  public void setLastUpdate(long lastUpdate) {
+    this.lastUpdate = lastUpdate;
   }
   }
 
 
   /** Sets number of active connections */
   /** Sets number of active connections */
-  public void setXceiverCount(int xceiverCount) { 
-    this.xceiverCount = xceiverCount; 
+  public void setXceiverCount(int xceiverCount) {
+    this.xceiverCount = xceiverCount;
   }
   }
 
 
   /** network location */
   /** network location */
   public synchronized String getNetworkLocation() {return location;}
   public synchronized String getNetworkLocation() {return location;}
-    
+
   /** Sets the network location */
   /** Sets the network location */
   public synchronized void setNetworkLocation(String location) {
   public synchronized void setNetworkLocation(String location) {
     this.location = NodeBase.normalize(location);
     this.location = NodeBase.normalize(location);
   }
   }
-  
+
   /** Add a hostname to a list of network dependencies */
   /** Add a hostname to a list of network dependencies */
   public void addDependentHostName(String hostname) {
   public void addDependentHostName(String hostname) {
     dependentHostNames.add(hostname);
     dependentHostNames.add(hostname);
   }
   }
-  
+
   /** List of Network dependencies */
   /** List of Network dependencies */
   public List<String> getDependentHostNames() {
   public List<String> getDependentHostNames() {
     return dependentHostNames;
     return dependentHostNames;
   }
   }
-  
+
   /** Sets the network dependencies */
   /** Sets the network dependencies */
   public void setDependentHostNames(List<String> dependencyList) {
   public void setDependentHostNames(List<String> dependencyList) {
     dependentHostNames = dependencyList;
     dependentHostNames = dependencyList;
   }
   }
-    
+
   /** A formatted string for reporting the status of the DataNode. */
   /** A formatted string for reporting the status of the DataNode. */
   public String getDatanodeReport() {
   public String getDatanodeReport() {
     StringBuilder buffer = new StringBuilder();
     StringBuilder buffer = new StringBuilder();
@@ -442,14 +441,13 @@ public class DatanodeInfo extends DatanodeID implements Node {
     }
     }
     return adminState;
     return adminState;
   }
   }
- 
+
   /**
   /**
-   * Check if the datanode is in stale state. Here if 
-   * the namenode has not received heartbeat msg from a 
-   * datanode for more than staleInterval (default value is
-   * {@link DFSConfigKeys#DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT}),
+   * Check if the datanode is in stale state. Here if
+   * the namenode has not received heartbeat msg from a
+   * datanode for more than staleInterval,
    * the datanode will be treated as stale node.
    * the datanode will be treated as stale node.
-   * 
+   *
    * @param staleInterval
    * @param staleInterval
    *          the time interval for marking the node as stale. If the last
    *          the time interval for marking the node as stale. If the last
    *          update time is beyond the given time interval, the node will be
    *          update time is beyond the given time interval, the node will be
@@ -459,7 +457,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
   public boolean isStale(long staleInterval) {
   public boolean isStale(long staleInterval) {
     return (Time.monotonicNow() - lastUpdateMonotonic) >= staleInterval;
     return (Time.monotonicNow() - lastUpdateMonotonic) >= staleInterval;
   }
   }
-  
+
   /**
   /**
    * Sets the admin state of this node.
    * Sets the admin state of this node.
    */
    */
@@ -480,7 +478,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
   public Node getParent() { return parent; }
   public Node getParent() { return parent; }
   @Override
   @Override
   public void setParent(Node parent) {this.parent = parent;}
   public void setParent(Node parent) {this.parent = parent;}
-   
+
   /** Return this node's level in the tree.
   /** Return this node's level in the tree.
    * E.g. the root of a tree returns 0 and its children return 1
    * E.g. the root of a tree returns 0 and its children return 1
    */
    */
@@ -494,7 +492,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     // Super implementation is sufficient
     // Super implementation is sufficient
     return super.hashCode();
     return super.hashCode();
   }
   }
-  
+
   @Override
   @Override
   public boolean equals(Object obj) {
   public boolean equals(Object obj) {
     // Sufficient to use super equality as datanodes are uniquely identified
     // Sufficient to use super equality as datanodes are uniquely identified

+ 0 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfoWithStorage.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfoWithStorage.java


+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ExtendedBlock.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ExtendedBlock.java

@@ -36,11 +36,11 @@ public class ExtendedBlock {
   public ExtendedBlock(final ExtendedBlock b) {
   public ExtendedBlock(final ExtendedBlock b) {
     this(b.poolId, new Block(b.block));
     this(b.poolId, new Block(b.block));
   }
   }
-  
+
   public ExtendedBlock(final String poolId, final long blockId) {
   public ExtendedBlock(final String poolId, final long blockId) {
     this(poolId, blockId, 0, 0);
     this(poolId, blockId, 0, 0);
   }
   }
-  
+
   public ExtendedBlock(String poolId, Block b) {
   public ExtendedBlock(String poolId, Block b) {
     this.poolId = poolId;
     this.poolId = poolId;
     this.block = b;
     this.block = b;
@@ -76,7 +76,7 @@ public class ExtendedBlock {
   public void setBlockId(final long bid) {
   public void setBlockId(final long bid) {
     block.setBlockId(bid);
     block.setBlockId(bid);
   }
   }
-  
+
   public void setGenerationStamp(final long genStamp) {
   public void setGenerationStamp(final long genStamp) {
     block.setGenerationStamp(genStamp);
     block.setGenerationStamp(genStamp);
   }
   }
@@ -84,7 +84,7 @@ public class ExtendedBlock {
   public void setNumBytes(final long len) {
   public void setNumBytes(final long len) {
     block.setNumBytes(len);
     block.setNumBytes(len);
   }
   }
-  
+
   public void set(String poolId, Block blk) {
   public void set(String poolId, Block blk) {
     this.poolId = poolId;
     this.poolId = poolId;
     this.block = blk;
     this.block = blk;
@@ -93,11 +93,11 @@ public class ExtendedBlock {
   public static Block getLocalBlock(final ExtendedBlock b) {
   public static Block getLocalBlock(final ExtendedBlock b) {
     return b == null ? null : b.getLocalBlock();
     return b == null ? null : b.getLocalBlock();
   }
   }
-  
+
   public Block getLocalBlock() {
   public Block getLocalBlock() {
     return block;
     return block;
   }
   }
-  
+
   @Override // Object
   @Override // Object
   public boolean equals(Object o) {
   public boolean equals(Object o) {
     if (this == o) {
     if (this == o) {
@@ -109,13 +109,13 @@ public class ExtendedBlock {
     ExtendedBlock b = (ExtendedBlock)o;
     ExtendedBlock b = (ExtendedBlock)o;
     return b.block.equals(block) && b.poolId.equals(poolId);
     return b.block.equals(block) && b.poolId.equals(poolId);
   }
   }
-  
+
   @Override // Object
   @Override // Object
   public int hashCode() {
   public int hashCode() {
     int result = 31 + poolId.hashCode();
     int result = 31 + poolId.hashCode();
     return (31 * result + block.hashCode());
     return (31 * result + block.hashCode());
   }
   }
-  
+
   @Override // Object
   @Override // Object
   public String toString() {
   public String toString() {
     return poolId + ":" + block;
     return poolId + ":" + block;

+ 0 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/FsPermissionExtension.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/FsPermissionExtension.java


+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstantsClient.java

@@ -31,4 +31,5 @@ public interface HdfsConstantsClient {
    * uses GRANDFATHER_INODE_ID for backward compatibility.
    * uses GRANDFATHER_INODE_ID for backward compatibility.
    */
    */
   long GRANDFATHER_INODE_ID = 0;
   long GRANDFATHER_INODE_ID = 0;
+  byte BLOCK_STORAGE_POLICY_ID_UNSPECIFIED = 0;
 }
 }

+ 19 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java

@@ -25,7 +25,7 @@ import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 
 
 /** Interface that represents the over the wire information for a file.
 /** Interface that represents the over the wire information for a file.
  */
  */
@@ -47,11 +47,11 @@ public class HdfsFileStatus {
   private final long fileId;
   private final long fileId;
 
 
   private final FileEncryptionInfo feInfo;
   private final FileEncryptionInfo feInfo;
-  
+
   // Used by dir, not including dot and dotdot. Always zero for a regular file.
   // Used by dir, not including dot and dotdot. Always zero for a regular file.
   private final int childrenNum;
   private final int childrenNum;
   private final byte storagePolicy;
   private final byte storagePolicy;
-  
+
   public static final byte[] EMPTY_NAME = new byte[0];
   public static final byte[] EMPTY_NAME = new byte[0];
 
 
   /**
   /**
@@ -80,9 +80,9 @@ public class HdfsFileStatus {
     this.blocksize = blocksize;
     this.blocksize = blocksize;
     this.modification_time = modification_time;
     this.modification_time = modification_time;
     this.access_time = access_time;
     this.access_time = access_time;
-    this.permission = (permission == null) ? 
-        ((isdir || symlink!=null) ? 
-            FsPermission.getDefault() : 
+    this.permission = (permission == null) ?
+        ((isdir || symlink!=null) ?
+            FsPermission.getDefault() :
             FsPermission.getFileDefault()) :
             FsPermission.getFileDefault()) :
         permission;
         permission;
     this.owner = (owner == null) ? "" : owner;
     this.owner = (owner == null) ? "" : owner;
@@ -118,7 +118,7 @@ public class HdfsFileStatus {
   public boolean isSymlink() {
   public boolean isSymlink() {
     return symlink != null;
     return symlink != null;
   }
   }
-  
+
   /**
   /**
    * Get the block size of the file.
    * Get the block size of the file.
    * @return the number of bytes
    * @return the number of bytes
@@ -158,7 +158,7 @@ public class HdfsFileStatus {
   public final FsPermission getPermission() {
   public final FsPermission getPermission() {
     return permission;
     return permission;
   }
   }
-  
+
   /**
   /**
    * Get the owner of the file.
    * Get the owner of the file.
    * @return owner of the file
    * @return owner of the file
@@ -166,15 +166,15 @@ public class HdfsFileStatus {
   public final String getOwner() {
   public final String getOwner() {
     return owner;
     return owner;
   }
   }
-  
+
   /**
   /**
    * Get the group associated with the file.
    * Get the group associated with the file.
-   * @return group for the file. 
+   * @return group for the file.
    */
    */
   public final String getGroup() {
   public final String getGroup() {
     return group;
     return group;
   }
   }
-  
+
   /**
   /**
    * Check if the local name is empty
    * Check if the local name is empty
    * @return true if the name is empty
    * @return true if the name is empty
@@ -188,9 +188,9 @@ public class HdfsFileStatus {
    * @return the local name in string
    * @return the local name in string
    */
    */
   public final String getLocalName() {
   public final String getLocalName() {
-    return DFSUtil.bytes2String(path);
+    return DFSUtilClient.bytes2String(path);
   }
   }
-  
+
   /**
   /**
    * Get the Java UTF8 representation of the local name
    * Get the Java UTF8 representation of the local name
    * @return the local name in java UTF8
    * @return the local name in java UTF8
@@ -208,7 +208,7 @@ public class HdfsFileStatus {
     if (isEmptyLocalName()) {
     if (isEmptyLocalName()) {
       return parent;
       return parent;
     }
     }
-    
+
     StringBuilder fullName = new StringBuilder(parent);
     StringBuilder fullName = new StringBuilder(parent);
     if (!parent.endsWith(Path.SEPARATOR)) {
     if (!parent.endsWith(Path.SEPARATOR)) {
       fullName.append(Path.SEPARATOR);
       fullName.append(Path.SEPARATOR);
@@ -226,7 +226,7 @@ public class HdfsFileStatus {
     if (isEmptyLocalName()) {
     if (isEmptyLocalName()) {
       return parent;
       return parent;
     }
     }
-    
+
     return new Path(parent, getLocalName());
     return new Path(parent, getLocalName());
   }
   }
 
 
@@ -235,17 +235,17 @@ public class HdfsFileStatus {
    * @return the symlink as a string.
    * @return the symlink as a string.
    */
    */
   public final String getSymlink() {
   public final String getSymlink() {
-    return DFSUtil.bytes2String(symlink);
+    return DFSUtilClient.bytes2String(symlink);
   }
   }
-  
+
   public final byte[] getSymlinkInBytes() {
   public final byte[] getSymlinkInBytes() {
     return symlink;
     return symlink;
   }
   }
-  
+
   public final long getFileId() {
   public final long getFileId() {
     return fileId;
     return fileId;
   }
   }
-  
+
   public final FileEncryptionInfo getFileEncryptionInfo() {
   public final FileEncryptionInfo getFileEncryptionInfo() {
     return feInfo;
     return feInfo;
   }
   }

+ 5 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java

@@ -24,8 +24,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
@@ -48,7 +46,7 @@ public class LocatedBlock {
   /** Cached storage type for each replica, if reported. */
   /** Cached storage type for each replica, if reported. */
   private final StorageType[] storageTypes;
   private final StorageType[] storageTypes;
   // corrupt flag is true if all of the replicas of a block are corrupt.
   // corrupt flag is true if all of the replicas of a block are corrupt.
-  // else false. If block has few corrupt replicas, they are filtered and 
+  // else false. If block has few corrupt replicas, they are filtered and
   // their locations are not part of this object
   // their locations are not part of this object
   private boolean corrupt;
   private boolean corrupt;
   private Token<BlockTokenIdentifier> blockToken = new Token<BlockTokenIdentifier>();
   private Token<BlockTokenIdentifier> blockToken = new Token<BlockTokenIdentifier>();
@@ -71,14 +69,6 @@ public class LocatedBlock {
     this(b, locs, storageIDs, storageTypes, -1, false, EMPTY_LOCS);
     this(b, locs, storageIDs, storageTypes, -1, false, EMPTY_LOCS);
   }
   }
 
 
-  public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages,
-      long startOffset, boolean corrupt) {
-    this(b, DatanodeStorageInfo.toDatanodeInfos(storages),
-        DatanodeStorageInfo.toStorageIDs(storages),
-        DatanodeStorageInfo.toStorageTypes(storages),
-        startOffset, corrupt, EMPTY_LOCS); // startOffset is unknown
-  }
-
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs,
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs,
                       StorageType[] storageTypes, long startOffset,
                       StorageType[] storageTypes, long startOffset,
                       boolean corrupt, DatanodeInfo[] cachedLocs) {
                       boolean corrupt, DatanodeInfo[] cachedLocs) {
@@ -132,7 +122,7 @@ public class LocatedBlock {
   public StorageType[] getStorageTypes() {
   public StorageType[] getStorageTypes() {
     return storageTypes;
     return storageTypes;
   }
   }
-  
+
   public String[] getStorageIDs() {
   public String[] getStorageIDs() {
     return storageIDs;
     return storageIDs;
   }
   }
@@ -157,7 +147,7 @@ public class LocatedBlock {
   public long getStartOffset() {
   public long getStartOffset() {
     return offset;
     return offset;
   }
   }
-  
+
   public long getBlockSize() {
   public long getBlockSize() {
     return b.getNumBytes();
     return b.getNumBytes();
   }
   }
@@ -169,14 +159,14 @@ public class LocatedBlock {
   public void setCorrupt(boolean corrupt) {
   public void setCorrupt(boolean corrupt) {
     this.corrupt = corrupt;
     this.corrupt = corrupt;
   }
   }
-  
+
   public boolean isCorrupt() {
   public boolean isCorrupt() {
     return this.corrupt;
     return this.corrupt;
   }
   }
 
 
   /**
   /**
    * Add a the location of a cached replica of the block.
    * Add a the location of a cached replica of the block.
-   * 
+   *
    * @param loc of datanode with the cached replica
    * @param loc of datanode with the cached replica
    */
    */
   public void addCachedLoc(DatanodeInfo loc) {
   public void addCachedLoc(DatanodeInfo loc) {
@@ -214,4 +204,3 @@ public class LocatedBlock {
         + "}";
         + "}";
   }
   }
 }
 }
-

+ 11 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java

@@ -57,19 +57,19 @@ public class LocatedBlocks {
     this.isLastBlockComplete = isLastBlockCompleted;
     this.isLastBlockComplete = isLastBlockCompleted;
     this.fileEncryptionInfo = feInfo;
     this.fileEncryptionInfo = feInfo;
   }
   }
-  
+
   /**
   /**
    * Get located blocks.
    * Get located blocks.
    */
    */
   public List<LocatedBlock> getLocatedBlocks() {
   public List<LocatedBlock> getLocatedBlocks() {
     return blocks;
     return blocks;
   }
   }
-  
+
   /** Get the last located block. */
   /** Get the last located block. */
   public LocatedBlock getLastLocatedBlock() {
   public LocatedBlock getLastLocatedBlock() {
     return lastLocatedBlock;
     return lastLocatedBlock;
   }
   }
-  
+
   /** Is the last block completed? */
   /** Is the last block completed? */
   public boolean isLastBlockComplete() {
   public boolean isLastBlockComplete() {
     return isLastBlockComplete;
     return isLastBlockComplete;
@@ -81,7 +81,7 @@ public class LocatedBlocks {
   public LocatedBlock get(int index) {
   public LocatedBlock get(int index) {
     return blocks.get(index);
     return blocks.get(index);
   }
   }
-  
+
   /**
   /**
    * Get number of located blocks.
    * Get number of located blocks.
    */
    */
@@ -90,7 +90,7 @@ public class LocatedBlocks {
   }
   }
 
 
   /**
   /**
-   * 
+   *
    */
    */
   public long getFileLength() {
   public long getFileLength() {
     return this.fileLength;
     return this.fileLength;
@@ -113,7 +113,7 @@ public class LocatedBlocks {
 
 
   /**
   /**
    * Find block containing specified offset.
    * Find block containing specified offset.
-   * 
+   *
    * @return block if found, or null otherwise.
    * @return block if found, or null otherwise.
    */
    */
   public int findBlock(long offset) {
   public int findBlock(long offset) {
@@ -122,7 +122,7 @@ public class LocatedBlocks {
         new ExtendedBlock(), new DatanodeInfo[0]);
         new ExtendedBlock(), new DatanodeInfo[0]);
     key.setStartOffset(offset);
     key.setStartOffset(offset);
     key.getBlock().setNumBytes(1);
     key.getBlock().setNumBytes(1);
-    Comparator<LocatedBlock> comp = 
+    Comparator<LocatedBlock> comp =
       new Comparator<LocatedBlock>() {
       new Comparator<LocatedBlock>() {
         // Returns 0 iff a is inside b or b is inside a
         // Returns 0 iff a is inside b or b is inside a
         @Override
         @Override
@@ -131,7 +131,7 @@ public class LocatedBlocks {
           long bBeg = b.getStartOffset();
           long bBeg = b.getStartOffset();
           long aEnd = aBeg + a.getBlockSize();
           long aEnd = aBeg + a.getBlockSize();
           long bEnd = bBeg + b.getBlockSize();
           long bEnd = bBeg + b.getBlockSize();
-          if(aBeg <= bBeg && bEnd <= aEnd 
+          if(aBeg <= bBeg && bEnd <= aEnd
               || bBeg <= aBeg && aEnd <= bEnd)
               || bBeg <= aBeg && aEnd <= bEnd)
             return 0; // one of the blocks is inside the other
             return 0; // one of the blocks is inside the other
           if(aBeg < bBeg)
           if(aBeg < bBeg)
@@ -141,11 +141,11 @@ public class LocatedBlocks {
       };
       };
     return Collections.binarySearch(blocks, key, comp);
     return Collections.binarySearch(blocks, key, comp);
   }
   }
-  
+
   public void insertRange(int blockIdx, List<LocatedBlock> newBlocks) {
   public void insertRange(int blockIdx, List<LocatedBlock> newBlocks) {
     int oldIdx = blockIdx;
     int oldIdx = blockIdx;
     int insStart = 0, insEnd = 0;
     int insStart = 0, insEnd = 0;
-    for(int newIdx = 0; newIdx < newBlocks.size() && oldIdx < blocks.size(); 
+    for(int newIdx = 0; newIdx < newBlocks.size() && oldIdx < blocks.size();
                                                         newIdx++) {
                                                         newIdx++) {
       long newOff = newBlocks.get(newIdx).getStartOffset();
       long newOff = newBlocks.get(newIdx).getStartOffset();
       long oldOff = blocks.get(oldIdx).getStartOffset();
       long oldOff = blocks.get(oldIdx).getStartOffset();
@@ -169,7 +169,7 @@ public class LocatedBlocks {
       blocks.addAll(oldIdx, newBlocks.subList(insStart, insEnd));
       blocks.addAll(oldIdx, newBlocks.subList(insStart, insEnd));
     }
     }
   }
   }
-  
+
   public static int getInsertIndex(int binSearchResult) {
   public static int getInsertIndex(int binSearchResult) {
     return binSearchResult >= 0 ? binSearchResult : -(binSearchResult+1);
     return binSearchResult >= 0 ? binSearchResult : -(binSearchResult+1);
   }
   }

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenIdentifier.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenIdentifier.java

@@ -46,7 +46,7 @@ public class BlockTokenIdentifier extends TokenIdentifier {
   private final EnumSet<AccessMode> modes;
   private final EnumSet<AccessMode> modes;
 
 
   private byte [] cache;
   private byte [] cache;
-  
+
   public BlockTokenIdentifier() {
   public BlockTokenIdentifier() {
     this(null, null, 0, EnumSet.noneOf(AccessMode.class));
     this(null, null, 0, EnumSet.noneOf(AccessMode.class));
   }
   }
@@ -129,7 +129,7 @@ public class BlockTokenIdentifier extends TokenIdentifier {
     if (obj instanceof BlockTokenIdentifier) {
     if (obj instanceof BlockTokenIdentifier) {
       BlockTokenIdentifier that = (BlockTokenIdentifier) obj;
       BlockTokenIdentifier that = (BlockTokenIdentifier) obj;
       return this.expiryDate == that.expiryDate && this.keyId == that.keyId
       return this.expiryDate == that.expiryDate && this.keyId == that.keyId
-          && isEqual(this.userId, that.userId) 
+          && isEqual(this.userId, that.userId)
           && isEqual(this.blockPoolId, that.blockPoolId)
           && isEqual(this.blockPoolId, that.blockPoolId)
           && this.blockId == that.blockId
           && this.blockId == that.blockId
           && isEqual(this.modes, that.modes);
           && isEqual(this.modes, that.modes);
@@ -171,11 +171,11 @@ public class BlockTokenIdentifier extends TokenIdentifier {
       WritableUtils.writeEnum(out, aMode);
       WritableUtils.writeEnum(out, aMode);
     }
     }
   }
   }
-  
+
   @Override
   @Override
   public byte[] getBytes() {
   public byte[] getBytes() {
     if(cache == null) cache = super.getBytes();
     if(cache == null) cache = super.getBytes();
-    
+
     return cache;
     return cache;
   }
   }
 
 

+ 7 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenIdentifier.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenIdentifier.java

@@ -23,8 +23,7 @@ import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.IOException;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.web.SWebHdfsFileSystem;
-import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
@@ -33,7 +32,7 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti
  * A delegation token identifier that is specific to HDFS.
  * A delegation token identifier that is specific to HDFS.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
-public class DelegationTokenIdentifier 
+public class DelegationTokenIdentifier
     extends AbstractDelegationTokenIdentifier {
     extends AbstractDelegationTokenIdentifier {
   public static final Text HDFS_DELEGATION_KIND = new Text("HDFS_DELEGATION_TOKEN");
   public static final Text HDFS_DELEGATION_KIND = new Text("HDFS_DELEGATION_TOKEN");
 
 
@@ -68,7 +67,7 @@ public class DelegationTokenIdentifier
   public static String stringifyToken(final Token<?> token) throws IOException {
   public static String stringifyToken(final Token<?> token) throws IOException {
     DelegationTokenIdentifier ident = new DelegationTokenIdentifier();
     DelegationTokenIdentifier ident = new DelegationTokenIdentifier();
     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
     ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
-    DataInputStream in = new DataInputStream(buf);  
+    DataInputStream in = new DataInputStream(buf);
     ident.readFields(in);
     ident.readFields(in);
 
 
     if (token.getService().getLength() > 0) {
     if (token.getService().getLength() > 0) {
@@ -77,7 +76,7 @@ public class DelegationTokenIdentifier
       return ident.toString();
       return ident.toString();
     }
     }
   }
   }
-  
+
   public static class WebHdfsDelegationTokenIdentifier
   public static class WebHdfsDelegationTokenIdentifier
       extends DelegationTokenIdentifier {
       extends DelegationTokenIdentifier {
     public WebHdfsDelegationTokenIdentifier() {
     public WebHdfsDelegationTokenIdentifier() {
@@ -85,10 +84,10 @@ public class DelegationTokenIdentifier
     }
     }
     @Override
     @Override
     public Text getKind() {
     public Text getKind() {
-      return WebHdfsFileSystem.TOKEN_KIND;
+      return WebHdfsConstants.WEBHDFS_TOKEN_KIND;
     }
     }
   }
   }
-  
+
   public static class SWebHdfsDelegationTokenIdentifier
   public static class SWebHdfsDelegationTokenIdentifier
       extends WebHdfsDelegationTokenIdentifier {
       extends WebHdfsDelegationTokenIdentifier {
     public SWebHdfsDelegationTokenIdentifier() {
     public SWebHdfsDelegationTokenIdentifier() {
@@ -96,7 +95,7 @@ public class DelegationTokenIdentifier
     }
     }
     @Override
     @Override
     public Text getKind() {
     public Text getKind() {
-      return SWebHdfsFileSystem.TOKEN_KIND;
+      return WebHdfsConstants.SWEBHDFS_TOKEN_KIND;
     }
     }
   }
   }
 }
 }

+ 8 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsConstants.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsConstants.java

@@ -17,9 +17,16 @@
  */
  */
 package org.apache.hadoop.hdfs.web;
 package org.apache.hadoop.hdfs.web;
 
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.io.Text;
+
+@InterfaceAudience.Private
+public class WebHdfsConstants {
+  /** Delegation token kind */
+  public static final Text WEBHDFS_TOKEN_KIND = new Text("WEBHDFS delegation");
+  public static final Text SWEBHDFS_TOKEN_KIND = new Text("SWEBHDFS delegation");
 
 
-class WebHdfsConstants {
   enum PathType {
   enum PathType {
     FILE, DIRECTORY, SYMLINK;
     FILE, DIRECTORY, SYMLINK;
 
 

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

@@ -132,6 +132,8 @@ Release 2.8.0 - UNRELEASED
 
 
     HDFS-8152. Refactoring of lazy persist storage cases. (Arpit Agarwal)
     HDFS-8152. Refactoring of lazy persist storage cases. (Arpit Agarwal)
 
 
+    HDFS-8169. Move LocatedBlocks and related classes to hdfs-client. (wheat9)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

+ 0 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -1235,21 +1235,6 @@ public class DFSUtil {
       }
       }
     }
     }
   }
   }
-  
-  /** Return used as percentage of capacity */
-  public static float getPercentUsed(long used, long capacity) {
-    return capacity <= 0 ? 100 : (used * 100.0f)/capacity; 
-  }
-  
-  /** Return remaining as percentage of capacity */
-  public static float getPercentRemaining(long remaining, long capacity) {
-    return capacity <= 0 ? 0 : (remaining * 100.0f)/capacity; 
-  }
-
-  /** Convert percentage to a string. */
-  public static String percent2String(double percentage) {
-    return StringUtils.format("%.2f%%", percentage);
-  }
 
 
   /**
   /**
    * Round bytes to GiB (gibibyte)
    * Round bytes to GiB (gibibyte)

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java

@@ -25,7 +25,6 @@ import java.util.Date;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 
 
 /**
 /**
  * Metadata about a snapshottable directory
  * Metadata about a snapshottable directory
@@ -62,7 +61,7 @@ public class SnapshottableDirectoryStatus {
       int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
       int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
     this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
     this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
         access_time, permission, owner, group, null, localName, inodeId,
         access_time, permission, owner, group, null, localName, inodeId,
-        childrenNum, null, BlockStoragePolicySuite.ID_UNSPECIFIED);
+        childrenNum, null, HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
     this.snapshotNumber = snapshotNumber;
     this.snapshotNumber = snapshotNumber;
     this.snapshotQuota = snapshotQuota;
     this.snapshotQuota = snapshotQuota;
     this.parentFullPath = parentFullPath;
     this.parentFullPath = parentFullPath;

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -184,7 +184,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.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
@@ -1443,7 +1442,7 @@ public class PBHelper {
         fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
         fs.hasChildrenNum() ? fs.getChildrenNum() : -1,
         fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
         fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null,
         fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
         fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy()
-            : BlockStoragePolicySuite.ID_UNSPECIFIED);
+            : HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
   }
   }
 
 
   public static SnapshottableDirectoryStatus convert(
   public static SnapshottableDirectoryStatus convert(

+ 15 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -123,7 +123,7 @@ public class BlockManager {
   private final AtomicLong excessBlocksCount = new AtomicLong(0L);
   private final AtomicLong excessBlocksCount = new AtomicLong(0L);
   private final AtomicLong postponedMisreplicatedBlocksCount = new AtomicLong(0L);
   private final AtomicLong postponedMisreplicatedBlocksCount = new AtomicLong(0L);
   private final long startupDelayBlockDeletionInMs;
   private final long startupDelayBlockDeletionInMs;
-  
+
   /** Used by metrics */
   /** Used by metrics */
   public long getPendingReplicationBlocksCount() {
   public long getPendingReplicationBlocksCount() {
     return pendingReplicationBlocksCount;
     return pendingReplicationBlocksCount;
@@ -836,7 +836,7 @@ public class BlockManager {
           (BlockInfoContiguousUnderConstruction) blk;
           (BlockInfoContiguousUnderConstruction) blk;
       final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
       final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
       final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
       final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
-      return new LocatedBlock(eb, storages, pos, false);
+      return newLocatedBlock(eb, storages, pos, false);
     }
     }
 
 
     // get block locations
     // get block locations
@@ -868,7 +868,7 @@ public class BlockManager {
       " numCorrupt: " + numCorruptNodes +
       " numCorrupt: " + numCorruptNodes +
       " numCorruptRepls: " + numCorruptReplicas;
       " numCorruptRepls: " + numCorruptReplicas;
     final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
     final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
-    return new LocatedBlock(eb, machines, pos, isCorrupt);
+    return newLocatedBlock(eb, machines, pos, isCorrupt);
   }
   }
 
 
   /** Create a LocatedBlocks. */
   /** Create a LocatedBlocks. */
@@ -3726,7 +3726,18 @@ public class BlockManager {
     postponedMisreplicatedBlocks.clear();
     postponedMisreplicatedBlocks.clear();
     postponedMisreplicatedBlocksCount.set(0);
     postponedMisreplicatedBlocksCount.set(0);
   };
   };
-  
+
+  public static LocatedBlock newLocatedBlock(
+      ExtendedBlock b, DatanodeStorageInfo[] storages,
+      long startOffset, boolean corrupt) {
+    // startOffset is unknown
+    return new LocatedBlock(
+        b, DatanodeStorageInfo.toDatanodeInfos(storages),
+        DatanodeStorageInfo.toStorageIDs(storages),
+        DatanodeStorageInfo.toStorageTypes(storages),
+        startOffset, corrupt,
+        null);
+  }
 
 
   private static class ReplicationWork {
   private static class ReplicationWork {
 
 

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

@@ -42,7 +42,6 @@ public class BlockStoragePolicySuite {
   public static final XAttr.NameSpace XAttrNS = XAttr.NameSpace.SYSTEM;
   public static final XAttr.NameSpace XAttrNS = XAttr.NameSpace.SYSTEM;
 
 
   public static final int ID_BIT_LENGTH = 4;
   public static final int ID_BIT_LENGTH = 4;
-  public static final byte ID_UNSPECIFIED = 0;
 
 
   @VisibleForTesting
   @VisibleForTesting
   public static BlockStoragePolicySuite createDefaultSuite() {
   public static BlockStoragePolicySuite createDefaultSuite() {

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java

@@ -25,6 +25,9 @@ import com.google.common.annotations.VisibleForTesting;
 
 
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java

@@ -22,7 +22,7 @@ import java.util.List;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
@@ -115,7 +115,7 @@ class HeartbeatManager implements DatanodeStatistics {
 
 
   @Override
   @Override
   public synchronized float getCapacityUsedPercent() {
   public synchronized float getCapacityUsedPercent() {
-    return DFSUtil.getPercentUsed(stats.capacityUsed, stats.capacityTotal);
+    return DFSUtilClient.getPercentUsed(stats.capacityUsed, stats.capacityTotal);
   }
   }
 
 
   @Override
   @Override
@@ -125,8 +125,8 @@ class HeartbeatManager implements DatanodeStatistics {
 
 
   @Override
   @Override
   public synchronized float getCapacityRemainingPercent() {
   public synchronized float getCapacityRemainingPercent() {
-    return DFSUtil.getPercentRemaining(
-        stats.capacityRemaining, stats.capacityTotal);
+    return DFSUtilClient.getPercentRemaining(stats.capacityRemaining,
+                                             stats.capacityTotal);
   }
   }
 
 
   @Override
   @Override
@@ -136,7 +136,8 @@ class HeartbeatManager implements DatanodeStatistics {
 
 
   @Override
   @Override
   public synchronized float getPercentBlockPoolUsed() {
   public synchronized float getPercentBlockPoolUsed() {
-    return DFSUtil.getPercentUsed(stats.blockPoolUsed, stats.capacityTotal);
+    return DFSUtilClient.getPercentUsed(stats.blockPoolUsed,
+                                        stats.capacityTotal);
   }
   }
 
 
   @Override
   @Override

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java

@@ -53,7 +53,6 @@ import org.apache.hadoop.util.ToolRunner;
 
 
 import java.io.BufferedReader;
 import java.io.BufferedReader;
 import java.io.FileInputStream;
 import java.io.FileInputStream;
-import java.io.FileReader;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
 import java.net.URI;
 import java.net.URI;
@@ -331,7 +330,7 @@ public class Mover {
     private boolean processFile(String fullPath, HdfsLocatedFileStatus status) {
     private boolean processFile(String fullPath, HdfsLocatedFileStatus status) {
       final byte policyId = status.getStoragePolicy();
       final byte policyId = status.getStoragePolicy();
       // currently we ignore files with unspecified storage policy
       // currently we ignore files with unspecified storage policy
-      if (policyId == BlockStoragePolicySuite.ID_UNSPECIFIED) {
+      if (policyId == HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
         return false;
         return false;
       }
       }
       final BlockStoragePolicy policy = blockStoragePolicies[policyId];
       final BlockStoragePolicy policy = blockStoragePolicies[policyId];

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java

@@ -31,12 +31,12 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
@@ -140,7 +140,7 @@ class FSDirStatAndListingOp {
   }
   }
 
 
   private static byte getStoragePolicyID(byte inodePolicy, byte parentPolicy) {
   private static byte getStoragePolicyID(byte inodePolicy, byte parentPolicy) {
-    return inodePolicy != BlockStoragePolicySuite.ID_UNSPECIFIED ? inodePolicy :
+    return inodePolicy != HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED ? inodePolicy :
         parentPolicy;
         parentPolicy;
   }
   }
 
 
@@ -176,8 +176,8 @@ class FSDirStatAndListingOp {
       if (targetNode == null)
       if (targetNode == null)
         return null;
         return null;
       byte parentStoragePolicy = isSuperUser ?
       byte parentStoragePolicy = isSuperUser ?
-          targetNode.getStoragePolicyID() : BlockStoragePolicySuite
-          .ID_UNSPECIFIED;
+          targetNode.getStoragePolicyID() : HdfsConstantsClient
+          .BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
 
 
       if (!targetNode.isDirectory()) {
       if (!targetNode.isDirectory()) {
         return new DirectoryListing(
         return new DirectoryListing(
@@ -199,7 +199,7 @@ class FSDirStatAndListingOp {
         INode cur = contents.get(startChild+i);
         INode cur = contents.get(startChild+i);
         byte curPolicy = isSuperUser && !cur.isSymlink()?
         byte curPolicy = isSuperUser && !cur.isSymlink()?
             cur.getLocalStoragePolicyID():
             cur.getLocalStoragePolicyID():
-            BlockStoragePolicySuite.ID_UNSPECIFIED;
+            HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
         listing[i] = createFileStatus(fsd, src, cur.getLocalNameBytes(), cur,
         listing[i] = createFileStatus(fsd, src, cur.getLocalNameBytes(), cur,
             needLocation, getStoragePolicyID(curPolicy,
             needLocation, getStoragePolicyID(curPolicy,
                 parentStoragePolicy), snapshot, isRawPath, iip);
                 parentStoragePolicy), snapshot, isRawPath, iip);
@@ -254,7 +254,7 @@ class FSDirStatAndListingOp {
     for (int i = 0; i < numOfListing; i++) {
     for (int i = 0; i < numOfListing; i++) {
       Snapshot.Root sRoot = snapshots.get(i + skipSize).getRoot();
       Snapshot.Root sRoot = snapshots.get(i + skipSize).getRoot();
       listing[i] = createFileStatus(fsd, src, sRoot.getLocalNameBytes(), sRoot,
       listing[i] = createFileStatus(fsd, src, sRoot.getLocalNameBytes(), sRoot,
-          BlockStoragePolicySuite.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
+          HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
           false, INodesInPath.fromINode(sRoot));
           false, INodesInPath.fromINode(sRoot));
     }
     }
     return new DirectoryListing(
     return new DirectoryListing(
@@ -277,7 +277,7 @@ class FSDirStatAndListingOp {
     try {
     try {
       final INode i = src.getLastINode();
       final INode i = src.getLastINode();
       byte policyId = includeStoragePolicy && i != null && !i.isSymlink() ?
       byte policyId = includeStoragePolicy && i != null && !i.isSymlink() ?
-          i.getStoragePolicyID() : BlockStoragePolicySuite.ID_UNSPECIFIED;
+          i.getStoragePolicyID() : HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
       return i == null ? null : createFileStatus(
       return i == null ? null : createFileStatus(
           fsd, path, HdfsFileStatus.EMPTY_NAME, i, policyId,
           fsd, path, HdfsFileStatus.EMPTY_NAME, i, policyId,
           src.getPathSnapshotId(), isRawPath, src);
           src.getPathSnapshotId(), isRawPath, src);
@@ -295,7 +295,7 @@ class FSDirStatAndListingOp {
       if (fsd.getINode4DotSnapshot(srcs) != null) {
       if (fsd.getINode4DotSnapshot(srcs) != null) {
         return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
         return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
             HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
             HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
-            BlockStoragePolicySuite.ID_UNSPECIFIED);
+            HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
       }
       }
       return null;
       return null;
     }
     }
@@ -322,7 +322,7 @@ class FSDirStatAndListingOp {
     if (fsd.getINode4DotSnapshot(src) != null) {
     if (fsd.getINode4DotSnapshot(src) != null) {
       return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
       return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
           HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
           HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
-          BlockStoragePolicySuite.ID_UNSPECIFIED);
+          HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
     }
     }
     return null;
     return null;
   }
   }

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

@@ -49,6 +49,7 @@ import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
 import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
@@ -61,7 +62,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
-import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
 import org.apache.hadoop.hdfs.util.ByteArray;
 import org.apache.hadoop.hdfs.util.ByteArray;
 import org.apache.hadoop.hdfs.util.EnumCounters;
 import org.apache.hadoop.hdfs.util.EnumCounters;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
@@ -742,7 +742,7 @@ public class FSDirectory implements Closeable {
     EnumCounters<StorageType> typeSpaceDeltas =
     EnumCounters<StorageType> typeSpaceDeltas =
         new EnumCounters<StorageType>(StorageType.class);
         new EnumCounters<StorageType>(StorageType.class);
     // Storage type and its quota are only available when storage policy is set
     // Storage type and its quota are only available when storage policy is set
-    if (storagePolicyID != BlockStoragePolicySuite.ID_UNSPECIFIED) {
+    if (storagePolicyID != HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
       BlockStoragePolicy storagePolicy = getBlockManager().getStoragePolicy(storagePolicyID);
       BlockStoragePolicy storagePolicy = getBlockManager().getStoragePolicy(storagePolicyID);
 
 
       if (oldRep != newRep) {
       if (oldRep != newRep) {

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

@@ -37,7 +37,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -382,7 +381,7 @@ public class FSEditLogLoader {
         if (toAddRetryCache) {
         if (toAddRetryCache) {
           HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
           HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
               fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME, newFile,
               fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME, newFile,
-              BlockStoragePolicySuite.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
+              HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
               false, iip);
               false, iip);
           fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
           fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
               addCloseOp.rpcCallId, stat);
               addCloseOp.rpcCallId, stat);
@@ -402,7 +401,7 @@ public class FSEditLogLoader {
             HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
             HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
                 fsNamesys.dir, path,
                 fsNamesys.dir, path,
                 HdfsFileStatus.EMPTY_NAME, newFile,
                 HdfsFileStatus.EMPTY_NAME, newFile,
-                BlockStoragePolicySuite.ID_UNSPECIFIED,
+                HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
                 Snapshot.CURRENT_STATE_ID, false, iip);
                 Snapshot.CURRENT_STATE_ID, false, iip);
             fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
             fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
                 addCloseOp.rpcCallId, new LastBlockWithStatus(lb, stat));
                 addCloseOp.rpcCallId, new LastBlockWithStatus(lb, stat));
@@ -475,7 +474,7 @@ public class FSEditLogLoader {
         if (toAddRetryCache) {
         if (toAddRetryCache) {
           HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
           HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
               fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME, file,
               fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME, file,
-              BlockStoragePolicySuite.ID_UNSPECIFIED,
+              HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
               Snapshot.CURRENT_STATE_ID, false, iip);
               Snapshot.CURRENT_STATE_ID, false, iip);
           fsNamesys.addCacheEntryWithPayload(appendOp.rpcClientId,
           fsNamesys.addCacheEntryWithPayload(appendOp.rpcClientId,
               appendOp.rpcCallId, new LastBlockWithStatus(lb, stat));
               appendOp.rpcCallId, new LastBlockWithStatus(lb, stat));

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

@@ -108,7 +108,6 @@ import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEditLogProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrEditLogProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrEditLogProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
 import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
 import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
 import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
@@ -436,7 +435,7 @@ public abstract class FSEditLogOp {
     
     
     private AddCloseOp(FSEditLogOpCodes opCode) {
     private AddCloseOp(FSEditLogOpCodes opCode) {
       super(opCode);
       super(opCode);
-      storagePolicyId = BlockStoragePolicySuite.ID_UNSPECIFIED;
+      storagePolicyId = HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
       assert(opCode == OP_ADD || opCode == OP_CLOSE || opCode == OP_APPEND);
       assert(opCode == OP_ADD || opCode == OP_CLOSE || opCode == OP_APPEND);
     }
     }
 
 
@@ -638,7 +637,7 @@ public abstract class FSEditLogOp {
             NameNodeLayoutVersion.Feature.BLOCK_STORAGE_POLICY, logVersion)) {
             NameNodeLayoutVersion.Feature.BLOCK_STORAGE_POLICY, logVersion)) {
           this.storagePolicyId = FSImageSerialization.readByte(in);
           this.storagePolicyId = FSImageSerialization.readByte(in);
         } else {
         } else {
-          this.storagePolicyId = BlockStoragePolicySuite.ID_UNSPECIFIED;
+          this.storagePolicyId = HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
         }
         }
         // read clientId and callId
         // read clientId and callId
         readRpcIds(in, logVersion);
         readRpcIds(in, logVersion);

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

@@ -3287,7 +3287,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
 
   LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs,
   LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs,
                                         long offset) throws IOException {
                                         long offset) throws IOException {
-    LocatedBlock lBlk = new LocatedBlock(
+    LocatedBlock lBlk = BlockManager.newLocatedBlock(
         getExtendedBlock(blk), locs, offset, false);
         getExtendedBlock(blk), locs, offset, false);
     getBlockManager().setBlockToken(
     getBlockManager().setBlockToken(
         lBlk, BlockTokenIdentifier.AccessMode.WRITE);
         lBlk, BlockTokenIdentifier.AccessMode.WRITE);
@@ -3351,7 +3351,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final DatanodeStorageInfo[] targets = blockManager.chooseTarget4AdditionalDatanode(
     final DatanodeStorageInfo[] targets = blockManager.chooseTarget4AdditionalDatanode(
         src, numAdditionalNodes, clientnode, chosen, 
         src, numAdditionalNodes, clientnode, chosen, 
         excludes, preferredblocksize, storagePolicyID);
         excludes, preferredblocksize, storagePolicyID);
-    final LocatedBlock lb = new LocatedBlock(blk, targets, -1, false);
+    final LocatedBlock lb = BlockManager.newLocatedBlock(
+        blk, targets, -1, false);
     blockManager.setBlockToken(lb, BlockTokenIdentifier.AccessMode.COPY);
     blockManager.setBlockToken(lb, BlockTokenIdentifier.AccessMode.COPY);
     return lb;
     return lb;
   }
   }

+ 7 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -510,7 +511,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    */
    */
   public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) {
   public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) {
     final byte storagePolicyId = isSymlink() ?
     final byte storagePolicyId = isSymlink() ?
-        BlockStoragePolicySuite.ID_UNSPECIFIED : getStoragePolicyID();
+        HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
     return computeQuotaUsage(bsps, storagePolicyId,
     return computeQuotaUsage(bsps, storagePolicyId,
         new QuotaCounts.Builder().build(), true, Snapshot.CURRENT_STATE_ID);
         new QuotaCounts.Builder().build(), true, Snapshot.CURRENT_STATE_ID);
   }
   }
@@ -554,7 +555,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   public final QuotaCounts computeQuotaUsage(
   public final QuotaCounts computeQuotaUsage(
     BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache) {
     BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache) {
     final byte storagePolicyId = isSymlink() ?
     final byte storagePolicyId = isSymlink() ?
-        BlockStoragePolicySuite.ID_UNSPECIFIED : getStoragePolicyID();
+        HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getStoragePolicyID();
     return computeQuotaUsage(bsps, storagePolicyId, counts,
     return computeQuotaUsage(bsps, storagePolicyId, counts,
         useCache, Snapshot.CURRENT_STATE_ID);
         useCache, Snapshot.CURRENT_STATE_ID);
   }
   }
@@ -711,7 +712,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
 
 
   /**
   /**
    * @return the storage policy directly specified on the INode. Return
    * @return the storage policy directly specified on the INode. Return
-   * {@link BlockStoragePolicySuite#ID_UNSPECIFIED} if no policy has
+   * {@link HdfsConstantsClient#BLOCK_STORAGE_POLICY_ID_UNSPECIFIED} if no policy has
    * been specified.
    * been specified.
    */
    */
   public abstract byte getLocalStoragePolicyID();
   public abstract byte getLocalStoragePolicyID();
@@ -720,13 +721,13 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * Get the storage policy ID while computing quota usage
    * Get the storage policy ID while computing quota usage
    * @param parentStoragePolicyId the storage policy ID of the parent directory
    * @param parentStoragePolicyId the storage policy ID of the parent directory
    * @return the storage policy ID of this INode. Note that for an
    * @return the storage policy ID of this INode. Note that for an
-   * {@link INodeSymlink} we return {@link BlockStoragePolicySuite#ID_UNSPECIFIED}
+   * {@link INodeSymlink} we return {@link HdfsConstantsClient#BLOCK_STORAGE_POLICY_ID_UNSPECIFIED}
    * instead of throwing Exception
    * instead of throwing Exception
    */
    */
   public byte getStoragePolicyIDForQuota(byte parentStoragePolicyId) {
   public byte getStoragePolicyIDForQuota(byte parentStoragePolicyId) {
     byte localId = isSymlink() ?
     byte localId = isSymlink() ?
-        BlockStoragePolicySuite.ID_UNSPECIFIED : getLocalStoragePolicyID();
-    return localId != BlockStoragePolicySuite.ID_UNSPECIFIED ?
+        HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED : getLocalStoragePolicyID();
+    return localId != HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED ?
         localId : parentStoragePolicyId;
         localId : parentStoragePolicyId;
   }
   }
 
 

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

@@ -45,7 +45,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableList;
 
 
-import static org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite.ID_UNSPECIFIED;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
 
 
 /**
 /**
  * Directory INode class.
  * Directory INode class.
@@ -125,18 +125,17 @@ public class INodeDirectory extends INodeWithAdditionalFields
         return (xattr.getValue())[0];
         return (xattr.getValue())[0];
       }
       }
     }
     }
-    return ID_UNSPECIFIED;
+    return BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
   }
   }
 
 
   @Override
   @Override
   public byte getStoragePolicyID() {
   public byte getStoragePolicyID() {
     byte id = getLocalStoragePolicyID();
     byte id = getLocalStoragePolicyID();
-    if (id != ID_UNSPECIFIED) {
+    if (id != BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
       return id;
       return id;
     }
     }
     // if it is unspecified, check its parent
     // if it is unspecified, check its parent
-    return getParent() != null ? getParent().getStoragePolicyID() :
-        ID_UNSPECIFIED;
+    return getParent() != null ? getParent().getStoragePolicyID() : BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
   }
   }
 
 
   void setQuota(BlockStoragePolicySuite bsps, long nsQuota, long ssQuota, StorageType type) {
   void setQuota(BlockStoragePolicySuite bsps, long nsQuota, long ssQuota, StorageType type) {

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

@@ -17,7 +17,7 @@
  */
  */
 package org.apache.hadoop.hdfs.server.namenode;
 package org.apache.hadoop.hdfs.server.namenode;
 
 
-import static org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite.ID_UNSPECIFIED;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
 
 
@@ -393,7 +393,7 @@ public class INodeFile extends INodeWithAdditionalFields
   @Override
   @Override
   public byte getStoragePolicyID() {
   public byte getStoragePolicyID() {
     byte id = getLocalStoragePolicyID();
     byte id = getLocalStoragePolicyID();
-    if (id == ID_UNSPECIFIED) {
+    if (id == BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
       return this.getParent() != null ?
       return this.getParent() != null ?
           this.getParent().getStoragePolicyID() : id;
           this.getParent().getStoragePolicyID() : id;
     }
     }
@@ -584,7 +584,7 @@ public class INodeFile extends INodeWithAdditionalFields
     counts.addNameSpace(nsDelta);
     counts.addNameSpace(nsDelta);
     counts.addStorageSpace(ssDeltaNoReplication * replication);
     counts.addStorageSpace(ssDeltaNoReplication * replication);
 
 
-    if (blockStoragePolicyId != ID_UNSPECIFIED){
+    if (blockStoragePolicyId != BLOCK_STORAGE_POLICY_ID_UNSPECIFIED){
       BlockStoragePolicy bsp = bsps.getPolicy(blockStoragePolicyId);
       BlockStoragePolicy bsp = bsps.getPolicy(blockStoragePolicyId);
       List<StorageType> storageTypes = bsp.chooseStorageTypes(replication);
       List<StorageType> storageTypes = bsp.chooseStorageTypes(replication);
       for (StorageType t : storageTypes) {
       for (StorageType t : storageTypes) {
@@ -619,7 +619,7 @@ public class INodeFile extends INodeWithAdditionalFields
     counts.addContent(Content.LENGTH, fileLen);
     counts.addContent(Content.LENGTH, fileLen);
     counts.addContent(Content.DISKSPACE, storagespaceConsumed());
     counts.addContent(Content.DISKSPACE, storagespaceConsumed());
 
 
-    if (getStoragePolicyID() != ID_UNSPECIFIED){
+    if (getStoragePolicyID() != BLOCK_STORAGE_POLICY_ID_UNSPECIFIED){
       BlockStoragePolicy bsp = summary.getBlockStoragePolicySuite().
       BlockStoragePolicy bsp = summary.getBlockStoragePolicySuite().
           getPolicy(getStoragePolicyID());
           getPolicy(getStoragePolicyID());
       List<StorageType> storageTypes = bsp.chooseStorageTypes(getFileReplication());
       List<StorageType> storageTypes = bsp.chooseStorageTypes(getFileReplication());

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

@@ -22,6 +22,7 @@ import java.util.List;
 
 
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.GSet;
 import org.apache.hadoop.util.LightWeightGSet;
 import org.apache.hadoop.util.LightWeightGSet;
@@ -123,12 +124,12 @@ public class INodeMap {
 
 
       @Override
       @Override
       public byte getStoragePolicyID(){
       public byte getStoragePolicyID(){
-        return BlockStoragePolicySuite.ID_UNSPECIFIED;
+        return HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
       }
       }
 
 
       @Override
       @Override
       public byte getLocalStoragePolicyID() {
       public byte getLocalStoragePolicyID() {
-        return BlockStoragePolicySuite.ID_UNSPECIFIED;
+        return HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
       }
       }
     };
     };
       
       

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

@@ -21,6 +21,7 @@ import java.util.List;
 
 
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
@@ -150,7 +151,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
     BlockStoragePolicy bsp = null;
     BlockStoragePolicy bsp = null;
     EnumCounters<StorageType> typeSpaces =
     EnumCounters<StorageType> typeSpaces =
         new EnumCounters<StorageType>(StorageType.class);
         new EnumCounters<StorageType>(StorageType.class);
-    if (storagePolicyID != BlockStoragePolicySuite.ID_UNSPECIFIED) {
+    if (storagePolicyID != HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
       bsp = bsps.getPolicy(file.getStoragePolicyID());
       bsp = bsps.getPolicy(file.getStoragePolicyID());
     }
     }
 
 

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -73,7 +73,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.web.JsonUtil;
 import org.apache.hadoop.hdfs.web.JsonUtil;
 import org.apache.hadoop.hdfs.web.ParamFilter;
 import org.apache.hadoop.hdfs.web.ParamFilter;
-import org.apache.hadoop.hdfs.web.SWebHdfsFileSystem;
+import org.apache.hadoop.hdfs.web.WebHdfsConstants;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.resources.*;
 import org.apache.hadoop.hdfs.web.resources.*;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -246,8 +246,8 @@ public class NamenodeWebHdfsMethods {
       return null;
       return null;
     }
     }
     final Token<? extends TokenIdentifier> t = c.getAllTokens().iterator().next();
     final Token<? extends TokenIdentifier> t = c.getAllTokens().iterator().next();
-    Text kind = request.getScheme().equals("http") ? WebHdfsFileSystem.TOKEN_KIND
-        : SWebHdfsFileSystem.TOKEN_KIND;
+    Text kind = request.getScheme().equals("http") ? WebHdfsConstants.WEBHDFS_TOKEN_KIND
+        : WebHdfsConstants.SWEBHDFS_TOKEN_KIND;
     t.setKind(kind);
     t.setKind(kind);
     return t;
     return t;
   }
   }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/StoragePolicyAdmin.java

@@ -22,8 +22,8 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.tools.TableListing;
 import org.apache.hadoop.tools.TableListing;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.Tool;
@@ -151,7 +151,7 @@ public class StoragePolicyAdmin extends Configured implements Tool {
           return 2;
           return 2;
         }
         }
         byte storagePolicyId = status.getStoragePolicy();
         byte storagePolicyId = status.getStoragePolicy();
-        if (storagePolicyId == BlockStoragePolicySuite.ID_UNSPECIFIED) {
+        if (storagePolicyId == HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
           System.out.println("The storage policy of " + path + " is unspecified");
           System.out.println("The storage policy of " + path + " is unspecified");
           return 0;
           return 0;
         }
         }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java

@@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -128,7 +127,7 @@ class JsonUtilClient {
     final int childrenNum = getInt(m, "childrenNum", -1);
     final int childrenNum = getInt(m, "childrenNum", -1);
     final byte storagePolicy = m.containsKey("storagePolicy") ?
     final byte storagePolicy = m.containsKey("storagePolicy") ?
         (byte) ((Number) m.get("storagePolicy")).longValue() :
         (byte) ((Number) m.get("storagePolicy")).longValue() :
-        BlockStoragePolicySuite.ID_UNSPECIFIED;
+        HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
     return new HdfsFileStatus(len, type == WebHdfsConstants.PathType.DIRECTORY, replication,
     return new HdfsFileStatus(len, type == WebHdfsConstants.PathType.DIRECTORY, replication,
         blockSize, mTime, aTime, permission, owner, group,
         blockSize, mTime, aTime, permission, owner, group,
         symlink, DFSUtil.string2Bytes(localName), fileId, childrenNum, null,
         symlink, DFSUtil.string2Bytes(localName), fileId, childrenNum, null,

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/SWebHdfsFileSystem.java

@@ -24,7 +24,6 @@ import com.google.common.annotations.VisibleForTesting;
 
 
 public class SWebHdfsFileSystem extends WebHdfsFileSystem {
 public class SWebHdfsFileSystem extends WebHdfsFileSystem {
 
 
-  public static final Text TOKEN_KIND = new Text("SWEBHDFS delegation");
   public static final String SCHEME = "swebhdfs";
   public static final String SCHEME = "swebhdfs";
 
 
   @Override
   @Override
@@ -39,7 +38,7 @@ public class SWebHdfsFileSystem extends WebHdfsFileSystem {
 
 
   @Override
   @Override
   protected Text getTokenKind() {
   protected Text getTokenKind() {
-    return TOKEN_KIND;
+    return WebHdfsConstants.SWEBHDFS_TOKEN_KIND;
   }
   }
 
 
   @Override
   @Override

+ 4 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java

@@ -17,8 +17,6 @@
  */
  */
 package org.apache.hadoop.hdfs.web;
 package org.apache.hadoop.hdfs.web;
 
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.HA_DT_SERVICE_PREFIX;
-
 import java.io.IOException;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
@@ -59,8 +57,8 @@ final class TokenAspect<T extends FileSystem & Renewable> {
     public boolean handleKind(Text kind) {
     public boolean handleKind(Text kind) {
       return kind.equals(HftpFileSystem.TOKEN_KIND)
       return kind.equals(HftpFileSystem.TOKEN_KIND)
           || kind.equals(HsftpFileSystem.TOKEN_KIND)
           || kind.equals(HsftpFileSystem.TOKEN_KIND)
-          || kind.equals(WebHdfsFileSystem.TOKEN_KIND)
-          || kind.equals(SWebHdfsFileSystem.TOKEN_KIND);
+          || kind.equals(WebHdfsConstants.WEBHDFS_TOKEN_KIND)
+          || kind.equals(WebHdfsConstants.SWEBHDFS_TOKEN_KIND);
     }
     }
 
 
     @Override
     @Override
@@ -93,9 +91,9 @@ final class TokenAspect<T extends FileSystem & Renewable> {
         return HftpFileSystem.SCHEME;
         return HftpFileSystem.SCHEME;
       } else if (kind.equals(HsftpFileSystem.TOKEN_KIND)) {
       } else if (kind.equals(HsftpFileSystem.TOKEN_KIND)) {
         return HsftpFileSystem.SCHEME;
         return HsftpFileSystem.SCHEME;
-      } else if (kind.equals(WebHdfsFileSystem.TOKEN_KIND)) {
+      } else if (kind.equals(WebHdfsConstants.WEBHDFS_TOKEN_KIND)) {
         return WebHdfsFileSystem.SCHEME;
         return WebHdfsFileSystem.SCHEME;
-      } else if (kind.equals(SWebHdfsFileSystem.TOKEN_KIND)) {
+      } else if (kind.equals(WebHdfsConstants.SWEBHDFS_TOKEN_KIND)) {
         return SWebHdfsFileSystem.SCHEME;
         return SWebHdfsFileSystem.SCHEME;
       } else {
       } else {
         throw new IllegalArgumentException("Unsupported scheme");
         throw new IllegalArgumentException("Unsupported scheme");

+ 1 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -101,9 +101,6 @@ public class WebHdfsFileSystem extends FileSystem
   /** Default connection factory may be overridden in tests to use smaller timeout values */
   /** Default connection factory may be overridden in tests to use smaller timeout values */
   protected URLConnectionFactory connectionFactory;
   protected URLConnectionFactory connectionFactory;
 
 
-  /** Delegation token kind */
-  public static final Text TOKEN_KIND = new Text("WEBHDFS delegation");
-
   @VisibleForTesting
   @VisibleForTesting
   public static final String CANT_FALLBACK_TO_INSECURE_MSG =
   public static final String CANT_FALLBACK_TO_INSECURE_MSG =
       "The client is configured to only allow connecting to secure cluster";
       "The client is configured to only allow connecting to secure cluster";
@@ -139,7 +136,7 @@ public class WebHdfsFileSystem extends FileSystem
   }
   }
 
 
   protected Text getTokenKind() {
   protected Text getTokenKind() {
-    return TOKEN_KIND;
+    return WebHdfsConstants.WEBHDFS_TOKEN_KIND;
   }
   }
 
 
   @Override
   @Override

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java

@@ -17,7 +17,7 @@
  */
  */
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
-import static org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite.ID_UNSPECIFIED;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstantsClient.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
 
 
 import java.io.File;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
@@ -852,8 +852,10 @@ public class TestBlockStoragePolicy {
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
       HdfsFileStatus[] barList = fs.getClient().listPaths(barDir.toString(),
       HdfsFileStatus[] barList = fs.getClient().listPaths(barDir.toString(),
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
           HdfsFileStatus.EMPTY_NAME, true).getPartialListing();
-      checkDirectoryListing(dirList, ID_UNSPECIFIED, ID_UNSPECIFIED);
-      checkDirectoryListing(barList, ID_UNSPECIFIED, ID_UNSPECIFIED);
+      checkDirectoryListing(dirList, BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
+                            BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
+      checkDirectoryListing(barList, BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
+                            BLOCK_STORAGE_POLICY_ID_UNSPECIFIED);
 
 
       final Path invalidPath = new Path("/invalidPath");
       final Path invalidPath = new Path("/invalidPath");
       try {
       try {

+ 11 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java

@@ -34,7 +34,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -103,11 +103,12 @@ public class TestNamenodeCapacityReport {
             + " percentRemaining " + percentRemaining);
             + " percentRemaining " + percentRemaining);
         
         
         assertTrue(configCapacity == (used + remaining + nonDFSUsed));
         assertTrue(configCapacity == (used + remaining + nonDFSUsed));
-        assertTrue(percentUsed == DFSUtil.getPercentUsed(used, configCapacity));
-        assertTrue(percentRemaining == DFSUtil.getPercentRemaining(remaining,
-            configCapacity));
-        assertTrue(percentBpUsed == DFSUtil.getPercentUsed(bpUsed,
-            configCapacity));
+        assertTrue(percentUsed == DFSUtilClient.getPercentUsed(used,
+                                                               configCapacity));
+        assertTrue(percentRemaining == DFSUtilClient.getPercentRemaining(
+            remaining, configCapacity));
+        assertTrue(percentBpUsed == DFSUtilClient.getPercentUsed(bpUsed,
+                                                                 configCapacity));
       }   
       }   
       
       
       DF df = new DF(new File(cluster.getDataDirectory()), conf);
       DF df = new DF(new File(cluster.getDataDirectory()), conf);
@@ -152,10 +153,12 @@ public class TestNamenodeCapacityReport {
       assertTrue(configCapacity == (used + remaining + nonDFSUsed));
       assertTrue(configCapacity == (used + remaining + nonDFSUsed));
 
 
       // Ensure percent used is calculated based on used and present capacity
       // Ensure percent used is calculated based on used and present capacity
-      assertTrue(percentUsed == DFSUtil.getPercentUsed(used, configCapacity));
+      assertTrue(percentUsed == DFSUtilClient.getPercentUsed(used,
+                                                             configCapacity));
 
 
       // Ensure percent used is calculated based on used and present capacity
       // Ensure percent used is calculated based on used and present capacity
-      assertTrue(percentBpUsed == DFSUtil.getPercentUsed(bpUsed, configCapacity));
+      assertTrue(percentBpUsed == DFSUtilClient.getPercentUsed(bpUsed,
+                                                               configCapacity));
 
 
       // Ensure percent used is calculated based on used and present capacity
       // Ensure percent used is calculated based on used and present capacity
       assertTrue(percentRemaining == ((float)remaining * 100.0f)/(float)configCapacity);
       assertTrue(percentRemaining == ((float)remaining * 100.0f)/(float)configCapacity);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java

@@ -328,7 +328,7 @@ public class TestWebHdfsUrl {
           dtId, dtSecretManager);
           dtId, dtSecretManager);
       SecurityUtil.setTokenService(
       SecurityUtil.setTokenService(
           token, NetUtils.createSocketAddr(uri.getAuthority()));
           token, NetUtils.createSocketAddr(uri.getAuthority()));
-      token.setKind(WebHdfsFileSystem.TOKEN_KIND);
+      token.setKind(WebHdfsConstants.WEBHDFS_TOKEN_KIND);
       ugi.addToken(token);
       ugi.addToken(token);
     }
     }
     return (WebHdfsFileSystem) FileSystem.get(uri, conf);
     return (WebHdfsFileSystem) FileSystem.get(uri, conf);