HBASE-15430 Failed taking snapshot - Manifest proto-message too large (JunHo Cho)
This commit is contained in:
parent
9eb25bd5c9
commit
3bf0945a11
|
@ -18,6 +18,8 @@
|
|||
|
||||
package org.apache.hadoop.hbase.snapshot;
|
||||
|
||||
import com.google.protobuf.CodedInputStream;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -30,7 +32,6 @@ import java.util.concurrent.TimeUnit;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
|
@ -41,6 +42,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
|
|||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
|
@ -68,7 +70,9 @@ import org.apache.hadoop.hbase.util.Threads;
|
|||
public final class SnapshotManifest {
|
||||
private static final Log LOG = LogFactory.getLog(SnapshotManifest.class);
|
||||
|
||||
private static final String DATA_MANIFEST_NAME = "data.manifest";
|
||||
public static final String SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY = "snapshot.manifest.size.limit";
|
||||
|
||||
public static final String DATA_MANIFEST_NAME = "data.manifest";
|
||||
|
||||
private List<SnapshotRegionManifest> regionManifests;
|
||||
private SnapshotDescription desc;
|
||||
|
@ -78,6 +82,7 @@ public final class SnapshotManifest {
|
|||
private final Configuration conf;
|
||||
private final Path workingDir;
|
||||
private final FileSystem fs;
|
||||
private int manifestSizeLimit;
|
||||
|
||||
private SnapshotManifest(final Configuration conf, final FileSystem fs,
|
||||
final Path workingDir, final SnapshotDescription desc,
|
||||
|
@ -87,6 +92,8 @@ public final class SnapshotManifest {
|
|||
this.workingDir = workingDir;
|
||||
this.conf = conf;
|
||||
this.fs = fs;
|
||||
|
||||
this.manifestSizeLimit = conf.getInt(SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY, 64 * 1024 * 1024);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -512,7 +519,9 @@ public final class SnapshotManifest {
|
|||
FSDataInputStream in = null;
|
||||
try {
|
||||
in = fs.open(new Path(workingDir, DATA_MANIFEST_NAME));
|
||||
return SnapshotDataManifest.parseFrom(in);
|
||||
CodedInputStream cin = CodedInputStream.newInstance(in);
|
||||
cin.setSizeLimit(manifestSizeLimit);
|
||||
return SnapshotDataManifest.parseFrom(cin);
|
||||
} catch (FileNotFoundException e) {
|
||||
return null;
|
||||
} finally {
|
||||
|
|
|
@ -550,15 +550,30 @@ public class SnapshotTestingUtils {
|
|||
return createSnapshot(snapshotName, tableName, SnapshotManifestV1.DESCRIPTOR_VERSION);
|
||||
}
|
||||
|
||||
public SnapshotBuilder createSnapshotV1(final String snapshotName, final String tableName,
|
||||
final int numRegions) throws IOException {
|
||||
return createSnapshot(snapshotName, tableName, numRegions, SnapshotManifestV1.DESCRIPTOR_VERSION);
|
||||
}
|
||||
|
||||
public SnapshotBuilder createSnapshotV2(final String snapshotName, final String tableName)
|
||||
throws IOException {
|
||||
return createSnapshot(snapshotName, tableName, SnapshotManifestV2.DESCRIPTOR_VERSION);
|
||||
}
|
||||
|
||||
public SnapshotBuilder createSnapshotV2(final String snapshotName, final String tableName,
|
||||
final int numRegions) throws IOException {
|
||||
return createSnapshot(snapshotName, tableName, numRegions, SnapshotManifestV2.DESCRIPTOR_VERSION);
|
||||
}
|
||||
|
||||
private SnapshotBuilder createSnapshot(final String snapshotName, final String tableName,
|
||||
final int version) throws IOException {
|
||||
return createSnapshot(snapshotName, tableName, TEST_NUM_REGIONS, version);
|
||||
}
|
||||
|
||||
private SnapshotBuilder createSnapshot(final String snapshotName, final String tableName,
|
||||
final int numRegions, final int version) throws IOException {
|
||||
HTableDescriptor htd = createHtd(tableName);
|
||||
RegionData[] regions = createTable(htd, TEST_NUM_REGIONS);
|
||||
RegionData[] regions = createTable(htd, numRegions);
|
||||
|
||||
SnapshotDescription desc = SnapshotDescription.newBuilder()
|
||||
.setTable(htd.getNameAsString())
|
||||
|
|
|
@ -0,0 +1,143 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.snapshot;
|
||||
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
@Category({MasterTests.class, SmallTests.class})
|
||||
public class TestSnapshotManifest {
|
||||
private final Log LOG = LogFactory.getLog(getClass());
|
||||
|
||||
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 HBaseTestingUtility TEST_UTIL;
|
||||
private Configuration conf;
|
||||
private FileSystem fs;
|
||||
private Path rootDir;
|
||||
private Path snapshotDir;
|
||||
private SnapshotDescription snapshotDesc;
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
TEST_UTIL = HBaseTestingUtility.createLocalHTU();
|
||||
|
||||
rootDir = TEST_UTIL.getDataTestDir(TABLE_NAME_STR);
|
||||
fs = TEST_UTIL.getTestFileSystem();
|
||||
conf = TEST_UTIL.getConfiguration();
|
||||
|
||||
SnapshotTestingUtils.SnapshotMock snapshotMock =
|
||||
new SnapshotTestingUtils.SnapshotMock(conf, fs, rootDir);
|
||||
SnapshotTestingUtils.SnapshotMock.SnapshotBuilder builder =
|
||||
snapshotMock.createSnapshotV2("snapshot", TABLE_NAME_STR, 0);
|
||||
snapshotDir = builder.commit();
|
||||
snapshotDesc = builder.getSnapshotDescription();
|
||||
|
||||
SnapshotDataManifest.Builder dataManifestBuilder =
|
||||
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();
|
||||
|
||||
for (HColumnDescriptor hcd: builder.getTableDescriptor().getFamilies()) {
|
||||
SnapshotRegionManifest.FamilyFiles.Builder family =
|
||||
SnapshotRegionManifest.FamilyFiles.newBuilder();
|
||||
family.setFamilyName(ByteStringer.wrap(hcd.getName()));
|
||||
for (int j = 0; j < 100; ++j) {
|
||||
SnapshotRegionManifest.StoreFile.Builder sfManifest =
|
||||
SnapshotRegionManifest.StoreFile.newBuilder();
|
||||
sfManifest.setName(String.format("%032d", i));
|
||||
sfManifest.setFileSize((1 + i) * (1 + i) * 1024);
|
||||
family.addStoreFiles(sfManifest.build());
|
||||
}
|
||||
dataRegionManifestBuilder.addFamilyFiles(family.build());
|
||||
}
|
||||
|
||||
dataRegionManifestBuilder.setRegionInfo(HRegionInfo.convert(regionInfo));
|
||||
dataManifestBuilder.addRegionManifests(dataRegionManifestBuilder.build());
|
||||
|
||||
startKey = stopKey;
|
||||
}
|
||||
|
||||
dataManifestBuilder.setTableSchema(builder.getTableDescriptor().convert());
|
||||
|
||||
SnapshotDataManifest dataManifest = dataManifestBuilder.build();
|
||||
writeDataManifest(dataManifest);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
fs.delete(rootDir,true);
|
||||
}
|
||||
|
||||
@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 (InvalidProtocolBufferException ipbe) {
|
||||
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 (InvalidProtocolBufferException ipbe2) {
|
||||
fail("fail to take snapshot because Manifest proto-message too large.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void writeDataManifest(final SnapshotDataManifest manifest)
|
||||
throws IOException {
|
||||
FSDataOutputStream stream = fs.create(new Path(snapshotDir, SnapshotManifest.DATA_MANIFEST_NAME));
|
||||
try {
|
||||
manifest.writeTo(stream);
|
||||
} finally {
|
||||
stream.close();
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue