You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ay...@apache.org on 2020/02/08 05:39:53 UTC

[hadoop] branch branch-3.2 updated: HDFS-15115. Namenode crash caused by NPE in BlockPlacementPolicyDefault when dynamically change logger to debug. Contributed by wangzhixiang

This is an automated email from the ASF dual-hosted git repository.

ayushsaxena pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
     new 7be104a  HDFS-15115. Namenode crash caused by NPE in BlockPlacementPolicyDefault when dynamically change logger to debug. Contributed by wangzhixiang
7be104a is described below

commit 7be104a9be14b10efdb5cdb6f6bc8169279f955f
Author: Ayush Saxena <ay...@apache.org>
AuthorDate: Sat Feb 8 10:33:57 2020 +0530

    HDFS-15115. Namenode crash caused by NPE in BlockPlacementPolicyDefault when dynamically change logger to debug. Contributed by wangzhixiang
---
 .../BlockPlacementPolicyDefault.java               |  3 +-
 ...estBlockPlacementPolicyDebugLoggingBuilder.java | 89 ++++++++++++++++++++++
 2 files changed, 90 insertions(+), 2 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
index 239ce9d..66c60be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
@@ -782,9 +782,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
                             boolean avoidStaleNodes,
                             EnumMap<StorageType, Integer> storageTypes)
                             throws NotEnoughReplicasException {
-    StringBuilder builder = null;
+    StringBuilder builder = debugLoggingBuilder.get();
     if (LOG.isDebugEnabled()) {
-      builder = debugLoggingBuilder.get();
       builder.setLength(0);
       builder.append("[");
     }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementPolicyDebugLoggingBuilder.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementPolicyDebugLoggingBuilder.java
new file mode 100644
index 0000000..24b3660
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockPlacementPolicyDebugLoggingBuilder.java
@@ -0,0 +1,89 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.blockmanagement;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.net.Node;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.util.ArrayList;
+import java.util.EnumMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
+
+public class TestBlockPlacementPolicyDebugLoggingBuilder extends
+    BaseReplicationPolicyTest {
+
+  public TestBlockPlacementPolicyDebugLoggingBuilder() {
+    this.blockPlacementPolicy = BlockPlacementPolicyDefault.class.getName();
+  }
+
+  @Override
+  DatanodeDescriptor[] getDatanodeDescriptors(Configuration conf) {
+    final String[] racks = {
+        "/d1/r1/n1",
+        "/d1/r1/n2",
+        "/d1/r2/n3",
+    };
+    storages = DFSTestUtil.createDatanodeStorageInfos(racks);
+    return DFSTestUtil.toDatanodeDescriptor(storages);
+  }
+
+  @Test
+  public void testChooseRandomDynamicallyChangeLogger() throws Exception {
+    BlockPlacementPolicyDefault repl =
+        spy((BlockPlacementPolicyDefault) replicator);
+
+    GenericTestUtils.setLogLevel(BlockPlacementPolicy.LOG,
+        org.slf4j.event.Level.INFO);
+    List<DatanodeStorageInfo> results = new ArrayList<DatanodeStorageInfo>();
+    results.add(storages[0]);
+    results.add(storages[1]);
+    results.add(storages[2]);
+    Set<Node> excludeNodes = new HashSet<>();
+    doAnswer(new Answer() {
+      @Override
+      public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+        GenericTestUtils.setLogLevel(BlockPlacementPolicy.LOG,
+            org.slf4j.event.Level.DEBUG);
+        return dataNodes[0];
+      }
+    }).when(repl).chooseDataNode("/", excludeNodes);
+
+    doAnswer(new Answer() {
+      @Override
+      public Object answer(InvocationOnMock invocationOnMock) throws Throwable {
+        GenericTestUtils.setLogLevel(BlockPlacementPolicy.LOG,
+            org.slf4j.event.Level.DEBUG);
+        return dataNodes[0];
+      }
+    }).when(repl).chooseDataNode("/", excludeNodes, StorageType.DISK);
+    EnumMap<StorageType, Integer> types = new EnumMap<>(StorageType.class);
+    types.put(StorageType.DISK, 1);
+    repl.chooseRandom(1, "/", excludeNodes, 1024L, 3, results, false, types);
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org