From bec23593b55e6b35366a1d804d0fb335d954ad6a Mon Sep 17 00:00:00 2001 From: Zhe Zhang Date: Thu, 27 Apr 2017 09:50:30 -0700 Subject: [PATCH] HDFS-11709. StandbyCheckpointer should handle an non-existing legacyOivImageDir gracefully. Contributed by Erik Krogen. (cherry picked from commit d8a33098309f17dfb0e3a000934f68394de44bf7) (cherry picked from commit fc46f398d9192e83c7197115589eca7805c49943) (cherry picked from commit 70f18911c71c61f9b3918e195bfb9a99d22d230c) --- hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++ .../hdfs/server/namenode/ha/StandbyCheckpointer.java | 7 ++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 07c2666bdb2..da640ffc5a5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -244,6 +244,9 @@ Release 2.7.4 - UNRELEASED HDFS-10455. Logging the username when deny the setOwner operation. (Tianyi Xu and Rakesh R via Brahma Reddy Battula) + HDFS-11709. StandbyCheckpointer should handle an non-existing legacyOivImageDir + gracefully. (Erik Krogen via zhz) + Release 2.7.3 - 2016-08-25 INCOMPATIBLE CHANGES diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java index 31fa5a2fa9a..5c09043c3f6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java @@ -190,7 +190,12 @@ private void doCheckpoint() throws InterruptedException, IOException { // Save the legacy OIV image, if the output dir is defined. String outputDir = checkpointConf.getLegacyOivImageDir(); if (outputDir != null && !outputDir.isEmpty()) { - img.saveLegacyOIVImage(namesystem, outputDir, canceler); + try { + img.saveLegacyOIVImage(namesystem, outputDir, canceler); + } catch (IOException ioe) { + LOG.error("Exception encountered while saving legacy OIV image; " + + "continuing with other checkpointing steps", ioe); + } } } finally { namesystem.cpUnlock();