You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by me...@apache.org on 2021/12/30 04:40:27 UTC

[hbase] branch branch-2.4 updated: HBASE-26625 ExportSnapshot tool failed to copy data files for tables with merge region (#3981)

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

meiyi pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
     new a7c4a39  HBASE-26625 ExportSnapshot tool failed to copy data files for tables with merge region (#3981)
a7c4a39 is described below

commit a7c4a394a9c5ccfc353c9edefdd7ba20cc1d4473
Author: meiyi <my...@gmail.com>
AuthorDate: Thu Dec 30 10:40:33 2021 +0800

    HBASE-26625 ExportSnapshot tool failed to copy data files for tables with merge region (#3981)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../hadoop/hbase/snapshot/ExportSnapshot.java      | 41 ++++++++++------
 .../hadoop/hbase/snapshot/TestExportSnapshot.java  | 57 ++++++++++++++++++++--
 .../snapshot/TestExportSnapshotV1NoCluster.java    | 19 +++++++-
 .../hadoop/hbase/regionserver/StoreFileInfo.java   | 19 ++++++++
 4 files changed, 116 insertions(+), 20 deletions(-)

diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
index a95a9f4..da6d83a 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.io.WALLink;
 import org.apache.hadoop.hbase.io.hadoopbackport.ThrottledInputStream;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -580,31 +581,39 @@ public class ExportSnapshot extends AbstractHBaseTool implements Tool {
         @Override
         public void storeFile(final RegionInfo regionInfo, final String family,
             final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
-          // for storeFile.hasReference() case, copied as part of the manifest
+          Pair<SnapshotFileInfo, Long> snapshotFileAndSize = null;
           if (!storeFile.hasReference()) {
             String region = regionInfo.getEncodedName();
             String hfile = storeFile.getName();
-            Path path = HFileLink.createPath(table, region, family, hfile);
-
-            SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder()
-              .setType(SnapshotFileInfo.Type.HFILE)
-              .setHfile(path.toString())
-              .build();
-
-            long size;
-            if (storeFile.hasFileSize()) {
-              size = storeFile.getFileSize();
-            } else {
-              size = HFileLink.buildFromHFileLinkPattern(conf, path).getFileStatus(fs).getLen();
-            }
-            files.add(new Pair<>(fileInfo, size));
+            snapshotFileAndSize = getSnapshotFileAndSize(fs, conf, table, region, family, hfile,
+              storeFile.hasFileSize() ? storeFile.getFileSize() : -1);
+          } else {
+            Pair<String, String> referredToRegionAndFile =
+                StoreFileInfo.getReferredToRegionAndFile(storeFile.getName());
+            String referencedRegion = referredToRegionAndFile.getFirst();
+            String referencedHFile = referredToRegionAndFile.getSecond();
+            snapshotFileAndSize = getSnapshotFileAndSize(fs, conf, table, referencedRegion, family,
+              referencedHFile, storeFile.hasFileSize() ? storeFile.getFileSize() : -1);
           }
+          files.add(snapshotFileAndSize);
         }
-    });
+      });
 
     return files;
   }
 
