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

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


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

commit 0ecb6fe713fe819d53ddc74a1395e96647601c2d
Author: Bryan Beaudreault <bbeaudrea...@apache.org>
AuthorDate: Thu May 16 08:51:00 2024 -0400

    HBASE-26625 ExportSnapshot tool failed to copy data files for tables with 
merge region (#3981) (#5897)
    
    Signed-off-by: Duo Zhang <zhang...@apache.org>
    Co-authored-by: meiyi <myime...@gmail.com>
---
 .../hadoop/hbase/snapshot/ExportSnapshot.java      | 37 +++++++++-----
 .../hadoop/hbase/snapshot/TestExportSnapshot.java  | 57 ++++++++++++++++++++--
 .../snapshot/TestExportSnapshotV1NoCluster.java    | 19 +++++++-
 .../hadoop/hbase/regionserver/StoreFileInfo.java   | 19 ++++++++
 4 files changed, 115 insertions(+), 17 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 e3a7b805121..2e50762357e 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
@@ -51,6 +51,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.EnvironmentEdgeManager;
@@ -583,29 +584,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 a9bd94f713b..5ce670c0cb0 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
@@ -31,6 +31,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 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;
@@ -40,13 +41,19 @@ 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.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -60,6 +67,8 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
 import 
org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
 
@@ -162,6 +171,39 @@ public class TestExportSnapshot {
     testExportFileSystemState(tableName, snapshotName, snapshotName, 
tableNumFiles);
   }
 
+  @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);
@@ -330,12 +372,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 7d27bfea79a..576834deb8c 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,11 +30,13 @@ 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.EnvironmentEdgeManager;
+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(), 
tableName,
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 ae840bcfa7b..dfc3fb70045 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
@@ -39,6 +39,7 @@ import 
org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType;
 import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder;
 import org.apache.hadoop.hbase.mob.MobUtils;
 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;
@@ -518,6 +519,24 @@ public class StoreFileInfo implements Configurable {
       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

Reply via email to