Browse Source

HDFS-8218. Move classes that used by ClientProtocol into hdfs-client. Contributed by Haohui Mai.

Haohui Mai 10 năm trước cách đây
mục cha
commit
12f4df043f
21 tập tin đã thay đổi với 52 bổ sung42 xóa
  1. 7 0
      hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
  2. 0 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/CacheFlag.java
  3. 11 11
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/XAttr.java
  4. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java
  5. 0 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/EventBatch.java
  6. 0 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/EventBatchList.java
  7. 0 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AlreadyBeingCreatedException.java
  8. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
  9. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CorruptFileBlocks.java
  10. 0 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DSQuotaExceededException.java
  11. 0 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeLocalInfo.java
  12. 5 5
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DirectoryListing.java
  13. 0 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZone.java
  14. 0 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LastBlockWithStatus.java
  15. 6 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/NSQuotaExceededException.java
  16. 10 10
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaExceededException.java
  17. 4 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeInfo.java
  18. 0 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeStatus.java
  19. 0 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotAccessControlException.java
  20. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/block/DataEncryptionKey.java
  21. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

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

@@ -1,9 +1,16 @@
 <FindBugsFilter>
   <Match>
     <Or>
+      <Class name="org.apache.hadoop.fs.XAttr"/>
+      <Class name="org.apache.hadoop.fs.XAttr$Builder"/>
+      <Class name="org.apache.hadoop.hdfs.inotify.EventBatch"/>
       <Class name="org.apache.hadoop.hdfs.protocol.HdfsFileStatus"/>
       <Class name="org.apache.hadoop.hdfs.protocol.LocatedBlock"/>
+      <Class name="org.apache.hadoop.hdfs.protocol.BlockStoragePolicy"/>
+      <Class name="org.apache.hadoop.hdfs.protocol.CorruptFileBlocks"/>
+      <Class name="org.apache.hadoop.hdfs.protocol.DirectoryListing"/>
       <Class name="org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier"/>
+      <Class name="org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey"/>
     </Or>
     <Bug pattern="EI_EXPOSE_REP,EI_EXPOSE_REP2" />
   </Match>

+ 0 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/CacheFlag.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/CacheFlag.java


+ 11 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/XAttr.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/XAttr.java

@@ -64,54 +64,54 @@ public class XAttr {
     SYSTEM,
     RAW;
   }
-  
+
   private final NameSpace ns;
   private final String name;
   private final byte[] value;
-  
+
   public static class Builder {
     private NameSpace ns = NameSpace.USER;
     private String name;
     private byte[] value;
-    
+
     public Builder setNameSpace(NameSpace ns) {
       this.ns = ns;
       return this;
     }
-    
+
     public Builder setName(String name) {
       this.name = name;
       return this;
     }
-    
+
     public Builder setValue(byte[] value) {
       this.value = value;
       return this;
     }
-    
+
     public XAttr build() {
       return new XAttr(ns, name, value);
     }
   }
-  
+
   private XAttr(NameSpace ns, String name, byte[] value) {
     this.ns = ns;
     this.name = name;
     this.value = value;
   }
-  
+
   public NameSpace getNameSpace() {
     return ns;
   }
-  
+
   public String getName() {
     return name;
   }
-  
+
   public byte[] getValue() {
     return value;
   }
