[hadoop] 07/10: HDFS-15551. Tiny Improve for DeadNode detector (#2265)

2021-03-23 Thread weichiu
This is an automated email from the ASF dual-hosted git repository.

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

commit 4f79df4da12d02b66cbd9aae77bc39299025dd6a
Author: imbajin 
AuthorDate: Fri Sep 11 15:44:03 2020 +0800

HDFS-15551. Tiny Improve for DeadNode detector (#2265)

Contributed by imbajin.

Reviewed-by: leosunli 
Signed-off-by: He Xiaoqiao 
(cherry picked from commit 89428f142fe7cee17bd1a0f5f207b6952ec79d32)
---
 .../src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java  |  3 +++
 .../main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java| 11 ++-
 2 files changed, 9 insertions(+), 5 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 0a7fc8f..a918101 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -181,10 +181,13 @@ public class DFSInputStream extends FSInputStream
   private byte[] oneByteBuf; // used for 'int read()'
 
   protected void addToLocalDeadNodes(DatanodeInfo dnInfo) {
+DFSClient.LOG.debug("Add {} to local dead nodes, previously was {}.",
+dnInfo, deadNodes);
 deadNodes.put(dnInfo, dnInfo);
   }
 
   protected void removeFromLocalDeadNodes(DatanodeInfo dnInfo) {
+DFSClient.LOG.debug("Remove {} from local dead nodes.", dnInfo);
 deadNodes.remove(dnInfo);
   }
 
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java
index 8066b8f..112bc04 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java
@@ -294,7 +294,7 @@ public class DeadNodeDetector extends Daemon {
   }
 
   /**
-   * Prode datanode by probe byte.
+   * Prode datanode by probe type.
*/
   private void scheduleProbe(ProbeType type) {
 LOG.debug("Schedule probe datanode for probe type: {}.", type);
@@ -376,9 +376,8 @@ public class DeadNodeDetector extends Daemon {
   } catch (Exception e) {
 LOG.error("Probe failed, datanode: {}, type: {}.", datanodeInfo, type,
 e);
+deadNodeDetector.probeCallBack(this, false);
   }
-
-  deadNodeDetector.probeCallBack(this, false);
 }
   }
 
@@ -402,7 +401,7 @@ public class DeadNodeDetector extends Daemon {
   }
 } else {
   if (probe.getType() == ProbeType.CHECK_SUSPECT) {
-LOG.info("Add the node to dead node list: {}.",
+LOG.warn("Probe failed, add suspect node to dead node list: {}.",
 probe.getDatanodeInfo());
 addToDead(probe.getDatanodeInfo());
   }
@@ -415,11 +414,12 @@ public class DeadNodeDetector extends Daemon {
   private void checkDeadNodes() {
 Set datanodeInfos = clearAndGetDetectedDeadNodes();
 for (DatanodeInfo datanodeInfo : datanodeInfos) {
-  LOG.debug("Add dead node to check: {}.", datanodeInfo);
   if (!deadNodesProbeQueue.offer(datanodeInfo)) {
 LOG.debug("Skip to add dead node {} to check " +
 "since the probe queue is full.", datanodeInfo);
 break;
+  } else {
+LOG.debug("Add dead node to check: {}.", datanodeInfo);
   }
 }
 state = State.IDLE;
@@ -475,6 +475,7 @@ public class DeadNodeDetector extends Daemon {
   datanodeInfos.add(datanodeInfo);
 }
 
+LOG.debug("Add datanode {} to suspectAndDeadNodes.", datanodeInfo);
 addSuspectNodeToDetect(datanodeInfo);
   }
 

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



[hadoop] 06/10: HDFS-15661. The DeadNodeDetector should not be shared by different DFSClients. Contributed by Jinglun.

2021-03-23 Thread weichiu
This is an automated email from the ASF dual-hosted git repository.

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

commit 3ecd02ac78091617f8fae90f92a2155c65d19a54
Author: sunlisheng 
AuthorDate: Thu Jan 28 10:10:39 2021 +0800

HDFS-15661. The DeadNodeDetector should not be shared by different 
DFSClients. Contributed by Jinglun.

(cherry picked from commit f8769e0f4b917d9fda8ff7a9fddb4d755d246a1e)
---
 .../java/org/apache/hadoop/hdfs/ClientContext.java | 45 +++---
 .../java/org/apache/hadoop/hdfs/DFSClient.java |  9 -
 .../org/apache/hadoop/hdfs/DeadNodeDetector.java   |  2 +-
 .../apache/hadoop/hdfs/TestDeadNodeDetection.java  | 37 ++
 4 files changed, 77 insertions(+), 16 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
index b34420d..47e985b 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
@@ -40,10 +40,10 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.ScriptBasedMapping;
-import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.ReflectionUtils;
 
 import 
org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -119,8 +119,6 @@ public class ClientContext {
   private NodeBase clientNode;
   private boolean topologyResolutionEnabled;
 
-  private Daemon deadNodeDetectorThr = null;
-
   /**
* The switch to DeadNodeDetector.
*/
@@ -130,12 +128,18 @@ public class ClientContext {
* Detect the dead datanodes in advance, and share this information among all
* the DFSInputStreams in the same client.
*/
-  private DeadNodeDetector deadNodeDetector = null;
+  private volatile DeadNodeDetector deadNodeDetector = null;
+
+  /**
+   * Count the reference of ClientContext.
+   */
+  private int counter = 0;
 
   /**
* ShortCircuitCache array size.
*/
   private final int clientShortCircuitNum;
+  private Configuration configuration;
 
   private ClientContext(String name, DfsClientConf conf,
   Configuration config) {
@@ -149,6 +153,7 @@ public class ClientContext {
   this.shortCircuitCache[i] = ShortCircuitCache.fromConf(scConf);
 }
 
+this.configuration = config;
 this.peerCache = new PeerCache(scConf.getSocketCacheCapacity(),
 scConf.getSocketCacheExpiry());
 this.keyProviderCache = new KeyProviderCache(
@@ -159,11 +164,6 @@ public class ClientContext {
 this.byteArrayManager = ByteArrayManager.newInstance(
 conf.getWriteByteArrayManagerConf());
 this.deadNodeDetectionEnabled = conf.isDeadNodeDetectionEnabled();
-if (deadNodeDetectionEnabled && deadNodeDetector == null) {
-  deadNodeDetector = new DeadNodeDetector(name, config);
-  deadNodeDetectorThr = new Daemon(deadNodeDetector);
-  deadNodeDetectorThr.start();
-}
 initTopologyResolution(config);
   }
 
@@ -201,6 +201,7 @@ public class ClientContext {
 context.printConfWarningIfNeeded(conf);
   }
 }
+context.reference();
 return context;
   }
 
@@ -301,17 +302,33 @@ public class ClientContext {
   }
 
   /**
-   * Close dead node detector thread.
+   * Increment the counter. Start the dead node detector thread if there is no
+   * reference.
+   */
+  synchronized void reference() {
+counter++;
+if (deadNodeDetectionEnabled && deadNodeDetector == null) {
+  deadNodeDetector = new DeadNodeDetector(name, configuration);
+  deadNodeDetector.start();
+}
+  }
+
+  /**
+   * Decrement the counter. Close the dead node detector thread if there is no
+   * reference.
*/
-  public void stopDeadNodeDetectorThread() {
-if (deadNodeDetectorThr != null) {
-  deadNodeDetectorThr.interrupt();
+  synchronized void unreference() {
+Preconditions.checkState(counter > 0);
+counter--;
+if (counter == 0 && deadNodeDetectionEnabled && deadNodeDetector != null) {
+  deadNodeDetector.interrupt();
   try {
-deadNodeDetectorThr.join();
+deadNodeDetector.join();
   } catch (InterruptedException e) {
 LOG.warn("Encountered exception while waiting to join on dead " +
 "node detector thread.", e);
   }
+  deadNodeDetector = null;
 }
   }
 }
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index e9ca5

[hadoop] 10/10: HDFS-15743. Fix -Pdist build failure of hadoop-hdfs-native-client. (#2569)

2021-03-23 Thread weichiu
This is an automated email from the ASF dual-hosted git repository.

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

commit 2d83369f6f612a1ba91909e4e06102ff4dc3788e
Author: Masatake Iwasaki 
AuthorDate: Tue Dec 22 07:19:44 2020 +0900

HDFS-15743. Fix -Pdist build failure of hadoop-hdfs-native-client. (#2569)

(cherry picked from commit 9aefe72ed68fdf3ae6015a6fed15d8bb20b0c6fb)
---
 dev-support/bin/dist-copynativelibs | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/dev-support/bin/dist-copynativelibs 
b/dev-support/bin/dist-copynativelibs
index 7f2b6ad..95de186 100755
--- a/dev-support/bin/dist-copynativelibs
+++ b/dev-support/bin/dist-copynativelibs
@@ -164,7 +164,7 @@ fi
 
 # Windows doesn't have a LIB_DIR, everything goes into bin
 
-if [[ -d "${BIN_DIR}" ]] ; then
+if [[ -d "${BIN_DIR}" && $(ls -A "${BIN_DIR}") ]] ; then
   mkdir -p "${TARGET_BIN_DIR}"
   cd "${BIN_DIR}"  || exit 1
   ${TAR} ./* | (cd "${TARGET_BIN_DIR}"/ || exit 1; ${UNTAR})

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



[hadoop] 09/10: HDFS-15809. DeadNodeDetector does not remove live nodes from dead node set. Contributed by Jinglun.

2021-03-23 Thread weichiu
This is an automated email from the ASF dual-hosted git repository.

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

commit a8c0083b1bea830454bfee37805c67828cd05f4f
Author: sunlisheng 
AuthorDate: Mon Mar 15 11:34:13 2021 +0800

HDFS-15809. DeadNodeDetector does not remove live nodes from dead node set. 
Contributed by Jinglun.

(cherry picked from commit 7025f39944e628345109b43cba2cd4d49ca8cc6b)
---
 .../org/apache/hadoop/hdfs/DeadNodeDetector.java   | 85 +-
 .../hadoop/hdfs/client/HdfsClientConfigKeys.java   | 10 +--
 .../src/main/resources/hdfs-default.xml| 20 ++---
 .../apache/hadoop/hdfs/TestDeadNodeDetection.java  | 71 --
 4 files changed, 107 insertions(+), 79 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java
index e17f261..cd46551 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java
@@ -29,9 +29,9 @@ import org.slf4j.LoggerFactory;
 
 import java.util.HashSet;
 import java.util.Map;
-import java.util.Queue;
 import java.util.Set;
-import java.util.concurrent.ArrayBlockingQueue;
+import java.util.Deque;
+import java.util.LinkedList;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
@@ -40,8 +40,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_DEAD_NODE_QUEUE_MAX_DEFAULT;
-import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_DEAD_NODE_QUEUE_MAX_KEY;
 import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_CONNECTION_TIMEOUT_MS_DEFAULT;
 import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_CONNECTION_TIMEOUT_MS_KEY;
 import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_DEAD_NODE_INTERVAL_MS_DEFAULT;
@@ -54,9 +52,9 @@ import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD
 import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_PROBE_SUSPECT_NODE_THREADS_KEY;
 import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_RPC_THREADS_DEFAULT;
 import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_RPC_THREADS_KEY;
-import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_SUSPECT_NODE_QUEUE_MAX_DEFAULT;
-import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_SUSPECT_NODE_QUEUE_MAX_KEY;
 import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
+import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_IDLE_SLEEP_MS_KEY;
+import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DEAD_NODE_DETECTION_IDLE_SLEEP_MS_DEFAULT;
 
 /**
  * Detect the dead nodes in advance, and share this information among all the
@@ -74,7 +72,7 @@ public class DeadNodeDetector extends Daemon {
   /**
* Waiting time when DeadNodeDetector's state is idle.
*/
-  private static final long IDLE_SLEEP_MS = 1;
+  private final long idleSleepMs;
 
   /**
* Client context name.
@@ -114,16 +112,6 @@ public class DeadNodeDetector extends Daemon {
   private long suspectNodeDetectInterval = 0;
 
   /**
-   * The max queue size of probing dead node.
-   */
-  private int maxDeadNodesProbeQueueLen = 0;
-
-  /**
-   * The max queue size of probing suspect node.
-   */
-  private int maxSuspectNodesProbeQueueLen;
-
-  /**
* Connection timeout for probing dead node in milliseconds.
*/
   private long probeConnectionTimeoutMs;
@@ -131,12 +119,12 @@ public class DeadNodeDetector extends Daemon {
   /**
* The dead node probe queue.
*/
-  private Queue deadNodesProbeQueue;
+  private UniqueQueue deadNodesProbeQueue;
 
   /**
* The suspect node probe queue.
*/
-  private Queue suspectNodesProbeQueue;
+  private UniqueQueue suspectNodesProbeQueue;
 
   /**
* The thread pool of probing dead node.
@@ -182,6 +170,32 @@ public class DeadNodeDetector extends Daemon {
   }
 
   /**
+   * The thread safe unique queue.
+   */
+  static class UniqueQueue {
+private Deque queue = new LinkedList<>();
+private Set set = new HashSet<>();
+
+synchronized boolean offer(T dn) {
+  if (set.add(dn)

[hadoop] 08/10: HDFS-15806. DeadNodeDetector should close all the threads when it is closed. Contributed by Jinglun.

2021-03-23 Thread weichiu
This is an automated email from the ASF dual-hosted git repository.

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

commit 94766fdb13759e53828cd296a9b1c8094bca24ed
Author: Ayush Saxena 
AuthorDate: Sat Feb 20 19:48:00 2021 +0530

HDFS-15806. DeadNodeDetector should close all the threads when it is 
closed. Contributed by Jinglun.

(cherry picked from commit ff84a57483320abf81bf941b81d13d5bee14084b)
---
 .../java/org/apache/hadoop/hdfs/ClientContext.java |  8 +-
 .../org/apache/hadoop/hdfs/DeadNodeDetector.java   | 31 ++
 .../apache/hadoop/hdfs/TestDeadNodeDetection.java  | 12 +
 3 files changed, 44 insertions(+), 7 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
index 47e985b..5bb7e03 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
@@ -321,13 +321,7 @@ public class ClientContext {
 Preconditions.checkState(counter > 0);
 counter--;
 if (counter == 0 && deadNodeDetectionEnabled && deadNodeDetector != null) {
-  deadNodeDetector.interrupt();
-  try {
-deadNodeDetector.join();
-  } catch (InterruptedException e) {
-LOG.warn("Encountered exception while waiting to join on dead " +
-"node detector thread.", e);
-  }
+  deadNodeDetector.shutdown();
   deadNodeDetector = null;
 }
   }
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java
index 112bc04..e17f261 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DeadNodeDetector.java
@@ -271,6 +271,37 @@ public class DeadNodeDetector extends Daemon {
 }
   }
 
+  /**
+   * Shutdown all the threads.
+   */
+  public void shutdown() {
+threadShutDown(this);
+threadShutDown(probeDeadNodesSchedulerThr);
+threadShutDown(probeSuspectNodesSchedulerThr);
+probeDeadNodesThreadPool.shutdown();
+probeSuspectNodesThreadPool.shutdown();
+rpcThreadPool.shutdown();
+  }
+
+  private static void threadShutDown(Thread thread) {
+if (thread != null && thread.isAlive()) {
+  thread.interrupt();
+  try {
+thread.join();
+  } catch (InterruptedException e) {
+  }
+}
+  }
+
+  @VisibleForTesting
+  boolean isThreadsShutdown() {
+return !this.isAlive() && !probeDeadNodesSchedulerThr.isAlive()
+&& !probeSuspectNodesSchedulerThr.isAlive()
+&& probeDeadNodesThreadPool.isShutdown()
+&& probeSuspectNodesThreadPool.isShutdown()
+&& rpcThreadPool.isShutdown();
+  }
+
   @VisibleForTesting
   static void setDisabledProbeThreadForTest(
   boolean disabledProbeThreadForTest) {
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDeadNodeDetection.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDeadNodeDetection.java
index 9c52fcd..9134f36 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDeadNodeDetection.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDeadNodeDetection.java
@@ -357,6 +357,18 @@ public class TestDeadNodeDetection {
 dfs1.close();
   }
 
+  @Test
+  public void testDeadNodeDetectorThreadsShutdown() throws Exception {
+DistributedFileSystem dfs = (DistributedFileSystem) FileSystem
+.newInstance(new URI("hdfs://127.0.0.1:2001/"), conf);
+DeadNodeDetector detector = dfs.getClient().getDeadNodeDetector();
+assertNotNull(detector);
+dfs.close();
+assertTrue(detector.isThreadsShutdown());
+detector = dfs.getClient().getDeadNodeDetector();
+assertNull(detector);
+  }
+
   private void createFile(FileSystem fs, Path filePath) throws IOException {
 FSDataOutputStream out = null;
 try {

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



[hadoop] branch branch-3.3 updated (6d3f5e8 -> 2d83369)

2021-03-23 Thread weichiu
This is an automated email from the ASF dual-hosted git repository.

weichiu pushed a change to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git.


from 6d3f5e8  HDFS-15911 : Provide blocks moved count in Balancer iteration 
result (#2796)
 new e9b06d9  HDFS-15249 ThrottledAsyncChecker is not thread-safe. (#1922)
 new a216ee3  HDFS-15910. Improve security with explicit_bzero (#2793)
 new 4abe6e6  HDFS-15908. Possible Resource Leak in 
org.apache.hadoop.hdfs.qjournal.server.Journal (#2790). Contributed by Narges 
Shadab.
 new 3a35528  HDFS-15868. Possible Resource Leak in EditLogFileOutputStream 
(#2736). Contributed by Narges Shadab.
 new f07bde9  HDFS-15791. Possible Resource Leak in FSImageFormatProtobuf. 
(#2652)
 new 3ecd02a  HDFS-15661. The DeadNodeDetector should not be shared by 
different DFSClients. Contributed by Jinglun.
 new 4f79df4  HDFS-15551. Tiny Improve for DeadNode detector (#2265)
 new 94766fd  HDFS-15806. DeadNodeDetector should close all the threads 
when it is closed. Contributed by Jinglun.
 new a8c0083  HDFS-15809. DeadNodeDetector does not remove live nodes from 
dead node set. Contributed by Jinglun.
 new 2d83369  HDFS-15743. Fix -Pdist build failure of 
hadoop-hdfs-native-client. (#2569)

The 10 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 dev-support/bin/dist-copynativelibs|   2 +-
 .../java/org/apache/hadoop/hdfs/ClientContext.java |  49 +---
 .../java/org/apache/hadoop/hdfs/DFSClient.java |   9 +-
 .../org/apache/hadoop/hdfs/DFSInputStream.java |   3 +
 .../org/apache/hadoop/hdfs/DeadNodeDetector.java   | 129 ++---
 .../hadoop/hdfs/client/HdfsClientConfigKeys.java   |  10 +-
 .../main/native/libhdfspp/lib/bindings/c/hdfs.cc   |   2 +-
 .../main/native/libhdfspp/tests/hdfs_ext_test.cc   |   2 +-
 .../main/native/libhdfspp/tests/hdfspp_mini_dfs.h  |   2 +-
 .../hadoop/hdfs/qjournal/server/Journal.java   |   2 +-
 .../datanode/checker/ThrottledAsyncChecker.java|   4 +-
 .../server/namenode/EditLogFileOutputStream.java   |   7 +-
 .../server/namenode/FSImageFormatProtobuf.java |  20 ++--
 .../src/main/resources/hdfs-default.xml|  20 +---
 .../apache/hadoop/hdfs/TestDeadNodeDetection.java  | 120 +++
 15 files changed, 261 insertions(+), 120 deletions(-)

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



[hadoop] 02/10: HDFS-15910. Improve security with explicit_bzero (#2793)

2021-03-23 Thread weichiu
This is an automated email from the ASF dual-hosted git repository.

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

commit a216ee3b852ff0447aa8810f4e245a074928bd56
Author: Gautham B A 
AuthorDate: Tue Mar 23 22:28:19 2021 +0530

HDFS-15910. Improve security with explicit_bzero (#2793)

(cherry picked from commit 714427c41401eacf3cab7c087d9e5d61a658716b)
---
 .../src/main/native/libhdfspp/lib/bindings/c/hdfs.cc| 2 +-
 .../src/main/native/libhdfspp/tests/hdfs_ext_test.cc| 2 +-
 .../src/main/native/libhdfspp/tests/hdfspp_mini_dfs.h   | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
 
b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
index 6b2468f..424bb6b 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/bindings/c/hdfs.cc
@@ -1402,7 +1402,7 @@ int hdfsGetBlockLocations(hdfsFS fs, const char *path, 
struct hdfsBlockLocations
 hdfsBlockLocations *locations = new struct hdfsBlockLocations();
 (*locations_out) = locations;
 
-bzero(locations, sizeof(*locations));
+explicit_bzero(locations, sizeof(*locations));
 locations->fileLength = ppLocations->getFileLength();
 locations->isLastBlockComplete = ppLocations->isLastBlockComplete();
 locations->isUnderConstruction = ppLocations->isUnderConstruction();
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ext_test.cc
 
b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ext_test.cc
index 79771f0..e69ddb2 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ext_test.cc
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfs_ext_test.cc
@@ -475,7 +475,7 @@ TEST_F(HdfsExtTest, TestReadStats) {
   hdfsFile file = hdfsOpenFile(fs, path.c_str(), O_WRONLY, 0, 0, 0);
   EXPECT_NE(nullptr, file);
   void * buf = malloc(size);
-  bzero(buf, size);
+  explicit_bzero(buf, size);
   EXPECT_EQ(size, hdfsWrite(fs, file, buf, size));
   free(buf);
   EXPECT_EQ(0, hdfsCloseFile(fs, file));
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfspp_mini_dfs.h
 
b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfspp_mini_dfs.h
index aecced1..98edbdc 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfspp_mini_dfs.h
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/hdfspp_mini_dfs.h
@@ -92,7 +92,7 @@ public:
 hdfsFile file = hdfsOpenFile(*this, path.c_str(), O_WRONLY, 0, 0, 0);
 EXPECT_NE(nullptr, file);
 void * buf = malloc(size);
-bzero(buf, size);
+explicit_bzero(buf, size);
 EXPECT_EQ(1024, hdfsWrite(*this, file, buf, size));
 EXPECT_EQ(0, hdfsCloseFile(*this, file));
 free(buf);

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



[hadoop] 05/10: HDFS-15791. Possible Resource Leak in FSImageFormatProtobuf. (#2652)

2021-03-23 Thread weichiu
This is an automated email from the ASF dual-hosted git repository.

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

commit f07bde90c9e9df591464fd0b2b3e3354f439e14d
Author: Narges Shadab <54193416+narges...@users.noreply.github.com>
AuthorDate: Mon Feb 1 01:56:59 2021 -0800

HDFS-15791. Possible Resource Leak in FSImageFormatProtobuf. (#2652)

(cherry picked from commit 115623a6ee4eb7391c7c26e1728fa153c0698803)
---
 .../hdfs/server/namenode/FSImageFormatProtobuf.java  | 20 +---
 1 file changed, 13 insertions(+), 7 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
index 92a2083..183449f 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
@@ -271,14 +271,20 @@ public final class FSImageFormatProtobuf {
 String compressionCodec)
 throws IOException {
   FileInputStream fin = new FileInputStream(filename);
-  FileChannel channel = fin.getChannel();
-  channel.position(section.getOffset());
-  InputStream in = new BufferedInputStream(new LimitInputStream(fin,
-  section.getLength()));
+  try {
 
-  in = FSImageUtil.wrapInputStreamForCompression(conf,
-  compressionCodec, in);
-  return in;
+  FileChannel channel = fin.getChannel();
+  channel.position(section.getOffset());
+  InputStream in = new BufferedInputStream(new LimitInputStream(fin,
+  section.getLength()));
+
+  in = FSImageUtil.wrapInputStreamForCompression(conf,
+  compressionCodec, in);
+  return in;
+  } catch (IOException e) {
+  fin.close();
+  throw e;
+  }
 }
 
 /**

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



[hadoop] 03/10: HDFS-15908. Possible Resource Leak in org.apache.hadoop.hdfs.qjournal.server.Journal (#2790). Contributed by Narges Shadab.

2021-03-23 Thread weichiu
This is an automated email from the ASF dual-hosted git repository.

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

commit 4abe6e6ac1d79351320160b2f15e929fadc4bd5f
Author: Narges Shadab <54193416+narges...@users.noreply.github.com>
AuthorDate: Tue Mar 23 04:36:58 2021 -0700

HDFS-15908. Possible Resource Leak in 
org.apache.hadoop.hdfs.qjournal.server.Journal (#2790). Contributed by Narges 
Shadab.

Signed-off-by: Ayush Saxena 
(cherry picked from commit d77c7ab4e0604484ec3c48ac2acb32ea71a53d9a)
---
 .../src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
index 99a087e..238a000 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
@@ -264,9 +264,9 @@ public class Journal implements Closeable {
*/
   @Override // Closeable
   public void close() throws IOException {
-storage.close();
 IOUtils.closeStream(committedTxnId);
 IOUtils.closeStream(curSegment);
+storage.close();
   }
   
   JNStorage getStorage() {

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



[hadoop] 04/10: HDFS-15868. Possible Resource Leak in EditLogFileOutputStream (#2736). Contributed by Narges Shadab.

2021-03-23 Thread weichiu
This is an automated email from the ASF dual-hosted git repository.

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

commit 3a355282c932bdbbdcbd9a773779413da188d638
Author: Narges Shadab <54193416+narges...@users.noreply.github.com>
AuthorDate: Fri Mar 19 01:36:48 2021 -0700

HDFS-15868. Possible Resource Leak in EditLogFileOutputStream (#2736). 
Contributed by Narges Shadab.

Reviewed-by: He Xiaoqiao 
(cherry picked from commit f5f3fc6fe90e3bb2ee783f956432db11e655b9e3)
---
 .../hadoop/hdfs/server/namenode/EditLogFileOutputStream.java   | 7 ++-
 1 file changed, 6 insertions(+), 1 deletion(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
index 4919ea4..7fe8430 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
@@ -88,7 +88,12 @@ public class EditLogFileOutputStream extends 
EditLogOutputStream {
 } else {
   rp = new RandomAccessFile(name, "rw");
 }
-fp = new FileOutputStream(rp.getFD()); // open for append
+try {
+  fp = new FileOutputStream(rp.getFD()); // open for append
+} catch (IOException e) {
+  IOUtils.closeStream(rp);
+  throw e;
+}
 fc = rp.getChannel();
 fc.position(fc.size());
   }

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



[hadoop] 01/10: HDFS-15249 ThrottledAsyncChecker is not thread-safe. (#1922)

2021-03-23 Thread weichiu
This is an automated email from the ASF dual-hosted git repository.

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

commit e9b06d942c01eecdfcde39bfdb9b2524b775aab6
Author: Toshihiro Suzuki 
AuthorDate: Tue Apr 7 14:38:09 2020 +0900

HDFS-15249 ThrottledAsyncChecker is not thread-safe. (#1922)

(cherry picked from commit c12ddbd1de40b32bbe2f6a3e484abf843d6d92ae)
---
 .../hadoop/hdfs/server/datanode/checker/ThrottledAsyncChecker.java| 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/ThrottledAsyncChecker.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/ThrottledAsyncChecker.java
index 4ad32ae..f969c7a 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/ThrottledAsyncChecker.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/ThrottledAsyncChecker.java
@@ -117,8 +117,8 @@ public class ThrottledAsyncChecker implements 
AsyncChecker {
* will receive the same Future.
*/
   @Override
-  public Optional> schedule(Checkable target,
-K context) {
+  public synchronized Optional> schedule(
+  Checkable target, K context) {
 if (checksInProgress.containsKey(target)) {
   return Optional.empty();
 }

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



[hadoop] branch branch-3.3 updated: HDFS-15911 : Provide blocks moved count in Balancer iteration result (#2796)

2021-03-23 Thread liuml07
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 6d3f5e8  HDFS-15911 : Provide blocks moved count in Balancer iteration 
result (#2796)
6d3f5e8 is described below

commit 6d3f5e844b1ad25b5e357e7fa01fd64f637a48b2
Author: Viraj Jasani 
AuthorDate: Wed Mar 24 11:21:21 2021 +0530

HDFS-15911 : Provide blocks moved count in Balancer iteration result (#2796)

Contributed by Viraj Jasani.

Signed-off-by: Mingliang Liu 
Signed-off-by: Ayush Saxena 
---
 .../hadoop/hdfs/server/balancer/Balancer.java  | 40 +-
 .../hadoop/hdfs/server/balancer/TestBalancer.java  | 11 +++---
 2 files changed, 38 insertions(+), 13 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
index 33b5fa4..8d97d2e 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
@@ -596,36 +596,60 @@ public class Balancer {
   }
 
   static class Result {
-final ExitStatus exitStatus;
-final long bytesLeftToMove;
-final long bytesBeingMoved;
-final long bytesAlreadyMoved;
+private final ExitStatus exitStatus;
+private final long bytesLeftToMove;
+private final long bytesBeingMoved;
+private final long bytesAlreadyMoved;
+private final long blocksMoved;
 
 Result(ExitStatus exitStatus, long bytesLeftToMove, long bytesBeingMoved,
-long bytesAlreadyMoved) {
+   long bytesAlreadyMoved, long blocksMoved) {
   this.exitStatus = exitStatus;
   this.bytesLeftToMove = bytesLeftToMove;
   this.bytesBeingMoved = bytesBeingMoved;
   this.bytesAlreadyMoved = bytesAlreadyMoved;
+  this.blocksMoved = blocksMoved;
+}
+
+public ExitStatus getExitStatus() {
+  return exitStatus;
+}
+
+public long getBytesLeftToMove() {
+  return bytesLeftToMove;
+}
+
+public long getBytesBeingMoved() {
+  return bytesBeingMoved;
+}
+
+public long getBytesAlreadyMoved() {
+  return bytesAlreadyMoved;
+}
+
+public long getBlocksMoved() {
+  return blocksMoved;
 }
 
 void print(int iteration, NameNodeConnector nnc, PrintStream out) {
-  out.printf("%-24s %10d  %19s  %18s  %17s  %s%n",
+  out.printf("%-24s %10d  %19s  %18s  %17s  %17s  %s%n",
   DateFormat.getDateTimeInstance().format(new Date()), iteration,
   StringUtils.byteDesc(bytesAlreadyMoved),
   StringUtils.byteDesc(bytesLeftToMove),
   StringUtils.byteDesc(bytesBeingMoved),
+  blocksMoved,
   nnc.getNameNodeUri());
 }
   }
 
   Result newResult(ExitStatus exitStatus, long bytesLeftToMove, long 
bytesBeingMoved) {
 return new Result(exitStatus, bytesLeftToMove, bytesBeingMoved,
-dispatcher.getBytesMoved());
+dispatcher.getBytesMoved(), dispatcher.getBblocksMoved());
   }
 
   Result newResult(ExitStatus exitStatus) {
-return new Result(exitStatus, -1, -1, dispatcher.getBytesMoved());
+return new Result(exitStatus, -1, -1, dispatcher.getBytesMoved(),
+dispatcher.getBblocksMoved());
   }
 
   /** Run an iteration for all datanodes. */
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index bb3ad65..f44bbb2 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -1022,14 +1022,14 @@ public class TestBalancer {
 
   // clean all lists
   b.resetData(conf);
-  if (r.exitStatus == ExitStatus.IN_PROGRESS) {
+  if (r.getExitStatus() == ExitStatus.IN_PROGRESS) {
 done = false;
-  } else if (r.exitStatus != ExitStatus.SUCCESS) {
+  } else if (r.getExitStatus() != ExitStatus.SUCCESS) {
 //must be an error statue, return.
-return r.exitStatus.getExitCode();
+return r.getExitStatus().getExitCode();
   } else {
 if (iteration > 0) {
-  assertTrue(r.bytesAlreadyMoved > 0);
+  assertTrue(r.getBytesAlreadyMoved() > 0);
 }
   }
 }
@@ -1655,7 +1655,8 @@ public class TestBalancer {
   // When a block move is not canceled in 2 seconds properly and then
   // a block is moved unexpectedly, IN_PROGRESS will be re

[hadoop] branch trunk updated: HDFS-15911 : Provide blocks moved count in Balancer iteration result (#2794)

2021-03-23 Thread liuml07
This is an automated email from the ASF dual-hosted git repository.

liuml07 pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 4b4ccce  HDFS-15911 : Provide blocks moved count in Balancer iteration 
result (#2794)
4b4ccce is described below

commit 4b4ccce02f591f63dff7db346de39c8d996e8f1d
Author: Viraj Jasani 
AuthorDate: Wed Mar 24 11:17:45 2021 +0530

HDFS-15911 : Provide blocks moved count in Balancer iteration result (#2794)

Contributed by Viraj Jasani.

Signed-off-by: Mingliang Liu 
Signed-off-by: Ayush Saxena 
---
 .../hadoop/hdfs/server/balancer/Balancer.java  | 28 ++
 .../hadoop/hdfs/server/balancer/TestBalancer.java  |  8 +--
 2 files changed, 30 insertions(+), 6 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
index 0024ba5..33650ea 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
@@ -38,6 +38,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.commons.lang3.builder.ToStringBuilder;
 import 
org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.slf4j.Logger;
@@ -638,13 +639,15 @@ public class Balancer {
 private final long bytesLeftToMove;
 private final long bytesBeingMoved;
 private final long bytesAlreadyMoved;
+private final long blocksMoved;
 
 Result(ExitStatus exitStatus, long bytesLeftToMove, long bytesBeingMoved,
-long bytesAlreadyMoved) {
+   long bytesAlreadyMoved, long blocksMoved) {
   this.exitStatus = exitStatus;
   this.bytesLeftToMove = bytesLeftToMove;
   this.bytesBeingMoved = bytesBeingMoved;
   this.bytesAlreadyMoved = bytesAlreadyMoved;
+  this.blocksMoved = blocksMoved;
 }
 
 public ExitStatus getExitStatus() {
@@ -663,23 +666,40 @@ public class Balancer {
   return bytesAlreadyMoved;
 }
 
+public long getBlocksMoved() {
+  return blocksMoved;
+}
+
 void print(int iteration, NameNodeConnector nnc, PrintStream out) {
-  out.printf("%-24s %10d  %19s  %18s  %17s  %s%n",
+  out.printf("%-24s %10d  %19s  %18s  %17s  %17s  %s%n",
   DateFormat.getDateTimeInstance().format(new Date()), iteration,
   StringUtils.byteDesc(bytesAlreadyMoved),
   StringUtils.byteDesc(bytesLeftToMove),
   StringUtils.byteDesc(bytesBeingMoved),
+  blocksMoved,
   nnc.getNameNodeUri());
 }
+
+@Override
+public String toString() {
+  return new ToStringBuilder(this)
+  .append("exitStatus", exitStatus)
+  .append("bytesLeftToMove", bytesLeftToMove)
+  .append("bytesBeingMoved", bytesBeingMoved)
+  .append("bytesAlreadyMoved", bytesAlreadyMoved)
+  .append("blocksMoved", blocksMoved)
+  .toString();
+}
   }
 
   Result newResult(ExitStatus exitStatus, long bytesLeftToMove, long 
bytesBeingMoved) {
 return new Result(exitStatus, bytesLeftToMove, bytesBeingMoved,
-dispatcher.getBytesMoved());
+dispatcher.getBytesMoved(), dispatcher.getBblocksMoved());
   }
 
   Result newResult(ExitStatus exitStatus) {
-return new Result(exitStatus, -1, -1, dispatcher.getBytesMoved());
+return new Result(exitStatus, -1, -1, dispatcher.getBytesMoved(),
+dispatcher.getBblocksMoved());
   }
 
   /** Run an iteration for all datanodes. */
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index 343faf6..f59743f 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -1658,6 +1658,7 @@ public class TestBalancer {
   // a block is moved unexpectedly, IN_PROGRESS will be reported.
   assertEquals("We expect ExitStatus.NO_MOVE_PROGRESS to be reported.",
   ExitStatus.NO_MOVE_PROGRESS, r.getExitStatus());
+  assertEquals(0, r.getBlocksMoved());
 }
   } finally {
 for (NameNodeConnector nnc : connectors) {
@@ -2309,8 +2310,11 @@ public class TestBalancer {
 // Hence, overall total blocks moved by HDFS balancer would be either of 
these 2 options:
 // a) 2 blocks of total size (100B + 100B)
 // b) 3 blocks of total size (50B + 100B

[hadoop] branch trunk updated: HADOOP-17531. DistCp: Reduce memory usage on copying huge directories. (#2732). Contributed by Ayush Saxena.

2021-03-23 Thread ayushsaxena
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/trunk by this push:
 new 03cfc85  HADOOP-17531. DistCp: Reduce memory usage on copying huge 
directories. (#2732). Contributed by Ayush Saxena.
03cfc85 is described below

commit 03cfc852791c14fad39db4e5b14104a276c08e59
Author: Ayush Saxena 
AuthorDate: Wed Mar 24 02:36:26 2021 +0530

HADOOP-17531. DistCp: Reduce memory usage on copying huge directories. 
(#2732). Contributed by Ayush Saxena.

Signed-off-by: Steve Loughran 
---
 .../util/functional/CommonCallableSupplier.java| 153 +++
 .../org/apache/hadoop/test/GenericTestUtils.java   | 149 ++-
 .../fs/s3a/impl/ITestPartialRenamesDeletes.java|  46 +---
 .../fs/s3a/scale/ITestS3ADeleteManyFiles.java  |   2 +-
 .../org/apache/hadoop/tools/DistCpConstants.java   |   1 +
 .../org/apache/hadoop/tools/DistCpContext.java |   4 +
 .../apache/hadoop/tools/DistCpOptionSwitch.java|   7 +-
 .../org/apache/hadoop/tools/DistCpOptions.java |  19 ++
 .../org/apache/hadoop/tools/OptionsParser.java |   4 +-
 .../org/apache/hadoop/tools/SimpleCopyListing.java | 294 +
 .../hadoop-distcp/src/site/markdown/DistCp.md.vm   |   1 +
 .../org/apache/hadoop/tools/TestDistCpOptions.java |   2 +-
 .../org/apache/hadoop/tools/TestDistCpSystem.java  |   5 -
 .../hadoop/tools/TestDistCpWithRawXAttrs.java  |  23 ++
 .../tools/contract/AbstractContractDistCpTest.java |  39 +++
 .../contract/OptionalTestHDFSContractDistCp.java   |  50 
 .../src/test/resources/contract/hdfs.xml   | 139 ++
 17 files changed, 773 insertions(+), 165 deletions(-)

diff --git 
a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CommonCallableSupplier.java
 
b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CommonCallableSupplier.java
new file mode 100644
index 000..e2cdc0f
--- /dev/null
+++ 
b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CommonCallableSupplier.java
@@ -0,0 +1,153 @@
+/*
+ * 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.util.functional;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.Executor;
+import java.util.function.Supplier;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.util.DurationInfo;
+
+import static org.apache.hadoop.fs.impl.FutureIOSupport.raiseInnerCause;
+
+/**
+ * A bridge from Callable to Supplier; catching exceptions
+ * raised by the callable and wrapping them as appropriate.
+ * @param  return type.
+ */
+public final class CommonCallableSupplier implements Supplier {
+
+  private static final Logger LOG =
+  LoggerFactory.getLogger(CommonCallableSupplier.class);
+
+  private final Callable call;
+
+  /**
+   * Create.
+   * @param call call to invoke.
+   */
+  public CommonCallableSupplier(final Callable call) {
+this.call = call;
+  }
+
+  @Override
+  public Object get() {
+try {
+  return call.call();
+} catch (RuntimeException e) {
+  throw e;
+} catch (IOException e) {
+  throw new UncheckedIOException(e);
+} catch (Exception e) {
+  throw new UncheckedIOException(new IOException(e));
+}
+  }
+
+  /**
+   * Submit a callable into a completable future.
+   * RTEs are rethrown.
+   * Non RTEs are caught and wrapped; IOExceptions to
+   * {@code RuntimeIOException} instances.
+   * @param executor executor.
+   * @param call call to invoke
+   * @param   type
+   * @return the future to wait for
+   */
+  @SuppressWarnings("unchecked")
+  public static  CompletableFuture submit(final Executor executor,
+  final Callable call) {
+return CompletableFuture
+.su

[hadoop] branch branch-3.3 updated: HDFS-15907. Reduce Memory Overhead of AclFeature by avoiding AtomicInteger. Contributed by Stephen O'Donnell.

2021-03-23 Thread ayushsaxena
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 76c40a5  HDFS-15907. Reduce Memory Overhead of AclFeature by avoiding 
AtomicInteger. Contributed by Stephen O'Donnell.
76c40a5 is described below

commit 76c40a52d76671df4c46d131e85478b50697dedf
Author: Ayush Saxena 
AuthorDate: Wed Mar 24 02:10:02 2021 +0530

HDFS-15907. Reduce Memory Overhead of AclFeature by avoiding AtomicInteger. 
Contributed by Stephen O'Donnell.
---
 .../apache/hadoop/hdfs/server/namenode/AclFeature.java| 15 +++
 1 file changed, 7 insertions(+), 8 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
index 6b2baa4..2dfe507 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.util.Arrays;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -35,7 +34,7 @@ import 
org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
 public class AclFeature implements INode.Feature, ReferenceCounter {
   public static final ImmutableList EMPTY_ENTRY_LIST =
 ImmutableList.of();
-  private AtomicInteger value = new AtomicInteger();
+  private int refCount = 0;
 
   private final int [] entries;
 
@@ -84,17 +83,17 @@ public class AclFeature implements INode.Feature, 
ReferenceCounter {
   }
 
   @Override
-  public int getRefCount() {
-return value.get();
+  public synchronized int getRefCount() {
+return refCount;
   }
 
   @Override
-  public int incrementAndGetRefCount() {
-return value.incrementAndGet();
+  public synchronized int incrementAndGetRefCount() {
+return ++refCount;
   }
 
   @Override
-  public int decrementAndGetRefCount() {
-return value.updateAndGet(i -> i > 0 ? i - 1 : i);
+  public synchronized int decrementAndGetRefCount() {
+return (refCount > 0) ? --refCount : 0;
   }
 }

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



[hadoop] branch trunk updated: HDFS-15907. Reduce Memory Overhead of AclFeature by avoiding AtomicInteger. Contributed by Stephen O'Donnell.

2021-03-23 Thread ayushsaxena
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/trunk by this push:
 new 569e407  HDFS-15907. Reduce Memory Overhead of AclFeature by avoiding 
AtomicInteger. Contributed by Stephen O'Donnell.
569e407 is described below

commit 569e407f646cdb422e83fd81d7948979a5a3c805
Author: Ayush Saxena 
AuthorDate: Wed Mar 24 02:10:02 2021 +0530

HDFS-15907. Reduce Memory Overhead of AclFeature by avoiding AtomicInteger. 
Contributed by Stephen O'Donnell.
---
 .../apache/hadoop/hdfs/server/namenode/AclFeature.java| 15 +++
 1 file changed, 7 insertions(+), 8 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
index 6b2baa4..2dfe507 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclFeature.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.util.Arrays;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -35,7 +34,7 @@ import 
org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
 public class AclFeature implements INode.Feature, ReferenceCounter {
   public static final ImmutableList EMPTY_ENTRY_LIST =
 ImmutableList.of();
-  private AtomicInteger value = new AtomicInteger();
+  private int refCount = 0;
 
   private final int [] entries;
 
@@ -84,17 +83,17 @@ public class AclFeature implements INode.Feature, 
ReferenceCounter {
   }
 
   @Override
-  public int getRefCount() {
-return value.get();
+  public synchronized int getRefCount() {
+return refCount;
   }
 
   @Override
-  public int incrementAndGetRefCount() {
-return value.incrementAndGet();
+  public synchronized int incrementAndGetRefCount() {
+return ++refCount;
   }
 
   @Override
-  public int decrementAndGetRefCount() {
-return value.updateAndGet(i -> i > 0 ? i - 1 : i);
+  public synchronized int decrementAndGetRefCount() {
+return (refCount > 0) ? --refCount : 0;
   }
 }

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



[hadoop] branch branch-2.10 updated: YARN-10697. Resources are displayed in bytes in UI for schedulers other than capacity. Contributed by Bilwa S T.

2021-03-23 Thread jbrennan
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-2.10 by this push:
 new 33c4d45  YARN-10697. Resources are displayed in bytes in UI for 
schedulers other than capacity. Contributed by Bilwa S T.
33c4d45 is described below

commit 33c4d4570d1e729e551a1cad551b9685d103a8c1
Author: Jim Brennan 
AuthorDate: Tue Mar 23 18:57:23 2021 +

YARN-10697. Resources are displayed in bytes in UI for schedulers other 
than capacity. Contributed by Bilwa S T.

(cherry picked from commit 34e507cb8c11d3b6ee561fd4aabde6dadadcee00)
---
 .../org/apache/hadoop/yarn/api/records/Resource.java | 16 +++-
 .../resourcemanager/webapp/MetricsOverviewTable.java | 12 ++--
 2 files changed, 21 insertions(+), 7 deletions(-)

diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
index bd6b315..977b286 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
@@ -29,6 +29,7 @@ import 
org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
 import org.apache.hadoop.yarn.api.records.impl.LightWeightResource;
@@ -472,9 +473,13 @@ public abstract class Resource implements 
Comparable {
 
   @Override
   public String toString() {
+return getFormattedString(String.valueOf(getMemorySize()));
+  }
+
+  private String getFormattedString(String memory) {
 StringBuilder sb = new StringBuilder();
 
-sb.append(" {
 return sb.toString();
   }
 
+  /**
+   * This method is to get memory in terms of KB|MB|GB.
+   * @return string containing all resources
+   */
+  public String getFormattedString() {
+return getFormattedString(
+StringUtils.byteDesc(getMemorySize() * 1024 * 1024));
+  }
+
   @Override
   public int hashCode() {
 final int prime = 47;
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
index f98fc01..0fc98df7 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
@@ -83,13 +83,13 @@ public class MetricsOverviewTable extends HtmlBlock {
 } else {
   allocatedContainers = clusterMetrics.getContainersAllocated();
   usedResources = Resource.newInstance(
-  clusterMetrics.getAllocatedMB() * BYTES_IN_MB,
+  clusterMetrics.getAllocatedMB(),
   (int) clusterMetrics.getAllocatedVirtualCores());
   totalResources = Resource.newInstance(
-  clusterMetrics.getTotalMB() * BYTES_IN_MB,
+  clusterMetrics.getTotalMB(),
   (int) clusterMetrics.getTotalVirtualCores());
   reservedResources = Resource.newInstance(
-  clusterMetrics.getReservedMB() * BYTES_IN_MB,
+  clusterMetrics.getReservedMB(),
   (int) clusterMetrics.getReservedVirtualCores());
 }
 
@@ -121,9 +121,9 @@ public class MetricsOverviewTable extends HtmlBlock {
 )
 ).
 td(String.valueOf(allocatedContainers)).
-td(usedResources.toString()).
-td(totalResources.toString()).
-td(reservedResources.toString()).
+td(usedResources.getFormattedString()).
+td(totalResources.getFormattedString()).
+td(reservedResources.getFormattedString()).
 td(String.valueOf(clusterMetrics.getUtilizedMBPercent())).
 td(String.valueOf(clusterMetrics.getUtilizedVirtualCoresPercent())).
   _().

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

[hadoop] branch branch-3.1 updated: YARN-10697. Resources are displayed in bytes in UI for schedulers other than capacity. Contributed by Bilwa S T.

2021-03-23 Thread jbrennan
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.1 by this push:
 new 34e507c  YARN-10697. Resources are displayed in bytes in UI for 
schedulers other than capacity. Contributed by Bilwa S T.
34e507c is described below

commit 34e507cb8c11d3b6ee561fd4aabde6dadadcee00
Author: Jim Brennan 
AuthorDate: Tue Mar 23 18:57:23 2021 +

YARN-10697. Resources are displayed in bytes in UI for schedulers other 
than capacity. Contributed by Bilwa S T.
---
 .../org/apache/hadoop/yarn/api/records/Resource.java | 16 +++-
 .../resourcemanager/webapp/MetricsOverviewTable.java | 12 ++--
 2 files changed, 21 insertions(+), 7 deletions(-)

diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
index 634a1d8..1558caf 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
@@ -30,6 +30,7 @@ import 
org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
 import org.apache.hadoop.yarn.api.records.impl.LightWeightResource;
@@ -478,9 +479,13 @@ public abstract class Resource implements 
Comparable {
 
   @Override
   public String toString() {
+return getFormattedString(String.valueOf(getMemorySize()));
+  }
+
+  private String getFormattedString(String memory) {
 StringBuilder sb = new StringBuilder();
 
-sb.append(" {
 return sb.toString();
   }
 
+  /**
+   * This method is to get memory in terms of KB|MB|GB.
+   * @return string containing all resources
+   */
+  public String getFormattedString() {
+return getFormattedString(
+StringUtils.byteDesc(getMemorySize() * 1024 * 1024));
+  }
+
   @Override
   public int hashCode() {
 final int prime = 47;
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
index 97e43e6..009a012 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
@@ -83,13 +83,13 @@ public class MetricsOverviewTable extends HtmlBlock {
 } else {
   allocatedContainers = clusterMetrics.getContainersAllocated();
   usedResources = Resource.newInstance(
-  clusterMetrics.getAllocatedMB() * BYTES_IN_MB,
+  clusterMetrics.getAllocatedMB(),
   (int) clusterMetrics.getAllocatedVirtualCores());
   totalResources = Resource.newInstance(
-  clusterMetrics.getTotalMB() * BYTES_IN_MB,
+  clusterMetrics.getTotalMB(),
   (int) clusterMetrics.getTotalVirtualCores());
   reservedResources = Resource.newInstance(
-  clusterMetrics.getReservedMB() * BYTES_IN_MB,
+  clusterMetrics.getReservedMB(),
   (int) clusterMetrics.getReservedVirtualCores());
 }
 
@@ -121,9 +121,9 @@ public class MetricsOverviewTable extends HtmlBlock {
 )
 ).
 td(String.valueOf(allocatedContainers)).
-td(usedResources.toString()).
-td(totalResources.toString()).
-td(reservedResources.toString()).
+td(usedResources.getFormattedString()).
+td(totalResources.getFormattedString()).
+td(reservedResources.getFormattedString()).
 td(String.valueOf(clusterMetrics.getUtilizedMBPercent())).
 td(String.valueOf(clusterMetrics.getUtilizedVirtualCoresPercent())).
 __().

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



[hadoop] branch branch-3.2 updated: YARN-10697. Resources are displayed in bytes in UI for schedulers other than capacity. Contributed by Bilwa S T.

2021-03-23 Thread jbrennan
This is an automated email from the ASF dual-hosted git repository.

jbrennan 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 0fcb2f2  YARN-10697. Resources are displayed in bytes in UI for 
schedulers other than capacity. Contributed by Bilwa S T.
0fcb2f2 is described below

commit 0fcb2f28ce2e12ae1633b135863c51d791b8b9fb
Author: Jim Brennan 
AuthorDate: Tue Mar 23 18:21:45 2021 +

YARN-10697. Resources are displayed in bytes in UI for schedulers other 
than capacity. Contributed by Bilwa S T.

(cherry picked from commit 174f3a96b10a0ab0fd8aed1b0f904ca5f0c3f268)
---
 .../org/apache/hadoop/yarn/api/records/Resource.java | 16 +++-
 .../resourcemanager/webapp/MetricsOverviewTable.java | 12 ++--
 2 files changed, 21 insertions(+), 7 deletions(-)

diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
index 7740354..5ab0306 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
 import org.apache.hadoop.yarn.api.records.impl.LightWeightResource;
@@ -470,9 +471,13 @@ public abstract class Resource implements 
Comparable {
 
   @Override
   public String toString() {
+return getFormattedString(String.valueOf(getMemorySize()));
+  }
+
+  private String getFormattedString(String memory) {
 StringBuilder sb = new StringBuilder();
 
-sb.append(" {
 return sb.toString();
   }
 
+  /**
+   * This method is to get memory in terms of KB|MB|GB.
+   * @return string containing all resources
+   */
+  public String getFormattedString() {
+return getFormattedString(
+StringUtils.byteDesc(getMemorySize() * 1024 * 1024));
+  }
+
   @Override
   public int hashCode() {
 final int prime = 47;
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
index 97e43e6..009a012 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
@@ -83,13 +83,13 @@ public class MetricsOverviewTable extends HtmlBlock {
 } else {
   allocatedContainers = clusterMetrics.getContainersAllocated();
   usedResources = Resource.newInstance(
-  clusterMetrics.getAllocatedMB() * BYTES_IN_MB,
+  clusterMetrics.getAllocatedMB(),
   (int) clusterMetrics.getAllocatedVirtualCores());
   totalResources = Resource.newInstance(
-  clusterMetrics.getTotalMB() * BYTES_IN_MB,
+  clusterMetrics.getTotalMB(),
   (int) clusterMetrics.getTotalVirtualCores());
   reservedResources = Resource.newInstance(
-  clusterMetrics.getReservedMB() * BYTES_IN_MB,
+  clusterMetrics.getReservedMB(),
   (int) clusterMetrics.getReservedVirtualCores());
 }
 
@@ -121,9 +121,9 @@ public class MetricsOverviewTable extends HtmlBlock {
 )
 ).
 td(String.valueOf(allocatedContainers)).
-td(usedResources.toString()).
-td(totalResources.toString()).
-td(reservedResources.toString()).
+td(usedResources.getFormattedString()).
+td(totalResources.getFormattedString()).
+td(reservedResources.getFormattedString()).
 td(String.valueOf(clusterMetrics.getUtilizedMBPercent())).
 td(String.valueOf(clusterMetrics.getUtilizedVirtualCoresPercent())).
 __().

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

[hadoop] branch branch-3.3 updated: YARN-10697. Resources are displayed in bytes in UI for schedulers other than capacity. Contributed by Bilwa S T.

2021-03-23 Thread jbrennan
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
 new 78bddd0  YARN-10697. Resources are displayed in bytes in UI for 
schedulers other than capacity. Contributed by Bilwa S T.
78bddd0 is described below

commit 78bddd0d9fadb5a34e8f41725592c35b5a88f630
Author: Jim Brennan 
AuthorDate: Tue Mar 23 18:21:45 2021 +

YARN-10697. Resources are displayed in bytes in UI for schedulers other 
than capacity. Contributed by Bilwa S T.

(cherry picked from commit 174f3a96b10a0ab0fd8aed1b0f904ca5f0c3f268)
---
 .../org/apache/hadoop/yarn/api/records/Resource.java | 16 +++-
 .../resourcemanager/webapp/MetricsOverviewTable.java | 12 ++--
 2 files changed, 21 insertions(+), 7 deletions(-)

diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
index 83e1c5f..b91658f 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
 import org.apache.hadoop.yarn.api.records.impl.LightWeightResource;
@@ -465,9 +466,13 @@ public abstract class Resource implements 
Comparable {
 
   @Override
   public String toString() {
+return getFormattedString(String.valueOf(getMemorySize()));
+  }
+
+  private String getFormattedString(String memory) {
 StringBuilder sb = new StringBuilder();
 
-sb.append(" {
 return sb.toString();
   }
 
+  /**
+   * This method is to get memory in terms of KB|MB|GB.
+   * @return string containing all resources
+   */
+  public String getFormattedString() {
+return getFormattedString(
+StringUtils.byteDesc(getMemorySize() * 1024 * 1024));
+  }
+
   @Override
   public int hashCode() {
 final int prime = 47;
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
index 97e43e6..009a012 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
@@ -83,13 +83,13 @@ public class MetricsOverviewTable extends HtmlBlock {
 } else {
   allocatedContainers = clusterMetrics.getContainersAllocated();
   usedResources = Resource.newInstance(
-  clusterMetrics.getAllocatedMB() * BYTES_IN_MB,
+  clusterMetrics.getAllocatedMB(),
   (int) clusterMetrics.getAllocatedVirtualCores());
   totalResources = Resource.newInstance(
-  clusterMetrics.getTotalMB() * BYTES_IN_MB,
+  clusterMetrics.getTotalMB(),
   (int) clusterMetrics.getTotalVirtualCores());
   reservedResources = Resource.newInstance(
-  clusterMetrics.getReservedMB() * BYTES_IN_MB,
+  clusterMetrics.getReservedMB(),
   (int) clusterMetrics.getReservedVirtualCores());
 }
 
@@ -121,9 +121,9 @@ public class MetricsOverviewTable extends HtmlBlock {
 )
 ).
 td(String.valueOf(allocatedContainers)).
-td(usedResources.toString()).
-td(totalResources.toString()).
-td(reservedResources.toString()).
+td(usedResources.getFormattedString()).
+td(totalResources.getFormattedString()).
+td(reservedResources.getFormattedString()).
 td(String.valueOf(clusterMetrics.getUtilizedMBPercent())).
 td(String.valueOf(clusterMetrics.getUtilizedVirtualCoresPercent())).
 __().

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

[hadoop] branch trunk updated: YARN-10697. Resources are displayed in bytes in UI for schedulers other than capacity. Contributed by Bilwa S T.

2021-03-23 Thread jbrennan
This is an automated email from the ASF dual-hosted git repository.

jbrennan pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 174f3a9  YARN-10697. Resources are displayed in bytes in UI for 
schedulers other than capacity. Contributed by Bilwa S T.
174f3a9 is described below

commit 174f3a96b10a0ab0fd8aed1b0f904ca5f0c3f268
Author: Jim Brennan 
AuthorDate: Tue Mar 23 18:21:45 2021 +

YARN-10697. Resources are displayed in bytes in UI for schedulers other 
than capacity. Contributed by Bilwa S T.
---
 .../org/apache/hadoop/yarn/api/records/Resource.java | 16 +++-
 .../resourcemanager/webapp/MetricsOverviewTable.java | 12 ++--
 2 files changed, 21 insertions(+), 7 deletions(-)

diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
index 83e1c5f..b91658f 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes;
 import org.apache.hadoop.yarn.api.records.impl.LightWeightResource;
@@ -465,9 +466,13 @@ public abstract class Resource implements 
Comparable {
 
   @Override
   public String toString() {
+return getFormattedString(String.valueOf(getMemorySize()));
+  }
+
+  private String getFormattedString(String memory) {
 StringBuilder sb = new StringBuilder();
 
-sb.append(" {
 return sb.toString();
   }
 
+  /**
+   * This method is to get memory in terms of KB|MB|GB.
+   * @return string containing all resources
+   */
+  public String getFormattedString() {
+return getFormattedString(
+StringUtils.byteDesc(getMemorySize() * 1024 * 1024));
+  }
+
   @Override
   public int hashCode() {
 final int prime = 47;
diff --git 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
index 97e43e6..009a012 100644
--- 
a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
+++ 
b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/MetricsOverviewTable.java
@@ -83,13 +83,13 @@ public class MetricsOverviewTable extends HtmlBlock {
 } else {
   allocatedContainers = clusterMetrics.getContainersAllocated();
   usedResources = Resource.newInstance(
-  clusterMetrics.getAllocatedMB() * BYTES_IN_MB,
+  clusterMetrics.getAllocatedMB(),
   (int) clusterMetrics.getAllocatedVirtualCores());
   totalResources = Resource.newInstance(
-  clusterMetrics.getTotalMB() * BYTES_IN_MB,
+  clusterMetrics.getTotalMB(),
   (int) clusterMetrics.getTotalVirtualCores());
   reservedResources = Resource.newInstance(
-  clusterMetrics.getReservedMB() * BYTES_IN_MB,
+  clusterMetrics.getReservedMB(),
   (int) clusterMetrics.getReservedVirtualCores());
 }
 
@@ -121,9 +121,9 @@ public class MetricsOverviewTable extends HtmlBlock {
 )
 ).
 td(String.valueOf(allocatedContainers)).
-td(usedResources.toString()).
-td(totalResources.toString()).
-td(reservedResources.toString()).
+td(usedResources.getFormattedString()).
+td(totalResources.getFormattedString()).
+td(reservedResources.getFormattedString()).
 td(String.valueOf(clusterMetrics.getUtilizedMBPercent())).
 td(String.valueOf(clusterMetrics.getUtilizedVirtualCoresPercent())).
 __().

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



[hadoop] branch trunk updated (9c2a712 -> 714427c)

2021-03-23 Thread inigoiri
This is an automated email from the ASF dual-hosted git repository.

inigoiri pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git.


from 9c2a712  HDFS-15913. Remove useless NULL checks before instanceof 
(#2805)
 add 714427c  HDFS-15910. Improve security with explicit_bzero (#2793)

No new revisions were added by this update.

Summary of changes:
 .../src/main/native/libhdfspp/lib/bindings/c/hdfs.cc| 2 +-
 .../src/main/native/libhdfspp/tests/hdfs_ext_test.cc| 2 +-
 .../src/main/native/libhdfspp/tests/hdfspp_mini_dfs.h   | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)

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



[hadoop] 02/02: HDFS-15093. RENAME.TO_TRASH is ignored When RENAME.OVERWRITE is specified. Contributed by Ayush Saxena.

2021-03-23 Thread sodonnell
This is an automated email from the ASF dual-hosted git repository.

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

commit b26d75ce37128dcfda37c52c8bfc8b465ce71d48
Author: Ayush Saxena 
AuthorDate: Sat May 23 18:38:32 2020 +0530

HDFS-15093. RENAME.TO_TRASH is ignored When RENAME.OVERWRITE is specified. 
Contributed by Ayush Saxena.

(cherry picked from commit e0ae232f669b2e2a6654cfacff22a090c462effc)
---
 .../ClientNamenodeProtocolTranslatorPB.java |  3 ++-
 ...lientNamenodeProtocolServerSideTranslatorPB.java |  3 ++-
 .../java/org/apache/hadoop/hdfs/TestDFSRename.java  | 21 +
 3 files changed, 25 insertions(+), 2 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 35b52b5..6cdce59 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -609,7 +609,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
   for (Rename option : options) {
 if (option == Rename.OVERWRITE) {
   overwrite = true;
-} else if (option == Rename.TO_TRASH) {
+}
+if (option == Rename.TO_TRASH) {
   toTrash = true;
 }
   }
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 9bd8248..e0afe00 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -688,7 +688,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB 
implements
 ArrayList optionList = new ArrayList();
 if(req.getOverwriteDest()) {
   optionList.add(Rename.OVERWRITE);
-} else if(req.hasMoveToTrash() && req.getMoveToTrash()) {
+}
+if (req.hasMoveToTrash() && req.getMoveToTrash()) {
   optionList.add(Rename.TO_TRASH);
 }
 
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java
index e7002c3..fe2eee2 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRename.java
@@ -30,7 +30,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 
 public class TestDFSRename {
@@ -175,4 +177,23 @@ public class TestDFSRename {
   }
 }
   }
+
+  @Test
+  public void testRename2Options() throws Exception {
+try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(
+new HdfsConfiguration()).build()) {
+  cluster.waitActive();
+  final DistributedFileSystem dfs = cluster.getFileSystem();
+  Path path = new Path("/test");
+  dfs.mkdirs(path);
+  GenericTestUtils.LogCapturer auditLog =
+  GenericTestUtils.LogCapturer.captureLogs(FSNamesystem.auditLog);
+  dfs.rename(path, new Path("/dir1"),
+  new Rename[] {Rename.OVERWRITE, Rename.TO_TRASH});
+  String auditOut = auditLog.getOutput();
+  assertTrue("Rename should have both OVERWRITE and TO_TRASH "
+  + "flags at namenode but had only " + auditOut,
+  auditOut.contains("options=[OVERWRITE, TO_TRASH]"));
+}
+  }
 }

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



[hadoop] branch branch-3.3 updated (91d229b -> b26d75c)

2021-03-23 Thread sodonnell
This is an automated email from the ASF dual-hosted git repository.

sodonnell pushed a change to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git.


from 91d229b  MAPREDUCE-7325. Intermediate data encryption is broken in 
LocalJobRunner. Contributed by Ahmed Hussein
 new 968c95b  HDFS-10792. RedundantEditLogInputStream should log caught 
exceptions. Contributed by Wei-Chiu Chuang.
 new b26d75c  HDFS-15093. RENAME.TO_TRASH is ignored When RENAME.OVERWRITE 
is specified. Contributed by Ayush Saxena.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../ClientNamenodeProtocolTranslatorPB.java |  3 ++-
 ...lientNamenodeProtocolServerSideTranslatorPB.java |  3 ++-
 .../namenode/RedundantEditLogInputStream.java   |  4 +++-
 .../java/org/apache/hadoop/hdfs/TestDFSRename.java  | 21 +
 4 files changed, 28 insertions(+), 3 deletions(-)

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



[hadoop] 01/02: HDFS-10792. RedundantEditLogInputStream should log caught exceptions. Contributed by Wei-Chiu Chuang.

2021-03-23 Thread sodonnell
This is an automated email from the ASF dual-hosted git repository.

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

commit 968c95bfba3a6932ad68a2050fc7a8b6454a7010
Author: Ayush Saxena 
AuthorDate: Sun May 31 17:06:44 2020 +0530

HDFS-10792. RedundantEditLogInputStream should log caught exceptions. 
Contributed by Wei-Chiu Chuang.

(cherry picked from commit ae13a5ccbea10fe86481adbbff574c528e03c7f6)
---
 .../hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java  | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java
index 80e6b39..15f799a 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java
@@ -170,6 +170,7 @@ class RedundantEditLogInputStream extends 
EditLogInputStream {
   }
   return nextOp();
 } catch (IOException e) {
+  LOG.warn("encountered an exception", e);
   return null;
 }
   }
@@ -228,7 +229,8 @@ class RedundantEditLogInputStream extends 
EditLogInputStream {
   "streams are shorter than the current one!  The best " +
   "remaining edit log ends at transaction " +
   newLast + ", but we thought we could read up to transaction " +
-  oldLast + ".  If you continue, metadata will be lost forever!");
+  oldLast + ".  If you continue, metadata will be lost forever!",
+  prevException);
 }
 LOG.error("Got error reading edit log input stream " +
   streams[curIdx].getName() + "; failing over to edit log " +

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



[hadoop] branch trunk updated: HDFS-15913. Remove useless NULL checks before instanceof (#2805)

2021-03-23 Thread inigoiri
This is an automated email from the ASF dual-hosted git repository.

inigoiri pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
 new 9c2a712  HDFS-15913. Remove useless NULL checks before instanceof 
(#2805)
9c2a712 is described below

commit 9c2a712597788bdd8a2eb77cfaa9bf7f93989b56
Author: Jack Jiang <81204396+tjj...@users.noreply.github.com>
AuthorDate: Tue Mar 23 23:51:35 2021 +0800

HDFS-15913. Remove useless NULL checks before instanceof (#2805)
---
 .../main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java   | 2 +-
 .../main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeInfo.java   | 2 +-
 .../main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeStatus.java | 2 +-
 .../main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java| 2 +-
 .../apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java  | 2 +-
 .../java/org/apache/hadoop/hdfs/server/datanode/ErrorReportAction.java  | 2 +-
 .../org/apache/hadoop/hdfs/server/datanode/ReportBadBlockAction.java| 2 +-
 .../java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java| 2 +-
 .../src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java | 2 +-
 .../main/java/org/apache/hadoop/hdfs/server/namenode/QuotaCounts.java   | 2 +-
 .../java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java  | 2 +-
 .../src/main/java/org/apache/hadoop/hdfs/util/EnumCounters.java | 2 +-
 .../src/main/java/org/apache/hadoop/hdfs/util/EnumDoubles.java  | 2 +-
 13 files changed, 13 insertions(+), 13 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
index 7bd3f96..b0ea3ea 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
@@ -219,7 +219,7 @@ public class BlockStoragePolicy implements 
BlockStoragePolicySpi {
   public boolean equals(Object obj) {
 if (obj == this) {
   return true;
-} else if (obj == null || !(obj instanceof BlockStoragePolicy)) {
+} else if (!(obj instanceof BlockStoragePolicy)) {
   return false;
 }
 final BlockStoragePolicy that = (BlockStoragePolicy)obj;
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeInfo.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeInfo.java
index b7a7e98..05d2e03 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeInfo.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeInfo.java
@@ -86,7 +86,7 @@ public class RollingUpgradeInfo extends RollingUpgradeStatus {
   public boolean equals(Object obj) {
 if (obj == this) {
   return true;
-} else if (obj == null || !(obj instanceof RollingUpgradeInfo)) {
+} else if (!(obj instanceof RollingUpgradeInfo)) {
   return false;
 }
 final RollingUpgradeInfo that = (RollingUpgradeInfo)obj;
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeStatus.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeStatus.java
index 1f969fb..2e8e0eb 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeStatus.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeStatus.java
@@ -51,7 +51,7 @@ public class RollingUpgradeStatus {
   public boolean equals(Object obj) {
 if (obj == this) {
   return true;
-} else if (obj == null || !(obj instanceof RollingUpgradeStatus)) {
+} else if (!(obj instanceof RollingUpgradeStatus)) {
   return false;
 }
 final RollingUpgradeStatus that = (RollingUpgradeStatus) obj;
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index 17f0d8f..a953800 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@ -622,7 +622,7 @@ public class Dispatcher {
   public boolean equals(Object obj) {
 if (this == obj) {
   return true;
-} else if (obj == null || !(obj instanceof StorageGroup)) {
+} els

[hadoop] branch trunk updated: HADOOP-17599. Remove NULL checks before instanceof (#2804)

2021-03-23 Thread inigoiri
This is an automated email from the ASF dual-hosted git repository.

inigoiri pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
 new d8ec8ab  HADOOP-17599. Remove NULL checks before instanceof (#2804)
d8ec8ab is described below

commit d8ec8ab9653f628fa6ed16baf4c9e7681266ae8f
Author: Jack Jiang <81204396+tjj...@users.noreply.github.com>
AuthorDate: Tue Mar 23 23:46:11 2021 +0800

HADOOP-17599. Remove NULL checks before instanceof (#2804)
---
 .../src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java| 2 +-
 .../src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java| 2 +-
 .../src/main/java/org/apache/hadoop/fs/FileChecksum.java  | 2 +-
 .../src/main/java/org/apache/hadoop/oncrpc/RpcCallCache.java  | 2 +-
 .../test/java/org/apache/hadoop/fs/azure/MockWasbAuthorizerImpl.java  | 3 +--
 .../yarn/api/protocolrecords/GetAllResourceProfilesResponse.java  | 2 +-
 .../yarn/api/protocolrecords/GetAllResourceTypeInfoResponse.java  | 2 +-
 .../hadoop/yarn/api/protocolrecords/GetResourceProfileRequest.java| 2 +-
 .../hadoop/yarn/api/protocolrecords/GetResourceProfileResponse.java   | 2 +-
 .../org/apache/hadoop/yarn/api/records/impl/LightWeightResource.java  | 2 +-
 .../org/apache/hadoop/yarn/util/Log4jWarningErrorMetricsAppender.java | 2 +-
 .../main/java/org/apache/hadoop/yarn/server/volume/csi/VolumeId.java  | 2 +-
 .../containermanager/linux/privileged/PrivilegedOperation.java| 2 +-
 .../resourceplugin/deviceframework/AssignedDevice.java| 4 ++--
 .../containermanager/resourceplugin/gpu/AssignedGpuDevice.java| 4 ++--
 .../nodemanager/containermanager/resourceplugin/gpu/GpuDevice.java| 4 ++--
 .../nodemanager/containermanager/runtime/ContainerRuntimeContext.java | 2 +-
 17 files changed, 20 insertions(+), 21 deletions(-)

diff --git 
a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
 
b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
index 5796f23..d50cf1e 100644
--- 
a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
+++ 
b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java
@@ -1353,7 +1353,7 @@ public abstract class AbstractFileSystem implements 
PathCapabilities {
   
   @Override //Object
   public boolean equals(Object other) {
-if (other == null || !(other instanceof AbstractFileSystem)) {
+if (!(other instanceof AbstractFileSystem)) {
   return false;
 }
 return myUri.equals(((AbstractFileSystem) other).myUri);
diff --git 
a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java
 
b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java
index 19c29cf..38a7344 100644
--- 
a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java
+++ 
b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java
@@ -97,7 +97,7 @@ public class DelegationTokenRenewer
 public boolean equals(final Object that) {
   if (this == that) {
 return true;
-  } else if (that == null || !(that instanceof RenewAction)) {
+  } else if (!(that instanceof RenewAction)) {
 return false;
   }
   return token.equals(((RenewAction)that).token);
diff --git 
a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileChecksum.java
 
b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileChecksum.java
index 62f1a9b..6822fa4 100644
--- 
a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileChecksum.java
+++ 
b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileChecksum.java
@@ -47,7 +47,7 @@ public abstract class FileChecksum implements Writable {
 if (other == this) {
   return true;
 }
-if (other == null || !(other instanceof FileChecksum)) {
+if (!(other instanceof FileChecksum)) {
   return false;
 }
 
diff --git 
a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcCallCache.java
 
b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcCallCache.java
index 8632a38..d6bd492 100644
--- 
a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcCallCache.java
+++ 
b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/RpcCallCache.java
@@ -93,7 +93,7 @@ public class RpcCallCache {
   if (this == obj) {
 return true;
   }
-  if (obj == null || !(obj instanceof ClientRequest)) {
+  if (!(obj instanceof ClientRequest)) {
 return false;
   }
   ClientRequest other = (ClientRequest) obj;
diff --git 
a/hadoo

[hadoop] branch trunk updated: HDFS-15908. Possible Resource Leak in org.apache.hadoop.hdfs.qjournal.server.Journal (#2790). Contributed by Narges Shadab.

2021-03-23 Thread ayushsaxena
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/trunk by this push:
 new d77c7ab  HDFS-15908. Possible Resource Leak in 
org.apache.hadoop.hdfs.qjournal.server.Journal (#2790). Contributed by Narges 
Shadab.
d77c7ab is described below

commit d77c7ab4e0604484ec3c48ac2acb32ea71a53d9a
Author: Narges Shadab <54193416+narges...@users.noreply.github.com>
AuthorDate: Tue Mar 23 04:36:58 2021 -0700

HDFS-15908. Possible Resource Leak in 
org.apache.hadoop.hdfs.qjournal.server.Journal (#2790). Contributed by Narges 
Shadab.

Signed-off-by: Ayush Saxena 
---
 .../src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
index 99a087e..238a000 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/Journal.java
@@ -264,9 +264,9 @@ public class Journal implements Closeable {
*/
   @Override // Closeable
   public void close() throws IOException {
-storage.close();
 IOUtils.closeStream(committedTxnId);
 IOUtils.closeStream(curSegment);
+storage.close();
   }
   
   JNStorage getStorage() {

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



[hadoop] branch branch-3.1 updated: HDFS-15906. Close FSImage and FSNamesystem after formatting is complete (#2800)

2021-03-23 Thread tasanuma
This is an automated email from the ASF dual-hosted git repository.

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


The following commit(s) were added to refs/heads/branch-3.1 by this push:
 new 75f2d47  HDFS-15906. Close FSImage and FSNamesystem after formatting 
is complete (#2800)
75f2d47 is described below

commit 75f2d47ff5736087fa8945f2d70527131bb3d957
Author: litao 
AuthorDate: Tue Mar 23 17:45:24 2021 +0800

HDFS-15906. Close FSImage and FSNamesystem after formatting is complete 
(#2800)

(cherry picked from commit d05d15620e320d28bfb2a4f5224e69062528)
---
 .../java/org/apache/hadoop/hdfs/server/namenode/NameNode.java | 11 +--
 1 file changed, 9 insertions(+), 2 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index c8cd8f7..e09aae6 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -1200,8 +1200,9 @@ public class NameNode extends ReconfigurableBase 
implements
 System.out.println("Formatting using clusterid: " + clusterId);
 
 FSImage fsImage = new FSImage(conf, nameDirsToFormat, editDirsToFormat);
+FSNamesystem fsn = null;
 try {
-  FSNamesystem fsn = new FSNamesystem(conf, fsImage);
+  fsn = new FSNamesystem(conf, fsImage);
   fsImage.getEditLog().initJournalsForWrite();
 
   // Abort NameNode format if reformat is disabled and if
@@ -1226,8 +1227,14 @@ public class NameNode extends ReconfigurableBase 
implements
   fsImage.format(fsn, clusterId);
 } catch (IOException ioe) {
   LOG.warn("Encountered exception during format: ", ioe);
-  fsImage.close();
   throw ioe;
+} finally {
+  if (fsImage != null) {
+fsImage.close();
+  }
+  if (fsn != null) {
+fsn.close();
+  }
 }
 return false;
   }

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



[hadoop] branch branch-3.2 updated: HDFS-15906. Close FSImage and FSNamesystem after formatting is complete (#2800)

2021-03-23 Thread tasanuma
This is an automated email from the ASF dual-hosted git repository.

tasanuma 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 d05d156  HDFS-15906. Close FSImage and FSNamesystem after formatting 
is complete (#2800)
d05d156 is described below

commit d05d15620e320d28bfb2a4f5224e69062528
Author: litao 
AuthorDate: Tue Mar 23 17:45:24 2021 +0800

HDFS-15906. Close FSImage and FSNamesystem after formatting is complete 
(#2800)
---
 .../java/org/apache/hadoop/hdfs/server/namenode/NameNode.java | 11 +--
 1 file changed, 9 insertions(+), 2 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
index 30bf4f85..7f62d2c 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
@@ -1204,8 +1204,9 @@ public class NameNode extends ReconfigurableBase 
implements
 System.out.println("Formatting using clusterid: " + clusterId);
 
 FSImage fsImage = new FSImage(conf, nameDirsToFormat, editDirsToFormat);
+FSNamesystem fsn = null;
 try {
-  FSNamesystem fsn = new FSNamesystem(conf, fsImage);
+  fsn = new FSNamesystem(conf, fsImage);
   fsImage.getEditLog().initJournalsForWrite();
 
   // Abort NameNode format if reformat is disabled and if
@@ -1230,8 +1231,14 @@ public class NameNode extends ReconfigurableBase 
implements
   fsImage.format(fsn, clusterId, force);
 } catch (IOException ioe) {
   LOG.warn("Encountered exception during format: ", ioe);
-  fsImage.close();
   throw ioe;
+} finally {
+  if (fsImage != null) {
+fsImage.close();
+  }
+  if (fsn != null) {
+fsn.close();
+  }
 }
 return false;
   }

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