HBASE-8760 possible loss of data in snapshot taken after region split

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1515965 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
mbertozzi 2013-08-20 20:48:44 +00:00
parent a07a9dbbe6
commit 422e98350f
15 changed files with 315 additions and 95 deletions

View File

@ -237,6 +237,23 @@ public class MetaEditor {
}
}
/**
* Adds a (single) META row for the specified new region and its daughters. Note that this does
* not add its daughter's as different rows, but adds information about the daughters
* in the same row as the parent. Use
* {@link #splitRegion(CatalogTracker, HRegionInfo, HRegionInfo, HRegionInfo, ServerName)}
* if you want to do that.
* @param catalogTracker CatalogTracker on whose back we will ride the edit.
* @param regionInfo region information
* @param splitA first split daughter of the parent regionInfo
* @param splitB second split daughter of the parent regionInfo
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(CatalogTracker catalogTracker, HRegionInfo regionInfo,
HRegionInfo splitA, HRegionInfo splitB) throws IOException {
addRegionToMeta(MetaReader.getMetaHTable(catalogTracker), regionInfo, splitA, splitB);
}
/**
* Adds a META row for each of the specified new regions.
* @param catalogTracker CatalogTracker

View File

@ -113,6 +113,37 @@ public class HFileLink extends FileLink {
setLocations(originPath, tempPath, archivePath);
}
/**
* Create an HFileLink relative path for the table/region/family/hfile location
* @param table Table name
* @param region Region Name
* @param family Family Name
* @param hfile HFile Name
* @return the relative Path to open the specified table/region/family/hfile link
*/
public static Path createPath(final TableName table, final String region,
final String family, final String hfile) {
if (HFileLink.isHFileLink(hfile)) {
return new Path(family, hfile);
}
return new Path(family, HFileLink.createHFileLinkName(table, region, hfile));
}
/**
* Create an HFileLink instance from table/region/family/hfile location
* @param conf {@link Configuration} from which to extract specific archive locations
* @param table Table name
* @param region Region Name
* @param family Family Name
* @param hfile HFile Name
* @return Link to the file with the specified table/region/family/hfile location
* @throws IOException on unexpected error.
*/
public static HFileLink create(final Configuration conf, final TableName table,
final String region, final String family, final String hfile) throws IOException {
return new HFileLink(conf, createPath(table, region, family, hfile));
}
/**
* @return the origin path of the hfile.
*/
@ -214,6 +245,15 @@ public class HFileLink extends FileLink {
return(TableName.valueOf(m.group(1), m.group(2)));
}
/**
* Returns true if the HFileLink exists
*/
public boolean exists(final FileSystem fs) throws IOException {
return fs.exists(this.originPath) ||
fs.exists(this.tempPath) ||
fs.exists(this.archivePath);
}
/**
* Create a new HFileLink name
*

View File

@ -220,7 +220,7 @@ public class CreateTableHandler extends EventHandler {
if (regionInfos != null && regionInfos.size() > 0) {
// 4. Add regions to META
MetaEditor.addRegionsToMeta(this.catalogTracker, regionInfos);
addRegionsToMeta(this.catalogTracker, regionInfos);
// 5. Trigger immediate assignment of the regions in round-robin fashion
try {
@ -265,4 +265,12 @@ public class CreateTableHandler extends EventHandler {
return ModifyRegionUtils.createRegions(conf, tableRootDir,
hTableDescriptor, newRegions, null);
}
/**
* Add the specified set of regions to the META table.
*/
protected void addRegionsToMeta(final CatalogTracker ct, final List<HRegionInfo> regionInfos)
throws IOException {
MetaEditor.addRegionsToMeta(this.catalogTracker, regionInfos);
}
}

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.master.MasterServices;
@ -67,6 +68,8 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
private final MetricsSnapshot metricsSnapshot = new MetricsSnapshot();
private final MonitoredTask status;
private RestoreSnapshotHelper.RestoreMetaChanges metaChanges;
private volatile boolean stopped = false;
public CloneSnapshotHandler(final MasterServices masterServices,
@ -106,7 +109,7 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir);
RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(conf, fs,
snapshot, snapshotDir, hTableDescriptor, tableRootDir, monitor, status);
RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreHdfsRegions();
metaChanges = restoreHelper.restoreHdfsRegions();
// Clone operation should not have stuff to restore or remove
Preconditions.checkArgument(!metaChanges.hasRegionsToRestore(),
@ -133,6 +136,13 @@ public class CloneSnapshotHandler extends CreateTableHandler implements Snapshot
}
}
@Override
protected void addRegionsToMeta(final CatalogTracker ct, final List<HRegionInfo> regionInfos)
throws IOException {
super.addRegionsToMeta(ct, regionInfos);
metaChanges.updateMetaParentRegions(ct, regionInfos);
}
@Override
protected void completed(final Throwable exception) {
this.stopped = true;

View File

@ -36,11 +36,7 @@ import org.apache.hadoop.hbase.master.MetricsMaster;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.CopyRecoveredEditsTask;
import org.apache.hadoop.hbase.snapshot.ReferenceRegionHFilesTask;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.TableInfoCopyTask;
import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
@ -89,11 +85,9 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
// 1. get all the regions hosting this table.
// extract each pair to separate lists
Set<String> serverNames = new HashSet<String>();
Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
regions.add(p.getFirst());
serverNames.add(p.getSecond().toString());
}
// 2. for each region, write all the info to disk
@ -102,26 +96,7 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler {
LOG.info(msg);
status.setStatus(msg);
for (HRegionInfo regionInfo : regions) {
// 2.1 copy the regionInfo files to the snapshot
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs,
snapshotDir, regionInfo);
// check for error for each region
monitor.rethrowException();
// 2.2 for each region, copy over its recovered.edits directory
Path regionDir = HRegion.getRegionDir(rootDir, regionInfo);
Path snapshotRegionDir = regionFs.getRegionDir();
new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir, snapshotRegionDir).call();
monitor.rethrowException();
status.setStatus("Completed copying recovered edits for offline snapshot of table: "
+ snapshotTable);
// 2.3 reference all the files in the region
new ReferenceRegionHFilesTask(snapshot, monitor, regionDir, fs, snapshotRegionDir).call();
monitor.rethrowException();
status.setStatus("Completed referencing HFiles for offline snapshot of table: " +
snapshotTable);
snapshotDisabledRegion(regionInfo);
}
// 3. write the table info to disk

