Forráskód Böngészése

HDFS-10559. DiskBalancer: Use SHA1 for Plan ID. Contributed by Xiaobing Zhou.

Anu Engineer 8 éve
szülő
commit
5628b36c08

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

@@ -175,7 +175,7 @@ public interface ClientDatanodeProtocol {
   /**
    * Cancel an executing plan.
    *
-   * @param planID - A SHA512 hash of the plan string.
+   * @param planID - A SHA-1 hash of the plan string.
    */
   void cancelDiskBalancePlan(String planID) throws IOException;
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java

@@ -369,7 +369,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
   /**
    * Cancels an executing disk balancer plan.
    *
-   * @param planID - A SHA512 hash of the plan string.
+   * @param planID - A SHA-1 hash of the plan string.
    * @throws IOException on error
    */
   @Override

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

@@ -154,7 +154,7 @@ message GetBalancerBandwidthResponseProto {
  * balancer plan to a data node.
  */
 message SubmitDiskBalancerPlanRequestProto {
-  required string planID = 1;         // A hash of the plan like SHA512
+  required string planID = 1;         // A hash of the plan like SHA-1
   required string plan = 2;           // Plan file data in Json format
   optional uint64 planVersion = 3;    // Plan version number
   optional bool ignoreDateCheck = 4;  // Ignore date checks on this plan.

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DiskBalancer.java

@@ -157,7 +157,7 @@ public class DiskBalancer {
    * Takes a client submitted plan and converts into a set of work items that
    * can be executed by the blockMover.
    *
-   * @param planId      - A SHA512 of the plan string
+   * @param planId      - A SHA-1 of the plan string
    * @param planVersion - version of the plan string - for future use.
    * @param planFileName    - Plan file name
    * @param planData    - Plan data in json format
@@ -308,7 +308,7 @@ public class DiskBalancer {
   /**
    * Verifies that user provided plan is valid.
    *
-   * @param planID      - SHA 512 of the plan.
+   * @param planID      - SHA-1 of the plan.
    * @param planVersion - Version of the plan, for future use.
    * @param plan        - Plan String in Json.
    * @param force       - Skip verifying when the plan was generated.
@@ -345,15 +345,15 @@ public class DiskBalancer {
   }
 
   /**
-   * Verifies that plan matches the SHA512 provided by the client.
+   * Verifies that plan matches the SHA-1 provided by the client.
    *
-   * @param planID - Sha512 Hex Bytes
+   * @param planID - SHA-1 Hex Bytes
    * @param plan   - Plan String
    * @throws DiskBalancerException
    */
   private NodePlan verifyPlanHash(String planID, String plan)
       throws DiskBalancerException {
-    final long sha512Length = 128;
+    final long sha1Length = 40;
     if (plan == null || plan.length() == 0) {
       LOG.error("Disk Balancer -  Invalid plan.");
       throw new DiskBalancerException("Invalid plan.",
@@ -361,8 +361,8 @@ public class DiskBalancer {
     }
 
     if ((planID == null) ||
-        (planID.length() != sha512Length) ||
-        !DigestUtils.sha512Hex(plan.getBytes(Charset.forName("UTF-8")))
+        (planID.length() != sha1Length) ||
+        !DigestUtils.shaHex(plan.getBytes(Charset.forName("UTF-8")))
             .equalsIgnoreCase(planID)) {
       LOG.error("Disk Balancer - Invalid plan hash.");
       throw new DiskBalancerException("Invalid or mis-matched hash.",

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/CancelCommand.java

@@ -93,7 +93,7 @@ public class CancelCommand extends Command {
     String dataNodeAddress = plan.getNodeName() + ":" + plan.getPort();
     Preconditions.checkNotNull(dataNodeAddress);
     ClientDatanodeProtocol dataNode = getDataNodeProxy(dataNodeAddress);
-    String planHash = DigestUtils.sha512Hex(planData);
+    String planHash = DigestUtils.shaHex(planData);
     try {
       dataNode.cancelDiskBalancePlan(planHash);
     } catch (DiskBalancerException ex) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/command/ExecuteCommand.java

@@ -85,7 +85,7 @@ public class ExecuteCommand extends Command {
     String dataNodeAddress = plan.getNodeName() + ":" + plan.getPort();
     Preconditions.checkNotNull(dataNodeAddress);
     ClientDatanodeProtocol dataNode = getDataNodeProxy(dataNodeAddress);
-    String planHash = DigestUtils.sha512Hex(planData);
+    String planHash = DigestUtils.shaHex(planData);
     try {
       // TODO : Support skipping date check.
       dataNode.submitDiskBalancerPlan(planHash, DiskBalancer.PLAN_VERSION,

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancer.java

@@ -191,7 +191,7 @@ public class TestDiskBalancer {
       plan.setNodeUUID(dnNode.getDatanodeUuid());
       plan.setTimeStamp(Time.now());
       String planJson = plan.toJson();
-      String planID = DigestUtils.sha512Hex(planJson);
+      String planID = DigestUtils.shaHex(planJson);
       assertNotNull(plan.getVolumeSetPlans());
       assertTrue(plan.getVolumeSetPlans().size() > 0);
       plan.getVolumeSetPlans().get(0).setTolerancePercent(10);
@@ -307,7 +307,7 @@ public class TestDiskBalancer {
       plan.setNodeUUID(dnNode.getDatanodeUuid());
       plan.setTimeStamp(Time.now());
       String planJson = plan.toJson();
-      String planID = DigestUtils.sha512Hex(planJson);
+      String planID = DigestUtils.shaHex(planJson);
 
       dataNode.submitDiskBalancerPlan(planID, 1, PLAN_FILE, planJson, false);
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerRPC.java

@@ -317,7 +317,7 @@ public class TestDiskBalancerRPC {
       plan = new NodePlan(node.getDataNodeName(), node.getDataNodePort());
       planner.balanceVolumeSet(node, node.getVolumeSets().get("DISK"), plan);
       planVersion = 1;
-      planHash = DigestUtils.sha512Hex(plan.toJson());
+      planHash = DigestUtils.shaHex(plan.toJson());
       return this;
     }
   }

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/TestDiskBalancerWithMockMover.java

@@ -125,7 +125,7 @@ public class TestDiskBalancerWithMockMover {
   private void executeSubmitPlan(NodePlan plan, DiskBalancer balancer,
                                  int version) throws IOException {
     String planJson = plan.toJson();
-    String planID = DigestUtils.sha512Hex(planJson);
+    String planID = DigestUtils.shaHex(planJson);
     balancer.submitPlan(planID, version, PLAN_FILE, planJson, false);
   }
 
@@ -214,7 +214,7 @@ public class TestDiskBalancerWithMockMover {
     NodePlan plan = mockMoverHelper.getPlan();
     DiskBalancer balancer = mockMoverHelper.getBalancer();
     String planJson = plan.toJson();
-    String planID = DigestUtils.sha512Hex(planJson);
+    String planID = DigestUtils.shaHex(planJson);
 
     thrown.expect(DiskBalancerException.class);
     thrown.expect(new DiskBalancerResultVerifier(DiskBalancerException
@@ -231,7 +231,7 @@ public class TestDiskBalancerWithMockMover {
 
 
     String planJson = plan.toJson();
-    String planID = DigestUtils.sha512Hex(planJson);
+    String planID = DigestUtils.shaHex(planJson);
     char repChar = planID.charAt(0);
     repChar++;
 
@@ -261,7 +261,7 @@ public class TestDiskBalancerWithMockMover {
 
 
     String planJson = plan.toJson();
-    String planID = DigestUtils.sha512Hex(planJson);
+    String planID = DigestUtils.shaHex(planJson);
     balancer.cancelPlan(planID);
 
     DiskBalancerWorkStatus status = balancer.queryWorkStatus();