Explorar o código

HDFS-12447. Rename AddECPolicyResponse to AddErasureCodingPolicyResponse. Contributed by SammiChen.

(cherry picked from commit a12f09ba3c4a3aa4c4558090c5e1b7bcaebe3b94)
Andrew Wang %!s(int64=7) %!d(string=hai) anos
pai
achega
1792093bae
Modificáronse 18 ficheiros con 85 adicións e 73 borrados
  1. 6 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  2. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  3. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  4. 6 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AddErasureCodingPolicyResponse.java
  5. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  6. 6 5
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  7. 12 10
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  8. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto
  9. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
  10. 7 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  11. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  12. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  13. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/ECAdmin.java
  14. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  15. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
  16. 13 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
  17. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
  18. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java

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

@@ -102,7 +102,7 @@ import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.AclException;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -2807,13 +2807,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
-  public AddECPolicyResponse[] addErasureCodingPolicies(
+  public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
     checkOpen();
     try (TraceScope ignored = tracer.newScope("addErasureCodingPolicies")) {
       return namenode.addErasureCodingPolicies(policies);
     } catch (RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class);
+      throw re.unwrapRemoteException(AccessControlException.class,
+          SafeModeException.class);
     }
   }
 
@@ -2823,7 +2824,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     try (TraceScope ignored = tracer.newScope("removeErasureCodingPolicy")) {
       namenode.removeErasureCodingPolicy(ecPolicyName);
     } catch (RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class);
