HBASE-26700 The way we bypass broken track file is not enough in StoreFileListFile...
[hbase.git] / hbase-server / src / test / java / org / apache / hadoop / hbase / regionserver / TestHdfsSnapshotHRegion.java
blobd17e30107f529c943c67a1cbf352c01bf8ae2402
1 /**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org.apache.hadoop.hbase.regionserver;
20 import java.io.IOException;
21 import org.apache.commons.lang3.StringUtils;
22 import org.apache.hadoop.conf.Configuration;
23 import org.apache.hadoop.fs.Path;
24 import org.apache.hadoop.hbase.HBaseClassTestRule;
25 import org.apache.hadoop.hbase.HBaseTestingUtil;
26 import org.apache.hadoop.hbase.TableName;
27 import org.apache.hadoop.hbase.client.RegionInfo;
28 import org.apache.hadoop.hbase.client.Table;
29 import org.apache.hadoop.hbase.testclassification.MediumTests;
30 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
31 import org.apache.hadoop.hbase.util.Bytes;
32 import org.apache.hadoop.hbase.util.CommonFSUtils;
33 import org.apache.hadoop.hdfs.DFSClient;
34 import org.junit.After;
35 import org.junit.Assert;
36 import org.junit.Before;
37 import org.junit.ClassRule;
38 import org.junit.Test;
39 import org.junit.experimental.categories.Category;
41 @Category({RegionServerTests.class, MediumTests.class})
42 public class TestHdfsSnapshotHRegion {
44 @ClassRule
45 public static final HBaseClassTestRule CLASS_RULE =
46 HBaseClassTestRule.forClass(TestHdfsSnapshotHRegion.class);
48 private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
49 private static final String SNAPSHOT_NAME = "foo_snapshot";
50 private Table table;
51 public static final TableName TABLE_NAME = TableName.valueOf("foo");
52 public static final byte[] FAMILY = Bytes.toBytes("f1");
53 private DFSClient client;
54 private String baseDir;
57 @Before
58 public void setUp() throws Exception {
59 Configuration c = TEST_UTIL.getConfiguration();
60 c.setBoolean("dfs.support.append", true);
61 TEST_UTIL.startMiniCluster(1);
62 table = TEST_UTIL.createMultiRegionTable(TABLE_NAME, FAMILY);
63 TEST_UTIL.loadTable(table, FAMILY);
65 // setup the hdfssnapshots
66 client = new DFSClient(TEST_UTIL.getDFSCluster().getURI(), TEST_UTIL.getConfiguration());
67 String fullUrIPath = TEST_UTIL.getDefaultRootDirPath().toString();
68 String uriString = TEST_UTIL.getTestFileSystem().getUri().toString();
69 baseDir = StringUtils.removeStart(fullUrIPath, uriString);
70 client.allowSnapshot(baseDir);
73 @After
74 public void tearDown() throws Exception {
75 client.deleteSnapshot(baseDir, SNAPSHOT_NAME);
76 TEST_UTIL.shutdownMiniCluster();
79 @Test
80 public void testOpeningReadOnlyRegionBasic() throws Exception {
81 String snapshotDir = client.createSnapshot(baseDir, SNAPSHOT_NAME);
82 RegionInfo firstRegion = TEST_UTIL.getConnection().getRegionLocator(
83 table.getName()).getAllRegionLocations().stream().findFirst().get().getRegion();
84 Path tableDir = CommonFSUtils.getTableDir(new Path(snapshotDir), TABLE_NAME);
85 HRegion snapshottedRegion = openSnapshotRegion(firstRegion, tableDir);
86 Assert.assertNotNull(snapshottedRegion);
87 snapshottedRegion.close();
90 @Test
91 public void testSnapshottingWithTmpSplitsAndMergeDirectoriesPresent() throws Exception {
92 // lets get a region and create those directories and make sure we ignore them
93 RegionInfo firstRegion = TEST_UTIL.getConnection().getRegionLocator(
94 table.getName()).getAllRegionLocations().stream().findFirst().get().getRegion();
95 String encodedName = firstRegion.getEncodedName();
96 Path tableDir = CommonFSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), TABLE_NAME);
97 Path regionDirectoryPath = new Path(tableDir, encodedName);
98 TEST_UTIL.getTestFileSystem().create(
99 new Path(regionDirectoryPath, HRegionFileSystem.REGION_TEMP_DIR));
100 TEST_UTIL.getTestFileSystem().create(
101 new Path(regionDirectoryPath, HRegionFileSystem.REGION_SPLITS_DIR));
102 TEST_UTIL.getTestFileSystem().create(
103 new Path(regionDirectoryPath, HRegionFileSystem.REGION_MERGES_DIR));
104 // now snapshot
105 String snapshotDir = client.createSnapshot(baseDir, "foo_snapshot");
106 // everything should still open just fine
107 HRegion snapshottedRegion = openSnapshotRegion(firstRegion,
108 CommonFSUtils.getTableDir(new Path(snapshotDir), TABLE_NAME));
109 Assert.assertNotNull(snapshottedRegion); // no errors and the region should open
110 snapshottedRegion.close();
113 private HRegion openSnapshotRegion(RegionInfo firstRegion, Path tableDir) throws IOException {
114 return HRegion.openReadOnlyFileSystemHRegion(
115 TEST_UTIL.getConfiguration(),
116 TEST_UTIL.getTestFileSystem(),
117 tableDir,
118 firstRegion,
119 table.getDescriptor()