View File

@ -25,6 +25,7 @@ import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.errorhandling.ForeignException;
@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.procedure.Procedure;
import org.apache.hadoop.hbase.procedure.ProcedureCoordinator;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.util.Pair;
import com.google.common.collect.Lists;
@ -70,10 +72,14 @@ public class EnabledTableSnapshotHandler extends TakeSnapshotHandler {
*/
@Override
protected void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regions)
throws HBaseSnapshotException {
throws HBaseSnapshotException, IOException {
Set<String> regionServers = new HashSet<String>(regions.size());
for (Pair<HRegionInfo, ServerName> region : regions) {
regionServers.add(region.getSecond().toString());
if (region != null && region.getFirst() != null && region.getSecond() != null) {
HRegionInfo hri = region.getFirst();
if (hri.isOffline() && (hri.isSplit() || hri.isSplitParent())) continue;
regionServers.add(region.getSecond().toString());
}
}
// start the snapshot on the RS
@ -90,7 +96,19 @@ public class EnabledTableSnapshotHandler extends TakeSnapshotHandler {
// wait for the snapshot to complete. A timer thread is kicked off that should cancel this
// if it takes too long.
proc.waitForCompleted();
LOG.info("Done waiting - snapshot for " + this.snapshot.getName() + " finished!");
LOG.info("Done waiting - online snapshot for " + this.snapshot.getName());
// Take the offline regions as disabled
Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir);
for (Pair<HRegionInfo, ServerName> region : regions) {
HRegionInfo regionInfo = region.getFirst();
if (regionInfo.isOffline() && (regionInfo.isSplit() || regionInfo.isSplitParent())) {
if (!fs.exists(new Path(snapshotDir, regionInfo.getEncodedName()))) {
LOG.info("Take disabled snapshot of offline region=" + regionInfo);
snapshotDisabledRegion(regionInfo);
}
}
}
} catch (InterruptedException e) {
ForeignException ee =
new ForeignException("Interrupted while waiting for snapshot to finish", e);

View File

@ -21,6 +21,8 @@ import java.io.IOException;
import java.util.List;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileStatus;
@ -31,16 +33,20 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.FSVisitor;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
/**
@ -74,6 +80,7 @@ import org.apache.hadoop.hbase.util.HFileArchiveUtil;
@InterfaceAudience.Private
@InterfaceStability.Unstable
public final class MasterSnapshotVerifier {
private static final Log LOG = LogFactory.getLog(MasterSnapshotVerifier.class);
private SnapshotDescription snapshot;
private FileSystem fs;
@ -141,10 +148,28 @@ public final class MasterSnapshotVerifier {
private void verifyRegions(Path snapshotDir) throws IOException {
List<HRegionInfo> regions = MetaReader.getTableRegions(this.services.getCatalogTracker(),
tableName);
Set<String> snapshotRegions = SnapshotReferenceUtil.getSnapshotRegionNames(fs, snapshotDir);
if (snapshotRegions == null) {
String msg = "Snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot) + " looks empty";
LOG.error(msg);
throw new CorruptedSnapshotException(msg);
}
if (snapshotRegions.size() != regions.size()) {
String msg = "Regions moved during the snapshot '" +
ClientSnapshotDescriptionUtils.toString(snapshot) + "'. expected=" +
regions.size() + " snapshotted=" + snapshotRegions.size();
LOG.error(msg);
throw new CorruptedSnapshotException(msg);
}
for (HRegionInfo region : regions) {
// if offline split parent, skip it
if (region.isOffline() && (region.isSplit() || region.isSplitParent())) {
continue;
if (!snapshotRegions.contains(region.getEncodedName())) {
// could happen due to a move or split race.
String msg = "No region directory found for region:" + region;
LOG.error(msg);
throw new CorruptedSnapshotException(msg, snapshot);
}
verifyRegion(fs, snapshotDir, region);
@ -157,14 +182,11 @@ public final class MasterSnapshotVerifier {
* @param snapshotDir snapshot directory to check
* @param region the region to check
*/
private void verifyRegion(FileSystem fs, Path snapshotDir, HRegionInfo region) throws IOException {
private void verifyRegion(final FileSystem fs, final Path snapshotDir, final HRegionInfo region)
throws IOException {
// make sure we have region in the snapshot
Path regionDir = new Path(snapshotDir, region.getEncodedName());
if (!fs.exists(regionDir)) {
// could happen due to a move or split race.
throw new CorruptedSnapshotException("No region directory found for region:" + region,
snapshot);
}
// make sure we have the region info in the snapshot
Path regionInfo = new Path(regionDir, HRegionFileSystem.REGION_INFO_FILE);
// make sure the file exists
@ -181,46 +203,44 @@ public final class MasterSnapshotVerifier {
// make sure we have the expected recovered edits files
TakeSnapshotUtils.verifyRecoveredEdits(fs, snapshotDir, found, snapshot);
// check for the existance of each hfile
PathFilter familiesDirs = new FSUtils.FamilyDirFilter(fs);
FileStatus[] columnFamilies = FSUtils.listStatus(fs, regionDir, familiesDirs);
// should we do some checking here to make sure the cfs are correct?
if (columnFamilies == null) return;
// setup the suffixes for the snapshot directories
Path tableNameSuffix = FSUtils.getTableDir(new Path("./"), tableName);
Path regionNameSuffix = new Path(tableNameSuffix, region.getEncodedName());
// get the potential real paths
Path archivedRegion = new Path(HFileArchiveUtil.getArchivePath(services.getConfiguration()),
regionNameSuffix);
Path realRegion = new Path(rootDir, regionNameSuffix);
// loop through each cf and check we can find each of the hfiles
for (FileStatus cf : columnFamilies) {
FileStatus[] hfiles = FSUtils.listStatus(fs, cf.getPath(), null);
// should we check if there should be hfiles?
if (hfiles == null || hfiles.length == 0) continue;
Path realCfDir = new Path(realRegion, cf.getPath().getName());
Path archivedCfDir = new Path(archivedRegion, cf.getPath().getName());
for (FileStatus hfile : hfiles) {
// make sure the name is correct
if (!StoreFileInfo.validateStoreFileName(hfile.getPath().getName())) {
throw new CorruptedSnapshotException("HFile: " + hfile.getPath()
+ " is not a valid hfile name.", snapshot);
}
// check to see if hfile is present in the real table
String fileName = hfile.getPath().getName();
Path file = new Path(realCfDir, fileName);
Path archived = new Path(archivedCfDir, fileName);
if (!fs.exists(file) && !fs.exists(archived)) {
throw new CorruptedSnapshotException("Can't find hfile: " + hfile.getPath()
+ " in the real (" + realCfDir + ") or archive (" + archivedCfDir
+ ") directory for the primary table.", snapshot);
}
// make sure we have all the expected store files
SnapshotReferenceUtil.visitRegionStoreFiles(fs, regionDir, new FSVisitor.StoreFileVisitor() {
public void storeFile(final String regionNameSuffix, final String family,
final String hfileName) throws IOException {
verifyStoreFile(snapshotDir, region, family, hfileName);
}
});
}
private void verifyStoreFile(final Path snapshotDir, final HRegionInfo regionInfo,
final String family, final String fileName) throws IOException {
Path refPath = null;
if (StoreFileInfo.isReference(fileName)) {
// If is a reference file check if the parent file is present in the snapshot
Path snapshotHFilePath = new Path(new Path(
new Path(snapshotDir, regionInfo.getEncodedName()), family), fileName);
refPath = StoreFileInfo.getReferredToFile(snapshotHFilePath);
if (!fs.exists(refPath)) {
throw new CorruptedSnapshotException("Missing parent hfile for: " + fileName, snapshot);
}
}
Path linkPath;
if (refPath != null && HFileLink.isHFileLink(refPath)) {
linkPath = new Path(family, refPath.getName());
} else if (HFileLink.isHFileLink(fileName)) {
linkPath = new Path(family, fileName);
} else {
linkPath = new Path(family, HFileLink.createHFileLinkName(tableName,
regionInfo.getEncodedName(), fileName));
}
// check if the linked file exists (in the archive, or in the table dir)
HFileLink link = new HFileLink(services.getConfiguration(), linkPath);
if (!link.exists(fs)) {
throw new CorruptedSnapshotException("Can't find hfile: " + fileName
+ " in the real (" + link.getOriginPath() + ") or archive (" + link.getArchivePath()
+ ") directory for the primary table.", snapshot);
}
}
}

View File

@ -128,6 +128,7 @@ public class RestoreSnapshotHandler extends TableEventHandler implements Snapsho
if (metaChanges.hasRegionsToRestore()) hris.addAll(metaChanges.getRegionsToRestore());
List<HRegionInfo> hrisToRemove = metaChanges.getRegionsToRemove();
MetaEditor.mutateRegions(catalogTracker, hrisToRemove, hris);
metaChanges.updateMetaParentRegions(catalogTracker, hris);
// At this point the restore is complete. Next step is enabling the table.
LOG.info("Restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot) +

View File

@ -48,7 +48,11 @@ import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.CopyRecoveredEditsTask;
import org.apache.hadoop.hbase.snapshot.ReferenceRegionHFilesTask;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.TableInfoCopyTask;
@ -164,7 +168,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
List<Pair<HRegionInfo, ServerName>> regionsAndLocations =
MetaReader.getTableRegionsAndLocations(this.server.getCatalogTracker(),
snapshotTable, true);
snapshotTable, false);
// run the snapshot
snapshotRegions(regionsAndLocations);
@ -173,7 +177,11 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
// extract each pair to separate lists
Set<String> serverNames = new HashSet<String>();
for (Pair<HRegionInfo, ServerName> p : regionsAndLocations) {
serverNames.add(p.getSecond().toString());
if (p != null && p.getFirst() != null && p.getSecond() != null) {
HRegionInfo hri = p.getFirst();
if (hri.isOffline() && (hri.isSplit() || hri.isSplitParent())) continue;
serverNames.add(p.getSecond().toString());
}
}
// verify the snapshot is valid
@ -247,6 +255,33 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
protected abstract void snapshotRegions(List<Pair<HRegionInfo, ServerName>> regions)
throws IOException, KeeperException;
/**
* Take a snapshot of the specified disabled region
*/
protected void snapshotDisabledRegion(final HRegionInfo regionInfo)
throws IOException {
// 2 copy the regionInfo files to the snapshot
HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs,
workingDir, regionInfo);
// check for error for each region
monitor.rethrowException();
// 2 for each region, copy over its recovered.edits directory
Path regionDir = HRegion.getRegionDir(rootDir, regionInfo);
Path snapshotRegionDir = regionFs.getRegionDir();
new CopyRecoveredEditsTask(snapshot, monitor, fs, regionDir, snapshotRegionDir).call();
monitor.rethrowException();
status.setStatus("Completed copying recovered edits for offline snapshot of table: "
+ snapshotTable);
// 2 reference all the files in the region
new ReferenceRegionHFilesTask(snapshot, monitor, regionDir, fs, snapshotRegionDir).call();
monitor.rethrowException();
status.setStatus("Completed referencing HFiles for offline snapshot of table: " +
snapshotTable);
}
@Override
public void cancel(String why) {
if (finished) return;

View File

@ -2470,14 +2470,22 @@ public class HRegion implements HeapSize { // , Writable{
if (exnSnare != null) {
exnSnare.rethrowException();
}
Path file = storeFiles.get(i).getPath();
// create "reference" to this store file. It is intentionally an empty file -- all
// necessary infomration is captured by its fs location and filename. This allows us to
// only figure out what needs to be done via a single nn operation (instead of having to
// open and read the files as well).
StoreFile storeFile = storeFiles.get(i);
Path file = storeFile.getPath();
LOG.debug("Creating reference for file (" + (i+1) + "/" + sz + ") : " + file);
Path referenceFile = new Path(dstStoreDir, file.getName());
boolean success = fs.getFileSystem().createNewFile(referenceFile);
boolean success = true;
if (storeFile.isReference()) {
// write the Reference object to the snapshot
storeFile.getFileInfo().getReference().write(fs.getFileSystem(), referenceFile);
} else {
// create "reference" to this store file. It is intentionally an empty file -- all
// necessary information is captured by its fs location and filename. This allows us to
// only figure out what needs to be done via a single nn operation (instead of having to
// open and read the files as well).
success = fs.getFileSystem().createNewFile(referenceFile);
}
if (!success) {
throw new IOException("Failed to create reference file:" + referenceFile);
}

View File

@ -235,6 +235,14 @@ public class StoreFile {
this.modificationTimeStamp = fileInfo.getModificationTime();
}
/**
* @return the StoreFile object associated to this StoreFile.
* null if the StoreFile is not a reference.
*/
StoreFileInfo getFileInfo() {
return this.fileInfo;
}
/**
* @return Path or null if this StoreFile was made with a Stream.
*/

View File

@ -126,6 +126,14 @@ public class StoreFileInfo {
}
}
/*
* @return the Reference object associated to this StoreFileInfo.
* null if the StoreFile is not a reference.
*/
Reference getReference() {
return this.reference;
}
/** @return True if the store file is a Reference */
public boolean isReference() {
return this.reference != null;

View File

@ -385,7 +385,7 @@ public final class ExportSnapshot extends Configured implements Tool {
new SnapshotReferenceUtil.FileVisitor() {
public void storeFile (final String region, final String family, final String hfile)
throws IOException {
Path path = new Path(family, HFileLink.createHFileLinkName(table, region, hfile));
Path path = HFileLink.createPath(table, region, family, hfile);
long size = new HFileLink(conf, path).getFileStatus(fs).getLen();
files.add(new Pair<Path, Long>(path, size));
}

View File

@ -42,6 +42,8 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.io.HFileLink;
@ -55,6 +57,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.FSVisitor;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.io.IOUtils;
/**
@ -105,6 +108,9 @@ public class RestoreSnapshotHelper {
private final Map<byte[], byte[]> regionsMap =
new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
private final Map<String, Pair<String, String> > parentsMap =
new HashMap<String, Pair<String, String> >();
private final ForeignExceptionDispatcher monitor;
private final MonitoredTask status;
@ -152,7 +158,7 @@ public class RestoreSnapshotHelper {
return null;
}
RestoreMetaChanges metaChanges = new RestoreMetaChanges();
RestoreMetaChanges metaChanges = new RestoreMetaChanges(parentsMap);
// Identify which region are still available and which not.
// NOTE: we rely upon the region name as: "table name, start key, end key"
@ -216,10 +222,16 @@ public class RestoreSnapshotHelper {
* Describe the set of operations needed to update META after restore.
*/
public static class RestoreMetaChanges {
private final Map<String, Pair<String, String> > parentsMap;
private List<HRegionInfo> regionsToRestore = null;
private List<HRegionInfo> regionsToRemove = null;
private List<HRegionInfo> regionsToAdd = null;
RestoreMetaChanges(final Map<String, Pair<String, String> > parentsMap) {
this.parentsMap = parentsMap;
}
/**
* @return true if there're new regions
*/
@ -291,6 +303,43 @@ public class RestoreSnapshotHelper {
}
regionsToRestore.add(hri);
}
public void updateMetaParentRegions(final CatalogTracker catalogTracker,
final List<HRegionInfo> regionInfos) throws IOException {
if (regionInfos == null || parentsMap.isEmpty()) return;
// Extract region names and offlined regions
Map<String, HRegionInfo> regionsByName = new HashMap<String, HRegionInfo>(regionInfos.size());
List<HRegionInfo> parentRegions = new LinkedList();
for (HRegionInfo regionInfo: regionInfos) {
if (regionInfo.isSplitParent()) {
parentRegions.add(regionInfo);
} else {
regionsByName.put(regionInfo.getEncodedName(), regionInfo);
}
}
// Update Offline parents
for (HRegionInfo regionInfo: parentRegions) {
Pair<String, String> daughters = parentsMap.get(regionInfo.getEncodedName());
if (daughters == null) {
// The snapshot contains an unreferenced region.
// It will be removed by the CatalogJanitor.
LOG.warn("Skip update of unreferenced offline parent: " + regionInfo);
continue;
}
// One side of the split is already compacted
if (daughters.getSecond() == null) {
daughters.setSecond(daughters.getFirst());
}
LOG.debug("Update splits parent " + regionInfo.getEncodedName() + " -> " + daughters);
MetaEditor.addRegionToMeta(catalogTracker, regionInfo,
regionsByName.get(daughters.getFirst()),
regionsByName.get(daughters.getSecond()));
}
}
}
/**
@ -514,18 +563,40 @@ public class RestoreSnapshotHelper {
if (clonedRegionName == null) clonedRegionName = snapshotRegionName;
// The output file should be a reference link table=snapshotRegion-fileName.clonedRegionName
Path linkPath = null;
String refLink = fileName;
if (!HFileLink.isHFileLink(fileName)) {
refLink = HFileLink.createHFileLinkName(snapshotTable, snapshotRegionName, fileName);
linkPath = new Path(familyDir,
HFileLink.createHFileLinkName(snapshotTable, regionInfo.getEncodedName(), hfileName));
}
Path outPath = new Path(familyDir, refLink + '.' + clonedRegionName);
// Create the new reference
Path linkPath = new Path(familyDir,
HFileLink.createHFileLinkName(snapshotTable, regionInfo.getEncodedName(), hfileName));
InputStream in = new HFileLink(conf, linkPath).open(fs);
InputStream in;
if (linkPath != null) {
in = new HFileLink(conf, linkPath).open(fs);
} else {
linkPath = new Path(new Path(HRegion.getRegionDir(snapshotDir, regionInfo.getEncodedName()),
familyDir.getName()), hfileName);
in = fs.open(linkPath);
}
OutputStream out = fs.create(outPath);
IOUtils.copyBytes(in, out, conf);
// Add the daughter region to the map
String regionName = Bytes.toString(regionsMap.get(regionInfo.getEncodedNameAsBytes()));
LOG.debug("Restore reference " + regionName + " to " + clonedRegionName);
synchronized (parentsMap) {
Pair<String, String> daughters = parentsMap.get(clonedRegionName);
if (daughters == null) {
daughters = new Pair<String, String>(regionName, null);
parentsMap.put(clonedRegionName, daughters);
} else if (!regionName.equals(daughters.getFirst())) {
daughters.setSecond(regionName);
}
}
}
/**
@ -537,9 +608,11 @@ public class RestoreSnapshotHelper {
* @return the new HRegion instance
*/
public HRegionInfo cloneRegionInfo(final HRegionInfo snapshotRegionInfo) {
return new HRegionInfo(tableDesc.getTableName(),
HRegionInfo regionInfo = new HRegionInfo(tableDesc.getTableName(),
snapshotRegionInfo.getStartKey(), snapshotRegionInfo.getEndKey(),
snapshotRegionInfo.isSplit(), snapshotRegionInfo.getRegionId());
regionInfo.setOffline(snapshotRegionInfo.isOffline());
return regionInfo;
}
/**

View File

@ -187,8 +187,7 @@ public final class SnapshotInfo extends Configured implements Tool {
FileInfo addStoreFile(final String region, final String family, final String hfile)
throws IOException {
TableName table = snapshotTable;
Path path = new Path(family, HFileLink.createHFileLinkName(table, region, hfile));
HFileLink link = new HFileLink(conf, path);
HFileLink link = HFileLink.create(conf, table, region, family, hfile);
boolean inArchive = false;
long size = -1;
try {