HBASE-17452 Failed taking snapshot - region Manifest proto-message too large (huaxiang sun)

This commit is contained in:
tedyu 2017-01-13 07:59:31 -08:00
parent 2f8ddf6fc5
commit 6d98c48699
3 changed files with 98 additions and 30 deletions

View File

@ -365,7 +365,8 @@ public final class SnapshotManifest {
ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader");
try {
v1Regions = SnapshotManifestV1.loadRegionManifests(conf, tpool, fs, workingDir, desc);
v2Regions = SnapshotManifestV2.loadRegionManifests(conf, tpool, fs, workingDir, desc);
v2Regions = SnapshotManifestV2.loadRegionManifests(conf, tpool, fs, workingDir, desc,
manifestSizeLimit);
} catch (InvalidProtocolBufferException e) {
throw new CorruptedSnapshotException("unable to parse region manifest " +
e.getMessage(), e);
@ -458,7 +459,8 @@ public final class SnapshotManifest {
ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader");
try {
v1Regions = SnapshotManifestV1.loadRegionManifests(conf, tpool, fs, workingDir, desc);
v2Regions = SnapshotManifestV2.loadRegionManifests(conf, tpool, fs, workingDir, desc);
v2Regions = SnapshotManifestV2.loadRegionManifests(conf, tpool, fs, workingDir, desc,
manifestSizeLimit);
} finally {
tpool.shutdown();
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.snapshot;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
@ -127,7 +128,7 @@ public final class SnapshotManifestV2 {
static List<SnapshotRegionManifest> loadRegionManifests(final Configuration conf,
final Executor executor,final FileSystem fs, final Path snapshotDir,
final SnapshotDescription desc) throws IOException {
final SnapshotDescription desc, final int manifestSizeLimit) throws IOException {
FileStatus[] manifestFiles = FSUtils.listStatus(fs, snapshotDir, new PathFilter() {
@Override
public boolean accept(Path path) {
@ -144,8 +145,11 @@ public final class SnapshotManifestV2 {
@Override
public SnapshotRegionManifest call() throws IOException {
FSDataInputStream stream = fs.open(st.getPath());
CodedInputStream cin = CodedInputStream.newInstance(stream);
cin.setSizeLimit(manifestSizeLimit);
try {
return SnapshotRegionManifest.parseFrom(stream);
return SnapshotRegionManifest.parseFrom(cin);
} finally {
stream.close();
}

View File

@ -51,6 +51,7 @@ public class TestSnapshotManifest {
private static final String TABLE_NAME_STR = "testSnapshotManifest";
private static final TableName TABLE_NAME = TableName.valueOf(TABLE_NAME_STR);
private static final int TEST_NUM_REGIONS = 16000;
private static final int TEST_NUM_REGIONFILES = 1000000;
private static HBaseTestingUtility TEST_UTIL;
private Configuration conf;
@ -58,6 +59,7 @@ public class TestSnapshotManifest {
private Path rootDir;
private Path snapshotDir;
private SnapshotDescription snapshotDesc;
private SnapshotTestingUtils.SnapshotMock.SnapshotBuilder builder;
@Before
public void setup() throws Exception {
@ -69,20 +71,63 @@ public class TestSnapshotManifest {
SnapshotTestingUtils.SnapshotMock snapshotMock =
new SnapshotTestingUtils.SnapshotMock(conf, fs, rootDir);
SnapshotTestingUtils.SnapshotMock.SnapshotBuilder builder =
snapshotMock.createSnapshotV2("snapshot", TABLE_NAME_STR, 0);
builder = snapshotMock.createSnapshotV2("snapshot", TABLE_NAME_STR, 0);
snapshotDir = builder.commit();
snapshotDesc = builder.getSnapshotDescription();
}
@After
public void tearDown() throws Exception {
fs.delete(rootDir,true);
}
@Test
public void testReadSnapshotManifest() throws IOException {
Path p = createDataManifest();
try {
SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc);
fail("fail to test snapshot manifest because message size is too small.");
} catch (CorruptedSnapshotException cse) {
try {
conf.setInt(SnapshotManifest.SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY, 128 * 1024 * 1024);
SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc);
LOG.info("open snapshot manifest succeed.");
} catch (CorruptedSnapshotException cse2) {
fail("fail to take snapshot because Manifest proto-message too large.");
}
} finally {
fs.delete(p, false);
}
}
@Test
public void testReadSnapshotRegionManifest() throws IOException {
// remove datamanifest file
fs.delete(new Path(snapshotDir, SnapshotManifest.DATA_MANIFEST_NAME), true);
Path regionPath = createRegionManifest();
try {
conf.setInt(SnapshotManifest.SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY, 128 * 1024 * 1024);
SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc);
} catch (CorruptedSnapshotException e) {
fail("fail to test snapshot manifest because region message size is too small.");
} finally {
fs.delete(regionPath, false);
}
}
private Path createDataManifest() throws IOException {
SnapshotDataManifest.Builder dataManifestBuilder =
SnapshotDataManifest.newBuilder();
SnapshotDataManifest.newBuilder();
byte[] startKey = null;
byte[] stopKey = null;
for (int i = 1; i <= TEST_NUM_REGIONS; i++) {
stopKey = Bytes.toBytes(String.format("%016d", i));
HRegionInfo regionInfo = new HRegionInfo(TABLE_NAME, startKey, stopKey, false);
SnapshotRegionManifest.Builder dataRegionManifestBuilder =
SnapshotRegionManifest.newBuilder();
SnapshotRegionManifest.newBuilder();
for (HColumnDescriptor hcd: builder.getTableDescriptor().getFamilies()) {
SnapshotRegionManifest.FamilyFiles.Builder family =
@ -90,7 +135,7 @@ public class TestSnapshotManifest {
family.setFamilyName(UnsafeByteOperations.unsafeWrap(hcd.getName()));
for (int j = 0; j < 100; ++j) {
SnapshotRegionManifest.StoreFile.Builder sfManifest =
SnapshotRegionManifest.StoreFile.newBuilder();
SnapshotRegionManifest.StoreFile.newBuilder();
sfManifest.setName(String.format("%032d", i));
sfManifest.setFileSize((1 + i) * (1 + i) * 1024);
family.addStoreFiles(sfManifest.build());
@ -108,37 +153,54 @@ public class TestSnapshotManifest {
.setTableSchema(ProtobufUtil.convertToTableSchema(builder.getTableDescriptor()));
SnapshotDataManifest dataManifest = dataManifestBuilder.build();
writeDataManifest(dataManifest);
return writeDataManifest(dataManifest);
}
@After
public void tearDown() throws Exception {
fs.delete(rootDir,true);
}
private Path createRegionManifest() throws IOException {
byte[] startKey = Bytes.toBytes("AAAAAA");
byte[] stopKey = Bytes.toBytes("BBBBBB");
HRegionInfo regionInfo = new HRegionInfo(TABLE_NAME, startKey, stopKey, false);
SnapshotRegionManifest.Builder dataRegionManifestBuilder = SnapshotRegionManifest.newBuilder();
dataRegionManifestBuilder.setRegionInfo(HRegionInfo.convert(regionInfo));
@Test
public void testReadSnapshotManifest() throws IOException {
try {
SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc);
fail("fail to test snapshot manifest because message size is too small.");
} catch (CorruptedSnapshotException cse) {
try {
conf.setInt(SnapshotManifest.SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY, 128 * 1024 * 1024);
SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc);
LOG.info("open snapshot manifest succeed.");
} catch (CorruptedSnapshotException cse2) {
fail("fail to take snapshot because Manifest proto-message too large.");
for (HColumnDescriptor hcd: builder.getTableDescriptor().getFamilies()) {
SnapshotRegionManifest.FamilyFiles.Builder family =
SnapshotRegionManifest.FamilyFiles.newBuilder();
family.setFamilyName(UnsafeByteOperations.unsafeWrap(hcd.getName()));
for (int j = 0; j < TEST_NUM_REGIONFILES; ++j) {
SnapshotRegionManifest.StoreFile.Builder sfManifest =
SnapshotRegionManifest.StoreFile.newBuilder();
sfManifest.setName(String.format("%064d", j));
sfManifest.setFileSize(j * 1024);
family.addStoreFiles(sfManifest.build());
}
dataRegionManifestBuilder.addFamilyFiles(family.build());
}
}
private void writeDataManifest(final SnapshotDataManifest manifest)
throws IOException {
FSDataOutputStream stream = fs.create(new Path(snapshotDir, SnapshotManifest.DATA_MANIFEST_NAME));
SnapshotRegionManifest manifest = dataRegionManifestBuilder.build();
Path regionPath = new Path(snapshotDir,
SnapshotManifestV2.SNAPSHOT_MANIFEST_PREFIX + regionInfo.getEncodedName());
FSDataOutputStream stream = fs.create(regionPath);
try {
manifest.writeTo(stream);
} finally {
stream.close();
}
return regionPath;
}
private Path writeDataManifest(final SnapshotDataManifest manifest)
throws IOException {
Path dataRegionPath = new Path(snapshotDir, SnapshotManifest.DATA_MANIFEST_NAME);
FSDataOutputStream stream = fs.create(dataRegionPath);
try {
manifest.writeTo(stream);
} finally {
stream.close();
}
return dataRegionPath;
}
}