+  private static Pair<SnapshotFileInfo, Long> getSnapshotFileAndSize(FileSystem fs,
+      Configuration conf, TableName table, String region, String family, String hfile, long size)
+      throws IOException {
+    Path path = HFileLink.createPath(table, region, family, hfile);
+    SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder().setType(SnapshotFileInfo.Type.HFILE)
+        .setHfile(path.toString()).build();
+    if (size == -1) {
+      size = HFileLink.buildFromHFileLinkPattern(conf, path).getFileStatus(fs).getLen();
+    }
+    return new Pair<>(fileInfo, size);
+  }
+
   /**
    * Given a list of file paths and sizes, create around ngroups in as balanced a way as possible.
    * The groups created will have similar amounts of bytes.
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
index 8363fb6..f1d50a5 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java
@@ -28,6 +28,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
+import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -37,12 +38,18 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.Pair;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -58,6 +65,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 /**
  * Test Export Snapshot Tool
@@ -159,6 +167,40 @@ public class TestExportSnapshot {
   }
 
   @Test
+  public void testExportFileSystemStateWithMergeRegion() throws Exception {
+    // disable compaction
+    admin.compactionSwitch(false,
+      admin.getRegionServers().stream().map(a -> a.getServerName()).collect(Collectors.toList()));
+    // create Table
+    TableName tableName0 = TableName.valueOf("testtb-" + testName.getMethodName() + "-1");
+    byte[] snapshotName0 = Bytes.toBytes("snaptb0-" + testName.getMethodName() + "-1");
+    admin.createTable(
+      TableDescriptorBuilder.newBuilder(tableName0)
+          .setColumnFamilies(
+            Lists.newArrayList(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build()))
+          .build(),
+      new byte[][] { Bytes.toBytes("2") });
+    // put some data
+    try (Table table = admin.getConnection().getTable(tableName0)) {
+      table.put(new Put(Bytes.toBytes("1")).addColumn(FAMILY, null, Bytes.toBytes("1")));
+      table.put(new Put(Bytes.toBytes("2")).addColumn(FAMILY, null, Bytes.toBytes("2")));
+    }
+    List<RegionInfo> regions = admin.getRegions(tableName0);
+    assertEquals(2, regions.size());
+    tableNumFiles = regions.size();
+    // merge region
+    admin.mergeRegionsAsync(new byte[][] { regions.get(0).getEncodedNameAsBytes(),
+        regions.get(1).getEncodedNameAsBytes() },
+      true).get();
+    // take a snapshot
+    admin.snapshot(snapshotName0, tableName0);
+    // export snapshot and verify
+    testExportFileSystemState(tableName0, snapshotName0, snapshotName0, tableNumFiles);
+    // delete table
+    TEST_UTIL.deleteTable(tableName0);
+  }
+
+  @Test
   public void testExportFileSystemStateWithSkipTmp() throws Exception {
     TEST_UTIL.getConfiguration().setBoolean(ExportSnapshot.CONF_SKIP_TMP, true);
     try {
@@ -285,12 +327,21 @@ public class TestExportSnapshot {
             return;
           }
 
-          String hfile = storeFile.getName();
-          snapshotFiles.add(hfile);
-          if (!storeFile.hasReference()) {
+          if (!storeFile.hasReference() && !StoreFileInfo.isReference(storeFile.getName())) {
+            String hfile = storeFile.getName();
+            snapshotFiles.add(hfile);
             verifyNonEmptyFile(new Path(exportedArchive,
               new Path(CommonFSUtils.getTableDir(new Path("./"), tableName),
                   new Path(regionInfo.getEncodedName(), new Path(family, hfile)))));
+          } else {
+            Pair<String, String> referredToRegionAndFile =
+                StoreFileInfo.getReferredToRegionAndFile(storeFile.getName());
+            String region = referredToRegionAndFile.getFirst();
+            String hfile = referredToRegionAndFile.getSecond();
+            snapshotFiles.add(hfile);
+            verifyNonEmptyFile(new Path(exportedArchive,
+              new Path(CommonFSUtils.getTableDir(new Path("./"), tableName),
+                new Path(region, new Path(family, hfile)))));
           }
         }
 
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotV1NoCluster.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotV1NoCluster.java
index d476025..d4e400b 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotV1NoCluster.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotV1NoCluster.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.snapshot;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
@@ -28,10 +30,12 @@ import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
+import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils.SnapshotMock;
 import org.apache.hadoop.hbase.testclassification.MapReduceTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -104,7 +108,20 @@ public class TestExportSnapshotV1NoCluster {
     Path[] r1Files = builder.addRegion();
     Path[] r2Files = builder.addRegion();
     builder.commit();
-    int snapshotFilesCount = r1Files.length + r2Files.length;
+    // remove references, only keep data files
+    Set<String> dataFiles = new HashSet<>();
+    for (Path[] files: new Path[][]{r1Files, r2Files}) {
+      for (Path file : files) {
+        if (StoreFileInfo.isReference(file.getName())) {
+          Pair<String, String> referredToRegionAndFile =
+            StoreFileInfo.getReferredToRegionAndFile(file.getName());
+          dataFiles.add(referredToRegionAndFile.getSecond());
+        } else {
+          dataFiles.add(file.getName());
+        }
+      }
+    }
+    int snapshotFilesCount = dataFiles.size();
     byte[] snapshotName = Bytes.toBytes(builder.getSnapshotDescription().getName());
     TableName tableName = builder.getTableDescriptor().getTableName();
     TestExportSnapshot.testExportFileSystemState(testUtil.getConfiguration(),
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
index abfb44f..d088ef0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.io.hfile.ReaderContext;
 import org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType;
 import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -511,6 +512,24 @@ public class StoreFileInfo {
       p.getParent().getName()), nameStrippedOfSuffix);
   }
 
+  /*
+   * Return region and file name referred to by a Reference.
+   * @param referenceFile HFile name which is a Reference.
+   * @return Calculated referenced region and file name.
+   * @throws IllegalArgumentException when referenceFile regex fails to match.
+   */
+  public static Pair<String, String> getReferredToRegionAndFile(final String referenceFile) {
+    Matcher m = REF_NAME_PATTERN.matcher(referenceFile);
+    if (m == null || !m.matches()) {
+      LOG.warn("Failed match of store file name {}", referenceFile);
+      throw new IllegalArgumentException("Failed match of store file name " + referenceFile);
+    }
+    String referencedRegion = m.group(2);
+    String referencedFile = m.group(1);
+    LOG.trace("reference {} to region={} file={}", referenceFile, referencedRegion, referencedFile);
+    return new Pair<>(referencedRegion, referencedFile);
+  }
+
   /**
    * Validate the store file name.
    * @param fileName name of the file to validate