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

HDFS-14512. ONE_SSD policy will be violated while write data with DistributedFileSystem.create(....favoredNodes). Contributed by Ayush Saxena.

Signed-off-by: Wei-Chiu Chuang <weichiu@apache.org>
(cherry picked from commit c1caab40f27e3e4f58ff1b5ef3e93efc56bbecbe)
(cherry picked from commit 2cd7577a5a58db42a14a659b85fbd475091d0cb2)
Ayush Saxena 6 éve
szülő
commit
90cc5ddd49

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.EnumMap;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
@@ -101,6 +102,17 @@ public abstract class BlockPlacementPolicy {
         excludedNodes, blocksize, storagePolicy, flags);
   }
 
+  /**
+   * @param storageTypes storage types that should be used as targets.
+   */
+  public DatanodeStorageInfo[] chooseTarget(String srcPath, int numOfReplicas,
+      Node writer, List<DatanodeStorageInfo> chosen, boolean returnChosenNodes,
+      Set<Node> excludedNodes, long blocksize, BlockStoragePolicy storagePolicy,
+      EnumSet<AddBlockFlag> flags, EnumMap<StorageType, Integer> storageTypes) {
+    return chooseTarget(srcPath, numOfReplicas, writer, chosen,
+        returnChosenNodes, excludedNodes, blocksize, storagePolicy, flags);
+  }
+
   /**
    * Verify if the block's placement meets requirement of placement policy,
    * i.e. replicas are placed on no less than minRacks racks in the system.

+ 25 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -142,7 +142,16 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                                     final BlockStoragePolicy storagePolicy,
                                     EnumSet<AddBlockFlag> flags) {
     return chooseTarget(numOfReplicas, writer, chosenNodes, returnChosenNodes,
-        excludedNodes, blocksize, storagePolicy, flags);
+        excludedNodes, blocksize, storagePolicy, flags, null);
+  }
+
+  @Override
+  public DatanodeStorageInfo[] chooseTarget(String srcPath, int numOfReplicas,
+      Node writer, List<DatanodeStorageInfo> chosen, boolean returnChosenNodes,
+      Set<Node> excludedNodes, long blocksize, BlockStoragePolicy storagePolicy,
+      EnumSet<AddBlockFlag> flags, EnumMap<StorageType, Integer> storageTypes) {
+    return chooseTarget(numOfReplicas, writer, chosen, returnChosenNodes,
+        excludedNodes, blocksize, storagePolicy, flags, storageTypes);
   }
 
   @Override
@@ -194,7 +203,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         DatanodeStorageInfo[] remainingTargets =
             chooseTarget(src, numOfReplicas, writer,
                 new ArrayList<DatanodeStorageInfo>(numOfReplicas), false,
-                favoriteAndExcludedNodes, blocksize, storagePolicy, flags);
+                favoriteAndExcludedNodes, blocksize, storagePolicy, flags,
+                storageTypes);
         for (int i = 0; i < remainingTargets.length; i++) {
           results.add(remainingTargets[i]);
         }
@@ -244,7 +254,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                                     Set<Node> excludedNodes,
                                     long blocksize,
                                     final BlockStoragePolicy storagePolicy,
-                                    EnumSet<AddBlockFlag> addBlockFlags) {
+                                    EnumSet<AddBlockFlag> addBlockFlags,
+                                    EnumMap<StorageType, Integer> sTypes) {
     if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
       return DatanodeStorageInfo.EMPTY_ARRAY;
     }
@@ -282,7 +293,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       localNode = chooseTarget(numOfReplicas, writer,
           excludedNodeCopy, blocksize, maxNodesPerRack, results,
           avoidStaleNodes, storagePolicy,
-          EnumSet.noneOf(StorageType.class), results.isEmpty());
+          EnumSet.noneOf(StorageType.class), results.isEmpty(), sTypes);
       if (results.size() < numOfReplicas) {
         // not enough nodes; discard results and fall back
         results = null;
@@ -292,7 +303,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       results = new ArrayList<>(chosenStorage);
       localNode = chooseTarget(numOfReplicas, writer, excludedNodes,
           blocksize, maxNodesPerRack, results, avoidStaleNodes,
-          storagePolicy, EnumSet.noneOf(StorageType.class), results.isEmpty());
+          storagePolicy, EnumSet.noneOf(StorageType.class), results.isEmpty(),
+          sTypes);
     }
 
     if (!returnChosenNodes) {  
@@ -372,6 +384,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
    * @param maxNodesPerRack max nodes allowed per rack
    * @param results the target nodes already chosen
    * @param avoidStaleNodes avoid stale nodes in replica choosing
+   * @param storageTypes storage type to be considered for target
    * @return local node of writer (not chosen node)
    */
   private Node chooseTarget(int numOfReplicas,
@@ -383,7 +396,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                             final boolean avoidStaleNodes,
                             final BlockStoragePolicy storagePolicy,
                             final EnumSet<StorageType> unavailableStorages,
-                            final boolean newBlock) {
+                            final boolean newBlock,
+                            EnumMap<StorageType, Integer> storageTypes) {
     if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
       return (writer instanceof DatanodeDescriptor) ? writer : null;
     }
