All Downloads are FREE. Search and download functionalities are using the official Maven repository.

org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper Maven / Gradle / Ivy

There is a newer version: 3.0.0-beta-1
Show newest version
/**
 * 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 java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.ThreadPoolExecutor;

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.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.Reference;
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.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.io.IOUtils;

/**
 * Helper to Restore/Clone a Snapshot
 *
 * 

The helper assumes that a table is already created, and by calling restore() * the content present in the snapshot will be restored as the new content of the table. * *

Clone from Snapshot: If the target table is empty, the restore operation * is just a "clone operation", where the only operations are: *

    *
  • for each region in the snapshot create a new region * (note that the region will have a different name, since the encoding contains the table name) *
  • for each file in the region create a new HFileLink to point to the original file. *
  • restore the logs, if any *
* *

Restore from Snapshot: *

    *
  • for each region in the table verify which are available in the snapshot and which are not *
      *
    • if the region is not present in the snapshot, remove it. *
    • if the region is present in the snapshot *
        *
      • for each file in the table region verify which are available in the snapshot *
          *
        • if the hfile is not present in the snapshot, remove it *
        • if the hfile is present, keep it (nothing to do) *
        *
      • for each file in the snapshot region but not in the table *
          *
        • create a new HFileLink that point to the original file *
        *
      *
    *
  • for each region in the snapshot not present in the current table state *
      *
    • create a new region and for each file in the region create a new HFileLink * (This is the same as the clone operation) *
    *
  • restore the logs, if any *
*/ @InterfaceAudience.Private public class RestoreSnapshotHelper { private static final Log LOG = LogFactory.getLog(RestoreSnapshotHelper.class); private final Map regionsMap = new TreeMap(Bytes.BYTES_COMPARATOR); private final Map > parentsMap = new HashMap >(); private final ForeignExceptionDispatcher monitor; private final MonitoredTask status; private final SnapshotManifest snapshotManifest; private final SnapshotDescription snapshotDesc; private final TableName snapshotTable; private final HTableDescriptor tableDesc; private final Path rootDir; private final Path tableDir; private final Configuration conf; private final FileSystem fs; private final boolean createBackRefs; public RestoreSnapshotHelper(final Configuration conf, final FileSystem fs, final SnapshotManifest manifest, final HTableDescriptor tableDescriptor, final Path rootDir, final ForeignExceptionDispatcher monitor, final MonitoredTask status) { this(conf, fs, manifest, tableDescriptor, rootDir, monitor, status, true); } public RestoreSnapshotHelper(final Configuration conf, final FileSystem fs, final SnapshotManifest manifest, final HTableDescriptor tableDescriptor, final Path rootDir, final ForeignExceptionDispatcher monitor, final MonitoredTask status, final boolean createBackRefs) { this.fs = fs; this.conf = conf; this.snapshotManifest = manifest; this.snapshotDesc = manifest.getSnapshotDescription(); this.snapshotTable = TableName.valueOf(snapshotDesc.getTable()); this.tableDesc = tableDescriptor; this.rootDir = rootDir; this.tableDir = FSUtils.getTableDir(rootDir, tableDesc.getTableName()); this.monitor = monitor; this.status = status; this.createBackRefs = createBackRefs; } /** * Restore the on-disk table to a specified snapshot state. * @return the set of regions touched by the restore operation */ public RestoreMetaChanges restoreHdfsRegions() throws IOException { ThreadPoolExecutor exec = SnapshotManifest.createExecutor(conf, "RestoreSnapshot"); try { return restoreHdfsRegions(exec); } finally { exec.shutdown(); } } private RestoreMetaChanges restoreHdfsRegions(final ThreadPoolExecutor exec) throws IOException { LOG.debug("starting restore"); Map regionManifests = snapshotManifest.getRegionManifestsMap(); if (regionManifests == null) { LOG.warn("Nothing to restore. Snapshot " + snapshotDesc + " looks empty"); return null; } RestoreMetaChanges metaChanges = new RestoreMetaChanges(tableDesc, parentsMap); // Take a copy of the manifest.keySet() since we are going to modify // this instance, by removing the regions already present in the restore dir. Set regionNames = new HashSet(regionManifests.keySet()); // Identify which region are still available and which not. // NOTE: we rely upon the region name as: "table name, start key, end key" List tableRegions = getTableRegions(); if (tableRegions != null) { monitor.rethrowException(); for (HRegionInfo regionInfo: tableRegions) { String regionName = regionInfo.getEncodedName(); if (regionNames.contains(regionName)) { LOG.info("region to restore: " + regionName); regionNames.remove(regionName); // Add the regionInfo from snapshot manifest, so that will not miss parent region details metaChanges.addRegionToRestore( HRegionInfo.convert(regionManifests.get(regionName).getRegionInfo())); } else { LOG.info("region to remove: " + regionName); metaChanges.addRegionToRemove(regionInfo); } } // Restore regions using the snapshot data monitor.rethrowException(); status.setStatus("Restoring table regions..."); restoreHdfsRegions(exec, regionManifests, metaChanges.getRegionsToRestore()); status.setStatus("Finished restoring all table regions."); // Remove regions from the current table monitor.rethrowException(); status.setStatus("Starting to delete excess regions from table"); removeHdfsRegions(exec, metaChanges.getRegionsToRemove()); status.setStatus("Finished deleting excess regions from table."); } // Regions to Add: present in the snapshot but not in the current table if (regionNames.size() > 0) { List regionsToAdd = new ArrayList(regionNames.size()); monitor.rethrowException(); for (String regionName: regionNames) { LOG.info("region to add: " + regionName); regionsToAdd.add(HRegionInfo.convert(regionManifests.get(regionName).getRegionInfo())); } // Create new regions cloning from the snapshot monitor.rethrowException(); status.setStatus("Cloning regions..."); HRegionInfo[] clonedRegions = cloneHdfsRegions(exec, regionManifests, regionsToAdd); metaChanges.setNewRegions(clonedRegions); status.setStatus("Finished cloning regions."); } return metaChanges; } /** * Describe the set of operations needed to update hbase:meta after restore. */ public static class RestoreMetaChanges { private final Map > parentsMap; private final HTableDescriptor htd; private List regionsToRestore = null; private List regionsToRemove = null; private List regionsToAdd = null; RestoreMetaChanges(HTableDescriptor htd, Map > parentsMap) { this.parentsMap = parentsMap; this.htd = htd; } public HTableDescriptor getTableDescriptor() { return htd; } /** * @return true if there're new regions */ public boolean hasRegionsToAdd() { return this.regionsToAdd != null && this.regionsToAdd.size() > 0; } /** * Returns the list of new regions added during the on-disk restore. * The caller is responsible to add the regions to META. * e.g MetaTableAccessor.addRegionsToMeta(...) * @return the list of regions to add to META */ public List getRegionsToAdd() { return this.regionsToAdd; } /** * @return true if there're regions to restore */ public boolean hasRegionsToRestore() { return this.regionsToRestore != null && this.regionsToRestore.size() > 0; } /** * Returns the list of 'restored regions' during the on-disk restore. * The caller is responsible to add the regions to hbase:meta if not present. * @return the list of regions restored */ public List getRegionsToRestore() { return this.regionsToRestore; } /** * @return true if there're regions to remove */ public boolean hasRegionsToRemove() { return this.regionsToRemove != null && this.regionsToRemove.size() > 0; } /** * Returns the list of regions removed during the on-disk restore. * The caller is responsible to remove the regions from META. * e.g. MetaTableAccessor.deleteRegions(...) * @return the list of regions to remove from META */ public List getRegionsToRemove() { return this.regionsToRemove; } void setNewRegions(final HRegionInfo[] hris) { if (hris != null) { regionsToAdd = Arrays.asList(hris); } else { regionsToAdd = null; } } void addRegionToRemove(final HRegionInfo hri) { if (regionsToRemove == null) { regionsToRemove = new LinkedList(); } regionsToRemove.add(hri); } void addRegionToRestore(final HRegionInfo hri) { if (regionsToRestore == null) { regionsToRestore = new LinkedList(); } regionsToRestore.add(hri); } public void updateMetaParentRegions(Connection connection, final List regionInfos) throws IOException { if (regionInfos == null || parentsMap.isEmpty()) return; // Extract region names and offlined regions Map regionsByName = new HashMap(regionInfos.size()); List 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 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); MetaTableAccessor.addRegionToMeta(connection, regionInfo, regionsByName.get(daughters.getFirst()), regionsByName.get(daughters.getSecond())); } } } /** * Remove specified regions from the file-system, using the archiver. */ private void removeHdfsRegions(final ThreadPoolExecutor exec, final List regions) throws IOException { if (regions == null || regions.size() == 0) return; ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() { @Override public void editRegion(final HRegionInfo hri) throws IOException { HFileArchiver.archiveRegion(conf, fs, hri); } }); } /** * Restore specified regions by restoring content to the snapshot state. */ private void restoreHdfsRegions(final ThreadPoolExecutor exec, final Map regionManifests, final List regions) throws IOException { if (regions == null || regions.size() == 0) return; ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() { @Override public void editRegion(final HRegionInfo hri) throws IOException { restoreRegion(hri, regionManifests.get(hri.getEncodedName())); } }); } private Map> getRegionHFileReferences( final SnapshotRegionManifest manifest) { Map> familyMap = new HashMap>(manifest.getFamilyFilesCount()); for (SnapshotRegionManifest.FamilyFiles familyFiles: manifest.getFamilyFilesList()) { familyMap.put(familyFiles.getFamilyName().toStringUtf8(), new ArrayList(familyFiles.getStoreFilesList())); } return familyMap; } /** * Restore region by removing files not in the snapshot * and adding the missing ones from the snapshot. */ private void restoreRegion(final HRegionInfo regionInfo, final SnapshotRegionManifest regionManifest) throws IOException { Map> snapshotFiles = getRegionHFileReferences(regionManifest); Path regionDir = new Path(tableDir, regionInfo.getEncodedName()); String tableName = tableDesc.getTableName().getNameAsString(); // Restore families present in the table for (Path familyDir: FSUtils.getFamilyDirs(fs, regionDir)) { byte[] family = Bytes.toBytes(familyDir.getName()); Set familyFiles = getTableRegionFamilyFiles(familyDir); List snapshotFamilyFiles = snapshotFiles.remove(familyDir.getName()); if (snapshotFamilyFiles != null) { List hfilesToAdd = new ArrayList(); for (SnapshotRegionManifest.StoreFile storeFile: snapshotFamilyFiles) { if (familyFiles.contains(storeFile.getName())) { // HFile already present familyFiles.remove(storeFile.getName()); } else { // HFile missing hfilesToAdd.add(storeFile); } } // Remove hfiles not present in the snapshot for (String hfileName: familyFiles) { Path hfile = new Path(familyDir, hfileName); LOG.trace("Removing hfile=" + hfileName + " from region=" + regionInfo.getEncodedName() + " table=" + tableName); HFileArchiver.archiveStoreFile(conf, fs, regionInfo, tableDir, family, hfile); } // Restore Missing files for (SnapshotRegionManifest.StoreFile storeFile: hfilesToAdd) { LOG.debug("Adding HFileLink " + storeFile.getName() + " to region=" + regionInfo.getEncodedName() + " table=" + tableName); restoreStoreFile(familyDir, regionInfo, storeFile, createBackRefs); } } else { // Family doesn't exists in the snapshot LOG.trace("Removing family=" + Bytes.toString(family) + " from region=" + regionInfo.getEncodedName() + " table=" + tableName); HFileArchiver.archiveFamily(fs, conf, regionInfo, tableDir, family); fs.delete(familyDir, true); } } // Add families not present in the table for (Map.Entry> familyEntry: snapshotFiles.entrySet()) { Path familyDir = new Path(regionDir, familyEntry.getKey()); if (!fs.mkdirs(familyDir)) { throw new IOException("Unable to create familyDir=" + familyDir); } for (SnapshotRegionManifest.StoreFile storeFile: familyEntry.getValue()) { LOG.trace("Adding HFileLink " + storeFile.getName() + " to table=" + tableName); restoreStoreFile(familyDir, regionInfo, storeFile, createBackRefs); } } } /** * @return The set of files in the specified family directory. */ private Set getTableRegionFamilyFiles(final Path familyDir) throws IOException { FileStatus[] hfiles = FSUtils.listStatus(fs, familyDir); if (hfiles == null) return Collections.emptySet(); Set familyFiles = new HashSet(hfiles.length); for (int i = 0; i < hfiles.length; ++i) { String hfileName = hfiles[i].getPath().getName(); familyFiles.add(hfileName); } return familyFiles; } /** * Clone specified regions. For each region create a new region * and create a HFileLink for each hfile. */ private HRegionInfo[] cloneHdfsRegions(final ThreadPoolExecutor exec, final Map regionManifests, final List regions) throws IOException { if (regions == null || regions.size() == 0) return null; final Map snapshotRegions = new HashMap(regions.size()); // clone region info (change embedded tableName with the new one) HRegionInfo[] clonedRegionsInfo = new HRegionInfo[regions.size()]; for (int i = 0; i < clonedRegionsInfo.length; ++i) { // clone the region info from the snapshot region info HRegionInfo snapshotRegionInfo = regions.get(i); clonedRegionsInfo[i] = cloneRegionInfo(snapshotRegionInfo); // add the region name mapping between snapshot and cloned String snapshotRegionName = snapshotRegionInfo.getEncodedName(); String clonedRegionName = clonedRegionsInfo[i].getEncodedName(); regionsMap.put(Bytes.toBytes(snapshotRegionName), Bytes.toBytes(clonedRegionName)); LOG.info("clone region=" + snapshotRegionName + " as " + clonedRegionName); // Add mapping between cloned region name and snapshot region info snapshotRegions.put(clonedRegionName, snapshotRegionInfo); } // create the regions on disk ModifyRegionUtils.createRegions(exec, conf, rootDir, tableDir, tableDesc, clonedRegionsInfo, new ModifyRegionUtils.RegionFillTask() { @Override public void fillRegion(final HRegion region) throws IOException { HRegionInfo snapshotHri = snapshotRegions.get(region.getRegionInfo().getEncodedName()); cloneRegion(region, snapshotHri, regionManifests.get(snapshotHri.getEncodedName())); } }); return clonedRegionsInfo; } /** * Clone region directory content from the snapshot info. * * Each region is encoded with the table name, so the cloned region will have * a different region name. * * Instead of copying the hfiles a HFileLink is created. * * @param region {@link HRegion} cloned * @param snapshotRegionInfo */ private void cloneRegion(final HRegion region, final HRegionInfo snapshotRegionInfo, final SnapshotRegionManifest manifest) throws IOException { final Path regionDir = new Path(tableDir, region.getRegionInfo().getEncodedName()); final String tableName = tableDesc.getTableName().getNameAsString(); for (SnapshotRegionManifest.FamilyFiles familyFiles: manifest.getFamilyFilesList()) { Path familyDir = new Path(regionDir, familyFiles.getFamilyName().toStringUtf8()); for (SnapshotRegionManifest.StoreFile storeFile: familyFiles.getStoreFilesList()) { LOG.info("Adding HFileLink " + storeFile.getName() + " to table=" + tableName); restoreStoreFile(familyDir, snapshotRegionInfo, storeFile, createBackRefs); } } } /** * Create a new {@link HFileLink} to reference the store file. *

The store file in the snapshot can be a simple hfile, an HFileLink or a reference. *

    *
  • hfile: abc -> table=region-abc *
  • reference: abc.1234 -> table=region-abc.1234 *
  • hfilelink: table=region-hfile -> table=region-hfile *
* @param familyDir destination directory for the store file * @param regionInfo destination region info for the table * @param storeFile store file name (can be a Reference, HFileLink or simple HFile) * @param createBackRef - Whether back reference should be created. Defaults to true. */ private void restoreStoreFile(final Path familyDir, final HRegionInfo regionInfo, final SnapshotRegionManifest.StoreFile storeFile, final boolean createBackRef) throws IOException { String hfileName = storeFile.getName(); if (HFileLink.isHFileLink(hfileName)) { HFileLink.createFromHFileLink(conf, fs, familyDir, hfileName, createBackRef); } else if (StoreFileInfo.isReference(hfileName)) { restoreReferenceFile(familyDir, regionInfo, storeFile); } else { HFileLink.create(conf, fs, familyDir, regionInfo, hfileName, createBackRef); } } /** * Create a new {@link Reference} as copy of the source one. *

   * The source table looks like:
   *    1234/abc      (original file)
   *    5678/abc.1234 (reference file)
   *
   * After the clone operation looks like:
   *   wxyz/table=1234-abc
   *   stuv/table=1234-abc.wxyz
   *
   * NOTE that the region name in the clone changes (md5 of regioninfo)
   * and the reference should reflect that change.
   * 
* @param familyDir destination directory for the store file * @param regionInfo destination region info for the table * @param hfileName reference file name */ private void restoreReferenceFile(final Path familyDir, final HRegionInfo regionInfo, final SnapshotRegionManifest.StoreFile storeFile) throws IOException { String hfileName = storeFile.getName(); // Extract the referred information (hfile name and parent region) Path refPath = StoreFileInfo.getReferredToFile(new Path(new Path(new Path(new Path(snapshotTable .getNamespaceAsString(), snapshotTable.getQualifierAsString()), regionInfo .getEncodedName()), familyDir.getName()), hfileName)); String snapshotRegionName = refPath.getParent().getParent().getName(); String fileName = refPath.getName(); // The new reference should have the cloned region name as parent, if it is a clone. String clonedRegionName = Bytes.toString(regionsMap.get(Bytes.toBytes(snapshotRegionName))); 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 if (storeFile.hasReference()) { Reference reference = Reference.convert(storeFile.getReference()); reference.write(fs, outPath); } else { InputStream in; if (linkPath != null) { in = HFileLink.buildFromHFileLinkPattern(conf, linkPath).open(fs); } else { linkPath = new Path(new Path(HRegion.getRegionDir(snapshotManifest.getSnapshotDir(), 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 daughters = parentsMap.get(clonedRegionName); if (daughters == null) { daughters = new Pair(regionName, null); parentsMap.put(clonedRegionName, daughters); } else if (!regionName.equals(daughters.getFirst())) { daughters.setSecond(regionName); } } } /** * Create a new {@link HRegionInfo} from the snapshot region info. * Keep the same startKey, endKey, regionId and split information but change * the table name. * * @param snapshotRegionInfo Info for region to clone. * @return the new HRegion instance */ public HRegionInfo cloneRegionInfo(final HRegionInfo snapshotRegionInfo) { return cloneRegionInfo(tableDesc.getTableName(), snapshotRegionInfo); } public static HRegionInfo cloneRegionInfo(TableName tableName, HRegionInfo snapshotRegionInfo) { HRegionInfo regionInfo = new HRegionInfo(tableName, snapshotRegionInfo.getStartKey(), snapshotRegionInfo.getEndKey(), snapshotRegionInfo.isSplit(), snapshotRegionInfo.getRegionId()); regionInfo.setOffline(snapshotRegionInfo.isOffline()); return regionInfo; } /** * @return the set of the regions contained in the table */ private List getTableRegions() throws IOException { LOG.debug("get table regions: " + tableDir); FileStatus[] regionDirs = FSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs)); if (regionDirs == null) return null; List regions = new ArrayList(regionDirs.length); for (int i = 0; i < regionDirs.length; ++i) { HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDirs[i].getPath()); regions.add(hri); } LOG.debug("found " + regions.size() + " regions for table=" + tableDesc.getTableName().getNameAsString()); return regions; } /** * Copy the snapshot files for a snapshot scanner, discards meta changes. * @param conf * @param fs * @param rootDir * @param restoreDir * @param snapshotName * @throws IOException */ public static RestoreMetaChanges copySnapshotForScanner(Configuration conf, FileSystem fs, Path rootDir, Path restoreDir, String snapshotName) throws IOException { // ensure that restore dir is not under root dir if (!restoreDir.getFileSystem(conf).getUri().equals(rootDir.getFileSystem(conf).getUri())) { throw new IllegalArgumentException("Filesystems for restore directory and HBase root directory " + "should be the same"); } if (restoreDir.toUri().getPath().startsWith(rootDir.toUri().getPath() +"/")) { throw new IllegalArgumentException("Restore directory cannot be a sub directory of HBase " + "root directory. RootDir: " + rootDir + ", restoreDir: " + restoreDir); } Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir); SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir); SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc); MonitoredTask status = TaskMonitor.get().createStatus( "Restoring snapshot '" + snapshotName + "' to directory " + restoreDir); ForeignExceptionDispatcher monitor = new ForeignExceptionDispatcher(); // we send createBackRefs=false so that restored hfiles do not create back reference links // in the base hbase root dir. RestoreSnapshotHelper helper = new RestoreSnapshotHelper(conf, fs, manifest, manifest.getTableDescriptor(), restoreDir, monitor, status, false); RestoreMetaChanges metaChanges = helper.restoreHdfsRegions(); // TODO: parallelize. if (LOG.isDebugEnabled()) { LOG.debug("Restored table dir:" + restoreDir); FSUtils.logFileSystemState(fs, restoreDir, LOG); } return metaChanges; } }




© 2015 - 2024 Weber Informatics LLC | Privacy Policy