1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.snapshot;
19
20 import com.google.protobuf.InvalidProtocolBufferException;
21 import org.apache.commons.logging.Log;
22 import org.apache.commons.logging.LogFactory;
23 import org.apache.hadoop.conf.Configuration;
24 import org.apache.hadoop.fs.FSDataOutputStream;
25 import org.apache.hadoop.fs.FileSystem;
26 import org.apache.hadoop.fs.Path;
27 import org.apache.hadoop.hbase.HColumnDescriptor;
28 import org.apache.hadoop.hbase.HRegionInfo;
29 import org.apache.hadoop.hbase.TableName;
30 import org.apache.hadoop.hbase.HBaseTestingUtility;
31 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
32 import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
33 import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
34 import org.apache.hadoop.hbase.testclassification.MasterTests;
35 import org.apache.hadoop.hbase.testclassification.SmallTests;
36 import org.apache.hadoop.hbase.util.ByteStringer;
37 import org.apache.hadoop.hbase.util.Bytes;
38 import org.junit.After;
39 import org.junit.Before;
40 import org.junit.Test;
41 import org.junit.experimental.categories.Category;
42
43 import java.io.IOException;
44
45 import static org.junit.Assert.fail;
46
47 @Category({MasterTests.class, SmallTests.class})
48 public class TestSnapshotManifest {
49 private final Log LOG = LogFactory.getLog(getClass());
50
51 private static final String TABLE_NAME_STR = "testSnapshotManifest";
52 private static final TableName TABLE_NAME = TableName.valueOf(TABLE_NAME_STR);
53 private static final int TEST_NUM_REGIONS = 16000;
54
55 private static HBaseTestingUtility TEST_UTIL;
56 private Configuration conf;
57 private FileSystem fs;
58 private Path rootDir;
59 private Path snapshotDir;
60 private SnapshotDescription snapshotDesc;
61
62 @Before
63 public void setup() throws Exception {
64 TEST_UTIL = HBaseTestingUtility.createLocalHTU();
65
66 rootDir = TEST_UTIL.getDataTestDir(TABLE_NAME_STR);
67 fs = TEST_UTIL.getTestFileSystem();
68 conf = TEST_UTIL.getConfiguration();
69
70 SnapshotTestingUtils.SnapshotMock snapshotMock =
71 new SnapshotTestingUtils.SnapshotMock(conf, fs, rootDir);
72 SnapshotTestingUtils.SnapshotMock.SnapshotBuilder builder =
73 snapshotMock.createSnapshotV2("snapshot", TABLE_NAME_STR, 0);
74 snapshotDir = builder.commit();
75 snapshotDesc = builder.getSnapshotDescription();
76
77 SnapshotDataManifest.Builder dataManifestBuilder =
78 SnapshotDataManifest.newBuilder();
79 byte[] startKey = null;
80 byte[] stopKey = null;
81 for (int i = 1; i <= TEST_NUM_REGIONS; i++) {
82 stopKey = Bytes.toBytes(String.format("%016d", i));
83 HRegionInfo regionInfo = new HRegionInfo(TABLE_NAME, startKey, stopKey, false);
84 SnapshotRegionManifest.Builder dataRegionManifestBuilder =
85 SnapshotRegionManifest.newBuilder();
86
87 for (HColumnDescriptor hcd: builder.getTableDescriptor().getFamilies()) {
88 SnapshotRegionManifest.FamilyFiles.Builder family =
89 SnapshotRegionManifest.FamilyFiles.newBuilder();
90 family.setFamilyName(ByteStringer.wrap(hcd.getName()));
91 for (int j = 0; j < 100; ++j) {
92 SnapshotRegionManifest.StoreFile.Builder sfManifest =
93 SnapshotRegionManifest.StoreFile.newBuilder();
94 sfManifest.setName(String.format("%032d", i));
95 sfManifest.setFileSize((1 + i) * (1 + i) * 1024);
96 family.addStoreFiles(sfManifest.build());
97 }
98 dataRegionManifestBuilder.addFamilyFiles(family.build());
99 }
100
101 dataRegionManifestBuilder.setRegionInfo(HRegionInfo.convert(regionInfo));
102 dataManifestBuilder.addRegionManifests(dataRegionManifestBuilder.build());
103
104 startKey = stopKey;
105 }
106
107 dataManifestBuilder.setTableSchema(builder.getTableDescriptor().convert());
108
109 SnapshotDataManifest dataManifest = dataManifestBuilder.build();
110 writeDataManifest(dataManifest);
111 }
112
113 @After
114 public void tearDown() throws Exception {
115 fs.delete(rootDir,true);
116 }
117
118 @Test
119 public void testReadSnapshotManifest() throws IOException {
120 try {
121 SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc);
122 fail("fail to test snapshot manifest because message size is too small.");
123 } catch (InvalidProtocolBufferException ipbe) {
124 try {
125 conf.setInt(SnapshotManifest.SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY, 128 * 1024 * 1024);
126 SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc);
127 LOG.info("open snapshot manifest succeed.");
128 } catch (InvalidProtocolBufferException ipbe2) {
129 fail("fail to take snapshot because Manifest proto-message too large.");
130 }
131 }
132 }
133
134 private void writeDataManifest(final SnapshotDataManifest manifest)
135 throws IOException {
136 FSDataOutputStream stream = fs.create(new Path(snapshotDir, SnapshotManifest.DATA_MANIFEST_NAME));
137 try {
138 manifest.writeTo(stream);
139 } finally {
140 stream.close();
141 }
142 }
143 }