@@ -401,8 +415,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         .chooseStorageTypes((short) totalReplicasExpected,
             DatanodeStorageInfo.toStorageTypes(results),
             unavailableStorages, newBlock);
-    final EnumMap<StorageType, Integer> storageTypes =
-        getRequiredStorageTypes(requiredStorageTypes);
+    if (storageTypes == null) {
+      storageTypes = getRequiredStorageTypes(requiredStorageTypes);
+    }
     if (LOG.isTraceEnabled()) {
       LOG.trace("storageTypes=" + storageTypes);
     }
@@ -445,7 +460,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         numOfReplicas = totalReplicasExpected - results.size();
         return chooseTarget(numOfReplicas, writer, oldExcludedNodes, blocksize,
             maxNodesPerRack, results, false, storagePolicy, unavailableStorages,
-            newBlock);
+            newBlock, null);
       }
 
       boolean retry = false;
@@ -465,7 +480,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         numOfReplicas = totalReplicasExpected - results.size();
         return chooseTarget(numOfReplicas, writer, oldExcludedNodes, blocksize,
             maxNodesPerRack, results, false, storagePolicy, unavailableStorages,
-            newBlock);
+            newBlock, null);
       }
     }
     return writer;

+ 29 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -39,6 +39,7 @@ import java.net.SocketTimeoutException;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashSet;
@@ -48,6 +49,7 @@ import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.conf.Configuration;
@@ -75,6 +77,7 @@ import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DistributedFileSystem.HdfsDataOutputStreamBuilder;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
 import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType;
 import org.apache.hadoop.hdfs.net.Peer;
@@ -1763,4 +1766,30 @@ public class TestDistributedFileSystem {
       }
     }
   }
+
+  @Test
+  public void testStorageFavouredNodes()
+      throws IOException, InterruptedException, TimeoutException {
+    Configuration conf = new HdfsConfiguration();
+    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .storageTypes(new StorageType[] {StorageType.SSD, StorageType.DISK})
+        .numDataNodes(3).storagesPerDatanode(2).build()) {
+      DistributedFileSystem fs = cluster.getFileSystem();
+      Path file1 = new Path("/tmp/file1");
+      fs.mkdirs(new Path("/tmp"));
+      fs.setStoragePolicy(new Path("/tmp"), "ONE_SSD");
+      InetSocketAddress[] addrs =
+          {cluster.getDataNodes().get(0).getXferAddress()};
+      HdfsDataOutputStream stream = fs.create(file1, FsPermission.getDefault(),
+          false, 1024, (short) 3, 1024, null, addrs);
+      stream.write("Some Bytes".getBytes());
+      stream.close();
+      DFSTestUtil.waitReplication(fs, file1, (short) 3);
+      BlockLocation[] locations = fs.getClient()
+          .getBlockLocations(file1.toUri().getPath(), 0, Long.MAX_VALUE);
+      int numSSD = Collections.frequency(
+          Arrays.asList(locations[0].getStorageTypes()), StorageType.SSD);
+      assertEquals("Number of SSD should be 1 but was : " + numSSD, 1, numSSD);
+    }
+  }
 }