-  
+
   @Override
   public int hashCode() {
     return new HashCodeBuilder(811, 67)

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java

@@ -154,7 +154,7 @@ public abstract class Event {
         this.symlinkTarget = symlinkTarget;
         return this;
       }
-      
+
       public Builder overwrite(boolean overwrite) {
         this.overwrite = overwrite;
         return this;
@@ -224,7 +224,7 @@ public abstract class Event {
     public String getSymlinkTarget() {
       return symlinkTarget;
     }
-    
+
     public boolean getOverwrite() {
       return overwrite;
     }

+ 0 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/inotify/EventBatch.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/EventBatch.java


+ 0 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/inotify/EventBatchList.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/EventBatchList.java


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


+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java

@@ -123,7 +123,7 @@ public class BlockStoragePolicy {
    * Choose the storage types for storing the remaining replicas, given the
    * replication number, the storage types of the chosen replicas and
    * the unavailable storage types. It uses fallback storage in case that
-   * the desired storage type is unavailable.  
+   * the desired storage type is unavailable.
    *
    * @param replication the replication number.
    * @param chosen the storage types of the chosen replicas.

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CorruptFileBlocks.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CorruptFileBlocks.java

@@ -61,7 +61,7 @@ public class CorruptFileBlocks {
       Arrays.equals(files, other.files);
   }
 
-  
+
   @Override
   public int hashCode() {
     int result = cookie.hashCode();

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


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


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

@@ -28,13 +28,13 @@ import org.apache.hadoop.classification.InterfaceStability;
 public class DirectoryListing {
   private HdfsFileStatus[] partialListing;
   private int remainingEntries;
-  
+
   /**
    * constructor
    * @param partialListing a partial listing of a directory
    * @param remainingEntries number of entries that are left to be listed
    */
-  public DirectoryListing(HdfsFileStatus[] partialListing, 
+  public DirectoryListing(HdfsFileStatus[] partialListing,
       int remainingEntries) {
     if (partialListing == null) {
       throw new IllegalArgumentException("partial listing should not be null");
@@ -54,7 +54,7 @@ public class DirectoryListing {
   public HdfsFileStatus[] getPartialListing() {
     return partialListing;
   }
-  
+
   /**
    * Get the number of remaining entries that are left to be listed
    * @return the number of remaining entries that are left to be listed
@@ -62,7 +62,7 @@ public class DirectoryListing {
   public int getRemainingEntries() {
     return remainingEntries;
   }
-  
+
   /**
    * Check if there are more entries that are left to be listed
    * @return true if there are more entries that are left to be listed;
@@ -71,7 +71,7 @@ public class DirectoryListing {
   public boolean hasMore() {
     return remainingEntries != 0;
   }
-  
+
   /**
    * Get the last name in this list
    * @return the last name in the list if it is not empty; otherwise return null

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


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


+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/NSQuotaExceededException.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/NSQuotaExceededException.java

@@ -25,15 +25,15 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Evolving
 public final class NSQuotaExceededException extends QuotaExceededException {
   protected static final long serialVersionUID = 1L;
-  
+
   private String prefix;
-  
+
   public NSQuotaExceededException() {}
 
   public NSQuotaExceededException(String msg) {
     super(msg);
   }
-  
+
   public NSQuotaExceededException(long quota, long count) {
     super(quota, count);
   }
@@ -42,9 +42,9 @@ public final class NSQuotaExceededException extends QuotaExceededException {
   public String getMessage() {
     String msg = super.getMessage();
     if (msg == null) {
-      msg = "The NameSpace quota (directories and files)" + 
-      (pathName==null?"":(" of directory " + pathName)) + 
-          " is exceeded: quota=" + quota + " file count=" + count; 
+      msg = "The NameSpace quota (directories and files)" +
+      (pathName==null?"":(" of directory " + pathName)) +
+          " is exceeded: quota=" + quota + " file count=" + count;
 
       if (prefix != null) {
         msg = prefix + ": " + msg;

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

@@ -23,15 +23,15 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
-/** 
+/**
  * This exception is thrown when modification to HDFS results in violation
- * of a directory quota. A directory quota might be namespace quota (limit 
- * on number of files and directories) or a diskspace quota (limit on space 
+ * of a directory quota. A directory quota might be namespace quota (limit
+ * on number of files and directories) or a diskspace quota (limit on space
  * taken by all the file under the directory tree). <br> <br>
- * 
+ *
  * The message for the exception specifies the directory where the quota
- * was violated and actual quotas. Specific message is generated in the 
- * corresponding Exception class: 
+ * was violated and actual quotas. Specific message is generated in the
+ * corresponding Exception class:
  *  DSQuotaExceededException or
  *  NSQuotaExceededException
  */
@@ -42,22 +42,22 @@ public class QuotaExceededException extends IOException {
   protected String pathName=null;
   protected long quota; // quota
   protected long count; // actual value
-  
+
   protected QuotaExceededException() {}
 
   protected QuotaExceededException(String msg) {
     super(msg);
   }
-  
+
   protected QuotaExceededException(long quota, long count) {
     this.quota = quota;
     this.count = count;
   }
-  
+
   public void setPathName(String path) {
     this.pathName = path;
   }
-  
+
   @Override
   public String getMessage() {
     return super.getMessage();

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeInfo.java → hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeInfo.java

@@ -31,7 +31,7 @@ public class RollingUpgradeInfo extends RollingUpgradeStatus {
   private final long startTime;
   private long finalizeTime;
   private boolean createdRollbackImages;
-  
+
   public RollingUpgradeInfo(String blockPoolId, boolean createdRollbackImages,
       long startTime, long finalizeTime) {
     super(blockPoolId, finalizeTime != 0);
@@ -39,7 +39,7 @@ public class RollingUpgradeInfo extends RollingUpgradeStatus {
     this.startTime = startTime;
     this.finalizeTime = finalizeTime;
   }
-  
+
   public boolean createdRollbackImages() {
     return createdRollbackImages;
   }
@@ -51,7 +51,7 @@ public class RollingUpgradeInfo extends RollingUpgradeStatus {
   public boolean isStarted() {
     return startTime != 0;
   }
-  
+
   /** @return The rolling upgrade starting time. */
   public long getStartTime() {
     return startTime;
@@ -102,7 +102,7 @@ public class RollingUpgradeInfo extends RollingUpgradeStatus {
       +  "\n     Start Time: " + (startTime == 0? "<NOT STARTED>": timestamp2String(startTime))
       +  "\n  Finalize Time: " + (finalizeTime == 0? "<NOT FINALIZED>": timestamp2String(finalizeTime));
   }
-  
+
   private static String timestamp2String(long timestamp) {
     return new Date(timestamp) + " (=" + timestamp + ")";
   }

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


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


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

@@ -31,7 +31,7 @@ public class DataEncryptionKey {
   public final byte[] encryptionKey;
   public final long expiryDate;
   public final String encryptionAlgorithm;
-  
+
   public DataEncryptionKey(int keyId, String blockPoolId, byte[] nonce,
       byte[] encryptionKey, long expiryDate, String encryptionAlgorithm) {
     this.keyId = keyId;
@@ -41,7 +41,7 @@ public class DataEncryptionKey {
     this.expiryDate = expiryDate;
     this.encryptionAlgorithm = encryptionAlgorithm;
   }
-  
+
   @Override
   public String toString() {
     return keyId + "/" + blockPoolId + "/" + nonce.length + "/" +

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

@@ -458,6 +458,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8185. Separate client related routines in HAUtil into a new class.
     (wheat9)
 
+    HDFS-8218. Move classes that used by ClientProtocol into hdfs-client.
+    (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than