Procházet zdrojové kódy

HDFS-13448. HDFS Block Placement - Ignore Locality for First Block Replica
(Contributed by BELUGA BEHR via Daniel Templeton)

Change-Id: I965d1cfa642ad24296038b83e3d5c9983545267d

Daniel Templeton před 6 roky
rodič
revize
849c45db18

+ 8 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java

@@ -116,7 +116,14 @@ public enum CreateFlag {
    * Enforce the file to be a replicated file, no matter what its parent
    * directory's replication or erasure coding policy is.
    */
-  SHOULD_REPLICATE((short) 0x80);
+  SHOULD_REPLICATE((short) 0x80),
+
+  /**
+   * Advise that the first block replica NOT take into account DataNode
+   * locality. The first block replica should be placed randomly within the
+   * cluster. Subsequent block replicas should follow DataNode locality rules.
+   */
+  IGNORE_CLIENT_LOCALITY((short) 0x100);
 
   private final short mode;
 

+ 10 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/AddBlockFlag.java

@@ -36,7 +36,16 @@ public enum AddBlockFlag {
    *
    * @see CreateFlag#NO_LOCAL_WRITE
    */
-  NO_LOCAL_WRITE((short) 0x01);
+  NO_LOCAL_WRITE((short) 0x01),
+
+  /**
+   * Advise that the first block replica NOT take into account DataNode
+   * locality. The first block replica should be placed randomly within the
+   * cluster. Subsequent block replicas should follow DataNode locality rules.
+   *
+   * @see CreateFlag#IGNORE_CLIENT_LOCALITY
+   */
+  IGNORE_CLIENT_LOCALITY((short) 0x02);
 
   private final short mode;
 

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -201,6 +201,9 @@ public class DFSOutputStream extends FSOutputSummer
     if (flag.contains(CreateFlag.NO_LOCAL_WRITE)) {
       this.addBlockFlags.add(AddBlockFlag.NO_LOCAL_WRITE);
     }
+    if (flag.contains(CreateFlag.IGNORE_CLIENT_LOCALITY)) {
+      this.addBlockFlags.add(AddBlockFlag.IGNORE_CLIENT_LOCALITY);
+    }
     if (progress != null) {
       DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
           +"{}", src);

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

@@ -3205,6 +3205,17 @@ public class DistributedFileSystem extends FileSystem
       return this;
     }
 
+    /**
+     * Advise that the first block replica be written without regard to the
+     * client locality.
+     *
+     * @see CreateFlag for the details.
+     */
+    public HdfsDataOutputStreamBuilder ignoreClientLocality() {
+      getFlags().add(CreateFlag.IGNORE_CLIENT_LOCALITY);
+      return this;
+    }
+
     @VisibleForTesting
     @Override
     protected EnumSet<CreateFlag> getFlags() {

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

@@ -167,6 +167,7 @@ message AbandonBlockResponseProto { // void response
 
 enum AddBlockFlagProto {
   NO_LOCAL_WRITE = 1; // avoid writing to local node.
+  IGNORE_CLIENT_LOCALITY = 2; // write to a random node
 }
 
 message AddBlockRequestProto {

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -280,7 +280,9 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
     if (avoidLocalNode) {
       results = new ArrayList<>(chosenStorage);
       Set<Node> excludedNodeCopy = new HashSet<>(excludedNodes);
-      excludedNodeCopy.add(writer);
+      if (writer != null) {
+        excludedNodeCopy.add(writer);
+      }
       localNode = chooseTarget(numOfReplicas, writer,
           excludedNodeCopy, blocksize, maxNodesPerRack, results,
           avoidStaleNodes, storagePolicy,

+ 19 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java

@@ -269,19 +269,27 @@ class FSDirWriteFileOp {
       BlockManager bm, String src, DatanodeInfo[] excludedNodes,
       String[] favoredNodes, EnumSet<AddBlockFlag> flags,
       ValidateAddBlockResult r) throws IOException {
-    Node clientNode = bm.getDatanodeManager()
-        .getDatanodeByHost(r.clientMachine);
-    if (clientNode == null) {
-      clientNode = getClientNode(bm, r.clientMachine);
-    }
+    Node clientNode = null;
 
-    Set<Node> excludedNodesSet = null;
-    if (excludedNodes != null) {
-      excludedNodesSet = new HashSet<>(excludedNodes.length);
-      Collections.addAll(excludedNodesSet, excludedNodes);
+    boolean ignoreClientLocality = (flags != null
+            && flags.contains(AddBlockFlag.IGNORE_CLIENT_LOCALITY));
+
+    // If client locality is ignored, clientNode remains 'null' to indicate
+    if (!ignoreClientLocality) {
+      clientNode = bm.getDatanodeManager().getDatanodeByHost(r.clientMachine);
+      if (clientNode == null) {
+        clientNode = getClientNode(bm, r.clientMachine);
+      }
     }
-    List<String> favoredNodesList = (favoredNodes == null) ? null
-        : Arrays.asList(favoredNodes);
+
+    Set<Node> excludedNodesSet =
+        (excludedNodes == null) ? new HashSet<>()
+            : new HashSet<>(Arrays.asList(excludedNodes));
+
+    List<String> favoredNodesList =
+        (favoredNodes == null) ? Collections.emptyList()
+            : Arrays.asList(favoredNodes);
+
     // choose targets for the new block to be allocated.
     return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode,
                                     excludedNodesSet, r.blockSize,

+ 79 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSDirWriteFileOp.java

@@ -0,0 +1,79 @@
+/**
+ * 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.server.namenode;
+
+import static org.junit.Assert.assertNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyByte;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyList;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.anySet;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.EnumSet;
+
+import org.apache.hadoop.hdfs.AddBlockFlag;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.FSDirWriteFileOp.ValidateAddBlockResult;
+import org.apache.hadoop.net.Node;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+public class TestFSDirWriteFileOp {
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testIgnoreClientLocality() throws IOException {
+    ValidateAddBlockResult addBlockResult =
+        new ValidateAddBlockResult(1024L, 3, (byte) 0x01, null, null, null);
+
+    EnumSet<AddBlockFlag> addBlockFlags =
+        EnumSet.of(AddBlockFlag.IGNORE_CLIENT_LOCALITY);
+
+    BlockManager bmMock = mock(BlockManager.class);
+
+    ArgumentCaptor<Node> nodeCaptor = ArgumentCaptor.forClass(Node.class);
+
+    when(bmMock.chooseTarget4NewBlock(anyString(), anyInt(), any(), anySet(),
+        anyLong(), anyList(), anyByte(), any(), any(), any())).thenReturn(null);
+
+    FSDirWriteFileOp.chooseTargetForNewBlock(bmMock, "localhost", null, null,
+        addBlockFlags, addBlockResult);
+
+    // There should be no other interactions with the block manager when the
+    // IGNORE_CLIENT_LOCALITY is passed in because there is no need to discover
+    // the local node requesting the new block
+    verify(bmMock, times(1)).chooseTarget4NewBlock(anyString(), anyInt(),
+        nodeCaptor.capture(), anySet(), anyLong(), anyList(), anyByte(), any(),
+        any(), any());
+
+    verifyNoMoreInteractions(bmMock);
+
+    assertNull(
+        "Source node was assigned a value. Expected 'null' value because "
+            + "chooseTarget was flagged to ignore source node locality",
+        nodeCaptor.getValue());
+  }
+}