+      throw re.unwrapRemoteException(AccessControlException.class,
+          SafeModeException.class);
     }
   }
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -64,7 +64,7 @@ import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.impl.CorruptFileBlockIterator;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -2650,7 +2650,7 @@ public class DistributedFileSystem extends FileSystem {
    * @return Return the response list of adding operations.
    * @throws IOException
    */
-  public AddECPolicyResponse[] addErasureCodingPolicies(
+  public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies)  throws IOException {
     return dfs.addErasureCodingPolicies(policies);
   }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java

@@ -35,7 +35,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSInotifyEventInputStream;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
@@ -562,7 +562,7 @@ public class HdfsAdmin {
    * @return Return the response list of adding operations.
    * @throws IOException
    */
-  public AddECPolicyResponse[] addErasureCodingPolicies(
+  public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies)  throws IOException {
     return dfs.addErasureCodingPolicies(policies);
   }

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

@@ -22,25 +22,25 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 /**
  * A response of add an ErasureCoding policy.
  */
-public class AddECPolicyResponse {
+public class AddErasureCodingPolicyResponse {
   private boolean succeed;
   private ErasureCodingPolicy policy;
   private String errorMsg;
 
-  public AddECPolicyResponse(ErasureCodingPolicy policy) {
+  public AddErasureCodingPolicyResponse(ErasureCodingPolicy policy) {
     this.policy = policy;
     this.succeed = true;
   }
 
-  public AddECPolicyResponse(ErasureCodingPolicy policy,
-      String errorMsg) {
+  public AddErasureCodingPolicyResponse(ErasureCodingPolicy policy,
+                                        String errorMsg) {
     this.policy = policy;
     this.errorMsg = errorMsg;
     this.succeed = false;
   }
 
-  public AddECPolicyResponse(ErasureCodingPolicy policy,
-      HadoopIllegalArgumentException e) {
+  public AddErasureCodingPolicyResponse(ErasureCodingPolicy policy,
+                                        HadoopIllegalArgumentException e) {
     this(policy, e.getMessage());
   }
 

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

@@ -1584,7 +1584,7 @@ public interface ClientProtocol {
    * @throws IOException
    */
   @AtMostOnce
-  AddECPolicyResponse[] addErasureCodingPolicies(
+  AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException;
 
   /**

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -50,7 +50,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.AddBlockFlag;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -1718,7 +1718,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public AddECPolicyResponse[] addErasureCodingPolicies(
+  public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
     List<ErasureCodingPolicyProto> protos = Arrays.stream(policies)
         .map(PBHelperClient::convertErasureCodingPolicy)
@@ -1729,9 +1729,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       AddErasureCodingPoliciesResponseProto rep = rpcProxy
           .addErasureCodingPolicies(null, req);
-      AddECPolicyResponse[] responses = rep.getResponsesList().stream()
-          .map(PBHelperClient::convertAddECPolicyResponse)
-          .toArray(AddECPolicyResponse[]::new);
+      AddErasureCodingPolicyResponse[] responses =
+          rep.getResponsesList().stream()
+              .map(PBHelperClient::convertAddErasureCodingPolicyResponse)
+              .toArray(AddErasureCodingPolicyResponse[]::new);
       return responses;
     }  catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);

+ 12 - 10
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -58,7 +58,7 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.inotify.Event;
 import org.apache.hadoop.hdfs.inotify.EventBatch;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.BlockType;
@@ -137,7 +137,7 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ReencryptionS
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ZoneReencryptionStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.AccessModeProto;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.AddECPolicyResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.AddErasureCodingPolicyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTypeProto;
@@ -2981,10 +2981,11 @@ public class PBHelperClient {
     return builder.build();
   }
 
-  public static AddECPolicyResponseProto convertAddECPolicyResponse(
-      AddECPolicyResponse response) {
-    AddECPolicyResponseProto.Builder builder =
-        AddECPolicyResponseProto.newBuilder()
+  public static AddErasureCodingPolicyResponseProto
+      convertAddErasureCodingPolicyResponse(
+          AddErasureCodingPolicyResponse response) {
+    AddErasureCodingPolicyResponseProto.Builder builder =
+        AddErasureCodingPolicyResponseProto.newBuilder()
         .setPolicy(convertErasureCodingPolicy(response.getPolicy()))
         .setSucceed(response.isSucceed());
     if (!response.isSucceed()) {
@@ -2993,13 +2994,14 @@ public class PBHelperClient {
     return builder.build();
   }
 
-  public static AddECPolicyResponse convertAddECPolicyResponse(
-      AddECPolicyResponseProto proto) {
+  public static AddErasureCodingPolicyResponse
+      convertAddErasureCodingPolicyResponse(
+          AddErasureCodingPolicyResponseProto proto) {
     ErasureCodingPolicy policy = convertErasureCodingPolicy(proto.getPolicy());
     if (proto.getSucceed()) {
-      return new AddECPolicyResponse(policy);
+      return new AddErasureCodingPolicyResponse(policy);
     } else {
-      return new AddECPolicyResponse(policy, proto.getErrorMsg());
+      return new AddErasureCodingPolicyResponse(policy, proto.getErrorMsg());
     }
   }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto

@@ -58,7 +58,7 @@ message AddErasureCodingPoliciesRequestProto {
 }
 
 message AddErasureCodingPoliciesResponseProto {
-  repeated AddECPolicyResponseProto responses = 1;
+  repeated AddErasureCodingPolicyResponseProto responses = 1;
 }
 
 message RemoveErasureCodingPolicyRequestProto {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto

@@ -390,7 +390,7 @@ message ErasureCodingPolicyProto {
   optional ErasureCodingPolicyState state = 5 [default = ENABLED];
 }
 
-message AddECPolicyResponseProto {
+message AddErasureCodingPolicyResponseProto {
   required ErasureCodingPolicyProto policy = 1;
   required bool succeed = 2;
   optional string errorMsg = 3;

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

@@ -36,7 +36,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.QuotaUsage;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -1721,15 +1721,16 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       ErasureCodingPolicy[] policies = request.getEcPoliciesList().stream()
           .map(PBHelperClient::convertErasureCodingPolicy)
           .toArray(ErasureCodingPolicy[]::new);
-      AddECPolicyResponse[] result = server
+      AddErasureCodingPolicyResponse[] result = server
           .addErasureCodingPolicies(policies);
 
-      List<HdfsProtos.AddECPolicyResponseProto> responseProtos = Arrays
-          .stream(result).map(PBHelperClient::convertAddECPolicyResponse)
-          .collect(Collectors.toList());
+      List<HdfsProtos.AddErasureCodingPolicyResponseProto> responseProtos =
+          Arrays.stream(result)
+              .map(PBHelperClient::convertAddErasureCodingPolicyResponse)
+              .collect(Collectors.toList());
       AddErasureCodingPoliciesResponseProto response =
           AddErasureCodingPoliciesResponseProto.newBuilder()
-            .addAllResponses(responseProtos).build();
+              .addAllResponses(responseProtos).build();
       return response;
     } catch (IOException e) {
       throw new ServiceException(e);

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -180,7 +180,7 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.UnknownCryptoProtocolVersionException;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockType;
@@ -7193,12 +7193,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    *                      rebuilding
    * @return The according result of add operation.
    */
-  AddECPolicyResponse[] addErasureCodingPolicies(ErasureCodingPolicy[] policies,
-      final boolean logRetryCache) throws IOException {
+  AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
+      ErasureCodingPolicy[] policies, final boolean logRetryCache)
+      throws IOException {
     final String operationName = "addErasureCodingPolicies";
     String addECPolicyName = "";
     checkOperation(OperationCategory.WRITE);
-    List<AddECPolicyResponse> responses = new ArrayList<>();
+    List<AddErasureCodingPolicyResponse> responses = new ArrayList<>();
     boolean success = false;
     writeLock();
     try {
@@ -7210,13 +7211,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
               FSDirErasureCodingOp.addErasureCodingPolicy(this, policy,
                   logRetryCache);
           addECPolicyName = newPolicy.getName();
-          responses.add(new AddECPolicyResponse(newPolicy));
+          responses.add(new AddErasureCodingPolicyResponse(newPolicy));
         } catch (HadoopIllegalArgumentException e) {
-          responses.add(new AddECPolicyResponse(policy, e));
+          responses.add(new AddErasureCodingPolicyResponse(policy, e));
         }
       }
       success = true;
-      return responses.toArray(new AddECPolicyResponse[0]);
+      return responses.toArray(new AddErasureCodingPolicyResponse[0]);
     } finally {
       writeUnlock(operationName);
       if (success) {

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

@@ -85,7 +85,7 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
 import org.apache.hadoop.hdfs.inotify.EventBatch;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.AclException;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
@@ -2333,17 +2333,18 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override
-  public AddECPolicyResponse[] addErasureCodingPolicies(
+  public AddErasureCodingPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
     checkNNStartup();
     namesystem.checkSuperuserPrivilege();
     final CacheEntryWithPayload cacheEntry =
         RetryCache.waitForCompletion(retryCache, null);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
-      return (AddECPolicyResponse[]) cacheEntry.getPayload();
+      return (AddErasureCodingPolicyResponse[]) cacheEntry.getPayload();
     }
     boolean success = false;
-    AddECPolicyResponse[] responses = new AddECPolicyResponse[0];
+    AddErasureCodingPolicyResponse[] responses =
+        new AddErasureCodingPolicyResponse[0];
     try {
       responses =
           namesystem.addErasureCodingPolicies(policies, cacheEntry != null);

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

@@ -21,7 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.util.ECPolicyLoader;
 import org.apache.hadoop.io.erasurecode.ErasureCodeConstants;
@@ -175,9 +175,10 @@ public class ECAdmin extends Configured implements Tool {
         List<ErasureCodingPolicy> policies =
             new ECPolicyLoader().loadPolicy(filePath);
         if (policies.size() > 0) {
-          AddECPolicyResponse[] responses = dfs.addErasureCodingPolicies(
+          AddErasureCodingPolicyResponse[] responses =
+              dfs.addErasureCodingPolicies(
             policies.toArray(new ErasureCodingPolicy[policies.size()]));
-          for (AddECPolicyResponse response : responses) {
+          for (AddErasureCodingPolicyResponse response : responses) {
             System.out.println(response);
           }
         } else {

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

@@ -108,7 +108,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
@@ -1472,7 +1472,7 @@ public class DFSTestUtil {
     ErasureCodingPolicy newPolicy1 =
         new ErasureCodingPolicy(ErasureCodeConstants.RS_3_2_SCHEMA, 8 * 1024);
     ErasureCodingPolicy[] policyArray = new ErasureCodingPolicy[] {newPolicy1};
-    AddECPolicyResponse[] responses =
+    AddErasureCodingPolicyResponse[] responses =
         filesystem.addErasureCodingPolicies(policyArray);
     newPolicy1 = responses[0].getPolicy();
 

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

@@ -23,7 +23,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -686,7 +686,7 @@ public class TestErasureCodingPolicies {
     ErasureCodingPolicy newPolicy =
         new ErasureCodingPolicy(toAddSchema, 128 * 1024);
     ErasureCodingPolicy[] policyArray = new ErasureCodingPolicy[]{newPolicy};
-    AddECPolicyResponse[] responses =
+    AddErasureCodingPolicyResponse[] responses =
         fs.addErasureCodingPolicies(policyArray);
     assertEquals(1, responses.length);
     assertFalse(responses[0].isSucceed());
@@ -839,7 +839,8 @@ public class TestErasureCodingPolicies {
         new ErasureCodingPolicy(ErasureCodeConstants.RS_3_2_SCHEMA, 8 * 1024);
     ErasureCodingPolicy[] policyArray =
         new ErasureCodingPolicy[] {newPolicy1};
-    AddECPolicyResponse[] responses = fs.addErasureCodingPolicies(policyArray);
+    AddErasureCodingPolicyResponse[] responses =
+        fs.addErasureCodingPolicies(policyArray);
     assertEquals(1, responses.length);
     assertTrue(responses[0].isSucceed());
     newPolicy1 = responses[0].getPolicy();

+ 13 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java

@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.protocolPB;
 
 
 import com.google.protobuf.UninitializedMessageException;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports;
 
@@ -913,14 +913,15 @@ public class TestPBHelper {
     // Check conversion of the built-in policies.
     for (ErasureCodingPolicy policy :
         SystemErasureCodingPolicies.getPolicies()) {
-      AddECPolicyResponse response = new AddECPolicyResponse(policy);
-      HdfsProtos.AddECPolicyResponseProto proto = PBHelperClient
-          .convertAddECPolicyResponse(response);
+      AddErasureCodingPolicyResponse response =
+          new AddErasureCodingPolicyResponse(policy);
+      HdfsProtos.AddErasureCodingPolicyResponseProto proto = PBHelperClient
+          .convertAddErasureCodingPolicyResponse(response);
       // Optional fields should not be set.
       assertFalse("Unnecessary field is set.", proto.hasErrorMsg());
       // Convert proto back to an object and check for equality.
-      AddECPolicyResponse convertedResponse = PBHelperClient
-          .convertAddECPolicyResponse(proto);
+      AddErasureCodingPolicyResponse convertedResponse = PBHelperClient
+          .convertAddErasureCodingPolicyResponse(proto);
       assertEquals("Converted policy not equal", response.getPolicy(),
           convertedResponse.getPolicy());
       assertEquals("Converted policy not equal", response.isSucceed(),
@@ -929,13 +930,13 @@ public class TestPBHelper {
 
     ErasureCodingPolicy policy = SystemErasureCodingPolicies
         .getPolicies().get(0);
-    AddECPolicyResponse response =
-        new AddECPolicyResponse(policy, "failed");
-    HdfsProtos.AddECPolicyResponseProto proto = PBHelperClient
-        .convertAddECPolicyResponse(response);
+    AddErasureCodingPolicyResponse response =
+        new AddErasureCodingPolicyResponse(policy, "failed");
+    HdfsProtos.AddErasureCodingPolicyResponseProto proto = PBHelperClient
+        .convertAddErasureCodingPolicyResponse(response);
     // Convert proto back to an object and check for equality.
-    AddECPolicyResponse convertedResponse = PBHelperClient
-        .convertAddECPolicyResponse(proto);
+    AddErasureCodingPolicyResponse convertedResponse = PBHelperClient
+        .convertAddErasureCodingPolicyResponse(proto);
     assertEquals("Converted policy not equal", response.getPolicy(),
         convertedResponse.getPolicy());
     assertEquals("Converted policy not equal", response.getErrorMsg(),

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java

@@ -47,7 +47,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
@@ -735,7 +735,7 @@ public class TestFSEditLogLoader {
       ErasureCodingPolicy newPolicy =
           new ErasureCodingPolicy(schema, cellSize, (byte) 0);
       ErasureCodingPolicy[] policyArray = new ErasureCodingPolicy[]{newPolicy};
-      AddECPolicyResponse[] responses =
+      AddErasureCodingPolicyResponse[] responses =
           fs.addErasureCodingPolicies(policyArray);
       assertEquals(1, responses.length);
       assertTrue(responses[0].isSucceed());

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java

@@ -35,7 +35,7 @@ import java.io.IOException;
 import java.util.EnumSet;
 
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
-import org.apache.hadoop.hdfs.protocol.AddECPolicyResponse;
+import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
@@ -847,7 +847,8 @@ public class TestFSImage {
       ErasureCodingPolicy newPolicy =
           new ErasureCodingPolicy(newSchema, 2 * 1024, (byte) 254);
       ErasureCodingPolicy[] policies = new ErasureCodingPolicy[]{newPolicy};
-      AddECPolicyResponse[] ret = fs.addErasureCodingPolicies(policies);
+      AddErasureCodingPolicyResponse[] ret =
+          fs.addErasureCodingPolicies(policies);
       assertEquals(1, ret.length);
       assertEquals(true, ret[0].isSucceed());
       newPolicy = ret[0].getPolicy();