You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2021/11/16 21:27:29 UTC

[hbase] branch master updated: HBASE-26267 Don't try to recover WALs from a WAL dir which doesn't exist (#3679)

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

elserj pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new 1c47f80  HBASE-26267 Don't try to recover WALs from a WAL dir which doesn't exist (#3679)
1c47f80 is described below

commit 1c47f80d83f38f69f0b1c31baad2766a17d0b6d0
Author: Josh Elser <el...@apache.org>
AuthorDate: Tue Nov 16 16:26:55 2021 -0500

    HBASE-26267 Don't try to recover WALs from a WAL dir which doesn't exist (#3679)
    
    We currently cause an error to be thrown by trying to list a
    non-existent directory. We see that the master region directory exists
    on the filesystem, but forget to make sure that the master region's WAL
    directory also exists before we try to list it.
---
 .../hadoop/hbase/master/region/MasterRegion.java   | 29 +++++--
 .../hbase/master/region/MasterRegionTestBase.java  |  8 ++
 .../master/region/TestMasterRegionWALRecovery.java | 96 ++++++++++++++++++++++
 3 files changed, 128 insertions(+), 5 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java
index ad885ea..a794315 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/region/MasterRegion.java
@@ -227,7 +227,31 @@ public final class MasterRegion {
     if (!walFs.exists(replayEditsDir) && !walFs.mkdirs(replayEditsDir)) {
       throw new IOException("Failed to create replay directory: " + replayEditsDir);
     }
+
+    // Replay any WALs for the Master Region before opening it.
     Path walsDir = new Path(walRootDir, HREGION_LOGDIR_NAME);
+    // In open(...), we expect that the WAL directory for the MasterRegion to already exist.
+    // This is in contrast to bootstrap() where we create the MasterRegion data and WAL dir.
+    // However, it's possible that users directly remove the WAL directory. We expect walsDir
+    // to always exist in normal situations, but we should guard against users changing the
+    // filesystem outside of HBase's line of sight.
+    if (walFs.exists(walsDir)) {
+      replayWALs(conf, walFs, walRootDir, walsDir, regionInfo, serverName, replayEditsDir);
+    } else {
+      LOG.error("UNEXPECTED: WAL directory for MasterRegion is missing."
+          + " {} is unexpectedly missing.", walsDir);
+    }
+
+    // Create a new WAL
+    WAL wal = createWAL(walFactory, walRoller, serverName, walFs, walRootDir, regionInfo);
+    conf.set(HRegion.SPECIAL_RECOVERED_EDITS_DIR,
+      replayEditsDir.makeQualified(walFs.getUri(), walFs.getWorkingDirectory()).toString());
+    return HRegion.openHRegionFromTableDir(conf, fs, tableDir, regionInfo, td, wal, null, null);
+  }
+
+  private static void replayWALs(Configuration conf, FileSystem walFs, Path walRootDir,
+      Path walsDir, RegionInfo regionInfo, String serverName, Path replayEditsDir)
+          throws IOException {
     for (FileStatus walDir : walFs.listStatus(walsDir)) {
       if (!walDir.isDirectory()) {
         continue;
@@ -261,11 +285,6 @@ public final class MasterRegion {
       LOG.info("Delete empty local region wal dir {}", deadWALDir);
       walFs.delete(deadWALDir, true);
     }
-
-    WAL wal = createWAL(walFactory, walRoller, serverName, walFs, walRootDir, regionInfo);
-    conf.set(HRegion.SPECIAL_RECOVERED_EDITS_DIR,
-      replayEditsDir.makeQualified(walFs.getUri(), walFs.getWorkingDirectory()).toString());
-    return HRegion.openHRegionFromTableDir(conf, fs, tableDir, regionInfo, td, wal, null, null);
   }
 
   public static MasterRegion create(MasterRegionParams params) throws IOException {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/MasterRegionTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/MasterRegionTestBase.java
index 721e4d1..8080a31 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/MasterRegionTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/MasterRegionTestBase.java
@@ -80,6 +80,14 @@ public class MasterRegionTestBase {
     htu.getConfiguration().setBoolean(MemStoreLAB.USEMSLAB_KEY, false);
     // Runs on local filesystem. Test does not need sync. Turn off checks.
     htu.getConfiguration().setBoolean(CommonFSUtils.UNSAFE_STREAM_CAPABILITY_ENFORCE, false);
+
+    createMasterRegion();
+  }
+
+  /**
+   * Creates a new MasterRegion using an existing {@code htu} on this class.
+   */
+  protected void createMasterRegion() throws IOException {
     configure(htu.getConfiguration());
     choreService = new ChoreService(getClass().getSimpleName());
     hfileCleanerPool = DirScanPool.getHFileCleanerScanPool(htu.getConfiguration());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestMasterRegionWALRecovery.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestMasterRegionWALRecovery.java
new file mode 100644
index 0000000..ad04f16
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/region/TestMasterRegionWALRecovery.java
@@ -0,0 +1,96 @@
+/**
+ * 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.hbase.master.region;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ MasterTests.class, MediumTests.class })
+public class TestMasterRegionWALRecovery extends MasterRegionTestBase {
+  private static final Logger LOG = LoggerFactory.getLogger(TestMasterRegionWALRecovery.class);
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestMasterRegionWALRecovery.class);
+
+  private Path masterRegionDir;
+
+  @Override
+  protected void postSetUp() throws IOException {
+    Configuration conf = htu.getConfiguration();
+    Path testDir = htu.getDataTestDir();
+    FileSystem fs = testDir.getFileSystem(conf);
+    masterRegionDir = new Path(testDir, REGION_DIR_NAME);
+  }
+
+  @Test
+  public void test() throws IOException, InterruptedException {
+    region
+      .update(r -> r.put(new Put(Bytes.toBytes(1)).addColumn(CF1, QUALIFIER, Bytes.toBytes(1))));
+    region.flush(true);
+
+    Path testDir = htu.getDataTestDir();
+    FileSystem fs = testDir.getFileSystem(htu.getConfiguration());
+    region.close(false);
+
+    Path masterRegionWalDir = new Path(masterRegionDir, HConstants.HREGION_LOGDIR_NAME);
+    LOG.info("WAL dir: {}", masterRegionWalDir);
+    assertTrue(fs.exists(masterRegionWalDir));
+    // Make sure we have the WAL for the localhost "server"
+    FileStatus[] files = fs.listStatus(masterRegionWalDir);
+    LOG.info("WAL files: {}", Arrays.toString(files));
+    assertEquals(1, files.length);
+    LOG.info("Deleting {}", masterRegionWalDir);
+    // Delete the WAL directory
+    fs.delete(masterRegionWalDir, true);
+
+    // Re-create the MasterRegion and hit the MasterRegion#open() code-path
+    // (rather than bootstrap())
+    createMasterRegion();
+
+    // Make sure we can read the same data we wrote (we flushed before nuking the WALs,
+    // so data should be durable)
+    Result r = region.get(new Get(Bytes.toBytes(1)));
+    Cell c = r.getColumnLatestCell(CF1, QUALIFIER);
+    assertArrayEquals(Bytes.toBytes(1), CellUtil.cloneValue(c));
+  }
+}