001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.snapshot; 019 020import static org.junit.jupiter.api.Assertions.fail; 021 022import java.io.IOException; 023import org.apache.hadoop.conf.Configuration; 024import org.apache.hadoop.fs.FSDataOutputStream; 025import org.apache.hadoop.fs.FileSystem; 026import org.apache.hadoop.fs.Path; 027import org.apache.hadoop.hbase.HBaseTestingUtil; 028import org.apache.hadoop.hbase.TableName; 029import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; 030import org.apache.hadoop.hbase.client.RegionInfo; 031import org.apache.hadoop.hbase.client.RegionInfoBuilder; 032import org.apache.hadoop.hbase.testclassification.MasterTests; 033import org.apache.hadoop.hbase.testclassification.MediumTests; 034import org.apache.hadoop.hbase.util.Bytes; 035import org.junit.jupiter.api.AfterEach; 036import org.junit.jupiter.api.BeforeEach; 037import org.junit.jupiter.api.Tag; 038import org.junit.jupiter.api.Test; 039import org.slf4j.Logger; 040import org.slf4j.LoggerFactory; 041 042import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; 043 044import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 045import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest; 046import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; 047import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; 048 049@Tag(MasterTests.TAG) 050@Tag(MediumTests.TAG) 051public class TestSnapshotManifest { 052 053 private final Logger LOG = LoggerFactory.getLogger(getClass()); 054 055 private static final String TABLE_NAME_STR = "testSnapshotManifest"; 056 private static final TableName TABLE_NAME = TableName.valueOf(TABLE_NAME_STR); 057 private static final int TEST_NUM_REGIONS = 16000; 058 private static final int TEST_NUM_REGIONFILES = 1000000; 059 060 private static HBaseTestingUtil TEST_UTIL; 061 private Configuration conf; 062 private FileSystem fs; 063 private Path rootDir; 064 private Path snapshotDir; 065 private SnapshotDescription snapshotDesc; 066 private SnapshotTestingUtils.SnapshotMock.SnapshotBuilder builder; 067 068 @BeforeEach 069 public void setup() throws Exception { 070 TEST_UTIL = new HBaseTestingUtil(); 071 072 rootDir = TEST_UTIL.getDataTestDir(TABLE_NAME_STR); 073 fs = TEST_UTIL.getTestFileSystem(); 074 conf = TEST_UTIL.getConfiguration(); 075 076 SnapshotTestingUtils.SnapshotMock snapshotMock = 077 new SnapshotTestingUtils.SnapshotMock(conf, fs, rootDir); 078 builder = snapshotMock.createSnapshotV2("snapshot", TABLE_NAME_STR, 0); 079 snapshotDir = builder.commit(); 080 snapshotDesc = builder.getSnapshotDescription(); 081 } 082 083 @AfterEach 084 public void tearDown() throws Exception { 085 fs.delete(rootDir, true); 086 } 087 088 @Test 089 public void testReadSnapshotManifest() throws IOException { 090 091 Path p = createDataManifest(); 092 try { 093 SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc); 094 fail("fail to test snapshot manifest because message size is too small."); 095 } catch (CorruptedSnapshotException cse) { 096 try { 097 conf.setInt(SnapshotManifest.SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY, 128 * 1024 * 1024); 098 SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc); 099 LOG.info("open snapshot manifest succeed."); 100 } catch (CorruptedSnapshotException cse2) { 101 fail("fail to take snapshot because Manifest proto-message too large."); 102 } 103 } finally { 104 fs.delete(p, false); 105 } 106 } 107 108 @Test 109 public void testReadSnapshotRegionManifest() throws IOException { 110 111 // remove datamanifest file 112 fs.delete(new Path(snapshotDir, SnapshotManifest.DATA_MANIFEST_NAME), true); 113 Path regionPath = createRegionManifest(); 114 115 try { 116 conf.setInt(SnapshotManifest.SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY, 128 * 1024 * 1024); 117 SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc); 118 } catch (CorruptedSnapshotException e) { 119 fail("fail to test snapshot manifest because region message size is too small."); 120 } finally { 121 fs.delete(regionPath, false); 122 } 123 } 124 125 private Path createDataManifest() throws IOException { 126 SnapshotDataManifest.Builder dataManifestBuilder = SnapshotDataManifest.newBuilder(); 127 byte[] startKey = null; 128 byte[] stopKey = null; 129 for (int i = 1; i <= TEST_NUM_REGIONS; i++) { 130 stopKey = Bytes.toBytes(String.format("%016d", i)); 131 RegionInfo regionInfo = RegionInfoBuilder.newBuilder(TABLE_NAME).build(); 132 SnapshotRegionManifest.Builder dataRegionManifestBuilder = 133 SnapshotRegionManifest.newBuilder(); 134 135 for (ColumnFamilyDescriptor hcd : builder.getTableDescriptor().getColumnFamilies()) { 136 SnapshotRegionManifest.FamilyFiles.Builder family = 137 SnapshotRegionManifest.FamilyFiles.newBuilder(); 138 family.setFamilyName(UnsafeByteOperations.unsafeWrap(hcd.getName())); 139 for (int j = 0; j < 100; ++j) { 140 SnapshotRegionManifest.StoreFile.Builder sfManifest = 141 SnapshotRegionManifest.StoreFile.newBuilder(); 142 sfManifest.setName(String.format("%032d", i)); 143 sfManifest.setFileSize((1 + i) * (1 + i) * 1024); 144 family.addStoreFiles(sfManifest.build()); 145 } 146 dataRegionManifestBuilder.addFamilyFiles(family.build()); 147 } 148 149 dataRegionManifestBuilder.setRegionInfo(ProtobufUtil.toRegionInfo(regionInfo)); 150 dataManifestBuilder.addRegionManifests(dataRegionManifestBuilder.build()); 151 152 startKey = stopKey; 153 } 154 155 dataManifestBuilder.setTableSchema(ProtobufUtil.toTableSchema(builder.getTableDescriptor())); 156 157 SnapshotDataManifest dataManifest = dataManifestBuilder.build(); 158 return writeDataManifest(dataManifest); 159 } 160 161 private Path createRegionManifest() throws IOException { 162 byte[] startKey = Bytes.toBytes("AAAAAA"); 163 byte[] stopKey = Bytes.toBytes("BBBBBB"); 164 RegionInfo regionInfo = RegionInfoBuilder.newBuilder(TABLE_NAME).build(); 165 SnapshotRegionManifest.Builder dataRegionManifestBuilder = SnapshotRegionManifest.newBuilder(); 166 dataRegionManifestBuilder.setRegionInfo(ProtobufUtil.toRegionInfo(regionInfo)); 167 168 for (ColumnFamilyDescriptor hcd : builder.getTableDescriptor().getColumnFamilies()) { 169 SnapshotRegionManifest.FamilyFiles.Builder family = 170 SnapshotRegionManifest.FamilyFiles.newBuilder(); 171 family.setFamilyName(UnsafeByteOperations.unsafeWrap(hcd.getName())); 172 for (int j = 0; j < TEST_NUM_REGIONFILES; ++j) { 173 SnapshotRegionManifest.StoreFile.Builder sfManifest = 174 SnapshotRegionManifest.StoreFile.newBuilder(); 175 sfManifest.setName(String.format("%064d", j)); 176 sfManifest.setFileSize(j * 1024); 177 family.addStoreFiles(sfManifest.build()); 178 } 179 dataRegionManifestBuilder.addFamilyFiles(family.build()); 180 } 181 182 SnapshotRegionManifest manifest = dataRegionManifestBuilder.build(); 183 Path regionPath = new Path(snapshotDir, 184 SnapshotManifestV2.SNAPSHOT_MANIFEST_PREFIX + regionInfo.getEncodedName()); 185 186 FSDataOutputStream stream = fs.create(regionPath); 187 try { 188 manifest.writeTo(stream); 189 } finally { 190 stream.close(); 191 } 192 193 return regionPath; 194 } 195 196 private Path writeDataManifest(final SnapshotDataManifest manifest) throws IOException { 197 Path dataRegionPath = new Path(snapshotDir, SnapshotManifest.DATA_MANIFEST_NAME); 198 FSDataOutputStream stream = fs.create(dataRegionPath); 199 try { 200 manifest.writeTo(stream); 201 } finally { 202 stream.close(); 203 } 204 205 return dataRegionPath; 206 } 207}