From 9150c7cdccef4ca870ec54c03a804b163286ab6a Mon Sep 17 00:00:00 2001 From: Umesh Agashe Date: Tue, 1 Nov 2016 10:32:01 -0700 Subject: [PATCH] HBASE-16904 Snapshot related changes for FS redo work --- .../hadoop/hbase/client/SnapshotDescription.java | 13 + .../hadoop/hbase/client/TableSnapshotScanner.java | 13 +- .../org/apache/hadoop/hbase/fs/MasterStorage.java | 296 +++++- .../org/apache/hadoop/hbase/fs/StorageContext.java | 1 - .../hadoop/hbase/fs/legacy/LegacyLayout.java | 131 ++- .../hbase/fs/legacy/LegacyMasterStorage.java | 417 +++++++- .../hbase/fs/legacy/snapshot/ExportSnapshot.java | 1102 ++++++++++++++++++++ .../fs/legacy/snapshot/RestoreSnapshotHelper.java | 689 ++++++++++++ .../fs/legacy/snapshot/SnapshotFileCache.java | 11 +- .../fs/legacy/snapshot/SnapshotHFileCleaner.java | 3 +- .../hbase/fs/legacy/snapshot/SnapshotManifest.java | 570 ++++++++++ .../fs/legacy/snapshot/SnapshotManifestV1.java | 209 ++++ .../fs/legacy/snapshot/SnapshotManifestV2.java | 187 ++++ .../org/apache/hadoop/hbase/mapreduce/Driver.java | 2 +- .../MultiTableSnapshotInputFormatImpl.java | 22 +- .../hbase/mapreduce/TableSnapshotInputFormat.java | 2 +- .../mapreduce/TableSnapshotInputFormatImpl.java | 41 +- .../master/procedure/CloneSnapshotProcedure.java | 18 +- .../master/procedure/RestoreSnapshotProcedure.java | 19 +- .../snapshot/DisabledTableSnapshotHandler.java | 5 +- .../master/snapshot/MasterSnapshotVerifier.java | 137 ++- .../hbase/master/snapshot/SnapshotManager.java | 253 ++--- .../hbase/master/snapshot/TakeSnapshotHandler.java | 80 +- .../hadoop/hbase/mob/mapreduce/SweepReducer.java | 1 - .../apache/hadoop/hbase/regionserver/HRegion.java | 6 +- .../regionserver/DumpReplicationQueues.java | 3 - .../hadoop/hbase/snapshot/ExportSnapshot.java | 1084 ------------------- .../hbase/snapshot/RestoreSnapshotHelper.java | 823 --------------- .../hbase/snapshot/SnapshotDescriptionUtils.java | 239 +---- .../apache/hadoop/hbase/snapshot/SnapshotInfo.java | 187 ++-- .../hadoop/hbase/snapshot/SnapshotManifest.java | 570 ---------- .../hadoop/hbase/snapshot/SnapshotManifestV1.java | 209 ---- .../hadoop/hbase/snapshot/SnapshotManifestV2.java | 187 ---- .../hbase/snapshot/SnapshotReferenceUtil.java | 327 +++--- .../hbase/snapshot/SnapshotRestoreMetaChanges.java | 157 +++ .../hbase/client/TestSnapshotFromClient.java | 7 - .../fs/legacy/snapshot/TestExportSnapshot.java | 384 +++++++ .../legacy/snapshot/TestExportSnapshotHelpers.java | 96 ++ .../legacy/snapshot/TestRestoreSnapshotHelper.java | 181 ++++ .../fs/legacy/snapshot/TestSnapshotFileCache.java | 30 +- .../fs/legacy/snapshot/TestSnapshotManifest.java | 146 +++ .../hadoop/hbase/regionserver/TestStoreFile.java | 3 - .../hbase/snapshot/SnapshotTestingUtils.java | 5 +- .../hadoop/hbase/snapshot/TestExportSnapshot.java | 376 ------- .../hbase/snapshot/TestExportSnapshotHelpers.java | 96 -- .../hbase/snapshot/TestRestoreSnapshotHelper.java | 180 ---- .../hbase/snapshot/TestSnapshotManifest.java | 145 --- src/main/asciidoc/_chapters/ops_mgt.adoc | 10 +- 48 files changed, 4981 insertions(+), 4692 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/ExportSnapshot.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/RestoreSnapshotHelper.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotManifest.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotManifestV1.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotManifestV2.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotRestoreMetaChanges.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestExportSnapshot.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestExportSnapshotHelpers.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestRestoreSnapshotHelper.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestSnapshotManifest.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotHelpers.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotManifest.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotDescription.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotDescription.java index f737825..6bcf5c6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotDescription.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/SnapshotDescription.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; /** * The POJO equivalent of HBaseProtos.SnapshotDescription @@ -32,6 +34,17 @@ public class SnapshotDescription { private long creationTime = -1L; private int version = -1; + /** + * Converts hbase.protobuf.generated.HBaseProtos.SnapshotDescription to + * hbase.client.SnapshotDescription + * @param desc + */ + public static SnapshotDescription convert(HBaseProtos.SnapshotDescription desc) { + return new SnapshotDescription(desc.getName(), desc.getTable(), + ProtobufUtil.createSnapshotType(desc.getType()), desc.getOwner(), desc.getCreationTime(), + desc.getVersion()); + } + public SnapshotDescription(String name) { this(name, null); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java index 4601ae4..48b746d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/TableSnapshotScanner.java @@ -34,7 +34,10 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; -import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper; +import org.apache.hadoop.hbase.fs.MasterStorage; +import org.apache.hadoop.hbase.fs.StorageIdentifier; +import org.apache.hadoop.hbase.snapshot.SnapshotRestoreMetaChanges; +import org.apache.hadoop.hbase.fs.legacy.snapshot.RestoreSnapshotHelper; import org.apache.hadoop.hbase.util.FSUtils; /** @@ -45,7 +48,7 @@ import org.apache.hadoop.hbase.util.FSUtils; *

* This also allows one to run the scan from an * online or offline hbase cluster. The snapshot files can be exported by using the - * {@link org.apache.hadoop.hbase.snapshot.ExportSnapshot} tool, + * {@link org.apache.hadoop.hbase.fs.legacy.snapshot.ExportSnapshot} tool, * to a pure-hdfs cluster, and this scanner can be used to * run the scan directly over the snapshot files. The snapshot should not be deleted while there * are open scanners reading from snapshot files. @@ -121,9 +124,9 @@ public class TableSnapshotScanner extends AbstractClientScanner { } private void init() throws IOException { - final RestoreSnapshotHelper.RestoreMetaChanges meta = - RestoreSnapshotHelper.copySnapshotForScanner( - conf, fs, rootDir, restoreDir, snapshotName); + MasterStorage masterStorage = MasterStorage.open(conf, false); + final SnapshotRestoreMetaChanges meta = RestoreSnapshotHelper.copySnapshotForScanner( + masterStorage, restoreDir, snapshotName); final List restoredRegions = meta.getRegionsToAdd(); htd = meta.getTableDescriptor(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/MasterStorage.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/MasterStorage.java index 2f3b4a4..2476228 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/MasterStorage.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/MasterStorage.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; +import java.util.List; import java.util.Map; import org.apache.commons.logging.Log; @@ -38,9 +39,16 @@ import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; +import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare; import org.apache.hadoop.hbase.fs.legacy.LegacyMasterStorage; import org.apache.hadoop.hbase.fs.RegionStorage.StoreFileVisitor; import org.apache.hadoop.hbase.fs.legacy.LegacyPathIdentifier; +import org.apache.hadoop.hbase.monitoring.MonitoredTask; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; +import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; +import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException; +import org.apache.hadoop.hbase.snapshot.SnapshotRestoreMetaChanges; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; @@ -83,22 +91,6 @@ public abstract class MasterStorage { return new ArrayList<>(); } - /** - * This method should be called to prepare storage implementation/s for snapshots. The default - * implementation does nothing. MasterStorage subclasses need to override this method to - * provide specific preparatory steps. - */ - public void enableSnapshots() { - return; - } - - /** - * Returns true if MasterStorage is prepared for snapshots - */ - public boolean isSnapshotsEnabled() { - return true; - } - // ========================================================================== // PUBLIC Interfaces - Visitors // ========================================================================== @@ -114,6 +106,17 @@ public abstract class MasterStorage { void visitRegion(HRegionInfo regionInfo) throws IOException; } + public interface SnapshotVisitor { + void visitSnapshot(final String snapshotName, final SnapshotDescription snapshot, + StorageContext ctx); + } + + public interface SnapshotStoreFileVisitor { + // TODO: Instead of SnapshotRegionManifest.StoreFile return common object across all + void visitSnapshotStoreFile(SnapshotDescription snapshot, StorageContext ctx, HRegionInfo hri, + String familyName, final SnapshotRegionManifest.StoreFile storeFile) throws IOException; + } + // ========================================================================== // PUBLIC Methods - Namespace related // ========================================================================== @@ -199,7 +202,7 @@ public abstract class MasterStorage { } /** - * Archives specified table and all it's regions + * Archives a table and all it's regions * @param tableName * @throws IOException */ @@ -208,7 +211,7 @@ public abstract class MasterStorage { } /** - * Archives specified table and all it's regions + * Archives a table and all it's regions * @param ctx Storage context of the table. * @param tableName * @throws IOException @@ -289,6 +292,10 @@ public abstract class MasterStorage { return RegionStorage.open(conf, regionInfo, false); } + public RegionStorage createRegionStorage(HRegionInfo hri) throws IOException { + return RegionStorage.open(conf, hri, true); + } + /** * Returns true if region exists on the Storage * @param regionInfo @@ -301,13 +308,240 @@ public abstract class MasterStorage { } /** - * Archives the specified region's storage artifacts (files, directories etc) + * Archives region's storage artifacts (files, directories etc) * @param regionInfo * @throws IOException */ public abstract void archiveRegion(HRegionInfo regionInfo) throws IOException; // ========================================================================== + // PUBLIC Methods - Snapshot related + // ========================================================================== + /** + * This method should be called to prepare storage implementation/s for snapshots. The default + * implementation does nothing. MasterStorage subclasses need to override this method to + * provide specific preparatory steps. + */ + public void enableSnapshots() throws IOException { + return; + } + + /** + * Returns true if MasterStorage is prepared for snapshots + */ + public boolean isSnapshotsEnabled() { + return true; + } + + /** + * Gets the list of all snapshots. + * @return list of SnapshotDescriptions + * @throws IOException Storage exception + */ + public List getSnapshots() throws IOException { + return getSnapshots(StorageContext.DATA); + } + + public abstract List getSnapshots(StorageContext ctx) throws IOException; + + /** + * Gets snapshot description of a snapshot + * @return Snapshot description of a snapshot if found, null otherwise + * @throws IOException + */ + public SnapshotDescription getSnapshot(final String snapshotName) + throws IOException { + return getSnapshot(snapshotName, StorageContext.DATA); + } + + public abstract SnapshotDescription getSnapshot(final String snapshotName, StorageContext ctx) + throws IOException; + + /** + * @return {@link HTableDescriptor} for a snapshot + * @param snapshot + * @throws IOException if can't read from the storage + */ + public HTableDescriptor getTableDescriptorForSnapshot(final SnapshotDescription snapshot) + throws IOException { + return getTableDescriptorForSnapshot(snapshot, StorageContext.DATA); + } + + public abstract HTableDescriptor getTableDescriptorForSnapshot(final SnapshotDescription + snapshot, StorageContext ctx) throws IOException; + + /** + * Returns all {@link HRegionInfo} for a snapshot + * + * @param snapshot + * @return + * @throws IOException + */ + public Map getSnapshotRegions(final SnapshotDescription snapshot) + throws IOException { + return getSnapshotRegions(snapshot, StorageContext.DATA); + } + + public abstract Map getSnapshotRegions(final SnapshotDescription snapshot, + StorageContext ctx) throws IOException; + + /** + * Returns list of all store files {@link SnapshotRegionManifest.StoreFile} for a snapshot + * @param snapshot + * @return + * @throws IOException + */ + public List getSnapshotStoreFiles(final SnapshotDescription + snapshot) throws IOException { + return getSnapshotStoreFiles(snapshot, StorageContext.DATA); + } + + public abstract List getSnapshotStoreFiles(final + SnapshotDescription snapshot, StorageContext ctx) throws IOException; + + /** + * Check to see if the snapshot is one of the currently snapshots on the storage. + * + * @param snapshot + * @throws IOException + */ + public boolean snapshotExists(SnapshotDescription snapshot) throws IOException { + return snapshotExists(snapshot, StorageContext.DATA); + } + + public abstract boolean snapshotExists(SnapshotDescription snapshot, StorageContext ctx) + throws IOException; + + public boolean snapshotExists(String snapshotName) throws IOException { + return snapshotExists(snapshotName, StorageContext.DATA); + } + + public abstract boolean snapshotExists(String snapshotName, StorageContext ctx) throws + IOException; + + /** + * Cleans up all snapshots. + * + * @throws IOException if can't reach the storage + */ + public void cleanupAllSnapshots() throws IOException { + cleanupAllSnapshots(StorageContext.DATA); + } + + public abstract void cleanupAllSnapshots(StorageContext ctx) throws IOException; + + /** + * Deletes a snapshot + * @param snapshot + * @throws SnapshotDoesNotExistException If the specified snapshot does not exist. + * @throws IOException For storage IOExceptions + */ + public boolean deleteSnapshot(final SnapshotDescription snapshot) throws IOException { + return deleteSnapshot(snapshot, StorageContext.DATA) && + deleteSnapshot(snapshot, StorageContext.TEMP); + } + + public boolean deleteSnapshot(final String snapshotName) throws IOException { + return deleteSnapshot(snapshotName, StorageContext.DATA) && + deleteSnapshot(snapshotName, StorageContext.TEMP); + } + + public abstract boolean deleteSnapshot(final SnapshotDescription snapshot, + final StorageContext ctx) throws IOException; + + public abstract boolean deleteSnapshot(final String snapshotName, final StorageContext ctx) + throws IOException; + + /** + * Deletes old in-progress and/ or completed snapshot and prepares for new one with the same + * description + * + * @param snapshot + * @throws IOException for storage IOExceptions + */ + public abstract void resetSnapshot(SnapshotDescription snapshot) throws IOException; + + /** + * In general snapshot is created with following steps: + *

+ * @param htd + * @param snapshot + * @param monitor + * @throws IOException + */ + public void snapshotTable(HTableDescriptor htd, SnapshotDescription snapshot, final + ForeignExceptionSnare monitor) throws IOException { + snapshotTable(htd, snapshot, monitor, StorageContext.DATA); + } + + public abstract void snapshotTable(HTableDescriptor htd, SnapshotDescription snapshot, + final ForeignExceptionSnare monitor, StorageContext ctx) throws IOException; + + /** + * Consolidates added regions and verifies snapshot + * @param snapshot + * @throws IOException + */ + public void consolidateSnapshot(SnapshotDescription snapshot) throws IOException { + consolidateSnapshot(snapshot, StorageContext.DATA); + } + + public abstract void consolidateSnapshot(SnapshotDescription snapshot, StorageContext ctx) + throws IOException; + + /** + * Changes {@link StorageContext} of a snapshot from src to dest + * + * @param snapshot + * @param src Source {@link StorageContext} + * @param dest Destination {@link StorageContext} + * @return + * @throws IOException + */ + public abstract boolean changeSnapshotContext(SnapshotDescription snapshot, StorageContext src, + StorageContext dest) throws IOException; + + /** + * Adds given region to the snapshot. + * + * @param hri + * @param snapshot + * @throws IOException + */ + public void addRegionToSnapshot(HRegionInfo hri, SnapshotDescription snapshot) + throws IOException { + addRegionToSnapshot(hri, snapshot, StorageContext.DATA); + } + + public abstract void addRegionToSnapshot(HRegionInfo hri, SnapshotDescription snapshot, + StorageContext ctx) throws IOException; + + /** + * Restore snapshot to dest table and returns instance of {@link SnapshotRestoreMetaChanges} + * describing changes required for META. + * @param snapshot + * @param destHtd + * @param monitor + * @param status + * @return + * @throws IOException + */ + public SnapshotRestoreMetaChanges restoreSnapshot(final SnapshotDescription snapshot, + final HTableDescriptor destHtd, final ForeignExceptionDispatcher monitor, + final MonitoredTask status) throws IOException { + return restoreSnapshot(snapshot, StorageContext.DATA, destHtd, monitor, status); + } + + public abstract SnapshotRestoreMetaChanges restoreSnapshot(final SnapshotDescription snapshot, + final StorageContext snapshotCtx, final HTableDescriptor destHtd, + final ForeignExceptionDispatcher monitor, final MonitoredTask status) throws IOException; + + // ========================================================================== // PUBLIC Methods - WAL // ========================================================================== @@ -320,6 +554,8 @@ public abstract class MasterStorage { // ========================================================================== // PUBLIC Methods - visitors // ========================================================================== + // TODO: remove implementations. How to visit store files is up to implementation, may use + // threadpool etc. public void visitStoreFiles(StoreFileVisitor visitor) throws IOException { visitStoreFiles(StorageContext.DATA, visitor); @@ -356,6 +592,28 @@ public abstract class MasterStorage { } } + /** + * Visit all snapshots on a storage with visitor instance + * @param visitor + * @throws IOException + */ + public abstract void visitSnapshots(final SnapshotVisitor visitor) throws IOException; + + public abstract void visitSnapshots(StorageContext ctx, final SnapshotVisitor visitor) + throws IOException; + + /** + * Visit all store files of a snapshot with visitor instance + * + * @param snapshot + * @param ctx + * @param visitor + * @throws IOException + */ + public abstract void visitSnapshotStoreFiles(SnapshotDescription snapshot, StorageContext ctx, + SnapshotStoreFileVisitor visitor) throws IOException; + + // ========================================================================== // PUBLIC Methods - bootstrap // ========================================================================== diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/StorageContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/StorageContext.java index cc324a9..3b2cc9b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/StorageContext.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/StorageContext.java @@ -23,6 +23,5 @@ public enum StorageContext { TEMP, DATA, ARCHIVE, - SNAPSHOT, SIDELINE, } \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyLayout.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyLayout.java index 2906f91..a59edca 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyLayout.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyLayout.java @@ -23,8 +23,66 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifest; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifestV2; import org.apache.hadoop.hbase.mob.MobConstants; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; +/** + * This class helps manage legacy layout of directories and files on HDFS for HBase. The directories + * are laid out on disk as below (Note: transient files and directories are enclosed with [], + * multiple directories, files for namespaces, tables, regions etc. at the same directorydepth is + * indicated by ...): + *

+ *

+ * Table data           --->  /hbase/{@value HConstants#BASE_NAMESPACE_DIR}/
+ * Default namespace    --->    default/
+ * System namespace     --->    hbase/
+ * Namespace            --->    ns1/
+ * Table                --->        table1/
+ * Table details        --->          {@value LegacyTableDescriptor#TABLEINFO_DIR}/
+ * Table info           --->            {@value LegacyTableDescriptor#TABLEINFO_FILE_PREFIX}.0000000003
+ * Region name          --->          region1/
+ * Region details       --->            {@value #REGION_INFO_FILE}
+ * Column family        --->            cf1/
+ * Store file           --->              file1
+ * Store files          --->              ...
+ * Column families      --->            .../
+ * Regions              --->          .../
+ * Tables               --->        .../
+ * Namespaces           --->    .../
+ * Temp                 --->  /hbase/{@value HConstants#HBASE_TEMP_DIRECTORY}/
+ * Base MOB             --->  /hbase/{@value MobConstants#MOB_DIR_NAME}/
+ * Snapshot             --->  /hbase/{@value HConstants#SNAPSHOT_DIR_NAME}/
+ * Working              --->    {@value #SNAPSHOT_TMP_DIR_NAME}/
+ * In progress snapshot --->      snap5/
+ * Snapshot descriptor  --->        {@value #SNAPSHOTINFO_FILE}
+ * Snapshot manifest    --->        {@value SnapshotManifest#DATA_MANIFEST_NAME}
+ * Region manifest      --->        [{@value SnapshotManifestV2#SNAPSHOT_MANIFEST_PREFIX}region51]
+ * Region manifests     --->        ...
+ * Snapshots            --->      .../
+ * Completed snapshot   --->    snap1/
+ * Snapshot descriptor  --->        {@value #SNAPSHOTINFO_FILE}
+ * Snapshot manifest    --->        {@value SnapshotManifest#DATA_MANIFEST_NAME}
+ * OLD snapshot layout  --->    snap_old/
+ * Snapshot descriptor  --->      {@value #SNAPSHOTINFO_FILE}
+ * Table details        --->      {@value LegacyTableDescriptor#TABLEINFO_DIR}/
+ * Table info           --->        {@value LegacyTableDescriptor#TABLEINFO_FILE_PREFIX}.0000000006
+ * Snapshot region      --->      region6/
+ * Region details       --->        {@value #REGION_INFO_FILE}
+ * Column family        --->        cf3/
+ * Store file           --->          file3
+ * Store files          --->          ...
+ * Column families      --->        .../
+ * Regions              --->      .../
+ * Logs                 --->      .logs/
+ * Server name          --->        server1/
+ * Log files            --->          logfile1
+ * Snapshots            --->    .../
+ * Archive              --->  /hbase/{@value HConstants#HFILE_ARCHIVE_DIRECTORY}/
+ * 
+ *

+ */ public final class LegacyLayout { /** Name of the region info file that resides just under the region directory. */ public final static String REGION_INFO_FILE = ".regioninfo"; @@ -38,22 +96,87 @@ public final class LegacyLayout { /** Temporary subdirectory of the region directory used for compaction output. */ private static final String REGION_TEMP_DIR = ".tmp"; + // snapshot directory constants + /** + * The file contains the snapshot basic information and it is under the directory of a snapshot. + */ + public static final String SNAPSHOTINFO_FILE = ".snapshotinfo"; + + /** Temporary directory under the snapshot directory to store in-progress snapshots */ + public static final String SNAPSHOT_TMP_DIR_NAME = ".tmp"; + private LegacyLayout() {} public static Path getDataDir(final Path rootDir) { return new Path(rootDir, HConstants.BASE_NAMESPACE_DIR); } - public static Path getSidelineDir(Path rootDir) { + public static Path getSidelineDir(final Path rootDir) { return new Path(rootDir, HConstants.HBCK_SIDELINEDIR_NAME); } - public static Path getSnapshotDir(Path rootDir) { + /** + * Get the snapshot root directory. All the snapshots are kept under this directory, i.e. + * ${hbase.rootdir}/{@value HConstants#SNAPSHOT_DIR_NAME} + * @param rootDir hbase root directory + * @return the base directory in which all snapshots are kept + */ + public static Path getSnapshotDir(final Path rootDir) { return new Path(rootDir, HConstants.SNAPSHOT_DIR_NAME); } - public static Path getSnapshotDir(Path baseSnapshotDir, String snapshotName) { - return new Path(baseSnapshotDir, snapshotName); + /** + * Get the directory for a completed snapshot. This directory is a sub-directory of snapshot root + * directory and all the data files for a snapshot are kept under this directory. + * @param rootDir hbase root directory + * @param snapshotName name of the snapshot being taken + * @return the final directory for the completed snapshot + */ + public static Path getCompletedSnapshotDir(final Path rootDir, final String snapshotName) { + return new Path(getSnapshotDir(rootDir), snapshotName); + } + + /** + * Get the directory for a specified snapshot. This directory is a sub-directory of snapshot root + * directory and all the data files for a snapshot are kept under this directory. + * @param rootDir hbase root directory + * @param snapshot snapshot description + * @return the final directory for the completed snapshot + */ + public static Path getCompletedSnapshotDir(final Path rootDir, + final SnapshotDescription snapshot) { + return getCompletedSnapshotDir(rootDir, snapshot.getName()); + } + + /** + * Get the general working directory for snapshots - where they are built, where they are + * temporarily copied on export, etc. + * i.e.$ {hbase.rootdir}/{@value HConstants#SNAPSHOT_DIR_NAME}/{@value #SNAPSHOT_TMP_DIR_NAME} + * @param rootDir root directory of the HBase installation + * @return Path to the snapshot tmp directory, relative to the passed root directory + */ + public static Path getWorkingSnapshotDir(final Path rootDir) { + return new Path(getSnapshotDir(rootDir), SNAPSHOT_TMP_DIR_NAME); + } + + /** + * Get the directory to build a snapshot, before it is finalized + * @param rootDir root directory of the hbase installation + * @param snapshotName name of the snapshot + * @return {@link Path} where one can build a snapshot + */ + public static Path getWorkingSnapshotDir(final Path rootDir, final String snapshotName) { + return new Path(getWorkingSnapshotDir(rootDir), snapshotName); + } + + /** + * Get the directory to build a snapshot, before it is finalized + * @param rootDir root directory of the hbase installation + * @param snapshot snapshot that will be built + * @return {@link Path} where one can build a snapshot + */ + public static Path getWorkingSnapshotDir(final Path rootDir, final SnapshotDescription snapshot) { + return getWorkingSnapshotDir(rootDir, snapshot.getName()); } public static Path getArchiveDir(Path rootDir) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyMasterStorage.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyMasterStorage.java index aa4de2c..de228fe 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyMasterStorage.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/LegacyMasterStorage.java @@ -24,7 +24,9 @@ import java.io.FileNotFoundException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; @@ -32,9 +34,11 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hbase.ClusterId; @@ -45,12 +49,24 @@ import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; +import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.fs.legacy.cleaner.HFileCleaner; import org.apache.hadoop.hbase.fs.legacy.cleaner.HFileLinkCleaner; import org.apache.hadoop.hbase.fs.legacy.cleaner.LogCleaner; +import org.apache.hadoop.hbase.fs.legacy.snapshot.RestoreSnapshotHelper; import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotHFileCleaner; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifest; import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.hadoop.hbase.monitoring.MonitoredTask; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; +import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; +import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException; +import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; +import org.apache.hadoop.hbase.snapshot.SnapshotRestoreMetaChanges; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.fs.StorageContext; @@ -65,10 +81,13 @@ import org.apache.hadoop.hbase.backup.HFileArchiver; @InterfaceAudience.Private public class LegacyMasterStorage extends MasterStorage { + // TODO: Modify all APIs to use ExecutorService and support parallel HDFS queries + private static final Log LOG = LogFactory.getLog(LegacyMasterStorage.class); private final Path sidelineDir; private final Path snapshotDir; + private final Path tmpSnapshotDir; private final Path archiveDataDir; private final Path archiveDir; private final Path tmpDataDir; @@ -102,6 +121,7 @@ public class LegacyMasterStorage extends MasterStorage { // base directories this.sidelineDir = LegacyLayout.getSidelineDir(rootDir.path); this.snapshotDir = LegacyLayout.getSnapshotDir(rootDir.path); + this.tmpSnapshotDir = LegacyLayout.getWorkingSnapshotDir(rootDir.path); this.archiveDir = LegacyLayout.getArchiveDir(rootDir.path); this.archiveDataDir = LegacyLayout.getDataDir(this.archiveDir); this.dataDir = LegacyLayout.getDataDir(rootDir.path); @@ -128,39 +148,6 @@ public class LegacyMasterStorage extends MasterStorage { return chores; } - /** - * This method modifies chores configuration for snapshots. Please call this method before - * instantiating and scheduling list of chores with {@link #getChores(Stoppable, Map)}. - */ - @Override - public void enableSnapshots() { - super.enableSnapshots(); - if (!isSnapshotsEnabled()) { - // Extract cleaners from conf - Set hfileCleaners = new HashSet<>(); - String[] cleaners = getConfiguration().getStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS); - if (cleaners != null) Collections.addAll(hfileCleaners, cleaners); - - // add snapshot related cleaners - hfileCleaners.add(SnapshotHFileCleaner.class.getName()); - hfileCleaners.add(HFileLinkCleaner.class.getName()); - - // Set cleaners conf - getConfiguration().setStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, - hfileCleaners.toArray(new String[hfileCleaners.size()])); - } - } - - @Override - public boolean isSnapshotsEnabled() { - // Extract cleaners from conf - Set hfileCleaners = new HashSet<>(); - String[] cleaners = getConfiguration().getStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS); - if (cleaners != null) Collections.addAll(hfileCleaners, cleaners); - return hfileCleaners.contains(SnapshotHFileCleaner.class.getName()) && - hfileCleaners.contains(HFileLinkCleaner.class.getName()); - } - // ========================================================================== // PUBLIC Methods - Namespace related // ========================================================================== @@ -332,6 +319,353 @@ public class LegacyMasterStorage extends MasterStorage { } // ========================================================================== + // Methods - Snapshot related + // ========================================================================== + + /** + * Filter that only accepts completed snapshot directories + */ + public static class CompletedSnapshotDirectoriesFilter extends FSUtils.BlackListDirFilter { + /** + * @param fs + */ + public CompletedSnapshotDirectoriesFilter(FileSystem fs) { + super(fs, Collections.singletonList(LegacyLayout.SNAPSHOT_TMP_DIR_NAME)); + } + } + + /** + * This method modifies chores configuration for snapshots. Please call this method before + * instantiating and scheduling list of chores with {@link #getChores(Stoppable, Map)}. + */ + @Override + public void enableSnapshots() throws IOException { + super.enableSnapshots(); + + // check if an older version of snapshot directory was present + Path oldSnapshotDir = new Path(getRootContainer().path, HConstants.OLD_SNAPSHOT_DIR_NAME); + List oldSnapshots = getSnapshotDescriptions(oldSnapshotDir, + new CompletedSnapshotDirectoriesFilter(getFileSystem())); + if (oldSnapshots != null && !oldSnapshots.isEmpty()) { + LOG.error("Snapshots from an earlier release were found under '" + oldSnapshotDir + "'."); + LOG.error("Please rename the directory "); + } + + // TODO: add check for old snapshot dir that existed just before HBASE-14439 + + if (!isSnapshotsEnabled()) { + // Extract cleaners from conf + Set hfileCleaners = new HashSet<>(); + String[] cleaners = getConfiguration().getStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS); + if (cleaners != null) Collections.addAll(hfileCleaners, cleaners); + + // add snapshot related cleaners + hfileCleaners.add(SnapshotHFileCleaner.class.getName()); + hfileCleaners.add(HFileLinkCleaner.class.getName()); + + // Set cleaners conf + getConfiguration().setStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, + hfileCleaners.toArray(new String[hfileCleaners.size()])); + } + } + + @Override + public boolean isSnapshotsEnabled() { + // Extract cleaners from conf + Set hfileCleaners = new HashSet<>(); + String[] cleaners = getConfiguration().getStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS); + if (cleaners != null) Collections.addAll(hfileCleaners, cleaners); + return hfileCleaners.contains(SnapshotHFileCleaner.class.getName()) && + hfileCleaners.contains(HFileLinkCleaner.class.getName()); + } + + private List getSnapshotDescriptions(final Path dir, + final PathFilter filter) throws IOException { + List snapshotDescs = new ArrayList<>(); + if (!FSUtils.isExists(getFileSystem(), dir)) { + return snapshotDescs; + } + + for (FileStatus fileStatus : FSUtils.listStatus(getFileSystem(), dir, filter)) { + Path info = new Path(fileStatus.getPath(), LegacyLayout.SNAPSHOTINFO_FILE); + if (!FSUtils.isExists(getFileSystem(), info)) { + LOG.error("Snapshot information for '" + fileStatus.getPath() + "' doesn't exist!"); + continue; + } + + FSDataInputStream in = null; + try { + in = getFileSystem().open(info); + SnapshotDescription desc = SnapshotDescription.parseFrom(in); + snapshotDescs.add(desc); + } catch (IOException e) { + LOG.warn("Found a corrupted snapshot '" + fileStatus.getPath() + "'.", e); + } finally { + if (in != null) { + in.close(); + } + } + } + return snapshotDescs; + } + + @Override + public List getSnapshots(StorageContext ctx) throws IOException { + return getSnapshotDescriptions(getSnapshotDirFromContext(ctx), + new CompletedSnapshotDirectoriesFilter(getFileSystem())); + } + + @Override + public SnapshotDescription getSnapshot(String snapshotName, StorageContext ctx) + throws IOException { + SnapshotDescription retSnapshot = null; + + Path snapshotDir = getSnapshotDirFromContext(ctx, snapshotName); + Path info = new Path(snapshotDir, LegacyLayout.SNAPSHOTINFO_FILE); + if (!FSUtils.isExists(getFileSystem(), info)) { + LOG.warn("Snapshot information for '" + snapshotName + "' doesn't exist!"); + return retSnapshot; + } + + FSDataInputStream in = null; + try { + in = getFileSystem().open(info); + retSnapshot = SnapshotDescription.parseFrom(in); + } catch (IOException e) { + LOG.warn("Found a corrupted snapshot '" + snapshotName + "'.", e); + } finally { + if (in != null) { + in.close(); + } + } + + return retSnapshot; + } + + @Override + public void visitSnapshots(final SnapshotVisitor visitor) throws IOException { + visitSnapshots(StorageContext.DATA, visitor); + } + + @Override + public void visitSnapshots(StorageContext ctx, final SnapshotVisitor visitor) throws IOException { + for (SnapshotDescription s : getSnapshots(ctx)) { + visitor.visitSnapshot(s.getName(), s, ctx); + } + } + + private SnapshotManifest getSnapshotManifest(SnapshotDescription snapshot, StorageContext ctx) + throws IOException { + Path snapshotDir = getSnapshotDirFromContext(ctx, snapshot.getName()); + return SnapshotManifest.open(getConfiguration(), getFileSystem(), snapshotDir, snapshot); + } + + @Override + public HTableDescriptor getTableDescriptorForSnapshot(SnapshotDescription snapshot, + StorageContext ctx) throws IOException { + SnapshotManifest manifest = getSnapshotManifest(snapshot, ctx); + return manifest.getTableDescriptor(); + } + + private List getSnapshotRegionManifests(SnapshotDescription snapshot, + StorageContext ctx) throws IOException { + SnapshotManifest manifest = getSnapshotManifest(snapshot, ctx); + List regionManifests = manifest.getRegionManifests(); + if (regionManifests == null) { + regionManifests = new ArrayList<>(); + } + return regionManifests; + } + + @Override + public Map getSnapshotRegions(SnapshotDescription snapshot, + StorageContext ctx) throws IOException { + Map retRegions = new HashMap<>(); + for (SnapshotRegionManifest regionManifest: getSnapshotRegionManifests(snapshot, ctx)) { + HRegionInfo hri = HRegionInfo.convert(regionManifest.getRegionInfo()); + retRegions.put(hri.getEncodedName(), hri); + } + return retRegions; + } + + private List visitAndGetSnapshotStoreFiles( + SnapshotDescription snapshot, StorageContext ctx, String regionName, String familyName, + SnapshotStoreFileVisitor visitor) throws IOException { + List snapshotStoreFiles = new ArrayList<>(); + + for (SnapshotRegionManifest regionManifest: getSnapshotRegionManifests(snapshot, ctx)) { + HRegionInfo hri = HRegionInfo.convert(regionManifest.getRegionInfo()); + + // check for region name + if (regionName != null) { + if (!hri.getEncodedName().equals(regionName)) { + continue; + } + } + + for (SnapshotRegionManifest.FamilyFiles familyFiles: regionManifest.getFamilyFilesList()) { + String family = familyFiles.getFamilyName().toStringUtf8(); + // check for family name + if (familyName != null && !familyName.equals(family)) { + continue; + } + + List storeFiles = familyFiles.getStoreFilesList(); + snapshotStoreFiles.addAll(storeFiles); + + if (visitor != null) { + for(SnapshotRegionManifest.StoreFile storeFile: storeFiles) { + visitor.visitSnapshotStoreFile(snapshot, ctx, hri, family, storeFile); + } + } + } + } + + return snapshotStoreFiles; + } + + @Override + public List getSnapshotStoreFiles(SnapshotDescription snapshot, + StorageContext ctx) throws IOException { + return visitAndGetSnapshotStoreFiles(snapshot, ctx, null, null, null); + } + + @Override + public void visitSnapshotStoreFiles(SnapshotDescription snapshot, StorageContext ctx, + SnapshotStoreFileVisitor visitor) throws IOException { + visitAndGetSnapshotStoreFiles(snapshot, ctx, null, null, visitor); + } + + @Override + public boolean snapshotExists(SnapshotDescription snapshot, StorageContext ctx) + throws IOException { + return snapshotExists(snapshot.getName(), ctx); + } + + @Override + public boolean snapshotExists(String snapshotName, StorageContext ctx) throws IOException { + return getSnapshot(snapshotName, ctx) != null; + } + + @Override + public void cleanupAllSnapshots(StorageContext ctx) throws IOException { + Path snapshotDir = getSnapshotDirFromContext(ctx); + if (!FSUtils.deleteDirectory(getFileSystem(), snapshotDir)) { + LOG.warn("Couldn't delete working snapshot directory '" + snapshotDir + "."); + } + } + + private void deleteSnapshotDir(Path snapshotDir) throws IOException { + LOG.debug("Deleting snapshot directory '" + snapshotDir + "'."); + if (!FSUtils.deleteDirectory(getFileSystem(), snapshotDir)) { + throw new HBaseSnapshotException("Failed to delete snapshot directory '" + + snapshotDir + "'."); + } + } + + @Override + public boolean deleteSnapshot(final SnapshotDescription snapshot, final StorageContext ctx) + throws IOException { + return deleteSnapshot(snapshot.getName(), ctx); + } + + @Override + public boolean deleteSnapshot(final String snapshotName, final StorageContext ctx) + throws IOException { + deleteSnapshotDir(getSnapshotDirFromContext(ctx, snapshotName)); + return false; + } + + @Override + public void resetSnapshot(SnapshotDescription snapshot) throws IOException { + if (snapshot == null) return; + deleteSnapshot(snapshot); + Path snapshotDir = getSnapshotDirFromContext(StorageContext.TEMP, snapshot.getName()); + if (getFileSystem().mkdirs(snapshotDir)) { + throw new SnapshotCreationException("Couldn't create working directory '" + snapshotDir + + "' for snapshot", ProtobufUtil.createSnapshotDesc(snapshot)); + } + } + + @Override + public void snapshotTable(HTableDescriptor htd, SnapshotDescription snapshot, + final ForeignExceptionSnare monitor, StorageContext ctx) throws IOException { + Path snapshotDir = getSnapshotDirFromContext(ctx, snapshot.getName()); + + // write down the snapshot info in the working directory + writeSnapshotInfo(snapshot, snapshotDir); + + // create manifest + SnapshotManifest manifest = SnapshotManifest.create(getConfiguration(), getFileSystem(), + snapshotDir, snapshot, monitor); + manifest.addTableDescriptor(htd); + } + + @Override + public void consolidateSnapshot(SnapshotDescription snapshot, StorageContext ctx) + throws IOException { + SnapshotManifest manifest = getSnapshotManifest(snapshot, ctx); + manifest.consolidate(); + } + + @Override + public boolean changeSnapshotContext(SnapshotDescription snapshot, StorageContext src, + StorageContext dest) throws IOException { + Path srcDir = getSnapshotDirFromContext(src, snapshot.getName()); + Path destDir = getSnapshotDirFromContext(dest, snapshot.getName()); + return getFileSystem().rename(srcDir, destDir); + } + + @Override + public void addRegionToSnapshot(HRegionInfo hri, SnapshotDescription snapshot, + StorageContext ctx) throws IOException { + SnapshotManifest manifest = getSnapshotManifest(snapshot, ctx); + Path tableDir = LegacyLayout.getTableDir(LegacyLayout.getDataDir(getRootContainer().path), + hri.getTable()); + manifest.addRegion(tableDir, hri); + } + + @Override + public SnapshotRestoreMetaChanges restoreSnapshot(final SnapshotDescription snapshot, + final StorageContext snapshotCtx, final HTableDescriptor destHtd, + final ForeignExceptionDispatcher monitor, final MonitoredTask status) throws IOException { + // TODO: currently snapshotCtx is not used, modify RestoreSnapshotHelper to take ctx as an input + RestoreSnapshotHelper restoreSnapshotHelper = new RestoreSnapshotHelper(this, snapshot, + destHtd, monitor, status); + return restoreSnapshotHelper.restoreStorageRegions(); + } + + /** + * Write the snapshot description into the working directory of a snapshot + * + * @param snapshot description of the snapshot being taken + * @param workingDir working directory of the snapshot + * @throws IOException if we can't reach the filesystem and the file cannot be cleaned up on + * failure + */ + public void writeSnapshotInfo(SnapshotDescription snapshot, Path workingDir) + throws IOException { + FsPermission perms = FSUtils.getFilePermissions(getFileSystem(), getFileSystem().getConf(), + HConstants.DATA_FILE_UMASK_KEY); + Path snapshotInfo = new Path(workingDir, LegacyLayout.SNAPSHOTINFO_FILE); + try { + FSDataOutputStream out = FSUtils.create(getFileSystem(), snapshotInfo, perms, true); + try { + snapshot.writeTo(out); + } finally { + out.close(); + } + } catch (IOException e) { + // if we get an exception, try to remove the snapshot info + if (!getFileSystem().delete(snapshotInfo, false)) { + String msg = "Couldn't delete snapshot info file: " + snapshotInfo; + LOG.error(msg); + throw new IOException(msg); + } + } + } + + // ========================================================================== // PUBLIC - WAL // ========================================================================== @Override @@ -653,12 +987,27 @@ public class LegacyMasterStorage extends MasterStorage { return new LegacyPathIdentifier(tmpDir); } + protected Path getSnapshotDirFromContext(StorageContext ctx, String snapshot) { + switch(ctx) { + case TEMP: return LegacyLayout.getWorkingSnapshotDir(getRootContainer().path, snapshot); + case DATA: return LegacyLayout.getCompletedSnapshotDir(getRootContainer().path, snapshot); + default: throw new RuntimeException("Invalid context: " + ctx); + } + } + + protected Path getSnapshotDirFromContext(StorageContext ctx) { + switch (ctx) { + case TEMP: return tmpSnapshotDir; + case DATA: return snapshotDir; + default: throw new RuntimeException("Invalid context: " + ctx); + } + } + protected Path getBaseDirFromContext(StorageContext ctx) { switch (ctx) { case TEMP: return tmpDataDir; case DATA: return dataDir; case ARCHIVE: return archiveDataDir; - case SNAPSHOT: return snapshotDir; case SIDELINE: return sidelineDir; default: throw new RuntimeException("Invalid context: " + ctx); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/ExportSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/ExportSnapshot.java new file mode 100644 index 0000000..fb53ca6 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/ExportSnapshot.java @@ -0,0 +1,1102 @@ +/** + * 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.fs.legacy.snapshot; + +import java.io.BufferedInputStream; +import java.io.FileNotFoundException; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.LinkedList; +import java.util.List; +import java.util.Random; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileChecksum; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.fs.MasterStorage; +import org.apache.hadoop.hbase.fs.StorageContext; +import org.apache.hadoop.hbase.fs.legacy.LegacyLayout; +import org.apache.hadoop.hbase.fs.legacy.LegacyMasterStorage; +import org.apache.hadoop.hbase.fs.legacy.LegacyPathIdentifier; +import org.apache.hadoop.hbase.io.FileLink; +import org.apache.hadoop.hbase.io.HFileLink; +import org.apache.hadoop.hbase.io.WALLink; +import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; +import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; +import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo; +import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; +import org.apache.hadoop.hbase.snapshot.ExportSnapshotException; +import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.HFileArchiveUtil; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.InputFormat; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; +import org.apache.hadoop.mapreduce.security.TokenCache; +import org.apache.hadoop.hbase.io.hadoopbackport.ThrottledInputStream; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; + +/** + * Export the specified snapshot to a given FileSystem. + * + * The .snapshot/name folder is copied to the destination cluster + * and then all the hfiles/wals are copied using a Map-Reduce Job in the .archive/ location. + * When everything is done, the second cluster can restore the snapshot. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class ExportSnapshot extends Configured implements Tool { + public static final String NAME = "exportsnapshot"; + /** Configuration prefix for overrides for the source filesystem */ + public static final String CONF_SOURCE_PREFIX = NAME + ".from."; + /** Configuration prefix for overrides for the destination filesystem */ + public static final String CONF_DEST_PREFIX = NAME + ".to."; + + private static final Log LOG = LogFactory.getLog(ExportSnapshot.class); + + private static final String MR_NUM_MAPS = "mapreduce.job.maps"; + private static final String CONF_NUM_SPLITS = "snapshot.export.format.splits"; + private static final String CONF_SNAPSHOT_NAME = "snapshot.export.format.snapshot.name"; + private static final String CONF_SNAPSHOT_DIR = "snapshot.export.format.snapshot.dir"; + private static final String CONF_FILES_USER = "snapshot.export.files.attributes.user"; + private static final String CONF_FILES_GROUP = "snapshot.export.files.attributes.group"; + private static final String CONF_FILES_MODE = "snapshot.export.files.attributes.mode"; + private static final String CONF_CHECKSUM_VERIFY = "snapshot.export.checksum.verify"; + private static final String CONF_OUTPUT_ROOT = "snapshot.export.output.root"; + private static final String CONF_INPUT_ROOT = "snapshot.export.input.root"; + private static final String CONF_BUFFER_SIZE = "snapshot.export.buffer.size"; + private static final String CONF_MAP_GROUP = "snapshot.export.default.map.group"; + private static final String CONF_BANDWIDTH_MB = "snapshot.export.map.bandwidth.mb"; + protected static final String CONF_SKIP_TMP = "snapshot.export.skip.tmp"; + + static final String CONF_TEST_FAILURE = "test.snapshot.export.failure"; + static final String CONF_TEST_RETRY = "test.snapshot.export.failure.retry"; + + // Export Map-Reduce Counters, to keep track of the progress + public enum Counter { + MISSING_FILES, FILES_COPIED, FILES_SKIPPED, COPY_FAILED, + BYTES_EXPECTED, BYTES_SKIPPED, BYTES_COPIED + } + + private static class ExportMapper extends Mapper { + final static int REPORT_SIZE = 1 * 1024 * 1024; + final static int BUFFER_SIZE = 64 * 1024; + + private boolean testFailures; + private Random random; + + private boolean verifyChecksum; + private String filesGroup; + private String filesUser; + private short filesMode; + private int bufferSize; + + private FileSystem outputFs; + private Path outputArchive; + private Path outputRoot; + + private FileSystem inputFs; + private Path inputArchive; + private Path inputRoot; + + @Override + public void setup(Context context) throws IOException { + Configuration conf = context.getConfiguration(); + Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX); + Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX); + + verifyChecksum = conf.getBoolean(CONF_CHECKSUM_VERIFY, true); + + filesGroup = conf.get(CONF_FILES_GROUP); + filesUser = conf.get(CONF_FILES_USER); + filesMode = (short)conf.getInt(CONF_FILES_MODE, 0); + outputRoot = new Path(conf.get(CONF_OUTPUT_ROOT)); + inputRoot = new Path(conf.get(CONF_INPUT_ROOT)); + + inputArchive = new Path(inputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY); + outputArchive = new Path(outputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY); + + testFailures = conf.getBoolean(CONF_TEST_FAILURE, false); + + try { + srcConf.setBoolean("fs." + inputRoot.toUri().getScheme() + ".impl.disable.cache", true); + inputFs = FileSystem.get(inputRoot.toUri(), srcConf); + } catch (IOException e) { + throw new IOException("Could not get the input FileSystem with root=" + inputRoot, e); + } + + try { + destConf.setBoolean("fs." + outputRoot.toUri().getScheme() + ".impl.disable.cache", true); + outputFs = FileSystem.get(outputRoot.toUri(), destConf); + } catch (IOException e) { + throw new IOException("Could not get the output FileSystem with root="+ outputRoot, e); + } + + // Use the default block size of the outputFs if bigger + int defaultBlockSize = Math.max((int) outputFs.getDefaultBlockSize(outputRoot), BUFFER_SIZE); + bufferSize = conf.getInt(CONF_BUFFER_SIZE, defaultBlockSize); + LOG.info("Using bufferSize=" + StringUtils.humanReadableInt(bufferSize)); + + for (Counter c : Counter.values()) { + context.getCounter(c).increment(0); + } + } + + @Override + protected void cleanup(Context context) { + IOUtils.closeStream(inputFs); + IOUtils.closeStream(outputFs); + } + + @Override + public void map(BytesWritable key, NullWritable value, Context context) + throws InterruptedException, IOException { + SnapshotFileInfo inputInfo = SnapshotFileInfo.parseFrom(key.copyBytes()); + Path outputPath = getOutputPath(inputInfo); + + copyFile(context, inputInfo, outputPath); + } + + /** + * Returns the location where the inputPath will be copied. + */ + private Path getOutputPath(final SnapshotFileInfo inputInfo) throws IOException { + Path path = null; + switch (inputInfo.getType()) { + case HFILE: + Path inputPath = new Path(inputInfo.getHfile()); + String family = inputPath.getParent().getName(); + TableName table =HFileLink.getReferencedTableName(inputPath.getName()); + String region = HFileLink.getReferencedRegionName(inputPath.getName()); + String hfile = HFileLink.getReferencedHFileName(inputPath.getName()); + path = new Path(FSUtils.getTableDir(new Path("./"), table), + new Path(region, new Path(family, hfile))); + break; + case WAL: + LOG.warn("snapshot does not keeps WALs: " + inputInfo); + break; + default: + throw new IOException("Invalid File Type: " + inputInfo.getType().toString()); + } + return new Path(outputArchive, path); + } + + /* + * Used by TestExportSnapshot to simulate a failure + */ + private void injectTestFailure(final Context context, final SnapshotFileInfo inputInfo) + throws IOException { + if (testFailures) { + if (context.getConfiguration().getBoolean(CONF_TEST_RETRY, false)) { + if (random == null) { + random = new Random(); + } + + // FLAKY-TEST-WARN: lower is better, we can get some runs without the + // retry, but at least we reduce the number of test failures due to + // this test exception from the same map task. + if (random.nextFloat() < 0.03) { + throw new IOException("TEST RETRY FAILURE: Unable to copy input=" + inputInfo + + " time=" + System.currentTimeMillis()); + } + } else { + context.getCounter(Counter.COPY_FAILED).increment(1); + throw new IOException("TEST FAILURE: Unable to copy input=" + inputInfo); + } + } + } + + private void copyFile(final Context context, final SnapshotFileInfo inputInfo, + final Path outputPath) throws IOException { + injectTestFailure(context, inputInfo); + + // Get the file information + FileStatus inputStat = getSourceFileStatus(context, inputInfo); + + // Verify if the output file exists and is the same that we want to copy + if (outputFs.exists(outputPath)) { + FileStatus outputStat = outputFs.getFileStatus(outputPath); + if (outputStat != null && sameFile(inputStat, outputStat)) { + LOG.info("Skip copy " + inputStat.getPath() + " to " + outputPath + ", same file."); + context.getCounter(Counter.FILES_SKIPPED).increment(1); + context.getCounter(Counter.BYTES_SKIPPED).increment(inputStat.getLen()); + return; + } + } + + InputStream in = openSourceFile(context, inputInfo); + int bandwidthMB = context.getConfiguration().getInt(CONF_BANDWIDTH_MB, 100); + if (Integer.MAX_VALUE != bandwidthMB) { + in = new ThrottledInputStream(new BufferedInputStream(in), bandwidthMB * 1024 * 1024L); + } + + try { + context.getCounter(Counter.BYTES_EXPECTED).increment(inputStat.getLen()); + + // Ensure that the output folder is there and copy the file + createOutputPath(outputPath.getParent()); + FSDataOutputStream out = outputFs.create(outputPath, true); + try { + copyData(context, inputStat.getPath(), in, outputPath, out, inputStat.getLen()); + } finally { + out.close(); + } + + // Try to Preserve attributes + if (!preserveAttributes(outputPath, inputStat)) { + LOG.warn("You may have to run manually chown on: " + outputPath); + } + } finally { + in.close(); + } + } + + /** + * Create the output folder and optionally set ownership. + */ + private void createOutputPath(final Path path) throws IOException { + if (filesUser == null && filesGroup == null) { + outputFs.mkdirs(path); + } else { + Path parent = path.getParent(); + if (!outputFs.exists(parent) && !parent.isRoot()) { + createOutputPath(parent); + } + outputFs.mkdirs(path); + if (filesUser != null || filesGroup != null) { + // override the owner when non-null user/group is specified + outputFs.setOwner(path, filesUser, filesGroup); + } + if (filesMode > 0) { + outputFs.setPermission(path, new FsPermission(filesMode)); + } + } + } + + /** + * Try to Preserve the files attribute selected by the user copying them from the source file + * This is only required when you are exporting as a different user than "hbase" or on a system + * that doesn't have the "hbase" user. + * + * This is not considered a blocking failure since the user can force a chmod with the user + * that knows is available on the system. + */ + private boolean preserveAttributes(final Path path, final FileStatus refStat) { + FileStatus stat; + try { + stat = outputFs.getFileStatus(path); + } catch (IOException e) { + LOG.warn("Unable to get the status for file=" + path); + return false; + } + + try { + if (filesMode > 0 && stat.getPermission().toShort() != filesMode) { + outputFs.setPermission(path, new FsPermission(filesMode)); + } else if (refStat != null && !stat.getPermission().equals(refStat.getPermission())) { + outputFs.setPermission(path, refStat.getPermission()); + } + } catch (IOException e) { + LOG.warn("Unable to set the permission for file="+ stat.getPath() +": "+ e.getMessage()); + return false; + } + + boolean hasRefStat = (refStat != null); + String user = stringIsNotEmpty(filesUser) || !hasRefStat ? filesUser : refStat.getOwner(); + String group = stringIsNotEmpty(filesGroup) || !hasRefStat ? filesGroup : refStat.getGroup(); + if (stringIsNotEmpty(user) || stringIsNotEmpty(group)) { + try { + if (!(user.equals(stat.getOwner()) && group.equals(stat.getGroup()))) { + outputFs.setOwner(path, user, group); + } + } catch (IOException e) { + LOG.warn("Unable to set the owner/group for file="+ stat.getPath() +": "+ e.getMessage()); + LOG.warn("The user/group may not exist on the destination cluster: user=" + + user + " group=" + group); + return false; + } + } + + return true; + } + + private boolean stringIsNotEmpty(final String str) { + return str != null && str.length() > 0; + } + + private void copyData(final Context context, + final Path inputPath, final InputStream in, + final Path outputPath, final FSDataOutputStream out, + final long inputFileSize) + throws IOException { + final String statusMessage = "copied %s/" + StringUtils.humanReadableInt(inputFileSize) + + " (%.1f%%)"; + + try { + byte[] buffer = new byte[bufferSize]; + long totalBytesWritten = 0; + int reportBytes = 0; + int bytesRead; + + long stime = System.currentTimeMillis(); + while ((bytesRead = in.read(buffer)) > 0) { + out.write(buffer, 0, bytesRead); + totalBytesWritten += bytesRead; + reportBytes += bytesRead; + + if (reportBytes >= REPORT_SIZE) { + context.getCounter(Counter.BYTES_COPIED).increment(reportBytes); + context.setStatus(String.format(statusMessage, + StringUtils.humanReadableInt(totalBytesWritten), + (totalBytesWritten/(float)inputFileSize) * 100.0f) + + " from " + inputPath + " to " + outputPath); + reportBytes = 0; + } + } + long etime = System.currentTimeMillis(); + + context.getCounter(Counter.BYTES_COPIED).increment(reportBytes); + context.setStatus(String.format(statusMessage, + StringUtils.humanReadableInt(totalBytesWritten), + (totalBytesWritten/(float)inputFileSize) * 100.0f) + + " from " + inputPath + " to " + outputPath); + + // Verify that the written size match + if (totalBytesWritten != inputFileSize) { + String msg = "number of bytes copied not matching copied=" + totalBytesWritten + + " expected=" + inputFileSize + " for file=" + inputPath; + throw new IOException(msg); + } + + LOG.info("copy completed for input=" + inputPath + " output=" + outputPath); + LOG.info("size=" + totalBytesWritten + + " (" + StringUtils.humanReadableInt(totalBytesWritten) + ")" + + " time=" + StringUtils.formatTimeDiff(etime, stime) + + String.format(" %.3fM/sec", (totalBytesWritten / ((etime - stime)/1000.0))/1048576.0)); + context.getCounter(Counter.FILES_COPIED).increment(1); + } catch (IOException e) { + LOG.error("Error copying " + inputPath + " to " + outputPath, e); + context.getCounter(Counter.COPY_FAILED).increment(1); + throw e; + } + } + + /** + * Try to open the "source" file. + * Throws an IOException if the communication with the inputFs fail or + * if the file is not found. + */ + private FSDataInputStream openSourceFile(Context context, final SnapshotFileInfo fileInfo) + throws IOException { + try { + Configuration conf = context.getConfiguration(); + FileLink link = null; + switch (fileInfo.getType()) { + case HFILE: + Path inputPath = new Path(fileInfo.getHfile()); + link = getFileLink(inputPath, conf); + break; + case WAL: + String serverName = fileInfo.getWalServer(); + String logName = fileInfo.getWalName(); + link = new WALLink(inputRoot, serverName, logName); + break; + default: + throw new IOException("Invalid File Type: " + fileInfo.getType().toString()); + } + return link.open(inputFs); + } catch (IOException e) { + context.getCounter(Counter.MISSING_FILES).increment(1); + LOG.error("Unable to open source file=" + fileInfo.toString(), e); + throw e; + } + } + + private FileStatus getSourceFileStatus(Context context, final SnapshotFileInfo fileInfo) + throws IOException { + try { + Configuration conf = context.getConfiguration(); + FileLink link = null; + switch (fileInfo.getType()) { + case HFILE: + Path inputPath = new Path(fileInfo.getHfile()); + link = getFileLink(inputPath, conf); + break; + case WAL: + link = new WALLink(inputRoot, fileInfo.getWalServer(), fileInfo.getWalName()); + break; + default: + throw new IOException("Invalid File Type: " + fileInfo.getType().toString()); + } + return link.getFileStatus(inputFs); + } catch (FileNotFoundException e) { + context.getCounter(Counter.MISSING_FILES).increment(1); + LOG.error("Unable to get the status for source file=" + fileInfo.toString(), e); + throw e; + } catch (IOException e) { + LOG.error("Unable to get the status for source file=" + fileInfo.toString(), e); + throw e; + } + } + + private FileLink getFileLink(Path path, Configuration conf) throws IOException{ + String regionName = HFileLink.getReferencedRegionName(path.getName()); + TableName tableName = HFileLink.getReferencedTableName(path.getName()); + if(MobUtils.getMobRegionInfo(tableName).getEncodedName().equals(regionName)) { + return HFileLink.buildFromHFileLinkPattern(MobUtils.getQualifiedMobRootDir(conf), + HFileArchiveUtil.getArchivePath(conf), path); + } + return HFileLink.buildFromHFileLinkPattern(inputRoot, inputArchive, path); + } + + private FileChecksum getFileChecksum(final FileSystem fs, final Path path) { + try { + return fs.getFileChecksum(path); + } catch (IOException e) { + LOG.warn("Unable to get checksum for file=" + path, e); + return null; + } + } + + /** + * Check if the two files are equal by looking at the file length, + * and at the checksum (if user has specified the verifyChecksum flag). + */ + private boolean sameFile(final FileStatus inputStat, final FileStatus outputStat) { + // Not matching length + if (inputStat.getLen() != outputStat.getLen()) return false; + + // Mark files as equals, since user asked for no checksum verification + if (!verifyChecksum) return true; + + // If checksums are not available, files are not the same. + FileChecksum inChecksum = getFileChecksum(inputFs, inputStat.getPath()); + if (inChecksum == null) return false; + + FileChecksum outChecksum = getFileChecksum(outputFs, outputStat.getPath()); + if (outChecksum == null) return false; + + return inChecksum.equals(outChecksum); + } + } + + // ========================================================================== + // Input Format + // ========================================================================== + + /** + * Extract the list of files (HFiles/WALs) to copy using Map-Reduce. + * @return list of files referenced by the snapshot (pair of path and size) + */ + private static List> getSnapshotFiles(final Configuration conf, + final FileSystem fs, final String snapshotName, StorageContext ctx) throws IOException { + LegacyMasterStorage lms = new LegacyMasterStorage(conf, fs, + new LegacyPathIdentifier(FSUtils.getRootDir(conf))); + SnapshotDescription snapshotDesc = lms.getSnapshot(snapshotName, ctx); + + final List> files = new ArrayList>(); + final TableName table = TableName.valueOf(snapshotDesc.getTable()); + + // Get snapshot files + LOG.info("Loading Snapshot '" + snapshotDesc.getName() + "' hfile list"); + lms.visitSnapshotStoreFiles(snapshotDesc, ctx, new MasterStorage.SnapshotStoreFileVisitor() { + @Override + public void visitSnapshotStoreFile(SnapshotDescription snapshot, StorageContext ctx, + HRegionInfo hri, String familyName, SnapshotRegionManifest.StoreFile storeFile) + throws IOException { + // for storeFile.hasReference() case, copied as part of the manifest + if (!storeFile.hasReference()) { + String region = hri.getEncodedName(); + String hfile = storeFile.getName(); + Path path = HFileLink.createPath(table, region, familyName, hfile); + + SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder() + .setType(SnapshotFileInfo.Type.HFILE) + .setHfile(path.toString()) + .build(); + + long size; + if (storeFile.hasFileSize()) { + size = storeFile.getFileSize(); + } else { + size = HFileLink.buildFromHFileLinkPattern(conf, path).getFileStatus(fs).getLen(); + } + files.add(new Pair(fileInfo, size)); + } + } + }); + return files; + } + + /** + * Given a list of file paths and sizes, create around ngroups in as balanced a way as possible. + * The groups created will have similar amounts of bytes. + *

+ * The algorithm used is pretty straightforward; the file list is sorted by size, + * and then each group fetch the bigger file available, iterating through groups + * alternating the direction. + */ + static List>> getBalancedSplits( + final List> files, final int ngroups) { + // Sort files by size, from small to big + Collections.sort(files, new Comparator>() { + public int compare(Pair a, Pair b) { + long r = a.getSecond() - b.getSecond(); + return (r < 0) ? -1 : ((r > 0) ? 1 : 0); + } + }); + + // create balanced groups + List>> fileGroups = + new LinkedList>>(); + long[] sizeGroups = new long[ngroups]; + int hi = files.size() - 1; + int lo = 0; + + List> group; + int dir = 1; + int g = 0; + + while (hi >= lo) { + if (g == fileGroups.size()) { + group = new LinkedList>(); + fileGroups.add(group); + } else { + group = fileGroups.get(g); + } + + Pair fileInfo = files.get(hi--); + + // add the hi one + sizeGroups[g] += fileInfo.getSecond(); + group.add(fileInfo); + + // change direction when at the end or the beginning + g += dir; + if (g == ngroups) { + dir = -1; + g = ngroups - 1; + } else if (g < 0) { + dir = 1; + g = 0; + } + } + + if (LOG.isDebugEnabled()) { + for (int i = 0; i < sizeGroups.length; ++i) { + LOG.debug("export split=" + i + " size=" + StringUtils.humanReadableInt(sizeGroups[i])); + } + } + + return fileGroups; + } + + private static class ExportSnapshotInputFormat extends InputFormat { + @Override + public RecordReader createRecordReader(InputSplit split, + TaskAttemptContext tac) throws IOException, InterruptedException { + return new ExportSnapshotRecordReader(((ExportSnapshotInputSplit)split).getSplitKeys()); + } + + @Override + public List getSplits(JobContext context) throws IOException, InterruptedException { + Configuration conf = context.getConfiguration(); + String snapshotName = conf.get(CONF_SNAPSHOT_NAME); + Path snapshotDir = new Path(conf.get(CONF_SNAPSHOT_DIR)); + FileSystem fs = FileSystem.get(snapshotDir.toUri(), conf); + + List> snapshotFiles = getSnapshotFiles(conf, fs, snapshotName, + StorageContext.DATA); + int mappers = conf.getInt(CONF_NUM_SPLITS, 0); + if (mappers == 0 && snapshotFiles.size() > 0) { + mappers = 1 + (snapshotFiles.size() / conf.getInt(CONF_MAP_GROUP, 10)); + mappers = Math.min(mappers, snapshotFiles.size()); + conf.setInt(CONF_NUM_SPLITS, mappers); + conf.setInt(MR_NUM_MAPS, mappers); + } + + List>> groups = getBalancedSplits(snapshotFiles, mappers); + List splits = new ArrayList(groups.size()); + for (List> files: groups) { + splits.add(new ExportSnapshotInputSplit(files)); + } + return splits; + } + + private static class ExportSnapshotInputSplit extends InputSplit implements Writable { + private List> files; + private long length; + + public ExportSnapshotInputSplit() { + this.files = null; + } + + public ExportSnapshotInputSplit(final List> snapshotFiles) { + this.files = new ArrayList(snapshotFiles.size()); + for (Pair fileInfo: snapshotFiles) { + this.files.add(new Pair( + new BytesWritable(fileInfo.getFirst().toByteArray()), fileInfo.getSecond())); + this.length += fileInfo.getSecond(); + } + } + + private List> getSplitKeys() { + return files; + } + + @Override + public long getLength() throws IOException, InterruptedException { + return length; + } + + @Override + public String[] getLocations() throws IOException, InterruptedException { + return new String[] {}; + } + + @Override + public void readFields(DataInput in) throws IOException { + int count = in.readInt(); + files = new ArrayList>(count); + length = 0; + for (int i = 0; i < count; ++i) { + BytesWritable fileInfo = new BytesWritable(); + fileInfo.readFields(in); + long size = in.readLong(); + files.add(new Pair(fileInfo, size)); + length += size; + } + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(files.size()); + for (final Pair fileInfo: files) { + fileInfo.getFirst().write(out); + out.writeLong(fileInfo.getSecond()); + } + } + } + + private static class ExportSnapshotRecordReader + extends RecordReader { + private final List> files; + private long totalSize = 0; + private long procSize = 0; + private int index = -1; + + ExportSnapshotRecordReader(final List> files) { + this.files = files; + for (Pair fileInfo: files) { + totalSize += fileInfo.getSecond(); + } + } + + @Override + public void close() { } + + @Override + public BytesWritable getCurrentKey() { return files.get(index).getFirst(); } + + @Override + public NullWritable getCurrentValue() { return NullWritable.get(); } + + @Override + public float getProgress() { return (float)procSize / totalSize; } + + @Override + public void initialize(InputSplit split, TaskAttemptContext tac) { } + + @Override + public boolean nextKeyValue() { + if (index >= 0) { + procSize += files.get(index).getSecond(); + } + return(++index < files.size()); + } + } + } + + // ========================================================================== + // Tool + // ========================================================================== + + /** + * Run Map-Reduce Job to perform the files copy. + */ + private void runCopyJob(final Path inputRoot, final Path outputRoot, + final String snapshotName, final Path snapshotDir, final boolean verifyChecksum, + final String filesUser, final String filesGroup, final int filesMode, + final int mappers, final int bandwidthMB) + throws IOException, InterruptedException, ClassNotFoundException { + Configuration conf = getConf(); + if (filesGroup != null) conf.set(CONF_FILES_GROUP, filesGroup); + if (filesUser != null) conf.set(CONF_FILES_USER, filesUser); + if (mappers > 0) { + conf.setInt(CONF_NUM_SPLITS, mappers); + conf.setInt(MR_NUM_MAPS, mappers); + } + conf.setInt(CONF_FILES_MODE, filesMode); + conf.setBoolean(CONF_CHECKSUM_VERIFY, verifyChecksum); + conf.set(CONF_OUTPUT_ROOT, outputRoot.toString()); + conf.set(CONF_INPUT_ROOT, inputRoot.toString()); + conf.setInt(CONF_BANDWIDTH_MB, bandwidthMB); + conf.set(CONF_SNAPSHOT_NAME, snapshotName); + conf.set(CONF_SNAPSHOT_DIR, snapshotDir.toString()); + + Job job = new Job(conf); + job.setJobName("ExportSnapshot-" + snapshotName); + job.setJarByClass(ExportSnapshot.class); + TableMapReduceUtil.addDependencyJars(job); + job.setMapperClass(ExportMapper.class); + job.setInputFormatClass(ExportSnapshotInputFormat.class); + job.setOutputFormatClass(NullOutputFormat.class); + job.setMapSpeculativeExecution(false); + job.setNumReduceTasks(0); + + // Acquire the delegation Tokens + Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX); + TokenCache.obtainTokensForNamenodes(job.getCredentials(), + new Path[] { inputRoot }, srcConf); + Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX); + TokenCache.obtainTokensForNamenodes(job.getCredentials(), + new Path[] { outputRoot }, destConf); + + // Run the MR Job + if (!job.waitForCompletion(true)) { + // TODO: Replace the fixed string with job.getStatus().getFailureInfo() + // when it will be available on all the supported versions. + throw new ExportSnapshotException("Copy Files Map-Reduce Job failed"); + } + } + + private void verifySnapshot(final Configuration baseConf, + final FileSystem fs, final Path rootDir, final String snapshotName, StorageContext ctx) + throws IOException { + // Update the conf with the current root dir, since may be a different cluster + Configuration conf = new Configuration(baseConf); + FSUtils.setRootDir(conf, rootDir); + FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf)); + LegacyMasterStorage lms = + new LegacyMasterStorage(baseConf, fs, new LegacyPathIdentifier(rootDir)); + SnapshotDescription snapshotDesc = lms.getSnapshot(snapshotName, ctx); + SnapshotReferenceUtil.verifySnapshot(lms, snapshotDesc, ctx); + } + + /** + * Set path ownership. + */ + private void setOwner(final FileSystem fs, final Path path, final String user, + final String group, final boolean recursive) throws IOException { + if (user != null || group != null) { + if (recursive && fs.isDirectory(path)) { + for (FileStatus child : fs.listStatus(path)) { + setOwner(fs, child.getPath(), user, group, recursive); + } + } + fs.setOwner(path, user, group); + } + } + + /** + * Set path permission. + */ + private void setPermission(final FileSystem fs, final Path path, final short filesMode, + final boolean recursive) throws IOException { + if (filesMode > 0) { + FsPermission perm = new FsPermission(filesMode); + if (recursive && fs.isDirectory(path)) { + for (FileStatus child : fs.listStatus(path)) { + setPermission(fs, child.getPath(), filesMode, recursive); + } + } + fs.setPermission(path, perm); + } + } + + /** + * Execute the export snapshot by copying the snapshot metadata, hfiles and wals. + * @return 0 on success, and != 0 upon failure. + */ + @Override + public int run(String[] args) throws IOException { + boolean verifyTarget = true; + boolean verifyChecksum = true; + String snapshotName = null; + String targetName = null; + boolean overwrite = false; + String filesGroup = null; + String filesUser = null; + Path outputRoot = null; + int bandwidthMB = Integer.MAX_VALUE; + int filesMode = 0; + int mappers = 0; + + Configuration conf = getConf(); + Path inputRoot = FSUtils.getRootDir(conf); + + // Process command line args + for (int i = 0; i < args.length; i++) { + String cmd = args[i]; + if (cmd.equals("-snapshot")) { + snapshotName = args[++i]; + } else if (cmd.equals("-target")) { + targetName = args[++i]; + } else if (cmd.equals("-copy-to")) { + outputRoot = new Path(args[++i]); + } else if (cmd.equals("-copy-from")) { + inputRoot = new Path(args[++i]); + FSUtils.setRootDir(conf, inputRoot); + } else if (cmd.equals("-no-checksum-verify")) { + verifyChecksum = false; + } else if (cmd.equals("-no-target-verify")) { + verifyTarget = false; + } else if (cmd.equals("-mappers")) { + mappers = Integer.parseInt(args[++i]); + } else if (cmd.equals("-chuser")) { + filesUser = args[++i]; + } else if (cmd.equals("-chgroup")) { + filesGroup = args[++i]; + } else if (cmd.equals("-bandwidth")) { + bandwidthMB = Integer.parseInt(args[++i]); + } else if (cmd.equals("-chmod")) { + filesMode = Integer.parseInt(args[++i], 8); + } else if (cmd.equals("-overwrite")) { + overwrite = true; + } else if (cmd.equals("-h") || cmd.equals("--help")) { + printUsageAndExit(); + } else { + System.err.println("UNEXPECTED: " + cmd); + printUsageAndExit(); + } + } + + // Check user options + if (snapshotName == null) { + System.err.println("Snapshot name not provided."); + printUsageAndExit(); + } + + if (outputRoot == null) { + System.err.println("Destination file-system not provided."); + printUsageAndExit(); + } + + if (targetName == null) { + targetName = snapshotName; + } + + Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX); + srcConf.setBoolean("fs." + inputRoot.toUri().getScheme() + ".impl.disable.cache", true); + FileSystem inputFs = FileSystem.get(inputRoot.toUri(), srcConf); + LOG.debug("inputFs=" + inputFs.getUri().toString() + " inputRoot=" + inputRoot); + Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX); + destConf.setBoolean("fs." + outputRoot.toUri().getScheme() + ".impl.disable.cache", true); + FileSystem outputFs = FileSystem.get(outputRoot.toUri(), destConf); + LOG.debug("outputFs=" + outputFs.getUri().toString() + " outputRoot=" + outputRoot.toString()); + + boolean skipTmp = conf.getBoolean(CONF_SKIP_TMP, false); + + LegacyMasterStorage srcMasterStorage = new LegacyMasterStorage(srcConf, inputFs, + new LegacyPathIdentifier(inputRoot)); + LegacyMasterStorage destMasterStorage = new LegacyMasterStorage(destConf, outputFs, + new LegacyPathIdentifier(outputRoot)); + StorageContext destCtx = skipTmp ? StorageContext.TEMP : StorageContext.DATA; + + Path snapshotDir = LegacyLayout.getCompletedSnapshotDir(inputRoot, snapshotName); + Path snapshotTmpDir = LegacyLayout.getWorkingSnapshotDir(outputRoot, targetName); + Path outputSnapshotDir = LegacyLayout.getCompletedSnapshotDir(outputRoot, targetName); + Path initialOutputSnapshotDir = skipTmp ? outputSnapshotDir : snapshotTmpDir; + + // Check if the snapshot already exists + if (destMasterStorage.snapshotExists(targetName)) { + if (overwrite) { + if (!destMasterStorage.deleteSnapshot(targetName)) { + System.err.println("Unable to remove existing snapshot '" + targetName + "'."); + return 1; + } + } else { + System.err.println("The snapshot '" + targetName + "' already exists in the destination: " + + LegacyLayout.getCompletedSnapshotDir(outputRoot, targetName)); + return 1; + } + } + + if (!skipTmp) { + // Check if the snapshot already in-progress + if (destMasterStorage.snapshotExists(targetName, destCtx)) { + if (overwrite) { + if (!outputFs.delete(snapshotTmpDir, true)) { + System.err.println("Unable to remove existing snapshot tmp directory: "+snapshotTmpDir); + return 1; + } + } else { + System.err.println("A snapshot with the same name '"+ targetName +"' may be in-progress"); + System.err.println("Please check "+snapshotTmpDir+". If the snapshot has completed, "); + System.err.println("consider removing "+snapshotTmpDir+" by using the -overwrite option"); + return 1; + } + } + } + + // Step 1 - Copy fs1:/.snapshot/ to fs2:/.snapshot/.tmp/ + // The snapshot references must be copied before the hfiles otherwise the cleaner + // will remove them because they are unreferenced. + try { + LOG.info("Copy Snapshot Manifest"); + FileUtil.copy(inputFs, snapshotDir, outputFs, initialOutputSnapshotDir, false, false, conf); + if (filesUser != null || filesGroup != null) { + setOwner(outputFs, snapshotTmpDir, filesUser, filesGroup, true); + } + if (filesMode > 0) { + setPermission(outputFs, snapshotTmpDir, (short)filesMode, true); + } + } catch (IOException e) { + throw new ExportSnapshotException("Failed to copy the snapshot directory: from=" + + snapshotDir + " to=" + initialOutputSnapshotDir, e); + } + + // Write a new .snapshotinfo if the target name is different from the source name + if (!targetName.equals(snapshotName)) { + SnapshotDescription snapshotDesc = srcMasterStorage.getSnapshot(snapshotName) + .toBuilder() + .setName(targetName) + .build(); + destMasterStorage.writeSnapshotInfo(snapshotDesc, snapshotTmpDir); + } + + // Step 2 - Start MR Job to copy files + // The snapshot references must be copied before the files otherwise the files gets removed + // by the HFileArchiver, since they have no references. + try { + runCopyJob(inputRoot, outputRoot, snapshotName, snapshotDir, verifyChecksum, + filesUser, filesGroup, filesMode, mappers, bandwidthMB); + + LOG.info("Finalize the Snapshot Export"); + if (!skipTmp) { + // Step 3 - Rename fs2:/.snapshot/.tmp/ fs2:/.snapshot/ + if (!outputFs.rename(snapshotTmpDir, outputSnapshotDir)) { + throw new ExportSnapshotException("Unable to rename snapshot directory from=" + + snapshotTmpDir + " to=" + outputSnapshotDir); + } + } + + // Step 4 - Verify snapshot integrity + if (verifyTarget) { + LOG.info("Verify snapshot integrity"); + verifySnapshot(destConf, outputFs, outputRoot, targetName, StorageContext.DATA); + } + + LOG.info("Export Completed: " + targetName); + return 0; + } catch (Exception e) { + LOG.error("Snapshot export failed", e); + if (!skipTmp) { + outputFs.delete(snapshotTmpDir, true); + } + outputFs.delete(outputSnapshotDir, true); + return 1; + } finally { + IOUtils.closeStream(inputFs); + IOUtils.closeStream(outputFs); + } + } + + // ExportSnapshot + private void printUsageAndExit() { + System.err.printf("Usage: bin/hbase %s [options]%n", getClass().getName()); + System.err.println(" where [options] are:"); + System.err.println(" -h|-help Show this help and exit."); + System.err.println(" -snapshot NAME Snapshot to restore."); + System.err.println(" -copy-to NAME Remote destination hdfs://"); + System.err.println(" -copy-from NAME Input folder hdfs:// (default hbase.rootdir)"); + System.err.println(" -no-checksum-verify Do not verify checksum, use name+length only."); + System.err.println(" -no-target-verify Do not verify the integrity of the \\" + + "exported snapshot."); + System.err.println(" -overwrite Rewrite the snapshot manifest if already exists"); + System.err.println(" -chuser USERNAME Change the owner of the files " + + "to the specified one."); + System.err.println(" -chgroup GROUP Change the group of the files to " + + "the specified one."); + System.err.println(" -chmod MODE Change the permission of the files " + + "to the specified one."); + System.err.println(" -mappers Number of mappers to use during the " + + "copy (mapreduce.job.maps)."); + System.err.println(" -bandwidth Limit bandwidth to this value in MB/second."); + System.err.println(); + System.err.println("Examples:"); + System.err.println(" hbase snapshot export \\"); + System.err.println(" -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase \\"); + System.err.println(" -chuser MyUser -chgroup MyGroup -chmod 700 -mappers 16"); + System.err.println(); + System.err.println(" hbase snapshot export \\"); + System.err.println(" -snapshot MySnapshot -copy-from hdfs://srv2:8082/hbase \\"); + System.err.println(" -copy-to hdfs://srv1:50070/hbase \\"); + System.exit(1); + } + + /** + * The guts of the {@link #main} method. + * Call this method to avoid the {@link #main(String[])} System.exit. + * @param args + * @return errCode + * @throws Exception + */ + static int innerMain(final Configuration conf, final String [] args) throws Exception { + return ToolRunner.run(conf, new ExportSnapshot(), args); + } + + public static void main(String[] args) throws Exception { + System.exit(innerMain(HBaseConfiguration.create(), args)); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/RestoreSnapshotHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/RestoreSnapshotHelper.java new file mode 100644 index 0000000..a62f73f --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/RestoreSnapshotHelper.java @@ -0,0 +1,689 @@ +/** + * 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.fs.legacy.snapshot; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +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.fs.FileSystem; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +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.errorhandling.ForeignExceptionDispatcher; +import org.apache.hadoop.hbase.fs.MasterStorage; +import org.apache.hadoop.hbase.fs.RegionStorage; +import org.apache.hadoop.hbase.fs.StorageIdentifier; +import org.apache.hadoop.hbase.fs.legacy.LegacyPathIdentifier; +import org.apache.hadoop.hbase.io.HFileLink; +import org.apache.hadoop.hbase.io.Reference; +import org.apache.hadoop.hbase.mob.MobUtils; +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.StoreFileInfo; +import org.apache.hadoop.hbase.snapshot.SnapshotRestoreMetaChanges; +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 + *
+ * + * TODO update for MasterStorage / RegionStorage + */ +@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 tableDir; + + + private final Configuration conf; + private final FileSystem fs; + private final MasterStorage masterStorage; + private final boolean createBackRefs; + + public RestoreSnapshotHelper(final MasterStorage masterStorage, + final SnapshotDescription snapshotDesc, final HTableDescriptor tableDescriptor, + final ForeignExceptionDispatcher monitor, final MonitoredTask status) throws IOException { + this(masterStorage, snapshotDesc, tableDescriptor, monitor, status, true); + } + + public RestoreSnapshotHelper(final MasterStorage masterStorage, + final SnapshotDescription snapshotDesc, final HTableDescriptor tableDescriptor, + final ForeignExceptionDispatcher monitor, final MonitoredTask status, + final boolean createBackRefs) throws IOException { + this.masterStorage = masterStorage; + this.conf = masterStorage.getConfiguration(); + this.fs = masterStorage.getFileSystem(); + this.snapshotDesc = snapshotDesc; + this.snapshotManifest = SnapshotManifest.open(conf, snapshotDesc); + this.snapshotTable = TableName.valueOf(snapshotDesc.getTable()); + this.tableDesc = tableDescriptor; + this.tableDir = FSUtils.getTableDir(FSUtils.getRootDir(conf), 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 SnapshotRestoreMetaChanges restoreStorageRegions() throws IOException { + ThreadPoolExecutor exec = SnapshotManifest.createExecutor(conf, "RestoreSnapshot"); + try { + return restoreHdfsRegions(exec); + } finally { + exec.shutdown(); + } + } + + private SnapshotRestoreMetaChanges restoreHdfsRegions(final ThreadPoolExecutor exec) throws IOException { + LOG.info("starting restore table regions using snapshot=" + snapshotDesc); + + Map regionManifests = snapshotManifest.getRegionManifestsMap(); + if (regionManifests == null) { + LOG.warn("Nothing to restore. Snapshot " + snapshotDesc + " looks empty"); + return null; + } + + SnapshotRestoreMetaChanges metaChanges = new SnapshotRestoreMetaChanges(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()); + + HRegionInfo mobRegion = MobUtils.getMobRegionInfo(snapshotManifest.getTableDescriptor() + .getTableName()); + // 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); + metaChanges.addRegionToRestore(regionInfo); + } else { + LOG.info("region to remove: " + regionName); + metaChanges.addRegionToRemove(regionInfo); + } + } + + // Restore regions using the snapshot data + monitor.rethrowException(); + status.setStatus("Restoring table regions..."); + if (regionNames.contains(mobRegion.getEncodedName())) { + // restore the mob region in case + List mobRegions = new ArrayList(1); + mobRegions.add(mobRegion); + restoreHdfsMobRegions(exec, regionManifests, mobRegions); + regionNames.remove(mobRegion.getEncodedName()); + } + 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(); + // add the mob region + if (regionNames.contains(mobRegion.getEncodedName())) { + cloneHdfsMobRegion(regionManifests, mobRegion); + regionNames.remove(mobRegion.getEncodedName()); + } + 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."); + } + + LOG.info("finishing restore table regions using snapshot=" + snapshotDesc); + + return metaChanges; + } + + /** + * 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())); + } + }); + } + + /** + * Restore specified mob regions by restoring content to the snapshot state. + */ + private void restoreHdfsMobRegions(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 { + restoreMobRegion(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 { + restoreRegion(regionInfo, regionManifest, new Path(tableDir, regionInfo.getEncodedName())); + } + + /** + * Restore mob region by removing files not in the snapshot + * and adding the missing ones from the snapshot. + */ + private void restoreMobRegion(final HRegionInfo regionInfo, + final SnapshotRegionManifest regionManifest) throws IOException { + if (regionManifest == null) { + return; + } + restoreRegion(regionInfo, regionManifest, + MobUtils.getMobRegionPath(conf, tableDesc.getTableName())); + } + + /** + * 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, Path regionDir) throws IOException { + Map> snapshotFiles = + getRegionHFileReferences(regionManifest); + + 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, + 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 the mob region. For the region create a new region + * and create a HFileLink for each hfile. + */ + private void cloneHdfsMobRegion(final Map regionManifests, + final HRegionInfo region) throws IOException { + // clone region info (change embedded tableName with the new one) + Path clonedRegionPath = MobUtils.getMobRegionPath(conf, tableDesc.getTableName()); + cloneRegion(clonedRegionPath, region, regionManifests.get(region.getEncodedName())); + } + + /** + * 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 regionDir {@link Path} cloned dir + * @param snapshotRegionInfo + */ + private void cloneRegion(final Path regionDir, final HRegionInfo snapshotRegionInfo, + final SnapshotRegionManifest manifest) throws IOException { + 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); + } + } + } + + /** + * 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 { + cloneRegion(new Path(tableDir, region.getRegionInfo().getEncodedName()), snapshotRegionInfo, + manifest); + } + + /** + * 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 createBackRef - Whether back reference should be created. Defaults to true. + * @param storeFile store file name (can be a Reference, HFileLink or simple HFile) + */ + 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 storeFile 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 LinkedList(); + for (FileStatus regionDir: regionDirs) { + final RegionStorage rs = RegionStorage.open(conf, new LegacyPathIdentifier(regionDir.getPath()), false); + regions.add(rs.getRegionInfo()); + } + 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 masterStorage the {@link MasterStorage} to use + * @param restoreDir + * @param snapshotName + * @throws IOException + */ + public static SnapshotRestoreMetaChanges copySnapshotForScanner( + final MasterStorage masterStorage, Path restoreDir, + String snapshotName) throws IOException { + Configuration conf = masterStorage.getConfiguration(); + Path rootDir = ((LegacyPathIdentifier)masterStorage.getRootContainer()).path; + // 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); + } + + SnapshotDescription snapshotDesc = masterStorage.getSnapshot(snapshotName); + HTableDescriptor htd = masterStorage.getTableDescriptorForSnapshot(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(masterStorage, snapshotDesc, htd, + monitor, status, false); + SnapshotRestoreMetaChanges metaChanges = helper.restoreStorageRegions(); // TODO: parallelize. + + if (LOG.isDebugEnabled()) { + LOG.debug("Restored table dir:" + restoreDir); + FSUtils.logFileSystemState(masterStorage.getFileSystem(), restoreDir, LOG); + } + return metaChanges; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotFileCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotFileCache.java index f9f1c67..2653c8f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotFileCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotFileCache.java @@ -40,6 +40,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.fs.legacy.LegacyLayout; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; @@ -89,6 +90,7 @@ public class SnapshotFileCache implements Stoppable { private static final Log LOG = LogFactory.getLog(SnapshotFileCache.class); private volatile boolean stop = false; private final FileSystem fs; + private final Path rootDir; private final SnapshotFileInspector fileInspector; private final Path snapshotDir; private final Set cache = new HashSet(); @@ -133,8 +135,9 @@ public class SnapshotFileCache implements Stoppable { public SnapshotFileCache(FileSystem fs, Path rootDir, long cacheRefreshPeriod, long cacheRefreshDelay, String refreshThreadName, SnapshotFileInspector inspectSnapshotFiles) { this.fs = fs; + this.rootDir = rootDir; this.fileInspector = inspectSnapshotFiles; - this.snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir); + this.snapshotDir = LegacyLayout.getSnapshotDir(rootDir); // periodically refresh the file cache to make sure we aren't superfluously saving files. this.refreshTimer = new Timer(refreshThreadName, true); this.refreshTimer.scheduleAtFixedRate(new RefreshCacheTask(), cacheRefreshDelay, @@ -224,7 +227,7 @@ public class SnapshotFileCache implements Stoppable { // get the status of the snapshots temporary directory and check if it has changes // The top-level directory timestamp is not updated, so we have to check the inner-level. try { - Path snapshotTmpDir = new Path(snapshotDir, SnapshotDescriptionUtils.SNAPSHOT_TMP_DIR_NAME); + Path snapshotTmpDir = LegacyLayout.getWorkingSnapshotDir(rootDir); FileStatus tempDirStatus = fs.getFileStatus(snapshotTmpDir); lastTimestamp = Math.min(lastTimestamp, tempDirStatus.getModificationTime()); hasChanges |= (lastTimestamp >= lastModifiedTime); @@ -273,7 +276,7 @@ public class SnapshotFileCache implements Stoppable { for (FileStatus snapshot : snapshots) { String name = snapshot.getPath().getName(); // its not the tmp dir, - if (!name.equals(SnapshotDescriptionUtils.SNAPSHOT_TMP_DIR_NAME)) { + if (!name.equals(LegacyLayout.SNAPSHOT_TMP_DIR_NAME)) { SnapshotDirectoryInfo files = this.snapshots.remove(name); // 3.1.1 if we don't know about the snapshot or its been modified, we need to update the // files the latter could occur where I create a snapshot, then delete it, and then make a @@ -300,7 +303,7 @@ public class SnapshotFileCache implements Stoppable { final SnapshotManager snapshotManager) throws IOException { List snapshotInProgress = Lists.newArrayList(); // only add those files to the cache, but not to the known snapshots - Path snapshotTmpDir = new Path(snapshotDir, SnapshotDescriptionUtils.SNAPSHOT_TMP_DIR_NAME); + Path snapshotTmpDir = LegacyLayout.getWorkingSnapshotDir(rootDir); // only add those files to the cache, but not to the known snapshots FileStatus[] running = FSUtils.listStatus(fs, snapshotTmpDir); if (running != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotHFileCleaner.java index 89704f0..24c4274 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotHFileCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotHFileCleaner.java @@ -97,7 +97,8 @@ public class SnapshotHFileCleaner extends BaseHFileCleanerDelegate { "snapshot-hfile-cleaner-cache-refresher", new SnapshotFileCache.SnapshotFileInspector() { public Collection filesUnderSnapshot(final Path snapshotDir) throws IOException { - return SnapshotReferenceUtil.getHFileNames(conf, fs, snapshotDir); + return SnapshotReferenceUtil.getHFileNames(master.getMasterStorage(), + snapshotDir.getName()); } }); } catch (IOException e) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotManifest.java new file mode 100644 index 0000000..58f7bf1 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotManifest.java @@ -0,0 +1,570 @@ +/** + * 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.fs.legacy.snapshot; + +import com.google.protobuf.CodedInputStream; +import com.google.protobuf.InvalidProtocolBufferException; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +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.HTableDescriptor; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.fs.RegionStorage; +import org.apache.hadoop.hbase.fs.legacy.LegacyLayout; +import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor; +import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare; +import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +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.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException; +import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.Threads; + +/** + * Utility class to help read/write the Snapshot Manifest. + * + * The snapshot format is transparent for the users of this class, + * once the snapshot is written, it will never be modified. + * On open() the snapshot will be loaded to the current in-memory format. + */ +@InterfaceAudience.Private +public final class SnapshotManifest { + private static final Log LOG = LogFactory.getLog(SnapshotManifest.class); + + 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 regionManifests; + private SnapshotDescription desc; + private HTableDescriptor htd; + + private final ForeignExceptionSnare monitor; + 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, + final ForeignExceptionSnare monitor) { + this.monitor = monitor; + this.desc = desc; + this.workingDir = workingDir; + this.conf = conf; + this.fs = fs; + + this.manifestSizeLimit = conf.getInt(SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY, 64 * 1024 * 1024); + } + + /** + * Return a SnapshotManifest instance, used for writing a snapshot. + * + * There are two usage pattern: + * - The Master will create a manifest, add the descriptor, offline regions + * and consolidate the snapshot by writing all the pending stuff on-disk. + * manifest = SnapshotManifest.create(...) + * manifest.addRegion(tableDir, hri) + * manifest.consolidate() + * - The RegionServer will create a single region manifest + * manifest = SnapshotManifest.create(...) + * manifest.addRegion(region) + */ + public static SnapshotManifest create(final Configuration conf, final FileSystem fs, + final Path workingDir, final SnapshotDescription desc, + final ForeignExceptionSnare monitor) { + return new SnapshotManifest(conf, fs, workingDir, desc, monitor); + } + + /** + * Return a SnapshotManifest instance with the information already loaded in-memory. + * SnapshotManifest manifest = SnapshotManifest.open(...) + * HTableDescriptor htd = manifest.getTableDescriptor() + * for (SnapshotRegionManifest regionManifest: manifest.getRegionManifests()) + * hri = regionManifest.getRegionInfo() + * for (regionManifest.getFamilyFiles()) + * ... + */ + public static SnapshotManifest open(final Configuration conf, final FileSystem fs, + final Path workingDir, final SnapshotDescription desc) throws IOException { + SnapshotManifest manifest = new SnapshotManifest(conf, fs, workingDir, desc, null); + manifest.load(); + return manifest; + } + + public static SnapshotManifest open(final Configuration conf, final SnapshotDescription desc) + throws IOException { + Path snapshotDir = LegacyLayout.getCompletedSnapshotDir(FSUtils.getRootDir(conf), desc); + return open(conf, snapshotDir.getFileSystem(conf), snapshotDir, desc); + } + + /** + * Add the table descriptor to the snapshot manifest + */ + public void addTableDescriptor(final HTableDescriptor htd) throws IOException { + this.htd = htd; + } + + interface RegionVisitor { + TRegion regionOpen(final HRegionInfo regionInfo) throws IOException; + void regionClose(final TRegion region) throws IOException; + + TFamily familyOpen(final TRegion region, final byte[] familyName) throws IOException; + void familyClose(final TRegion region, final TFamily family) throws IOException; + + void storeFile(final TRegion region, final TFamily family, final StoreFileInfo storeFile) + throws IOException; + } + + private RegionVisitor createRegionVisitor(final SnapshotDescription desc) throws IOException { + switch (getSnapshotFormat(desc)) { + case SnapshotManifestV1.DESCRIPTOR_VERSION: + return new SnapshotManifestV1.ManifestBuilder(conf, fs, workingDir); + case SnapshotManifestV2.DESCRIPTOR_VERSION: + return new SnapshotManifestV2.ManifestBuilder(conf, fs, workingDir); + default: + throw new CorruptedSnapshotException("Invalid Snapshot version: " + desc.getVersion(), + ProtobufUtil.createSnapshotDesc(desc)); + } + } + + public void addMobRegion(HRegionInfo regionInfo, HColumnDescriptor[] hcds) throws IOException { + // 0. Get the ManifestBuilder/RegionVisitor + RegionVisitor visitor = createRegionVisitor(desc); + + // 1. dump region meta info into the snapshot directory + LOG.debug("Storing mob region '" + regionInfo + "' region-info for snapshot."); + Object regionData = visitor.regionOpen(regionInfo); + monitor.rethrowException(); + + // 2. iterate through all the stores in the region + LOG.debug("Creating references for mob files"); + + Path mobRegionPath = MobUtils.getMobRegionPath(conf, regionInfo.getTable()); + for (HColumnDescriptor hcd : hcds) { + // 2.1. build the snapshot reference for the store if it's a mob store + if (!hcd.isMobEnabled()) { + continue; + } + Object familyData = visitor.familyOpen(regionData, hcd.getName()); + monitor.rethrowException(); + + Path storePath = MobUtils.getMobFamilyPath(mobRegionPath, hcd.getNameAsString()); + List storeFiles = getStoreFiles(storePath); + if (storeFiles == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("No mob files under family: " + hcd.getNameAsString()); + } + continue; + } + + addReferenceFiles(visitor, regionData, familyData, storeFiles, true); + + visitor.familyClose(regionData, familyData); + } + visitor.regionClose(regionData); + } + + /** + * Creates a 'manifest' for the specified region, by reading directly from the HRegion object. + * This is used by the "online snapshot" when the table is enabled. + */ + public void addRegion(final HRegion region) throws IOException { + // 0. Get the ManifestBuilder/RegionVisitor + RegionVisitor visitor = createRegionVisitor(desc); + + // 1. dump region meta info into the snapshot directory + LOG.debug("Storing '" + region + "' region-info for snapshot."); + Object regionData = visitor.regionOpen(region.getRegionInfo()); + monitor.rethrowException(); + + // 2. iterate through all the stores in the region + LOG.debug("Creating references for hfiles"); + + for (Store store : region.getStores()) { + // 2.1. build the snapshot reference for the store + Object familyData = visitor.familyOpen(regionData, store.getFamily().getName()); + monitor.rethrowException(); + + List storeFiles = new ArrayList(store.getStorefiles()); + if (LOG.isDebugEnabled()) { + LOG.debug("Adding snapshot references for " + storeFiles + " hfiles"); + } + + // 2.2. iterate through all the store's files and create "references". + for (int i = 0, sz = storeFiles.size(); i < sz; i++) { + StoreFile storeFile = storeFiles.get(i); + monitor.rethrowException(); + + // create "reference" to this store file. + LOG.debug("Adding reference for file (" + (i+1) + "/" + sz + "): " + storeFile.getPath()); + visitor.storeFile(regionData, familyData, storeFile.getFileInfo()); + } + visitor.familyClose(regionData, familyData); + } + visitor.regionClose(regionData); + } + + /** + * Creates a 'manifest' for the specified region, by reading directly from the disk. + * This is used by the "offline snapshot" when the table is disabled. + */ + public void addRegion(final Path tableDir, final HRegionInfo regionInfo) throws IOException { + // 0. Get the ManifestBuilder/RegionVisitor + RegionVisitor visitor = createRegionVisitor(desc); + + boolean isMobRegion = MobUtils.isMobRegionInfo(regionInfo); + try { + // Open the RegionFS + RegionStorage regionFs = RegionStorage.open(conf, regionInfo, false); + monitor.rethrowException(); + + // 1. dump region meta info into the snapshot directory + LOG.debug("Storing region-info for snapshot."); + Object regionData = visitor.regionOpen(regionInfo); + monitor.rethrowException(); + + // 2. iterate through all the stores in the region + LOG.debug("Creating references for hfiles"); + + // This ensures that we have an atomic view of the directory as long as we have < ls limit + // (batch size of the files in a directory) on the namenode. Otherwise, we get back the files + // in batches and may miss files being added/deleted. This could be more robust (iteratively + // checking to see if we have all the files until we are sure), but the limit is currently + // 1000 files/batch, far more than the number of store files under a single column family. + Collection familyNames = regionFs.getFamilies(); + if (familyNames != null) { + for (String familyName: familyNames) { + Object familyData = visitor.familyOpen(regionData, Bytes.toBytes(familyName)); + monitor.rethrowException(); + + Collection storeFiles = null; + if (isMobRegion) { + Path regionPath = MobUtils.getMobRegionPath(conf, regionInfo.getTable()); + Path storePath = MobUtils.getMobFamilyPath(regionPath, familyName); + storeFiles = getStoreFiles(storePath); + } else { + storeFiles = regionFs.getStoreFiles(familyName); + } + + if (storeFiles == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("No files under family: " + familyName); + } + continue; + } + + // 2.1. build the snapshot reference for the store + // iterate through all the store's files and create "references". + addReferenceFiles(visitor, regionData, familyData, storeFiles, false); + + visitor.familyClose(regionData, familyData); + } + } + visitor.regionClose(regionData); + } catch (IOException e) { + // the mob directory might not be created yet, so do nothing when it is a mob region + if (!isMobRegion) { + throw e; + } + } + } + + private List getStoreFiles(Path storeDir) throws IOException { + FileStatus[] stats = FSUtils.listStatus(fs, storeDir); + if (stats == null) return null; + + ArrayList storeFiles = new ArrayList(stats.length); + for (int i = 0; i < stats.length; ++i) { + storeFiles.add(new StoreFileInfo(conf, fs, stats[i])); + } + return storeFiles; + } + + private void addReferenceFiles(RegionVisitor visitor, Object regionData, Object familyData, + Collection storeFiles, boolean isMob) throws IOException { + final String fileType = isMob ? "mob file" : "hfile"; + + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("Adding snapshot references for %s %ss", storeFiles, fileType)); + } + + int i = 0; + int sz = storeFiles.size(); + for (StoreFileInfo storeFile: storeFiles) { + monitor.rethrowException(); + + LOG.debug(String.format("Adding reference for %s (%d/%d): %s", + fileType, ++i, sz, storeFile.getPath())); + + // create "reference" to this store file. + visitor.storeFile(regionData, familyData, storeFile); + } + } + + /** + * Load the information in the SnapshotManifest. Called by SnapshotManifest.open() + * + * If the format is v2 and there is no data-manifest, means that we are loading an + * in-progress snapshot. Since we support rolling-upgrades, we loook for v1 and v2 + * regions format. + */ + private void load() throws IOException { + switch (getSnapshotFormat(desc)) { + case SnapshotManifestV1.DESCRIPTOR_VERSION: { + this.htd = LegacyTableDescriptor.getTableDescriptorFromFs(fs, workingDir); + ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader"); + try { + this.regionManifests = + SnapshotManifestV1.loadRegionManifests(conf, tpool, fs, workingDir, desc); + } finally { + tpool.shutdown(); + } + break; + } + case SnapshotManifestV2.DESCRIPTOR_VERSION: { + SnapshotDataManifest dataManifest = readDataManifest(); + if (dataManifest != null) { + htd = ProtobufUtil.convertToHTableDesc(dataManifest.getTableSchema()); + regionManifests = dataManifest.getRegionManifestsList(); + } else { + // Compatibility, load the v1 regions + // This happens only when the snapshot is in-progress and the cache wants to refresh. + List v1Regions, v2Regions; + ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader"); + try { + v1Regions = SnapshotManifestV1.loadRegionManifests(conf, tpool, fs, workingDir, desc); + v2Regions = SnapshotManifestV2.loadRegionManifests(conf, tpool, fs, workingDir, desc); + } catch (InvalidProtocolBufferException e) { + throw new CorruptedSnapshotException("unable to parse region manifest " + + e.getMessage(), e); + } finally { + tpool.shutdown(); + } + if (v1Regions != null && v2Regions != null) { + regionManifests = + new ArrayList(v1Regions.size() + v2Regions.size()); + regionManifests.addAll(v1Regions); + regionManifests.addAll(v2Regions); + } else if (v1Regions != null) { + regionManifests = v1Regions; + } else /* if (v2Regions != null) */ { + regionManifests = v2Regions; + } + } + break; + } + default: + throw new CorruptedSnapshotException("Invalid Snapshot version: " + desc.getVersion(), + ProtobufUtil.createSnapshotDesc(desc)); + } + } + + /** + * Get the current snapshot working dir + */ + public Path getSnapshotDir() { + return this.workingDir; + } + + /** + * Get the SnapshotDescription + */ + public SnapshotDescription getSnapshotDescription() { + return this.desc; + } + + /** + * Get the table descriptor from the Snapshot + */ + public HTableDescriptor getTableDescriptor() { + return this.htd; + } + + /** + * Get all the Region Manifest from the snapshot + */ + public List getRegionManifests() { + return this.regionManifests; + } + + /** + * Get all the Region Manifest from the snapshot. + * This is an helper to get a map with the region encoded name + */ + public Map getRegionManifestsMap() { + if (regionManifests == null || regionManifests.size() == 0) return null; + + HashMap regionsMap = + new HashMap(regionManifests.size()); + for (SnapshotRegionManifest manifest: regionManifests) { + String regionName = getRegionNameFromManifest(manifest); + regionsMap.put(regionName, manifest); + } + return regionsMap; + } + + public void consolidate() throws IOException { + if (getSnapshotFormat(desc) == SnapshotManifestV1.DESCRIPTOR_VERSION) { + Path rootDir = FSUtils.getRootDir(conf); + LOG.info("Using old Snapshot Format"); + // write a copy of descriptor to the snapshot directory + LegacyTableDescriptor.createTableDescriptor(fs, workingDir, htd, false); + } else { + LOG.debug("Convert to Single Snapshot Manifest"); + convertToV2SingleManifest(); + } + } + + /* + * In case of rolling-upgrade, we try to read all the formats and build + * the snapshot with the latest format. + */ + private void convertToV2SingleManifest() throws IOException { + // Try to load v1 and v2 regions + List v1Regions, v2Regions; + ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader"); + try { + v1Regions = SnapshotManifestV1.loadRegionManifests(conf, tpool, fs, workingDir, desc); + v2Regions = SnapshotManifestV2.loadRegionManifests(conf, tpool, fs, workingDir, desc); + } finally { + tpool.shutdown(); + } + + SnapshotDataManifest.Builder dataManifestBuilder = SnapshotDataManifest.newBuilder(); + dataManifestBuilder.setTableSchema(ProtobufUtil.convertToTableSchema(htd)); + + if (v1Regions != null && v1Regions.size() > 0) { + dataManifestBuilder.addAllRegionManifests(v1Regions); + } + if (v2Regions != null && v2Regions.size() > 0) { + dataManifestBuilder.addAllRegionManifests(v2Regions); + } + + // Write the v2 Data Manifest. + // Once the data-manifest is written, the snapshot can be considered complete. + // Currently snapshots are written in a "temporary" directory and later + // moved to the "complated" snapshot directory. + SnapshotDataManifest dataManifest = dataManifestBuilder.build(); + writeDataManifest(dataManifest); + this.regionManifests = dataManifest.getRegionManifestsList(); + + // Remove the region manifests. Everything is now in the data-manifest. + // The delete operation is "relaxed", unless we get an exception we keep going. + // The extra files in the snapshot directory will not give any problem, + // since they have the same content as the data manifest, and even by re-reading + // them we will get the same information. + if (v1Regions != null && v1Regions.size() > 0) { + for (SnapshotRegionManifest regionManifest: v1Regions) { + SnapshotManifestV1.deleteRegionManifest(fs, workingDir, regionManifest); + } + } + if (v2Regions != null && v2Regions.size() > 0) { + for (SnapshotRegionManifest regionManifest: v2Regions) { + SnapshotManifestV2.deleteRegionManifest(fs, workingDir, regionManifest); + } + } + } + + /* + * Write the SnapshotDataManifest file + */ + private void writeDataManifest(final SnapshotDataManifest manifest) + throws IOException { + FSDataOutputStream stream = fs.create(new Path(workingDir, DATA_MANIFEST_NAME)); + try { + manifest.writeTo(stream); + } finally { + stream.close(); + } + } + + /* + * Read the SnapshotDataManifest file + */ + private SnapshotDataManifest readDataManifest() throws IOException { + FSDataInputStream in = null; + try { + in = fs.open(new Path(workingDir, DATA_MANIFEST_NAME)); + CodedInputStream cin = CodedInputStream.newInstance(in); + cin.setSizeLimit(manifestSizeLimit); + return SnapshotDataManifest.parseFrom(cin); + } catch (FileNotFoundException e) { + return null; + } catch (InvalidProtocolBufferException e) { + throw new CorruptedSnapshotException("unable to parse data manifest " + e.getMessage(), e); + } finally { + if (in != null) in.close(); + } + } + + private ThreadPoolExecutor createExecutor(final String name) { + return createExecutor(conf, name); + } + + public static ThreadPoolExecutor createExecutor(final Configuration conf, final String name) { + int maxThreads = conf.getInt("hbase.snapshot.thread.pool.max", 8); + return Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS, + Threads.getNamedThreadFactory(name)); + } + + /** + * Extract the region encoded name from the region manifest + */ + static String getRegionNameFromManifest(final SnapshotRegionManifest manifest) { + byte[] regionName = HRegionInfo.createRegionName( + ProtobufUtil.toTableName(manifest.getRegionInfo().getTableName()), + manifest.getRegionInfo().getStartKey().toByteArray(), + manifest.getRegionInfo().getRegionId(), true); + return HRegionInfo.encodeRegionName(regionName); + } + + /* + * Return the snapshot format + */ + private static int getSnapshotFormat(final SnapshotDescription desc) { + return desc.hasVersion() ? desc.getVersion() : SnapshotManifestV1.DESCRIPTOR_VERSION; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotManifestV1.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotManifestV1.java new file mode 100644 index 0000000..88dea70 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotManifestV1.java @@ -0,0 +1,209 @@ +/** + * 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.fs.legacy.snapshot; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; + +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.fs.RegionStorage; +import org.apache.hadoop.hbase.fs.StorageIdentifier; +import org.apache.hadoop.hbase.fs.legacy.LegacyPathIdentifier; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; +import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.util.ByteStringer; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; + +/** + * DO NOT USE DIRECTLY. USE {@link SnapshotManifest}. + * + * Snapshot v1 layout format + * - Each region in the table is represented by a directory with the .hregioninfo file + * /snapshotName/regionName/.hregioninfo + * - Each file present in the table is represented by an empty file + * /snapshotName/regionName/familyName/fileName + */ +@InterfaceAudience.Private +public final class SnapshotManifestV1 { + private static final Log LOG = LogFactory.getLog(SnapshotManifestV1.class); + + public static final int DESCRIPTOR_VERSION = 0; + + private SnapshotManifestV1() { + } + + // TODO update for RegionStorage + static class ManifestBuilder implements SnapshotManifest.RegionVisitor { + private final Configuration conf; + private final StorageIdentifier snapshotDir; + private final FileSystem fs; + + public ManifestBuilder(final Configuration conf, final FileSystem fs, final Path snapshotDir) { + this.snapshotDir = new LegacyPathIdentifier(snapshotDir); + this.conf = conf; + this.fs = fs; + } + + public RegionStorage regionOpen(final HRegionInfo regionInfo) throws IOException { + RegionStorage snapshotRegionFs = RegionStorage.open(conf, fs, + snapshotDir, regionInfo, true); + return snapshotRegionFs; + } + + public void regionClose(final RegionStorage region) { + } + + public Path familyOpen(final RegionStorage snapshotRegionFs, final byte[] familyName) { + Path familyDir = ((LegacyPathIdentifier)snapshotRegionFs.getStoreContainer(Bytes.toString(familyName))).path; + return familyDir; + } + + public void familyClose(final RegionStorage region, final Path family) { + } + + public void storeFile(final RegionStorage region, final Path familyDir, + final StoreFileInfo storeFile) throws IOException { + Path referenceFile = new Path(familyDir, storeFile.getPath().getName()); + boolean success = true; + if (storeFile.isReference()) { + // write the Reference object to the snapshot + storeFile.getReference().write(fs, 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.createNewFile(referenceFile); + } + if (!success) { + throw new IOException("Failed to create reference file:" + referenceFile); + } + } + } + + static List loadRegionManifests(final Configuration conf, + final Executor executor,final FileSystem fs, final Path snapshotDir, + final SnapshotDescription desc) throws IOException { + FileStatus[] regions = FSUtils.listStatus(fs, snapshotDir, new FSUtils.RegionDirFilter(fs)); + if (regions == null) { + LOG.debug("No regions under directory:" + snapshotDir); + return null; + } + + final ExecutorCompletionService completionService = + new ExecutorCompletionService(executor); + for (final FileStatus region: regions) { + completionService.submit(new Callable() { + @Override + public SnapshotRegionManifest call() throws IOException { + final RegionStorage rs = RegionStorage.open(conf, new LegacyPathIdentifier(region.getPath()), true); + return buildManifestFromDisk(conf, fs, snapshotDir, rs); + } + }); + } + + ArrayList regionsManifest = + new ArrayList(regions.length); + try { + for (int i = 0; i < regions.length; ++i) { + regionsManifest.add(completionService.take().get()); + } + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + IOException ex = new IOException(); + ex.initCause(e.getCause()); + throw ex; + } + return regionsManifest; + } + + static void deleteRegionManifest(final FileSystem fs, final Path snapshotDir, + final SnapshotRegionManifest manifest) throws IOException { + String regionName = SnapshotManifest.getRegionNameFromManifest(manifest); + fs.delete(new Path(snapshotDir, regionName), true); + } + + static SnapshotRegionManifest buildManifestFromDisk(final Configuration conf, + final FileSystem fs, final Path tableDir, final RegionStorage regionFs) throws IOException { + SnapshotRegionManifest.Builder manifest = SnapshotRegionManifest.newBuilder(); + + // 1. dump region meta info into the snapshot directory + LOG.debug("Storing region-info for snapshot."); + manifest.setRegionInfo(HRegionInfo.convert(regionFs.getRegionInfo())); + + // 2. iterate through all the stores in the region + LOG.debug("Creating references for hfiles"); + + // This ensures that we have an atomic view of the directory as long as we have < ls limit + // (batch size of the files in a directory) on the namenode. Otherwise, we get back the files in + // batches and may miss files being added/deleted. This could be more robust (iteratively + // checking to see if we have all the files until we are sure), but the limit is currently 1000 + // files/batch, far more than the number of store files under a single column family. + Collection familyNames = regionFs.getFamilies(); + if (familyNames != null) { + for (String familyName: familyNames) { + Collection storeFiles = regionFs.getStoreFiles(familyName, false); + if (storeFiles == null) { + LOG.debug("No files under family: " + familyName); + continue; + } + + // 2.1. build the snapshot reference for the store + SnapshotRegionManifest.FamilyFiles.Builder family = + SnapshotRegionManifest.FamilyFiles.newBuilder(); + family.setFamilyName(ByteStringer.wrap(Bytes.toBytes(familyName))); + + if (LOG.isDebugEnabled()) { + LOG.debug("Adding snapshot references for " + storeFiles + " hfiles"); + } + + // 2.2. iterate through all the store's files and create "references". + int i = 0; + int sz = storeFiles.size(); + for (StoreFileInfo storeFile: storeFiles) { + // create "reference" to this store file. + LOG.debug("Adding reference for file ("+ (++i) +"/" + sz + "): " + storeFile.getPath()); + SnapshotRegionManifest.StoreFile.Builder sfManifest = + SnapshotRegionManifest.StoreFile.newBuilder(); + sfManifest.setName(storeFile.getPath().getName()); + family.addStoreFiles(sfManifest.build()); + } + manifest.addFamilyFiles(family.build()); + } + } + return manifest.build(); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotManifestV2.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotManifestV2.java new file mode 100644 index 0000000..dba70c0 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/fs/legacy/snapshot/SnapshotManifestV2.java @@ -0,0 +1,187 @@ +/** + * 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.fs.legacy.snapshot; + +import com.google.protobuf.InvalidProtocolBufferException; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; + +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; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; +import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.util.ByteStringer; +import org.apache.hadoop.hbase.util.FSUtils; + +/** + * DO NOT USE DIRECTLY. USE {@link SnapshotManifest}. + * + * Snapshot v2 layout format + * - Single Manifest file containing all the information of regions + * - In the online-snapshot case each region will write a "region manifest" + * /snapshotName/manifest.regionName + */ +@InterfaceAudience.Private +public final class SnapshotManifestV2 { + private static final Log LOG = LogFactory.getLog(SnapshotManifestV2.class); + + public static final int DESCRIPTOR_VERSION = 2; + + public static final String SNAPSHOT_MANIFEST_PREFIX = "region-manifest."; + + private SnapshotManifestV2() {} + + static class ManifestBuilder implements SnapshotManifest.RegionVisitor< + SnapshotRegionManifest.Builder, SnapshotRegionManifest.FamilyFiles.Builder> { + private final Configuration conf; + private final Path snapshotDir; + private final FileSystem fs; + + public ManifestBuilder(final Configuration conf, final FileSystem fs, final Path snapshotDir) { + this.snapshotDir = snapshotDir; + this.conf = conf; + this.fs = fs; + } + + public SnapshotRegionManifest.Builder regionOpen(final HRegionInfo regionInfo) { + SnapshotRegionManifest.Builder manifest = SnapshotRegionManifest.newBuilder(); + manifest.setRegionInfo(HRegionInfo.convert(regionInfo)); + return manifest; + } + + public void regionClose(final SnapshotRegionManifest.Builder region) throws IOException { + // we should ensure the snapshot dir exist, maybe it has been deleted by master + // see HBASE-16464 + if (fs.exists(snapshotDir)) { + SnapshotRegionManifest manifest = region.build(); + FSDataOutputStream stream = fs.create(getRegionManifestPath(snapshotDir, manifest)); + try { + manifest.writeTo(stream); + } finally { + stream.close(); + } + } else { + LOG.warn("can't write manifest without parent dir, maybe it has been deleted by master?"); + } + } + + public SnapshotRegionManifest.FamilyFiles.Builder familyOpen( + final SnapshotRegionManifest.Builder region, final byte[] familyName) { + SnapshotRegionManifest.FamilyFiles.Builder family = + SnapshotRegionManifest.FamilyFiles.newBuilder(); + family.setFamilyName(ByteStringer.wrap(familyName)); + return family; + } + + public void familyClose(final SnapshotRegionManifest.Builder region, + final SnapshotRegionManifest.FamilyFiles.Builder family) { + region.addFamilyFiles(family.build()); + } + + public void storeFile(final SnapshotRegionManifest.Builder region, + final SnapshotRegionManifest.FamilyFiles.Builder family, final StoreFileInfo storeFile) + throws IOException { + SnapshotRegionManifest.StoreFile.Builder sfManifest = + SnapshotRegionManifest.StoreFile.newBuilder(); + sfManifest.setName(storeFile.getPath().getName()); + if (storeFile.isReference()) { + sfManifest.setReference(storeFile.getReference().convert()); + } + sfManifest.setFileSize(storeFile.getReferencedFileStatus(fs).getLen()); + family.addStoreFiles(sfManifest.build()); + } + } + + static List loadRegionManifests(final Configuration conf, + final Executor executor,final FileSystem fs, final Path snapshotDir, + final SnapshotDescription desc) throws IOException { + FileStatus[] manifestFiles = FSUtils.listStatus(fs, snapshotDir, new PathFilter() { + @Override + public boolean accept(Path path) { + return path.getName().startsWith(SNAPSHOT_MANIFEST_PREFIX); + } + }); + + if (manifestFiles == null || manifestFiles.length == 0) return null; + + final ExecutorCompletionService completionService = + new ExecutorCompletionService(executor); + for (final FileStatus st: manifestFiles) { + completionService.submit(new Callable() { + @Override + public SnapshotRegionManifest call() throws IOException { + FSDataInputStream stream = fs.open(st.getPath()); + try { + return SnapshotRegionManifest.parseFrom(stream); + } finally { + stream.close(); + } + } + }); + } + + ArrayList regionsManifest = + new ArrayList(manifestFiles.length); + try { + for (int i = 0; i < manifestFiles.length; ++i) { + regionsManifest.add(completionService.take().get()); + } + } catch (InterruptedException e) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException e) { + Throwable t = e.getCause(); + + if(t instanceof InvalidProtocolBufferException) { + throw (InvalidProtocolBufferException)t; + } else { + IOException ex = new IOException("ExecutionException"); + ex.initCause(e.getCause()); + throw ex; + } + } + return regionsManifest; + } + + static void deleteRegionManifest(final FileSystem fs, final Path snapshotDir, + final SnapshotRegionManifest manifest) throws IOException { + fs.delete(getRegionManifestPath(snapshotDir, manifest), true); + } + + private static Path getRegionManifestPath(final Path snapshotDir, + final SnapshotRegionManifest manifest) { + String regionName = SnapshotManifest.getRegionNameFromManifest(manifest); + return new Path(snapshotDir, SNAPSHOT_MANIFEST_PREFIX + regionName); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Driver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Driver.java index 9737b55..ec0427a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Driver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/Driver.java @@ -22,7 +22,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication; -import org.apache.hadoop.hbase.snapshot.ExportSnapshot; +import org.apache.hadoop.hbase.fs.legacy.snapshot.ExportSnapshot; import org.apache.hadoop.util.ProgramDriver; /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormatImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormatImpl.java index 5c46f2a..2780906 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormatImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MultiTableSnapshotInputFormatImpl.java @@ -25,12 +25,13 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper; -import org.apache.hadoop.hbase.snapshot.SnapshotManifest; +import org.apache.hadoop.hbase.fs.MasterStorage; +import org.apache.hadoop.hbase.fs.StorageIdentifier; +import org.apache.hadoop.hbase.fs.legacy.snapshot.RestoreSnapshotHelper; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.util.ConfigurationUtil; import org.apache.hadoop.hbase.util.FSUtils; @@ -91,8 +92,7 @@ public class MultiTableSnapshotInputFormatImpl { */ public List getSplits(Configuration conf) throws IOException { - Path rootDir = FSUtils.getRootDir(conf); - FileSystem fs = rootDir.getFileSystem(conf); + MasterStorage masterStorage = MasterStorage.open(conf, false); List rtn = Lists.newArrayList(); @@ -103,14 +103,13 @@ public class MultiTableSnapshotInputFormatImpl { Path restoreDir = snapshotsToRestoreDirs.get(snapshotName); - SnapshotManifest manifest = - TableSnapshotInputFormatImpl.getSnapshotManifest(conf, snapshotName, rootDir, fs); - List regionInfos = - TableSnapshotInputFormatImpl.getRegionInfosFromManifest(manifest); + HBaseProtos.SnapshotDescription snapshot = masterStorage.getSnapshot(snapshotName); for (Scan scan : entry.getValue()) { List splits = - TableSnapshotInputFormatImpl.getSplits(scan, manifest, regionInfos, restoreDir, conf); + TableSnapshotInputFormatImpl.getSplits(scan, + masterStorage.getTableDescriptorForSnapshot(snapshot), + masterStorage.getSnapshotRegions(snapshot).values(), restoreDir, conf); rtn.addAll(splits); } } @@ -246,7 +245,8 @@ public class MultiTableSnapshotInputFormatImpl { void restoreSnapshot(Configuration conf, String snapshotName, Path rootDir, Path restoreDir, FileSystem fs) throws IOException { - RestoreSnapshotHelper.copySnapshotForScanner(conf, fs, rootDir, restoreDir, snapshotName); + MasterStorage masterStorage = MasterStorage.open(conf, false); + RestoreSnapshotHelper.copySnapshotForScanner(masterStorage, restoreDir, snapshotName); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java index c40396f..61c1a1b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormat.java @@ -49,7 +49,7 @@ import java.util.List; * wals, etc) directly to provide maximum performance. The snapshot is not required to be * restored to the live cluster or cloned. This also allows to run the mapreduce job from an * online or offline hbase cluster. The snapshot files can be exported by using the - * {@link org.apache.hadoop.hbase.snapshot.ExportSnapshot} tool, to a pure-hdfs cluster, + * {@link org.apache.hadoop.hbase.fs.legacy.snapshot.ExportSnapshot} tool, to a pure-hdfs cluster, * and this InputFormat can be used to run the mapreduce job directly over the snapshot files. * The snapshot should not be deleted while there are jobs reading from snapshot files. *

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java index 7ddde5b..d964b16 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableSnapshotInputFormatImpl.java @@ -35,17 +35,19 @@ import org.apache.hadoop.hbase.client.ClientSideRegionScanner; import org.apache.hadoop.hbase.client.IsolationLevel; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.fs.MasterStorage; +import org.apache.hadoop.hbase.fs.StorageIdentifier; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.generated.MapReduceProtos.TableSnapshotRegionSplit; import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper; -import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; -import org.apache.hadoop.hbase.snapshot.SnapshotManifest; +import org.apache.hadoop.hbase.fs.legacy.snapshot.RestoreSnapshotHelper; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifest; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.io.Writable; import java.io.ByteArrayOutputStream; @@ -53,6 +55,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.UUID; @@ -251,19 +254,16 @@ public class TableSnapshotInputFormatImpl { public static List getSplits(Configuration conf) throws IOException { String snapshotName = getSnapshotName(conf); - Path rootDir = FSUtils.getRootDir(conf); - FileSystem fs = rootDir.getFileSystem(conf); - - SnapshotManifest manifest = getSnapshotManifest(conf, snapshotName, rootDir, fs); - - List regionInfos = getRegionInfosFromManifest(manifest); + MasterStorage masterStorage = MasterStorage.open(conf, false); + SnapshotDescription snapshot = masterStorage.getSnapshot(snapshotName); // TODO: mapred does not support scan as input API. Work around for now. Scan scan = extractScanFromConf(conf); // the temp dir where the snapshot is restored Path restoreDir = new Path(conf.get(RESTORE_DIR_KEY)); - return getSplits(scan, manifest, regionInfos, restoreDir, conf); + return getSplits(scan, masterStorage.getTableDescriptorForSnapshot(snapshot), + masterStorage.getSnapshotRegions(snapshot).values(), restoreDir, conf); } public static List getRegionInfosFromManifest(SnapshotManifest manifest) { @@ -280,13 +280,6 @@ public class TableSnapshotInputFormatImpl { return regionInfos; } - public static SnapshotManifest getSnapshotManifest(Configuration conf, String snapshotName, - Path rootDir, FileSystem fs) throws IOException { - Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir); - SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir); - return SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc); - } - public static Scan extractScanFromConf(Configuration conf) throws IOException { Scan scan = null; if (conf.get(TableInputFormat.SCAN) != null) { @@ -304,15 +297,12 @@ public class TableSnapshotInputFormatImpl { return scan; } - public static List getSplits(Scan scan, SnapshotManifest manifest, - List regionManifests, Path restoreDir, Configuration conf) throws IOException { - // load table descriptor - HTableDescriptor htd = manifest.getTableDescriptor(); - + public static List getSplits(Scan scan, HTableDescriptor htd, + Collection regionInfos, Path restoreDir, Configuration conf) throws IOException { Path tableDir = FSUtils.getTableDir(restoreDir, htd.getTableName()); List splits = new ArrayList(); - for (HRegionInfo hri : regionManifests) { + for (HRegionInfo hri : regionInfos) { // load region descriptor if (CellUtil.overlappingKeys(scan.getStartRow(), scan.getStopRow(), hri.getStartKey(), @@ -395,13 +385,12 @@ public class TableSnapshotInputFormatImpl { throws IOException { conf.set(SNAPSHOT_NAME_KEY, snapshotName); - Path rootDir = FSUtils.getRootDir(conf); - FileSystem fs = rootDir.getFileSystem(conf); + MasterStorage masterStorage = MasterStorage.open(conf, false); restoreDir = new Path(restoreDir, UUID.randomUUID().toString()); // TODO: restore from record readers to parallelize. - RestoreSnapshotHelper.copySnapshotForScanner(conf, fs, rootDir, restoreDir, snapshotName); + RestoreSnapshotHelper.copySnapshotForScanner(masterStorage, restoreDir, snapshotName); conf.set(RESTORE_DIR_KEY, restoreDir.toString()); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java index e8e75c8..e4c31f3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; import org.apache.hadoop.hbase.fs.MasterStorage; +import org.apache.hadoop.hbase.fs.StorageIdentifier; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MetricsSnapshot; import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure.CreateStorageRegions; @@ -48,12 +49,12 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CloneSnapshotState; +import org.apache.hadoop.hbase.snapshot.SnapshotRestoreMetaChanges; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; -import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper; -import org.apache.hadoop.hbase.snapshot.SnapshotManifest; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifest; import com.google.common.base.Preconditions; @@ -301,7 +302,7 @@ public class CloneSnapshotProcedure throws IOException, InterruptedException { if (!getTableName().isSystemTable()) { // Check and update namespace quota - final MasterStorage ms = env.getMasterServices().getMasterStorage(); + final MasterStorage masterStorage = env.getMasterServices().getMasterStorage(); SnapshotManifest manifest = SnapshotManifest.open(env.getMasterConfiguration(), snapshot); @@ -354,11 +355,10 @@ public class CloneSnapshotProcedure try { // 1. Execute the on-disk Clone - SnapshotManifest manifest = SnapshotManifest.open(conf, snapshot); - RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper(conf, manifest, + MasterStorage masterStorage = + env.getMasterServices().getMasterStorage(); + SnapshotRestoreMetaChanges metaChanges = masterStorage.restoreSnapshot(snapshot, hTableDescriptor, monitorException, monitorStatus); - RestoreSnapshotHelper.RestoreMetaChanges metaChanges = - restoreHelper.restoreStorageRegions(); // Clone operation should not have stuff to restore or remove Preconditions.checkArgument( @@ -424,8 +424,8 @@ public class CloneSnapshotProcedure private void addRegionsToMeta(final MasterProcedureEnv env) throws IOException { newRegions = CreateTableProcedure.addTableToMeta(env, hTableDescriptor, newRegions); - RestoreSnapshotHelper.RestoreMetaChanges metaChanges = - new RestoreSnapshotHelper.RestoreMetaChanges( + SnapshotRestoreMetaChanges metaChanges = + new SnapshotRestoreMetaChanges( hTableDescriptor, parentsToChildrenPairMap); metaChanges.updateMetaParentRegions(env.getMasterServices().getConnection(), newRegions); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java index fdfa174..ea6d183 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RestoreSnapshotProcedure.java @@ -51,8 +51,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.RestoreSnapshotState; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; -import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper; -import org.apache.hadoop.hbase.snapshot.SnapshotManifest; +import org.apache.hadoop.hbase.snapshot.SnapshotRestoreMetaChanges; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifest; import org.apache.hadoop.hbase.util.Pair; @InterfaceAudience.Private @@ -362,13 +362,10 @@ public class RestoreSnapshotProcedure LOG.info("Starting restore snapshot=" + ClientSnapshotDescriptionUtils.toString(snapshot)); try { - SnapshotManifest manifest = SnapshotManifest.open(env.getMasterServices().getConfiguration(), - snapshot); - RestoreSnapshotHelper restoreHelper = new RestoreSnapshotHelper( - env.getMasterServices().getConfiguration(), manifest, modifiedHTableDescriptor, - monitorException, getMonitorStatus()); - - RestoreSnapshotHelper.RestoreMetaChanges metaChanges = restoreHelper.restoreStorageRegions(); + MasterStorage masterStorage = + env.getMasterServices().getMasterStorage(); + SnapshotRestoreMetaChanges metaChanges = masterStorage.restoreSnapshot(snapshot, + modifiedHTableDescriptor, monitorException, getMonitorStatus()); regionsToRestore = metaChanges.getRegionsToRestore(); regionsToRemove = metaChanges.getRegionsToRemove(); regionsToAdd = metaChanges.getRegionsToAdd(); @@ -437,8 +434,8 @@ public class RestoreSnapshotProcedure modifiedHTableDescriptor.getRegionReplication()); } - RestoreSnapshotHelper.RestoreMetaChanges metaChanges = - new RestoreSnapshotHelper.RestoreMetaChanges( + SnapshotRestoreMetaChanges metaChanges = + new SnapshotRestoreMetaChanges( modifiedHTableDescriptor, parentsToChildrenPairMap); metaChanges.updateMetaParentRegions(conn, regionsToAdd); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java index a7c2652..533dfc7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java @@ -36,9 +36,8 @@ import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; -import org.apache.hadoop.hbase.snapshot.SnapshotManifest; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifest; 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.zookeeper.KeeperException; @@ -101,7 +100,7 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler { ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() { @Override public void editRegion(final HRegionInfo regionInfo) throws IOException { - snapshotManifest.addRegion(FSUtils.getTableDir(rootDir, snapshotTable), regionInfo); + masterStorage.addRegionToSnapshot(regionInfo, snapshot); } }); } finally { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java index 6e14f47..4dfd7d3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java @@ -20,38 +20,35 @@ package org.apache.hadoop.hbase.master.snapshot; import java.io.IOException; import java.util.List; import java.util.Map; -import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.fs.MasterStorage; +import org.apache.hadoop.hbase.fs.StorageContext; +import org.apache.hadoop.hbase.fs.StorageIdentifier; import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; -import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; 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.SnapshotManifest; import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; /** * General snapshot verification on the master. *

- * This is a light-weight verification mechanism for all the files in a snapshot. It doesn't - * attempt to verify that the files are exact copies (that would be paramount to taking the - * snapshot again!), but instead just attempts to ensure that the files match the expected - * files and are the same length. + * This is a light-weight verification mechanism for verifying snapshot artifacts like snapshot + * description, table, regions and store files. It doesn't attempt to verify that the artifacts + * are exact copies (that would be paramount to taking the snapshot again!), but instead just + * attempts to ensure that the artifacts match the expected artifacts including the length etc. *

* Taking an online snapshots can race against other operations and this is an last line of * defense. For example, if meta changes between when snapshots are taken not all regions of a @@ -59,7 +56,7 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; * but snapshot took parent), or move (snapshots only checks lists of region servers, a move could * have caused a region to be skipped or done twice). *

- * Current snapshot files checked: + * Current snapshot artifacts checked: *

    *
  1. SnapshotDescription is readable
  2. *
  3. Table info is readable
  4. @@ -79,53 +76,57 @@ public final class MasterSnapshotVerifier { private static final Log LOG = LogFactory.getLog(MasterSnapshotVerifier.class); private SnapshotDescription snapshot; - private FileSystem fs; - private Path rootDir; private TableName tableName; + private MasterStorage masterStorage; private MasterServices services; + private StorageContext ctx; /** * @param services services for the master * @param snapshot snapshot to check - * @param rootDir root directory of the hbase installation. */ - public MasterSnapshotVerifier(MasterServices services, SnapshotDescription snapshot, Path rootDir) { - this.fs = services.getMasterStorage().getFileSystem(); + public MasterSnapshotVerifier(MasterServices services, SnapshotDescription snapshot, + StorageContext ctx) { this.services = services; + this.masterStorage = services.getMasterStorage(); this.snapshot = snapshot; - this.rootDir = rootDir; + this.ctx = ctx; this.tableName = TableName.valueOf(snapshot.getTable()); } /** - * Verify that the snapshot in the directory is a valid snapshot - * @param snapshotDir snapshot directory to check - * @param snapshotServers {@link org.apache.hadoop.hbase.ServerName} of the servers - * that are involved in the snapshot + * Verify that the snapshot persisted on a storage is a valid snapshot + * @param ctx {@link StorageContext} for a given snapshot * @throws CorruptedSnapshotException if the snapshot is invalid - * @throws IOException if there is an unexpected connection issue to the filesystem + * @throws IOException if there is an unexpected connection issue to the storage */ - public void verifySnapshot(Path snapshotDir, Set snapshotServers) + public void verifySnapshot(StorageContext ctx) throws CorruptedSnapshotException, IOException { - SnapshotManifest manifest = SnapshotManifest.open(services.getConfiguration(), fs, - snapshotDir, snapshot); // verify snapshot info matches - verifySnapshotDescription(snapshotDir); + verifySnapshotDescription(ctx); - // check that tableinfo is a valid table description - verifyTableInfo(manifest); + // check that table info is a valid table description + verifyTableInfo(ctx); // check that each region is valid - verifyRegions(manifest); + verifyRegions(ctx); } /** - * Check that the snapshot description written in the filesystem matches the current snapshot - * @param snapshotDir snapshot directory to check + * Check that the snapshot description written to storage matches the current snapshot + * @param ctx {@link StorageContext} for a given snapshot + * @throws CorruptedSnapshotException if verification fails */ - private void verifySnapshotDescription(Path snapshotDir) throws CorruptedSnapshotException { - SnapshotDescription found = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir); - if (!this.snapshot.equals(found)) { + private void verifySnapshotDescription(StorageContext ctx) throws CorruptedSnapshotException { + boolean match = false; + SnapshotDescription found = null; + try { + found = masterStorage.getSnapshot(snapshot.getName(), ctx); + match = this.snapshot.equals(found); + } catch (IOException e) { + LOG.warn("Failed to read snapshot '" + snapshot.getName() + "' from storage.", e); + } + if (!match) { throw new CorruptedSnapshotException( "Snapshot read (" + found + ") doesn't equal snapshot we ran (" + snapshot + ").", ProtobufUtil.createSnapshotDesc(snapshot)); @@ -133,11 +134,12 @@ public final class MasterSnapshotVerifier { } /** - * Check that the table descriptor for the snapshot is a valid table descriptor - * @param manifest snapshot manifest to inspect + * Check that the table descriptor written to storage for the snapshot is valid + * @param ctx {@link StorageContext} for a given snapshot + * @throws IOException if fails to read table descriptor from storage */ - private void verifyTableInfo(final SnapshotManifest manifest) throws IOException { - HTableDescriptor htd = manifest.getTableDescriptor(); + private void verifyTableInfo(StorageContext ctx) throws IOException { + HTableDescriptor htd = masterStorage.getTableDescriptorForSnapshot(snapshot, ctx); if (htd == null) { throw new CorruptedSnapshotException("Missing Table Descriptor", ProtobufUtil.createSnapshotDesc(snapshot)); @@ -152,10 +154,10 @@ public final class MasterSnapshotVerifier { /** * Check that all the regions in the snapshot are valid, and accounted for. - * @param manifest snapshot manifest to inspect - * @throws IOException if we can't reach hbase:meta or read the files from the FS + * @param ctx {@link StorageContext} for a given snapshot + * @throws IOException if fails to read region info for a snapshot from storage */ - private void verifyRegions(final SnapshotManifest manifest) throws IOException { + private void verifyRegions(StorageContext ctx) throws IOException { List regions; if (TableName.META_TABLE_NAME.equals(tableName)) { regions = new MetaTableLocator().getMetaRegions(services.getZooKeeper()); @@ -165,8 +167,8 @@ public final class MasterSnapshotVerifier { // Remove the non-default regions RegionReplicaUtil.removeNonDefaultRegions(regions); - Map regionManifests = manifest.getRegionManifestsMap(); - if (regionManifests == null) { + Map snapshotRegions = masterStorage.getSnapshotRegions(snapshot, ctx); + if (snapshotRegions == null) { String msg = "Snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot) + " looks empty"; LOG.error(msg); throw new CorruptedSnapshotException(msg); @@ -176,29 +178,31 @@ public final class MasterSnapshotVerifier { boolean hasMobStore = false; // the mob region is a dummy region, it's not a real region in HBase. // the mob region has a special name, it could be found by the region name. - if (regionManifests.get(MobUtils.getMobRegionInfo(tableName).getEncodedName()) != null) { + if (snapshotRegions.get(MobUtils.getMobRegionInfo(tableName).getEncodedName()) != null) { hasMobStore = true; } - int realRegionCount = hasMobStore ? regionManifests.size() - 1 : regionManifests.size(); + int realRegionCount = hasMobStore ? snapshotRegions.size() - 1 : snapshotRegions.size(); if (realRegionCount != regions.size()) { errorMsg = "Regions moved during the snapshot '" + ClientSnapshotDescriptionUtils.toString(snapshot) + "'. expected=" + regions.size() + " snapshotted=" + realRegionCount + "."; LOG.error(errorMsg); - } - - // Verify HRegionInfo - for (HRegionInfo region : regions) { - SnapshotRegionManifest regionManifest = regionManifests.get(region.getEncodedName()); - if (regionManifest == null) { - // could happen due to a move or split race. - String mesg = " No snapshot region directory found for region:" + region; - if (errorMsg.isEmpty()) errorMsg = mesg; - LOG.error(mesg); - continue; + } else { + // Verify HRegionInfo + for (HRegionInfo region : regions) { + HRegionInfo snapshotRegion = snapshotRegions.get(region.getEncodedName()); + if (snapshotRegion == null) { + // could happen due to a move or split race. + errorMsg = "No snapshot region directory found for region '" + region + "'"; + LOG.error(errorMsg); + break; + } else if (!region.equals(snapshotRegion)) { + errorMsg = "Snapshot region info '" + snapshotRegion + "' doesn't match expected region'" + + region + "'."; + LOG.error(errorMsg); + break; + } } - - verifyRegionInfo(region, regionManifest); } if (!errorMsg.isEmpty()) { @@ -206,21 +210,6 @@ public final class MasterSnapshotVerifier { } // Verify Snapshot HFiles - SnapshotReferenceUtil.verifySnapshot(services.getConfiguration(), fs, manifest); - } - - /** - * Verify that the regionInfo is valid - * @param region the region to check - * @param manifest snapshot manifest to inspect - */ - private void verifyRegionInfo(final HRegionInfo region, - final SnapshotRegionManifest manifest) throws IOException { - HRegionInfo manifestRegionInfo = HRegionInfo.convert(manifest.getRegionInfo()); - if (!region.equals(manifestRegionInfo)) { - String msg = "Manifest region info " + manifestRegionInfo + - "doesn't match expected region:" + region; - throw new CorruptedSnapshotException(msg, ProtobufUtil.createSnapshotDesc(snapshot)); - } + SnapshotReferenceUtil.verifySnapshot(services.getMasterStorage(), snapshot, ctx); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java index 75a1a17..9a8599c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java @@ -29,12 +29,7 @@ import java.util.concurrent.ThreadPoolExecutor; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.Stoppable; @@ -44,7 +39,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.executor.ExecutorService; -import org.apache.hadoop.hbase.fs.legacy.LegacyPathIdentifier; +import org.apache.hadoop.hbase.fs.StorageContext; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.fs.MasterStorage; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; @@ -74,12 +69,10 @@ import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException; import org.apache.hadoop.hbase.snapshot.SnapshotExistsException; -import org.apache.hadoop.hbase.snapshot.SnapshotManifest; import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil; import org.apache.hadoop.hbase.snapshot.TablePartiallyOpenException; import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.KeyLocker; import org.apache.zookeeper.KeeperException; @@ -154,7 +147,6 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable // snapshot using Procedure-V2. private Map restoreTableToProcIdMap = new HashMap(); - private Path rootDir; private ExecutorService executorService; /** @@ -179,131 +171,65 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable ProcedureCoordinator coordinator, ExecutorService pool) throws IOException, UnsupportedOperationException { this.master = master; - - this.rootDir = ((LegacyPathIdentifier) master.getMasterStorage().getRootContainer()).path; - checkSnapshotSupport(master.getConfiguration(), master.getMasterStorage()); + checkSnapshotSupport(master.getConfiguration()); this.coordinator = coordinator; this.executorService = pool; - resetTempDir(); + this.master.getMasterStorage().cleanupAllSnapshots(StorageContext.TEMP); } /** * Gets the list of all completed snapshots. * @return list of SnapshotDescriptions - * @throws IOException File system exception + * @throws IOException Storage exception */ public List getCompletedSnapshots() throws IOException { - return getCompletedSnapshots(SnapshotDescriptionUtils.getSnapshotsDir(rootDir)); - } + List snapshotDescs = new ArrayList<>(); - /** - * Gets the list of all completed snapshots. - * @param snapshotDir snapshot directory - * @return list of SnapshotDescriptions - * @throws IOException File system exception - */ - private List getCompletedSnapshots(Path snapshotDir) throws IOException { - List snapshotDescs = new ArrayList(); - // first create the snapshot root path and check to see if it exists - FileSystem fs = master.getMasterStorage().getFileSystem(); - if (snapshotDir == null) snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir); - - // if there are no snapshots, return an empty list - if (!fs.exists(snapshotDir)) { - return snapshotDescs; - } - - // ignore all the snapshots in progress - FileStatus[] snapshots = fs.listStatus(snapshotDir, - new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs)); MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost(); - // loop through all the completed snapshots - for (FileStatus snapshot : snapshots) { - Path info = new Path(snapshot.getPath(), SnapshotDescriptionUtils.SNAPSHOTINFO_FILE); - // if the snapshot is bad - if (!fs.exists(info)) { - LOG.error("Snapshot information for " + snapshot.getPath() + " doesn't exist"); - continue; - } - FSDataInputStream in = null; - try { - in = fs.open(info); - SnapshotDescription desc = SnapshotDescription.parseFrom(in); - if (cpHost != null) { - try { - cpHost.preListSnapshot(desc); - } catch (AccessDeniedException e) { - LOG.warn("Current user does not have access to " + desc.getName() + " snapshot. " - + "Either you should be owner of this snapshot or admin user."); - // Skip this and try for next snapshot - continue; + master.getMasterStorage().visitSnapshots(new MasterStorage.SnapshotVisitor() { + @Override + public void visitSnapshot(String snapshotName, SnapshotDescription snapshot, + StorageContext ctx) { + try { + if (cpHost != null) { + cpHost.preListSnapshot(snapshot); } - } - snapshotDescs.add(desc); - - // call coproc post hook - if (cpHost != null) { - cpHost.postListSnapshot(desc); - } - } catch (IOException e) { - LOG.warn("Found a corrupted snapshot " + snapshot.getPath(), e); - } finally { - if (in != null) { - in.close(); + snapshotDescs.add(snapshot); + } catch (AccessDeniedException e) { + LOG.warn("Current user does not have access to snapshot '" + snapshot.getName() + "'." + + " Either you should be owner of this snapshot or admin user."); + // Skip this and try for next snapshot + } catch (IOException e) { + LOG.warn("Error while checking access permissions for snapshot '" + snapshot.getName() + + "'.", e); } } - } - return snapshotDescs; - } + }); - /** - * Cleans up any snapshots in the snapshot/.tmp directory that were left from failed - * snapshot attempts. - * - * @throws IOException if we can't reach the filesystem - */ - void resetTempDir() throws IOException { - // cleanup any existing snapshots. - Path tmpdir = SnapshotDescriptionUtils.getWorkingSnapshotDir(rootDir); - if (master.getMasterStorage().getFileSystem().exists(tmpdir)) { - if (!master.getMasterStorage().getFileSystem().delete(tmpdir, true)) { - LOG.warn("Couldn't delete working snapshot directory: " + tmpdir); - } - } + return snapshotDescs; } /** * Delete the specified snapshot - * @param snapshot + * @param snapshot {@link SnapshotDescription} * @throws SnapshotDoesNotExistException If the specified snapshot does not exist. - * @throws IOException For filesystem IOExceptions + * @throws IOException For storage IOExceptions */ - public void deleteSnapshot(SnapshotDescription snapshot) throws SnapshotDoesNotExistException, IOException { - // check to see if it is completed - if (!isSnapshotCompleted(snapshot)) { + public void deleteSnapshot(SnapshotDescription snapshot) throws SnapshotDoesNotExistException, + IOException { + if (!master.getMasterStorage().snapshotExists(snapshot)) { throw new SnapshotDoesNotExistException(ProtobufUtil.createSnapshotDesc(snapshot)); } - String snapshotName = snapshot.getName(); - // first create the snapshot description and check to see if it exists - FileSystem fs = master.getMasterStorage().getFileSystem(); - Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir); - // Get snapshot info from file system. The one passed as parameter is a "fake" snapshotInfo with - // just the "name" and it does not contains the "real" snapshot information - snapshot = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir); - // call coproc pre hook MasterCoprocessorHost cpHost = master.getMasterCoprocessorHost(); if (cpHost != null) { cpHost.preDeleteSnapshot(snapshot); } - LOG.debug("Deleting snapshot: " + snapshotName); - // delete the existing snapshot - if (!fs.delete(snapshotDir, true)) { - throw new HBaseSnapshotException("Failed to delete snapshot directory: " + snapshotDir); - } + // delete snapshot from storage + master.getMasterStorage().deleteSnapshot(snapshot); // call coproc post hook if (cpHost != null) { @@ -343,7 +269,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable // - the snapshot was never requested // In those cases returns to the user the "done state" if the snapshots exists on disk, // otherwise raise an exception saying that the snapshot is not running and doesn't exist. - if (!isSnapshotCompleted(expected)) { + if (!master.getMasterStorage().snapshotExists(expected)) { throw new UnknownSnapshotException("Snapshot " + ssString + " is not currently running or one of the known completed snapshots."); } @@ -416,12 +342,10 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable * Check to make sure that we are OK to run the passed snapshot. Checks to make sure that we * aren't already running a snapshot or restore on the requested table. * @param snapshot description of the snapshot we want to start - * @throws HBaseSnapshotException if the filesystem could not be prepared to start the snapshot + * @throws HBaseSnapshotException if the storage could not be prepared to start the snapshot */ private synchronized void prepareToTakeSnapshot(SnapshotDescription snapshot) throws HBaseSnapshotException { - FileSystem fs = master.getMasterStorage().getFileSystem(); - Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir); TableName snapshotTable = TableName.valueOf(snapshot.getTable()); @@ -444,22 +368,12 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable } try { - // delete the working directory, since we aren't running the snapshot. Likely leftovers - // from a failed attempt. - fs.delete(workingDir, true); - - // recreate the working directory for the snapshot - if (!fs.mkdirs(workingDir)) { - throw new SnapshotCreationException( - "Couldn't create working directory (" + workingDir + ") for snapshot", - ProtobufUtil.createSnapshotDesc(snapshot)); - } + master.getMasterStorage().resetSnapshot(snapshot); } catch (HBaseSnapshotException e) { throw e; } catch (IOException e) { - throw new SnapshotCreationException( - "Exception while checking to see if snapshot could be started.", e, - ProtobufUtil.createSnapshotDesc(snapshot)); + throw new SnapshotCreationException("Exception while checking to see if snapshot could be " + + "started.", e, ProtobufUtil.createSnapshotDesc(snapshot)); } } @@ -513,15 +427,11 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable this.executorService.submit(handler); this.snapshotHandlers.put(TableName.valueOf(snapshot.getTable()), handler); } catch (Exception e) { - // cleanup the working directory by trying to delete it from the fs. - Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir); + // cleanup the storage by trying to delete it. try { - if (!this.master.getMasterStorage().getFileSystem().delete(workingDir, true)) { - LOG.error("Couldn't delete working directory (" + workingDir + " for snapshot:" + - ClientSnapshotDescriptionUtils.toString(snapshot)); - } + master.getMasterStorage().deleteSnapshot(snapshot); } catch (IOException e1) { - LOG.error("Couldn't delete working directory (" + workingDir + " for snapshot:" + + LOG.error("Couldn't delete in-progress snapshot:" + ClientSnapshotDescriptionUtils.toString(snapshot)); } // fail the snapshot @@ -539,10 +449,9 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable */ public void takeSnapshot(SnapshotDescription snapshot) throws IOException { // check to see if we already completed the snapshot - if (isSnapshotCompleted(snapshot)) { - throw new SnapshotExistsException( - "Snapshot '" + snapshot.getName() + "' already stored on the filesystem.", - ProtobufUtil.createSnapshotDesc(snapshot)); + if (master.getMasterStorage().snapshotExists(snapshot)) { + throw new SnapshotExistsException("Snapshot '" + snapshot.getName() + + "' already stored on the storage.", ProtobufUtil.createSnapshotDesc(snapshot)); } LOG.debug("No existing snapshot, attempting snapshot..."); @@ -642,27 +551,6 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable } /** - * Check to see if the snapshot is one of the currently completed snapshots - * Returns true if the snapshot exists in the "completed snapshots folder". - * - * @param snapshot expected snapshot to check - * @return true if the snapshot is stored on the {@link FileSystem}, false if is - * not stored - * @throws IOException if the filesystem throws an unexpected exception, - * @throws IllegalArgumentException if snapshot name is invalid. - */ - private boolean isSnapshotCompleted(SnapshotDescription snapshot) throws IOException { - try { - final Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir); - FileSystem fs = master.getMasterStorage().getFileSystem(); - // check to see if the snapshot already exists - return fs.exists(snapshotDir); - } catch (IllegalArgumentException iae) { - throw new UnknownSnapshotException("Unexpected exception thrown", iae); - } - } - - /** * Clone the specified snapshot. * The clone will fail if the destination table has a snapshot or restore in progress. * @@ -753,34 +641,29 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable * @param nonce unique value to prevent duplicated RPC * @throws IOException */ - public long restoreOrCloneSnapshot( - SnapshotDescription reqSnapshot, - final long nonceGroup, + public long restoreOrCloneSnapshot(SnapshotDescription reqSnapshot, final long nonceGroup, final long nonce) throws IOException { - FileSystem fs = master.getMasterStorage().getFileSystem(); - Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(reqSnapshot, rootDir); - // check if the snapshot exists - if (!fs.exists(snapshotDir)) { - LOG.error("A Snapshot named '" + reqSnapshot.getName() + "' does not exist."); - throw new SnapshotDoesNotExistException( - ProtobufUtil.createSnapshotDesc(reqSnapshot)); + if (!master.getMasterStorage().snapshotExists(reqSnapshot)) { + LOG.error("A Snapshot '" + reqSnapshot.getName() + "' does not exist."); + throw new SnapshotDoesNotExistException(ProtobufUtil.createSnapshotDesc(reqSnapshot)); } - // Get snapshot info from file system. The reqSnapshot is a "fake" snapshotInfo with + // Get snapshot info from storage. The reqSnapshot is a "fake" snapshotInfo with // just the snapshot "name" and table name to restore. It does not contains the "real" snapshot // information. - SnapshotDescription snapshot = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir); - SnapshotManifest manifest = SnapshotManifest.open(master.getConfiguration(), fs, - snapshotDir, snapshot); - HTableDescriptor snapshotTableDesc = manifest.getTableDescriptor(); + SnapshotDescription snapshot = + master.getMasterStorage().getSnapshot(reqSnapshot.getName()); + + HTableDescriptor snapshotTableDesc = + master.getMasterStorage().getTableDescriptorForSnapshot(snapshot); TableName tableName = TableName.valueOf(reqSnapshot.getTable()); // stop tracking "abandoned" handlers cleanupSentinels(); // Verify snapshot validity - SnapshotReferenceUtil.verifySnapshot(master.getConfiguration(), fs, manifest); + SnapshotReferenceUtil.verifySnapshot(master.getMasterStorage(), snapshot, StorageContext.DATA); // Execute the restore/clone operation long procId; @@ -1037,36 +920,25 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable * starting the master if there're snapshots present but the cleaners needed are missing. * Otherwise we can end up with snapshot data loss. * @param conf The {@link Configuration} object to use - * @param ms The MasterFileSystem to use - * @throws IOException in case of file-system operation failure + * @throws IOException in case of storage operation failure * @throws UnsupportedOperationException in case cleaners are missing and * there're snapshot in the system */ - private void checkSnapshotSupport(final Configuration conf, final MasterStorage ms) + private void checkSnapshotSupport(final Configuration conf) throws IOException, UnsupportedOperationException { // Verify if snapshot is disabled by the user String enabled = conf.get(HBASE_SNAPSHOT_ENABLED); boolean snapshotEnabled = conf.getBoolean(HBASE_SNAPSHOT_ENABLED, false); boolean userDisabled = (enabled != null && enabled.trim().length() > 0 && !snapshotEnabled); - // check if an older version of snapshot directory was present - Path oldSnapshotDir = new Path(((LegacyPathIdentifier) ms.getRootContainer()).path, HConstants - .OLD_SNAPSHOT_DIR_NAME); - FileSystem fs = ms.getFileSystem(); - List ss = getCompletedSnapshots(new Path(rootDir, oldSnapshotDir)); - if (ss != null && !ss.isEmpty()) { - LOG.error("Snapshots from an earlier release were found under: " + oldSnapshotDir); - LOG.error("Please rename the directory as " + HConstants.SNAPSHOT_DIR_NAME); - } - // If the user has enabled the snapshot, we force the cleaners to be present // otherwise we still need to check if cleaners are enabled or not and verify // that there're no snapshot in the .snapshot folder. if (snapshotEnabled) { - ms.enableSnapshots(); + master.getMasterStorage().enableSnapshots(); } else { // Verify if cleaners are present - snapshotEnabled = ms.isSnapshotsEnabled(); + snapshotEnabled = master.getMasterStorage().isSnapshotsEnabled(); // Warn if the cleaners are enabled but the snapshot.enabled property is false/not set. if (snapshotEnabled) { @@ -1083,15 +955,10 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable // otherwise we end up with snapshot data loss. if (!snapshotEnabled) { LOG.info("Snapshot feature is not enabled, missing log and hfile cleaners."); - Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(((LegacyPathIdentifier) ms - .getRootContainer()).path); - if (fs.exists(snapshotDir)) { - FileStatus[] snapshots = FSUtils.listStatus(fs, snapshotDir, - new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs)); - if (snapshots != null) { + List snapshots = master.getMasterStorage().getSnapshots(); + if (snapshots != null && !snapshots.isEmpty()) { LOG.error("Snapshots are present, but cleaners are not enabled."); checkSnapshotSupport(); - } } } } @@ -1100,9 +967,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable public void initialize(MasterServices master, MetricsMaster metricsMaster) throws KeeperException, IOException, UnsupportedOperationException { this.master = master; - - this.rootDir = ((LegacyPathIdentifier) master.getMasterStorage().getRootContainer()).path; - checkSnapshotSupport(master.getConfiguration(), master.getMasterStorage()); + checkSnapshotSupport(master.getConfiguration()); // get the configuration for the coordinator Configuration conf = master.getConfiguration(); @@ -1121,7 +986,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable this.coordinator = new ProcedureCoordinator(comms, tpool, timeoutMillis, wakeFrequency); this.executorService = master.getExecutorService(); - resetTempDir(); + this.master.getMasterStorage().cleanupAllSnapshots(StorageContext.TEMP); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java index 503f346..de871c9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java @@ -29,8 +29,6 @@ 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.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; @@ -41,7 +39,9 @@ import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare; import org.apache.hadoop.hbase.executor.EventHandler; import org.apache.hadoop.hbase.executor.EventType; -import org.apache.hadoop.hbase.fs.legacy.LegacyPathIdentifier; +import org.apache.hadoop.hbase.fs.MasterStorage; +import org.apache.hadoop.hbase.fs.StorageContext; +import org.apache.hadoop.hbase.fs.StorageIdentifier; import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MetricsSnapshot; import org.apache.hadoop.hbase.master.SnapshotSentinel; @@ -52,9 +52,6 @@ import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; -import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; -import org.apache.hadoop.hbase.snapshot.SnapshotManifest; -import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; import org.apache.zookeeper.KeeperException; @@ -78,17 +75,13 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh protected final MetricsSnapshot metricsSnapshot = new MetricsSnapshot(); protected final SnapshotDescription snapshot; protected final Configuration conf; - protected final FileSystem fs; - protected final Path rootDir; - private final Path snapshotDir; - protected final Path workingDir; + protected final MasterStorage masterStorage; private final MasterSnapshotVerifier verifier; protected final ForeignExceptionDispatcher monitor; protected final TableLockManager tableLockManager; protected final TableLock tableLock; protected final MonitoredTask status; protected final TableName snapshotTable; - protected final SnapshotManifest snapshotManifest; protected final SnapshotManager snapshotManager; protected HTableDescriptor htd; @@ -108,12 +101,8 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh this.snapshotManager = snapshotManager; this.snapshotTable = TableName.valueOf(snapshot.getTable()); this.conf = this.master.getConfiguration(); - this.fs = this.master.getMasterStorage().getFileSystem(); - this.rootDir = ((LegacyPathIdentifier) this.master.getMasterStorage().getRootContainer()).path; - this.snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir); - this.workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, rootDir); + this.masterStorage = masterServices.getMasterStorage(); this.monitor = new ForeignExceptionDispatcher(snapshot.getName()); - this.snapshotManifest = SnapshotManifest.create(conf, fs, workingDir, snapshot, monitor); this.tableLockManager = master.getTableLockManager(); this.tableLock = this.tableLockManager.writeLock( @@ -121,7 +110,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh EventType.C_M_SNAPSHOT_TABLE.toString()); // prepare the verify - this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, rootDir); + this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, StorageContext.TEMP); // update the running tasks this.status = TaskMonitor.get().createStatus( "Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable); @@ -129,8 +118,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh private HTableDescriptor loadTableDescriptor() throws FileNotFoundException, IOException { - HTableDescriptor htd = - this.master.getTableDescriptors().get(snapshotTable); + HTableDescriptor htd = this.master.getTableDescriptors().get(snapshotTable); if (htd == null) { throw new IOException("HTableDescriptor missing for " + snapshotTable); } @@ -171,9 +159,8 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh // If regions move after this meta scan, the region specific snapshot should fail, triggering // an external exception that gets captured here. - // write down the snapshot info in the working directory - SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, workingDir, fs); - snapshotManifest.addTableDescriptor(this.htd); + // initiate snapshot on storage + masterStorage.snapshotTable(htd, snapshot, monitor, StorageContext.TEMP); monitor.rethrowException(); List> regionsAndLocations; @@ -201,14 +188,14 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh // flush the in-memory state, and write the single manifest status.setStatus("Consolidate snapshot: " + snapshot.getName()); - snapshotManifest.consolidate(); + masterStorage.consolidateSnapshot(snapshot, StorageContext.TEMP); // verify the snapshot is valid status.setStatus("Verifying snapshot: " + snapshot.getName()); - verifier.verifySnapshot(this.workingDir, serverNames); + verifier.verifySnapshot(StorageContext.TEMP); - // complete the snapshot, atomically moving from tmp to .snapshot dir. - completeSnapshot(this.snapshotDir, this.workingDir, this.fs); + // complete the snapshot, atomically moving from TEMP storage context to DATA context. + completeSnapshot(snapshot); msg = "Snapshot " + snapshot.getName() + " of table " + snapshotTable + " completed"; status.markComplete(msg); LOG.info(msg); @@ -224,15 +211,19 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh // need to mark this completed to close off and allow cleanup to happen. cancel(reason); } finally { - LOG.debug("Launching cleanup of working dir:" + workingDir); + LOG.debug("Launching cleanup of artifacts in TEMP storage context for a snapshot '" + + snapshot.getName() + "'."); try { - // if the working dir is still present, the snapshot has failed. it is present we delete - // it. - if (fs.exists(workingDir) && !this.fs.delete(workingDir, true)) { - LOG.error("Couldn't delete snapshot working directory:" + workingDir); + // If the snapshot is still present in TEMP storage context, the snapshot has failed. + // Delete it. + if (masterStorage.snapshotExists(snapshot, StorageContext.TEMP) && + !masterStorage.deleteSnapshot(snapshot, StorageContext.TEMP)) { + LOG.error("Couldn't delete snapshot '" + snapshot.getName() + + "' from TEMP storage context."); } } catch (IOException e) { - LOG.error("Couldn't delete snapshot working directory:" + workingDir); + LOG.error("Couldn't delete snapshot '" + snapshot.getName() + + "' from TEMP storage context."); } lock.unlock(); releaseTableLock(); @@ -252,19 +243,17 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh /** * Reset the manager to allow another snapshot to proceed * - * @param snapshotDir final path of the snapshot - * @param workingDir directory where the in progress snapshot was built - * @param fs {@link FileSystem} where the snapshot was built + * @param snapshot * @throws SnapshotCreationException if the snapshot could not be moved - * @throws IOException the filesystem could not be reached + * @throws IOException the storage could not be reached */ - public void completeSnapshot(Path snapshotDir, Path workingDir, FileSystem fs) - throws SnapshotCreationException, IOException { - LOG.debug("Sentinel is done, just moving the snapshot from " + workingDir + " to " - + snapshotDir); - if (!fs.rename(workingDir, snapshotDir)) { - throw new SnapshotCreationException("Failed to move working directory(" + workingDir - + ") to completed directory(" + snapshotDir + ")."); + public void completeSnapshot(SnapshotDescription snapshot) throws SnapshotCreationException, + IOException { + LOG.debug("Sentinel is done, just moving the snapshot '" + snapshot.getName() + + "' from TEMP storae context to DATA context."); + if (!masterStorage.changeSnapshotContext(snapshot, StorageContext.TEMP, StorageContext.DATA)) { + throw new SnapshotCreationException("Failed to move snapshot '" + snapshot.getName() + + "' from TEMP storage context to DATA context."); } finished = true; } @@ -278,9 +267,8 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh /** * Take a snapshot of the specified disabled region */ - protected void snapshotDisabledRegion(final HRegionInfo regionInfo) - throws IOException { - snapshotManifest.addRegion(FSUtils.getTableDir(rootDir, snapshotTable), regionInfo); + protected void snapshotDisabledRegion(final HRegionInfo regionInfo) throws IOException { + masterStorage.addRegionToSnapshot(regionInfo, snapshot, StorageContext.TEMP); monitor.rethrowException(); status.setStatus("Completed referencing HFiles for offline region " + regionInfo.toString() + " of table: " + snapshotTable); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepReducer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepReducer.java index b6b4f67..3a26f48 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepReducer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/mapreduce/SweepReducer.java @@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.client.Admin; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index d0a4c94..919f016 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -135,6 +135,7 @@ import org.apache.hadoop.hbase.filter.FilterWrapper; import org.apache.hadoop.hbase.filter.IncompatibleFilterException; import org.apache.hadoop.hbase.fs.RegionStorage; import org.apache.hadoop.hbase.fs.StorageIdentifier; +import org.apache.hadoop.hbase.fs.legacy.LegacyLayout; import org.apache.hadoop.hbase.fs.legacy.LegacyPathIdentifier; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.TimeRange; @@ -174,8 +175,7 @@ import org.apache.hadoop.hbase.regionserver.wal.ReplayHLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; -import org.apache.hadoop.hbase.snapshot.SnapshotManifest; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifest; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.CancelableProgressable; @@ -3696,7 +3696,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi public void addRegionToSnapshot(SnapshotDescription desc, ForeignExceptionSnare exnSnare) throws IOException { Path rootDir = FSUtils.getRootDir(conf); - Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir); + Path snapshotDir = LegacyLayout.getWorkingSnapshotDir(rootDir, desc); SnapshotManifest manifest = SnapshotManifest.create(conf, getFilesystem(), snapshotDir, desc, exnSnare); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java index 5305149..b8750a4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java @@ -23,7 +23,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -32,13 +31,11 @@ import org.apache.hadoop.hbase.client.replication.ReplicationAdmin; import org.apache.hadoop.hbase.io.WALLink; import org.apache.hadoop.hbase.procedure2.util.StringUtils; import org.apache.hadoop.hbase.replication.*; -import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; import org.apache.zookeeper.KeeperException; -import org.mortbay.util.IO; import java.io.IOException; import java.util.*; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java deleted file mode 100644 index 9734f43..0000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/ExportSnapshot.java +++ /dev/null @@ -1,1084 +0,0 @@ -/** - * 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.BufferedInputStream; -import java.io.FileNotFoundException; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.LinkedList; -import java.util.List; -import java.util.Random; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileChecksum; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.io.FileLink; -import org.apache.hadoop.hbase.io.HFileLink; -import org.apache.hadoop.hbase.io.WALLink; -import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; -import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo; -import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.HFileArchiveUtil; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.JobContext; -import org.apache.hadoop.mapreduce.Mapper; -import org.apache.hadoop.mapreduce.InputFormat; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; -import org.apache.hadoop.mapreduce.security.TokenCache; -import org.apache.hadoop.hbase.io.hadoopbackport.ThrottledInputStream; -import org.apache.hadoop.util.StringUtils; -import org.apache.hadoop.util.Tool; -import org.apache.hadoop.util.ToolRunner; - -/** - * Export the specified snapshot to a given FileSystem. - * - * The .snapshot/name folder is copied to the destination cluster - * and then all the hfiles/wals are copied using a Map-Reduce Job in the .archive/ location. - * When everything is done, the second cluster can restore the snapshot. - */ -@InterfaceAudience.Public -@InterfaceStability.Evolving -public class ExportSnapshot extends Configured implements Tool { - public static final String NAME = "exportsnapshot"; - /** Configuration prefix for overrides for the source filesystem */ - public static final String CONF_SOURCE_PREFIX = NAME + ".from."; - /** Configuration prefix for overrides for the destination filesystem */ - public static final String CONF_DEST_PREFIX = NAME + ".to."; - - private static final Log LOG = LogFactory.getLog(ExportSnapshot.class); - - private static final String MR_NUM_MAPS = "mapreduce.job.maps"; - private static final String CONF_NUM_SPLITS = "snapshot.export.format.splits"; - private static final String CONF_SNAPSHOT_NAME = "snapshot.export.format.snapshot.name"; - private static final String CONF_SNAPSHOT_DIR = "snapshot.export.format.snapshot.dir"; - private static final String CONF_FILES_USER = "snapshot.export.files.attributes.user"; - private static final String CONF_FILES_GROUP = "snapshot.export.files.attributes.group"; - private static final String CONF_FILES_MODE = "snapshot.export.files.attributes.mode"; - private static final String CONF_CHECKSUM_VERIFY = "snapshot.export.checksum.verify"; - private static final String CONF_OUTPUT_ROOT = "snapshot.export.output.root"; - private static final String CONF_INPUT_ROOT = "snapshot.export.input.root"; - private static final String CONF_BUFFER_SIZE = "snapshot.export.buffer.size"; - private static final String CONF_MAP_GROUP = "snapshot.export.default.map.group"; - private static final String CONF_BANDWIDTH_MB = "snapshot.export.map.bandwidth.mb"; - protected static final String CONF_SKIP_TMP = "snapshot.export.skip.tmp"; - - static final String CONF_TEST_FAILURE = "test.snapshot.export.failure"; - static final String CONF_TEST_RETRY = "test.snapshot.export.failure.retry"; - - // Export Map-Reduce Counters, to keep track of the progress - public enum Counter { - MISSING_FILES, FILES_COPIED, FILES_SKIPPED, COPY_FAILED, - BYTES_EXPECTED, BYTES_SKIPPED, BYTES_COPIED - } - - private static class ExportMapper extends Mapper { - final static int REPORT_SIZE = 1 * 1024 * 1024; - final static int BUFFER_SIZE = 64 * 1024; - - private boolean testFailures; - private Random random; - - private boolean verifyChecksum; - private String filesGroup; - private String filesUser; - private short filesMode; - private int bufferSize; - - private FileSystem outputFs; - private Path outputArchive; - private Path outputRoot; - - private FileSystem inputFs; - private Path inputArchive; - private Path inputRoot; - - @Override - public void setup(Context context) throws IOException { - Configuration conf = context.getConfiguration(); - Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX); - Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX); - - verifyChecksum = conf.getBoolean(CONF_CHECKSUM_VERIFY, true); - - filesGroup = conf.get(CONF_FILES_GROUP); - filesUser = conf.get(CONF_FILES_USER); - filesMode = (short)conf.getInt(CONF_FILES_MODE, 0); - outputRoot = new Path(conf.get(CONF_OUTPUT_ROOT)); - inputRoot = new Path(conf.get(CONF_INPUT_ROOT)); - - inputArchive = new Path(inputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY); - outputArchive = new Path(outputRoot, HConstants.HFILE_ARCHIVE_DIRECTORY); - - testFailures = conf.getBoolean(CONF_TEST_FAILURE, false); - - try { - srcConf.setBoolean("fs." + inputRoot.toUri().getScheme() + ".impl.disable.cache", true); - inputFs = FileSystem.get(inputRoot.toUri(), srcConf); - } catch (IOException e) { - throw new IOException("Could not get the input FileSystem with root=" + inputRoot, e); - } - - try { - destConf.setBoolean("fs." + outputRoot.toUri().getScheme() + ".impl.disable.cache", true); - outputFs = FileSystem.get(outputRoot.toUri(), destConf); - } catch (IOException e) { - throw new IOException("Could not get the output FileSystem with root="+ outputRoot, e); - } - - // Use the default block size of the outputFs if bigger - int defaultBlockSize = Math.max((int) outputFs.getDefaultBlockSize(outputRoot), BUFFER_SIZE); - bufferSize = conf.getInt(CONF_BUFFER_SIZE, defaultBlockSize); - LOG.info("Using bufferSize=" + StringUtils.humanReadableInt(bufferSize)); - - for (Counter c : Counter.values()) { - context.getCounter(c).increment(0); - } - } - - @Override - protected void cleanup(Context context) { - IOUtils.closeStream(inputFs); - IOUtils.closeStream(outputFs); - } - - @Override - public void map(BytesWritable key, NullWritable value, Context context) - throws InterruptedException, IOException { - SnapshotFileInfo inputInfo = SnapshotFileInfo.parseFrom(key.copyBytes()); - Path outputPath = getOutputPath(inputInfo); - - copyFile(context, inputInfo, outputPath); - } - - /** - * Returns the location where the inputPath will be copied. - */ - private Path getOutputPath(final SnapshotFileInfo inputInfo) throws IOException { - Path path = null; - switch (inputInfo.getType()) { - case HFILE: - Path inputPath = new Path(inputInfo.getHfile()); - String family = inputPath.getParent().getName(); - TableName table =HFileLink.getReferencedTableName(inputPath.getName()); - String region = HFileLink.getReferencedRegionName(inputPath.getName()); - String hfile = HFileLink.getReferencedHFileName(inputPath.getName()); - path = new Path(FSUtils.getTableDir(new Path("./"), table), - new Path(region, new Path(family, hfile))); - break; - case WAL: - LOG.warn("snapshot does not keeps WALs: " + inputInfo); - break; - default: - throw new IOException("Invalid File Type: " + inputInfo.getType().toString()); - } - return new Path(outputArchive, path); - } - - /* - * Used by TestExportSnapshot to simulate a failure - */ - private void injectTestFailure(final Context context, final SnapshotFileInfo inputInfo) - throws IOException { - if (testFailures) { - if (context.getConfiguration().getBoolean(CONF_TEST_RETRY, false)) { - if (random == null) { - random = new Random(); - } - - // FLAKY-TEST-WARN: lower is better, we can get some runs without the - // retry, but at least we reduce the number of test failures due to - // this test exception from the same map task. - if (random.nextFloat() < 0.03) { - throw new IOException("TEST RETRY FAILURE: Unable to copy input=" + inputInfo - + " time=" + System.currentTimeMillis()); - } - } else { - context.getCounter(Counter.COPY_FAILED).increment(1); - throw new IOException("TEST FAILURE: Unable to copy input=" + inputInfo); - } - } - } - - private void copyFile(final Context context, final SnapshotFileInfo inputInfo, - final Path outputPath) throws IOException { - injectTestFailure(context, inputInfo); - - // Get the file information - FileStatus inputStat = getSourceFileStatus(context, inputInfo); - - // Verify if the output file exists and is the same that we want to copy - if (outputFs.exists(outputPath)) { - FileStatus outputStat = outputFs.getFileStatus(outputPath); - if (outputStat != null && sameFile(inputStat, outputStat)) { - LOG.info("Skip copy " + inputStat.getPath() + " to " + outputPath + ", same file."); - context.getCounter(Counter.FILES_SKIPPED).increment(1); - context.getCounter(Counter.BYTES_SKIPPED).increment(inputStat.getLen()); - return; - } - } - - InputStream in = openSourceFile(context, inputInfo); - int bandwidthMB = context.getConfiguration().getInt(CONF_BANDWIDTH_MB, 100); - if (Integer.MAX_VALUE != bandwidthMB) { - in = new ThrottledInputStream(new BufferedInputStream(in), bandwidthMB * 1024 * 1024L); - } - - try { - context.getCounter(Counter.BYTES_EXPECTED).increment(inputStat.getLen()); - - // Ensure that the output folder is there and copy the file - createOutputPath(outputPath.getParent()); - FSDataOutputStream out = outputFs.create(outputPath, true); - try { - copyData(context, inputStat.getPath(), in, outputPath, out, inputStat.getLen()); - } finally { - out.close(); - } - - // Try to Preserve attributes - if (!preserveAttributes(outputPath, inputStat)) { - LOG.warn("You may have to run manually chown on: " + outputPath); - } - } finally { - in.close(); - } - } - - /** - * Create the output folder and optionally set ownership. - */ - private void createOutputPath(final Path path) throws IOException { - if (filesUser == null && filesGroup == null) { - outputFs.mkdirs(path); - } else { - Path parent = path.getParent(); - if (!outputFs.exists(parent) && !parent.isRoot()) { - createOutputPath(parent); - } - outputFs.mkdirs(path); - if (filesUser != null || filesGroup != null) { - // override the owner when non-null user/group is specified - outputFs.setOwner(path, filesUser, filesGroup); - } - if (filesMode > 0) { - outputFs.setPermission(path, new FsPermission(filesMode)); - } - } - } - - /** - * Try to Preserve the files attribute selected by the user copying them from the source file - * This is only required when you are exporting as a different user than "hbase" or on a system - * that doesn't have the "hbase" user. - * - * This is not considered a blocking failure since the user can force a chmod with the user - * that knows is available on the system. - */ - private boolean preserveAttributes(final Path path, final FileStatus refStat) { - FileStatus stat; - try { - stat = outputFs.getFileStatus(path); - } catch (IOException e) { - LOG.warn("Unable to get the status for file=" + path); - return false; - } - - try { - if (filesMode > 0 && stat.getPermission().toShort() != filesMode) { - outputFs.setPermission(path, new FsPermission(filesMode)); - } else if (refStat != null && !stat.getPermission().equals(refStat.getPermission())) { - outputFs.setPermission(path, refStat.getPermission()); - } - } catch (IOException e) { - LOG.warn("Unable to set the permission for file="+ stat.getPath() +": "+ e.getMessage()); - return false; - } - - boolean hasRefStat = (refStat != null); - String user = stringIsNotEmpty(filesUser) || !hasRefStat ? filesUser : refStat.getOwner(); - String group = stringIsNotEmpty(filesGroup) || !hasRefStat ? filesGroup : refStat.getGroup(); - if (stringIsNotEmpty(user) || stringIsNotEmpty(group)) { - try { - if (!(user.equals(stat.getOwner()) && group.equals(stat.getGroup()))) { - outputFs.setOwner(path, user, group); - } - } catch (IOException e) { - LOG.warn("Unable to set the owner/group for file="+ stat.getPath() +": "+ e.getMessage()); - LOG.warn("The user/group may not exist on the destination cluster: user=" + - user + " group=" + group); - return false; - } - } - - return true; - } - - private boolean stringIsNotEmpty(final String str) { - return str != null && str.length() > 0; - } - - private void copyData(final Context context, - final Path inputPath, final InputStream in, - final Path outputPath, final FSDataOutputStream out, - final long inputFileSize) - throws IOException { - final String statusMessage = "copied %s/" + StringUtils.humanReadableInt(inputFileSize) + - " (%.1f%%)"; - - try { - byte[] buffer = new byte[bufferSize]; - long totalBytesWritten = 0; - int reportBytes = 0; - int bytesRead; - - long stime = System.currentTimeMillis(); - while ((bytesRead = in.read(buffer)) > 0) { - out.write(buffer, 0, bytesRead); - totalBytesWritten += bytesRead; - reportBytes += bytesRead; - - if (reportBytes >= REPORT_SIZE) { - context.getCounter(Counter.BYTES_COPIED).increment(reportBytes); - context.setStatus(String.format(statusMessage, - StringUtils.humanReadableInt(totalBytesWritten), - (totalBytesWritten/(float)inputFileSize) * 100.0f) + - " from " + inputPath + " to " + outputPath); - reportBytes = 0; - } - } - long etime = System.currentTimeMillis(); - - context.getCounter(Counter.BYTES_COPIED).increment(reportBytes); - context.setStatus(String.format(statusMessage, - StringUtils.humanReadableInt(totalBytesWritten), - (totalBytesWritten/(float)inputFileSize) * 100.0f) + - " from " + inputPath + " to " + outputPath); - - // Verify that the written size match - if (totalBytesWritten != inputFileSize) { - String msg = "number of bytes copied not matching copied=" + totalBytesWritten + - " expected=" + inputFileSize + " for file=" + inputPath; - throw new IOException(msg); - } - - LOG.info("copy completed for input=" + inputPath + " output=" + outputPath); - LOG.info("size=" + totalBytesWritten + - " (" + StringUtils.humanReadableInt(totalBytesWritten) + ")" + - " time=" + StringUtils.formatTimeDiff(etime, stime) + - String.format(" %.3fM/sec", (totalBytesWritten / ((etime - stime)/1000.0))/1048576.0)); - context.getCounter(Counter.FILES_COPIED).increment(1); - } catch (IOException e) { - LOG.error("Error copying " + inputPath + " to " + outputPath, e); - context.getCounter(Counter.COPY_FAILED).increment(1); - throw e; - } - } - - /** - * Try to open the "source" file. - * Throws an IOException if the communication with the inputFs fail or - * if the file is not found. - */ - private FSDataInputStream openSourceFile(Context context, final SnapshotFileInfo fileInfo) - throws IOException { - try { - Configuration conf = context.getConfiguration(); - FileLink link = null; - switch (fileInfo.getType()) { - case HFILE: - Path inputPath = new Path(fileInfo.getHfile()); - link = getFileLink(inputPath, conf); - break; - case WAL: - String serverName = fileInfo.getWalServer(); - String logName = fileInfo.getWalName(); - link = new WALLink(inputRoot, serverName, logName); - break; - default: - throw new IOException("Invalid File Type: " + fileInfo.getType().toString()); - } - return link.open(inputFs); - } catch (IOException e) { - context.getCounter(Counter.MISSING_FILES).increment(1); - LOG.error("Unable to open source file=" + fileInfo.toString(), e); - throw e; - } - } - - private FileStatus getSourceFileStatus(Context context, final SnapshotFileInfo fileInfo) - throws IOException { - try { - Configuration conf = context.getConfiguration(); - FileLink link = null; - switch (fileInfo.getType()) { - case HFILE: - Path inputPath = new Path(fileInfo.getHfile()); - link = getFileLink(inputPath, conf); - break; - case WAL: - link = new WALLink(inputRoot, fileInfo.getWalServer(), fileInfo.getWalName()); - break; - default: - throw new IOException("Invalid File Type: " + fileInfo.getType().toString()); - } - return link.getFileStatus(inputFs); - } catch (FileNotFoundException e) { - context.getCounter(Counter.MISSING_FILES).increment(1); - LOG.error("Unable to get the status for source file=" + fileInfo.toString(), e); - throw e; - } catch (IOException e) { - LOG.error("Unable to get the status for source file=" + fileInfo.toString(), e); - throw e; - } - } - - private FileLink getFileLink(Path path, Configuration conf) throws IOException{ - String regionName = HFileLink.getReferencedRegionName(path.getName()); - TableName tableName = HFileLink.getReferencedTableName(path.getName()); - if(MobUtils.getMobRegionInfo(tableName).getEncodedName().equals(regionName)) { - return HFileLink.buildFromHFileLinkPattern(MobUtils.getQualifiedMobRootDir(conf), - HFileArchiveUtil.getArchivePath(conf), path); - } - return HFileLink.buildFromHFileLinkPattern(inputRoot, inputArchive, path); - } - - private FileChecksum getFileChecksum(final FileSystem fs, final Path path) { - try { - return fs.getFileChecksum(path); - } catch (IOException e) { - LOG.warn("Unable to get checksum for file=" + path, e); - return null; - } - } - - /** - * Check if the two files are equal by looking at the file length, - * and at the checksum (if user has specified the verifyChecksum flag). - */ - private boolean sameFile(final FileStatus inputStat, final FileStatus outputStat) { - // Not matching length - if (inputStat.getLen() != outputStat.getLen()) return false; - - // Mark files as equals, since user asked for no checksum verification - if (!verifyChecksum) return true; - - // If checksums are not available, files are not the same. - FileChecksum inChecksum = getFileChecksum(inputFs, inputStat.getPath()); - if (inChecksum == null) return false; - - FileChecksum outChecksum = getFileChecksum(outputFs, outputStat.getPath()); - if (outChecksum == null) return false; - - return inChecksum.equals(outChecksum); - } - } - - // ========================================================================== - // Input Format - // ========================================================================== - - /** - * Extract the list of files (HFiles/WALs) to copy using Map-Reduce. - * @return list of files referenced by the snapshot (pair of path and size) - */ - private static List> getSnapshotFiles(final Configuration conf, - final FileSystem fs, final Path snapshotDir) throws IOException { - SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir); - - final List> files = new ArrayList>(); - final TableName table = TableName.valueOf(snapshotDesc.getTable()); - - // Get snapshot files - LOG.info("Loading Snapshot '" + snapshotDesc.getName() + "' hfile list"); - SnapshotReferenceUtil.visitReferencedFiles(conf, fs, snapshotDir, snapshotDesc, - new SnapshotReferenceUtil.SnapshotVisitor() { - @Override - public void storeFile(final HRegionInfo regionInfo, final String family, - final SnapshotRegionManifest.StoreFile storeFile) throws IOException { - // for storeFile.hasReference() case, copied as part of the manifest - if (!storeFile.hasReference()) { - String region = regionInfo.getEncodedName(); - String hfile = storeFile.getName(); - Path path = HFileLink.createPath(table, region, family, hfile); - - SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder() - .setType(SnapshotFileInfo.Type.HFILE) - .setHfile(path.toString()) - .build(); - - long size; - if (storeFile.hasFileSize()) { - size = storeFile.getFileSize(); - } else { - size = HFileLink.buildFromHFileLinkPattern(conf, path).getFileStatus(fs).getLen(); - } - files.add(new Pair(fileInfo, size)); - } - } - }); - - return files; - } - - /** - * Given a list of file paths and sizes, create around ngroups in as balanced a way as possible. - * The groups created will have similar amounts of bytes. - *

    - * The algorithm used is pretty straightforward; the file list is sorted by size, - * and then each group fetch the bigger file available, iterating through groups - * alternating the direction. - */ - static List>> getBalancedSplits( - final List> files, final int ngroups) { - // Sort files by size, from small to big - Collections.sort(files, new Comparator>() { - public int compare(Pair a, Pair b) { - long r = a.getSecond() - b.getSecond(); - return (r < 0) ? -1 : ((r > 0) ? 1 : 0); - } - }); - - // create balanced groups - List>> fileGroups = - new LinkedList>>(); - long[] sizeGroups = new long[ngroups]; - int hi = files.size() - 1; - int lo = 0; - - List> group; - int dir = 1; - int g = 0; - - while (hi >= lo) { - if (g == fileGroups.size()) { - group = new LinkedList>(); - fileGroups.add(group); - } else { - group = fileGroups.get(g); - } - - Pair fileInfo = files.get(hi--); - - // add the hi one - sizeGroups[g] += fileInfo.getSecond(); - group.add(fileInfo); - - // change direction when at the end or the beginning - g += dir; - if (g == ngroups) { - dir = -1; - g = ngroups - 1; - } else if (g < 0) { - dir = 1; - g = 0; - } - } - - if (LOG.isDebugEnabled()) { - for (int i = 0; i < sizeGroups.length; ++i) { - LOG.debug("export split=" + i + " size=" + StringUtils.humanReadableInt(sizeGroups[i])); - } - } - - return fileGroups; - } - - private static class ExportSnapshotInputFormat extends InputFormat { - @Override - public RecordReader createRecordReader(InputSplit split, - TaskAttemptContext tac) throws IOException, InterruptedException { - return new ExportSnapshotRecordReader(((ExportSnapshotInputSplit)split).getSplitKeys()); - } - - @Override - public List getSplits(JobContext context) throws IOException, InterruptedException { - Configuration conf = context.getConfiguration(); - Path snapshotDir = new Path(conf.get(CONF_SNAPSHOT_DIR)); - FileSystem fs = FileSystem.get(snapshotDir.toUri(), conf); - - List> snapshotFiles = getSnapshotFiles(conf, fs, snapshotDir); - int mappers = conf.getInt(CONF_NUM_SPLITS, 0); - if (mappers == 0 && snapshotFiles.size() > 0) { - mappers = 1 + (snapshotFiles.size() / conf.getInt(CONF_MAP_GROUP, 10)); - mappers = Math.min(mappers, snapshotFiles.size()); - conf.setInt(CONF_NUM_SPLITS, mappers); - conf.setInt(MR_NUM_MAPS, mappers); - } - - List>> groups = getBalancedSplits(snapshotFiles, mappers); - List splits = new ArrayList(groups.size()); - for (List> files: groups) { - splits.add(new ExportSnapshotInputSplit(files)); - } - return splits; - } - - private static class ExportSnapshotInputSplit extends InputSplit implements Writable { - private List> files; - private long length; - - public ExportSnapshotInputSplit() { - this.files = null; - } - - public ExportSnapshotInputSplit(final List> snapshotFiles) { - this.files = new ArrayList(snapshotFiles.size()); - for (Pair fileInfo: snapshotFiles) { - this.files.add(new Pair( - new BytesWritable(fileInfo.getFirst().toByteArray()), fileInfo.getSecond())); - this.length += fileInfo.getSecond(); - } - } - - private List> getSplitKeys() { - return files; - } - - @Override - public long getLength() throws IOException, InterruptedException { - return length; - } - - @Override - public String[] getLocations() throws IOException, InterruptedException { - return new String[] {}; - } - - @Override - public void readFields(DataInput in) throws IOException { - int count = in.readInt(); - files = new ArrayList>(count); - length = 0; - for (int i = 0; i < count; ++i) { - BytesWritable fileInfo = new BytesWritable(); - fileInfo.readFields(in); - long size = in.readLong(); - files.add(new Pair(fileInfo, size)); - length += size; - } - } - - @Override - public void write(DataOutput out) throws IOException { - out.writeInt(files.size()); - for (final Pair fileInfo: files) { - fileInfo.getFirst().write(out); - out.writeLong(fileInfo.getSecond()); - } - } - } - - private static class ExportSnapshotRecordReader - extends RecordReader { - private final List> files; - private long totalSize = 0; - private long procSize = 0; - private int index = -1; - - ExportSnapshotRecordReader(final List> files) { - this.files = files; - for (Pair fileInfo: files) { - totalSize += fileInfo.getSecond(); - } - } - - @Override - public void close() { } - - @Override - public BytesWritable getCurrentKey() { return files.get(index).getFirst(); } - - @Override - public NullWritable getCurrentValue() { return NullWritable.get(); } - - @Override - public float getProgress() { return (float)procSize / totalSize; } - - @Override - public void initialize(InputSplit split, TaskAttemptContext tac) { } - - @Override - public boolean nextKeyValue() { - if (index >= 0) { - procSize += files.get(index).getSecond(); - } - return(++index < files.size()); - } - } - } - - // ========================================================================== - // Tool - // ========================================================================== - - /** - * Run Map-Reduce Job to perform the files copy. - */ - private void runCopyJob(final Path inputRoot, final Path outputRoot, - final String snapshotName, final Path snapshotDir, final boolean verifyChecksum, - final String filesUser, final String filesGroup, final int filesMode, - final int mappers, final int bandwidthMB) - throws IOException, InterruptedException, ClassNotFoundException { - Configuration conf = getConf(); - if (filesGroup != null) conf.set(CONF_FILES_GROUP, filesGroup); - if (filesUser != null) conf.set(CONF_FILES_USER, filesUser); - if (mappers > 0) { - conf.setInt(CONF_NUM_SPLITS, mappers); - conf.setInt(MR_NUM_MAPS, mappers); - } - conf.setInt(CONF_FILES_MODE, filesMode); - conf.setBoolean(CONF_CHECKSUM_VERIFY, verifyChecksum); - conf.set(CONF_OUTPUT_ROOT, outputRoot.toString()); - conf.set(CONF_INPUT_ROOT, inputRoot.toString()); - conf.setInt(CONF_BANDWIDTH_MB, bandwidthMB); - conf.set(CONF_SNAPSHOT_NAME, snapshotName); - conf.set(CONF_SNAPSHOT_DIR, snapshotDir.toString()); - - Job job = new Job(conf); - job.setJobName("ExportSnapshot-" + snapshotName); - job.setJarByClass(ExportSnapshot.class); - TableMapReduceUtil.addDependencyJars(job); - job.setMapperClass(ExportMapper.class); - job.setInputFormatClass(ExportSnapshotInputFormat.class); - job.setOutputFormatClass(NullOutputFormat.class); - job.setMapSpeculativeExecution(false); - job.setNumReduceTasks(0); - - // Acquire the delegation Tokens - Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX); - TokenCache.obtainTokensForNamenodes(job.getCredentials(), - new Path[] { inputRoot }, srcConf); - Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX); - TokenCache.obtainTokensForNamenodes(job.getCredentials(), - new Path[] { outputRoot }, destConf); - - // Run the MR Job - if (!job.waitForCompletion(true)) { - // TODO: Replace the fixed string with job.getStatus().getFailureInfo() - // when it will be available on all the supported versions. - throw new ExportSnapshotException("Copy Files Map-Reduce Job failed"); - } - } - - private void verifySnapshot(final Configuration baseConf, - final FileSystem fs, final Path rootDir, final Path snapshotDir) throws IOException { - // Update the conf with the current root dir, since may be a different cluster - Configuration conf = new Configuration(baseConf); - FSUtils.setRootDir(conf, rootDir); - FSUtils.setFsDefault(conf, FSUtils.getRootDir(conf)); - SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir); - SnapshotReferenceUtil.verifySnapshot(conf, fs, snapshotDir, snapshotDesc); - } - - /** - * Set path ownership. - */ - private void setOwner(final FileSystem fs, final Path path, final String user, - final String group, final boolean recursive) throws IOException { - if (user != null || group != null) { - if (recursive && fs.isDirectory(path)) { - for (FileStatus child : fs.listStatus(path)) { - setOwner(fs, child.getPath(), user, group, recursive); - } - } - fs.setOwner(path, user, group); - } - } - - /** - * Set path permission. - */ - private void setPermission(final FileSystem fs, final Path path, final short filesMode, - final boolean recursive) throws IOException { - if (filesMode > 0) { - FsPermission perm = new FsPermission(filesMode); - if (recursive && fs.isDirectory(path)) { - for (FileStatus child : fs.listStatus(path)) { - setPermission(fs, child.getPath(), filesMode, recursive); - } - } - fs.setPermission(path, perm); - } - } - - /** - * Execute the export snapshot by copying the snapshot metadata, hfiles and wals. - * @return 0 on success, and != 0 upon failure. - */ - @Override - public int run(String[] args) throws IOException { - boolean verifyTarget = true; - boolean verifyChecksum = true; - String snapshotName = null; - String targetName = null; - boolean overwrite = false; - String filesGroup = null; - String filesUser = null; - Path outputRoot = null; - int bandwidthMB = Integer.MAX_VALUE; - int filesMode = 0; - int mappers = 0; - - Configuration conf = getConf(); - Path inputRoot = FSUtils.getRootDir(conf); - - // Process command line args - for (int i = 0; i < args.length; i++) { - String cmd = args[i]; - if (cmd.equals("-snapshot")) { - snapshotName = args[++i]; - } else if (cmd.equals("-target")) { - targetName = args[++i]; - } else if (cmd.equals("-copy-to")) { - outputRoot = new Path(args[++i]); - } else if (cmd.equals("-copy-from")) { - inputRoot = new Path(args[++i]); - FSUtils.setRootDir(conf, inputRoot); - } else if (cmd.equals("-no-checksum-verify")) { - verifyChecksum = false; - } else if (cmd.equals("-no-target-verify")) { - verifyTarget = false; - } else if (cmd.equals("-mappers")) { - mappers = Integer.parseInt(args[++i]); - } else if (cmd.equals("-chuser")) { - filesUser = args[++i]; - } else if (cmd.equals("-chgroup")) { - filesGroup = args[++i]; - } else if (cmd.equals("-bandwidth")) { - bandwidthMB = Integer.parseInt(args[++i]); - } else if (cmd.equals("-chmod")) { - filesMode = Integer.parseInt(args[++i], 8); - } else if (cmd.equals("-overwrite")) { - overwrite = true; - } else if (cmd.equals("-h") || cmd.equals("--help")) { - printUsageAndExit(); - } else { - System.err.println("UNEXPECTED: " + cmd); - printUsageAndExit(); - } - } - - // Check user options - if (snapshotName == null) { - System.err.println("Snapshot name not provided."); - printUsageAndExit(); - } - - if (outputRoot == null) { - System.err.println("Destination file-system not provided."); - printUsageAndExit(); - } - - if (targetName == null) { - targetName = snapshotName; - } - - Configuration srcConf = HBaseConfiguration.createClusterConf(conf, null, CONF_SOURCE_PREFIX); - srcConf.setBoolean("fs." + inputRoot.toUri().getScheme() + ".impl.disable.cache", true); - FileSystem inputFs = FileSystem.get(inputRoot.toUri(), srcConf); - LOG.debug("inputFs=" + inputFs.getUri().toString() + " inputRoot=" + inputRoot); - Configuration destConf = HBaseConfiguration.createClusterConf(conf, null, CONF_DEST_PREFIX); - destConf.setBoolean("fs." + outputRoot.toUri().getScheme() + ".impl.disable.cache", true); - FileSystem outputFs = FileSystem.get(outputRoot.toUri(), destConf); - LOG.debug("outputFs=" + outputFs.getUri().toString() + " outputRoot=" + outputRoot.toString()); - - boolean skipTmp = conf.getBoolean(CONF_SKIP_TMP, false); - - Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, inputRoot); - Path snapshotTmpDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(targetName, outputRoot); - Path outputSnapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(targetName, outputRoot); - Path initialOutputSnapshotDir = skipTmp ? outputSnapshotDir : snapshotTmpDir; - - // Check if the snapshot already exists - if (outputFs.exists(outputSnapshotDir)) { - if (overwrite) { - if (!outputFs.delete(outputSnapshotDir, true)) { - System.err.println("Unable to remove existing snapshot directory: " + outputSnapshotDir); - return 1; - } - } else { - System.err.println("The snapshot '" + targetName + - "' already exists in the destination: " + outputSnapshotDir); - return 1; - } - } - - if (!skipTmp) { - // Check if the snapshot already in-progress - if (outputFs.exists(snapshotTmpDir)) { - if (overwrite) { - if (!outputFs.delete(snapshotTmpDir, true)) { - System.err.println("Unable to remove existing snapshot tmp directory: "+snapshotTmpDir); - return 1; - } - } else { - System.err.println("A snapshot with the same name '"+ targetName +"' may be in-progress"); - System.err.println("Please check "+snapshotTmpDir+". If the snapshot has completed, "); - System.err.println("consider removing "+snapshotTmpDir+" by using the -overwrite option"); - return 1; - } - } - } - - // Step 1 - Copy fs1:/.snapshot/ to fs2:/.snapshot/.tmp/ - // The snapshot references must be copied before the hfiles otherwise the cleaner - // will remove them because they are unreferenced. - try { - LOG.info("Copy Snapshot Manifest"); - FileUtil.copy(inputFs, snapshotDir, outputFs, initialOutputSnapshotDir, false, false, conf); - if (filesUser != null || filesGroup != null) { - setOwner(outputFs, snapshotTmpDir, filesUser, filesGroup, true); - } - if (filesMode > 0) { - setPermission(outputFs, snapshotTmpDir, (short)filesMode, true); - } - } catch (IOException e) { - throw new ExportSnapshotException("Failed to copy the snapshot directory: from=" + - snapshotDir + " to=" + initialOutputSnapshotDir, e); - } - - // Write a new .snapshotinfo if the target name is different from the source name - if (!targetName.equals(snapshotName)) { - SnapshotDescription snapshotDesc = - SnapshotDescriptionUtils.readSnapshotInfo(inputFs, snapshotDir) - .toBuilder() - .setName(targetName) - .build(); - SnapshotDescriptionUtils.writeSnapshotInfo(snapshotDesc, snapshotTmpDir, outputFs); - } - - // Step 2 - Start MR Job to copy files - // The snapshot references must be copied before the files otherwise the files gets removed - // by the HFileArchiver, since they have no references. - try { - runCopyJob(inputRoot, outputRoot, snapshotName, snapshotDir, verifyChecksum, - filesUser, filesGroup, filesMode, mappers, bandwidthMB); - - LOG.info("Finalize the Snapshot Export"); - if (!skipTmp) { - // Step 3 - Rename fs2:/.snapshot/.tmp/ fs2:/.snapshot/ - if (!outputFs.rename(snapshotTmpDir, outputSnapshotDir)) { - throw new ExportSnapshotException("Unable to rename snapshot directory from=" + - snapshotTmpDir + " to=" + outputSnapshotDir); - } - } - - // Step 4 - Verify snapshot integrity - if (verifyTarget) { - LOG.info("Verify snapshot integrity"); - verifySnapshot(destConf, outputFs, outputRoot, outputSnapshotDir); - } - - LOG.info("Export Completed: " + targetName); - return 0; - } catch (Exception e) { - LOG.error("Snapshot export failed", e); - if (!skipTmp) { - outputFs.delete(snapshotTmpDir, true); - } - outputFs.delete(outputSnapshotDir, true); - return 1; - } finally { - IOUtils.closeStream(inputFs); - IOUtils.closeStream(outputFs); - } - } - - // ExportSnapshot - private void printUsageAndExit() { - System.err.printf("Usage: bin/hbase %s [options]%n", getClass().getName()); - System.err.println(" where [options] are:"); - System.err.println(" -h|-help Show this help and exit."); - System.err.println(" -snapshot NAME Snapshot to restore."); - System.err.println(" -copy-to NAME Remote destination hdfs://"); - System.err.println(" -copy-from NAME Input folder hdfs:// (default hbase.rootdir)"); - System.err.println(" -no-checksum-verify Do not verify checksum, use name+length only."); - System.err.println(" -no-target-verify Do not verify the integrity of the \\" + - "exported snapshot."); - System.err.println(" -overwrite Rewrite the snapshot manifest if already exists"); - System.err.println(" -chuser USERNAME Change the owner of the files " + - "to the specified one."); - System.err.println(" -chgroup GROUP Change the group of the files to " + - "the specified one."); - System.err.println(" -chmod MODE Change the permission of the files " + - "to the specified one."); - System.err.println(" -mappers Number of mappers to use during the " + - "copy (mapreduce.job.maps)."); - System.err.println(" -bandwidth Limit bandwidth to this value in MB/second."); - System.err.println(); - System.err.println("Examples:"); - System.err.println(" hbase snapshot export \\"); - System.err.println(" -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase \\"); - System.err.println(" -chuser MyUser -chgroup MyGroup -chmod 700 -mappers 16"); - System.err.println(); - System.err.println(" hbase snapshot export \\"); - System.err.println(" -snapshot MySnapshot -copy-from hdfs://srv2:8082/hbase \\"); - System.err.println(" -copy-to hdfs://srv1:50070/hbase \\"); - System.exit(1); - } - - /** - * The guts of the {@link #main} method. - * Call this method to avoid the {@link #main(String[])} System.exit. - * @param args - * @return errCode - * @throws Exception - */ - static int innerMain(final Configuration conf, final String [] args) throws Exception { - return ToolRunner.run(conf, new ExportSnapshot(), args); - } - - public static void main(String[] args) throws Exception { - System.exit(innerMain(HBaseConfiguration.create(), args)); - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java deleted file mode 100644 index 2d9d3d7..0000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/RestoreSnapshotHelper.java +++ /dev/null @@ -1,823 +0,0 @@ -/** - * 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.fs.RegionStorage; -import org.apache.hadoop.hbase.fs.legacy.LegacyPathIdentifier; -import org.apache.hadoop.hbase.io.HFileLink; -import org.apache.hadoop.hbase.io.Reference; -import org.apache.hadoop.hbase.mob.MobUtils; -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.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 - *
    - * - * TODO update for MasterStorage / RegionStorage - */ -@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 tableDir; - - private final Configuration conf; - private final FileSystem fs; - private final boolean createBackRefs; - - public RestoreSnapshotHelper(final Configuration conf, final SnapshotManifest manifest, - final HTableDescriptor tableDescriptor, final ForeignExceptionDispatcher monitor, - final MonitoredTask status) throws IOException { - this(conf, manifest, tableDescriptor, monitor, status, true); - } - - public RestoreSnapshotHelper(final Configuration conf, final SnapshotManifest manifest, - final HTableDescriptor tableDescriptor, final ForeignExceptionDispatcher monitor, - final MonitoredTask status, final boolean createBackRefs) throws IOException { - this.fs = FSUtils.getCurrentFileSystem(conf); - this.conf = conf; - this.snapshotManifest = manifest; - this.snapshotDesc = manifest.getSnapshotDescription(); - this.snapshotTable = TableName.valueOf(snapshotDesc.getTable()); - this.tableDesc = tableDescriptor; - this.tableDir = FSUtils.getTableDir(FSUtils.getRootDir(conf), 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 restoreStorageRegions() throws IOException { - ThreadPoolExecutor exec = SnapshotManifest.createExecutor(conf, "RestoreSnapshot"); - try { - return restoreHdfsRegions(exec); - } finally { - exec.shutdown(); - } - } - - private RestoreMetaChanges restoreHdfsRegions(final ThreadPoolExecutor exec) throws IOException { - LOG.info("starting restore table regions using snapshot=" + snapshotDesc); - - 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()); - - HRegionInfo mobRegion = MobUtils.getMobRegionInfo(snapshotManifest.getTableDescriptor() - .getTableName()); - // 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); - metaChanges.addRegionToRestore(regionInfo); - } else { - LOG.info("region to remove: " + regionName); - metaChanges.addRegionToRemove(regionInfo); - } - } - - // Restore regions using the snapshot data - monitor.rethrowException(); - status.setStatus("Restoring table regions..."); - if (regionNames.contains(mobRegion.getEncodedName())) { - // restore the mob region in case - List mobRegions = new ArrayList(1); - mobRegions.add(mobRegion); - restoreHdfsMobRegions(exec, regionManifests, mobRegions); - regionNames.remove(mobRegion.getEncodedName()); - } - 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(); - // add the mob region - if (regionNames.contains(mobRegion.getEncodedName())) { - cloneHdfsMobRegion(regionManifests, mobRegion); - regionNames.remove(mobRegion.getEncodedName()); - } - 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."); - } - - LOG.info("finishing restore table regions using snapshot=" + snapshotDesc); - - 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; - - public RestoreMetaChanges(HTableDescriptor htd, Map > parentsMap) { - this.parentsMap = parentsMap; - this.htd = htd; - } - - public HTableDescriptor getTableDescriptor() { - return htd; - } - - /** - * Returns the map of parent-children_pair. - * @return the map - */ - public Map> getParentToChildrenPairMap() { - return this.parentsMap; - } - - /** - * @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())); - } - }); - } - - /** - * Restore specified mob regions by restoring content to the snapshot state. - */ - private void restoreHdfsMobRegions(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 { - restoreMobRegion(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 { - restoreRegion(regionInfo, regionManifest, new Path(tableDir, regionInfo.getEncodedName())); - } - - /** - * Restore mob region by removing files not in the snapshot - * and adding the missing ones from the snapshot. - */ - private void restoreMobRegion(final HRegionInfo regionInfo, - final SnapshotRegionManifest regionManifest) throws IOException { - if (regionManifest == null) { - return; - } - restoreRegion(regionInfo, regionManifest, - MobUtils.getMobRegionPath(conf, tableDesc.getTableName())); - } - - /** - * 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, Path regionDir) throws IOException { - Map> snapshotFiles = - getRegionHFileReferences(regionManifest); - - 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, - 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 the mob region. For the region create a new region - * and create a HFileLink for each hfile. - */ - private void cloneHdfsMobRegion(final Map regionManifests, - final HRegionInfo region) throws IOException { - // clone region info (change embedded tableName with the new one) - Path clonedRegionPath = MobUtils.getMobRegionPath(conf, tableDesc.getTableName()); - cloneRegion(clonedRegionPath, region, regionManifests.get(region.getEncodedName())); - } - - /** - * 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 regionDir {@link Path} cloned dir - * @param snapshotRegionInfo - */ - private void cloneRegion(final Path regionDir, final HRegionInfo snapshotRegionInfo, - final SnapshotRegionManifest manifest) throws IOException { - 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); - } - } - } - - /** - * 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 { - cloneRegion(new Path(tableDir, region.getRegionInfo().getEncodedName()), snapshotRegionInfo, - manifest); - } - - /** - * 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 createBackRef - Whether back reference should be created. Defaults to true. - * @param storeFile store file name (can be a Reference, HFileLink or simple HFile) - */ - 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 storeFile 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 LinkedList(); - for (FileStatus regionDir: regionDirs) { - final RegionStorage rs = RegionStorage.open(conf, new LegacyPathIdentifier(regionDir.getPath()), false); - regions.add(rs.getRegionInfo()); - } - 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, - manifest, manifest.getTableDescriptor(), monitor, status, false); - RestoreMetaChanges metaChanges = helper.restoreStorageRegions(); // TODO: parallelize. - - if (LOG.isDebugEnabled()) { - LOG.debug("Restored table dir:" + restoreDir); - FSUtils.logFileSystemState(fs, restoreDir, LOG); - } - return metaChanges; - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java index 2fd619d..ce12ba9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotDescriptionUtils.java @@ -17,79 +17,21 @@ */ package org.apache.hadoop.hbase.snapshot; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.Collections; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifestV2; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.snapshot.SnapshotManifestV2; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.FSUtils; /** * Utility class to help manage {@link SnapshotDescription SnapshotDesriptions}. - *

    - * Snapshots are laid out on disk like this: - * - *

    - * /hbase/.snapshots
    - *          /.tmp                <---- working directory
    - *          /[snapshot name]     <----- completed snapshot
    - * 
    - * - * A completed snapshot named 'completed' then looks like (multiple regions, servers, files, etc. - * signified by '...' on the same directory depth). - * - *
    - * /hbase/.snapshots/completed
    - *                   .snapshotinfo          <--- Description of the snapshot
    - *                   .tableinfo             <--- Copy of the tableinfo
    - *                    /.logs
    - *                        /[server_name]
    - *                            /... [log files]
    - *                         ...
    - *                   /[region name]           <---- All the region's information
    - *                   .regioninfo              <---- Copy of the HRegionInfo
    - *                      /[column family name]
    - *                          /[hfile name]     <--- name of the hfile in the real region
    - *                          ...
    - *                      ...
    - *                    ...
    - * 
    - * - * Utility methods in this class are useful for getting the correct locations for different parts of - * the snapshot, as well as moving completed snapshots into place (see - * {@link #completeSnapshot}, and writing the - * {@link SnapshotDescription} to the working snapshot directory. */ @InterfaceAudience.Private public final class SnapshotDescriptionUtils { - /** - * Filter that only accepts completed snapshot directories - */ - public static class CompletedSnaphotDirectoriesFilter extends FSUtils.BlackListDirFilter { - - /** - * @param fs - */ - public CompletedSnaphotDirectoriesFilter(FileSystem fs) { - super(fs, Collections.singletonList(SNAPSHOT_TMP_DIR_NAME)); - } - } - private static final Log LOG = LogFactory.getLog(SnapshotDescriptionUtils.class); /** * Version of the fs layout for a snapshot. Future snapshots may have different file layouts, @@ -97,17 +39,6 @@ public final class SnapshotDescriptionUtils { */ public static final int SNAPSHOT_LAYOUT_VERSION = SnapshotManifestV2.DESCRIPTOR_VERSION; - // snapshot directory constants - /** - * The file contains the snapshot basic information and it is under the directory of a snapshot. - */ - public static final String SNAPSHOTINFO_FILE = ".snapshotinfo"; - - /** Temporary directory under the snapshot directory to store in-progress snapshots */ - public static final String SNAPSHOT_TMP_DIR_NAME = ".tmp"; - - /** This tag will be created in in-progess snapshots */ - public static final String SNAPSHOT_IN_PROGRESS = ".inprogress"; // snapshot operation values /** Default value if no start time is specified */ public static final long NO_SNAPSHOT_START_TIME_SPECIFIED = 0; @@ -157,86 +88,6 @@ public final class SnapshotDescriptionUtils { } /** - * Get the snapshot root directory. All the snapshots are kept under this directory, i.e. - * ${hbase.rootdir}/.snapshot - * @param rootDir hbase root directory - * @return the base directory in which all snapshots are kept - */ - public static Path getSnapshotRootDir(final Path rootDir) { - return new Path(rootDir, HConstants.SNAPSHOT_DIR_NAME); - } - - /** - * Get the directory for a specified snapshot. This directory is a sub-directory of snapshot root - * directory and all the data files for a snapshot are kept under this directory. - * @param snapshot snapshot being taken - * @param rootDir hbase root directory - * @return the final directory for the completed snapshot - */ - public static Path getCompletedSnapshotDir(final SnapshotDescription snapshot, final Path rootDir) { - return getCompletedSnapshotDir(snapshot.getName(), rootDir); - } - - /** - * Get the directory for a completed snapshot. This directory is a sub-directory of snapshot root - * directory and all the data files for a snapshot are kept under this directory. - * @param snapshotName name of the snapshot being taken - * @param rootDir hbase root directory - * @return the final directory for the completed snapshot - */ - public static Path getCompletedSnapshotDir(final String snapshotName, final Path rootDir) { - return getCompletedSnapshotDir(getSnapshotsDir(rootDir), snapshotName); - } - - /** - * Get the general working directory for snapshots - where they are built, where they are - * temporarily copied on export, etc. - * @param rootDir root directory of the HBase installation - * @return Path to the snapshot tmp directory, relative to the passed root directory - */ - public static Path getWorkingSnapshotDir(final Path rootDir) { - return new Path(getSnapshotsDir(rootDir), SNAPSHOT_TMP_DIR_NAME); - } - - /** - * Get the directory to build a snapshot, before it is finalized - * @param snapshot snapshot that will be built - * @param rootDir root directory of the hbase installation - * @return {@link Path} where one can build a snapshot - */ - public static Path getWorkingSnapshotDir(SnapshotDescription snapshot, final Path rootDir) { - return getCompletedSnapshotDir(getWorkingSnapshotDir(rootDir), snapshot.getName()); - } - - /** - * Get the directory to build a snapshot, before it is finalized - * @param snapshotName name of the snapshot - * @param rootDir root directory of the hbase installation - * @return {@link Path} where one can build a snapshot - */ - public static Path getWorkingSnapshotDir(String snapshotName, final Path rootDir) { - return getCompletedSnapshotDir(getWorkingSnapshotDir(rootDir), snapshotName); - } - - /** - * Get the directory to store the snapshot instance - * @param snapshotsDir hbase-global directory for storing all snapshots - * @param snapshotName name of the snapshot to take - * @return the final directory for the completed snapshot - */ - private static final Path getCompletedSnapshotDir(final Path snapshotsDir, String snapshotName) { - return new Path(snapshotsDir, snapshotName); - } - - /** - * @param rootDir hbase root directory - * @return the directory for all completed snapshots; - */ - public static final Path getSnapshotsDir(Path rootDir) { - return new Path(rootDir, HConstants.SNAPSHOT_DIR_NAME); - } - - /** * Convert the passed snapshot description into a 'full' snapshot description based on default * parameters, if none have been supplied. This resolves any 'optional' parameters that aren't * supplied to their default values. @@ -267,94 +118,6 @@ public final class SnapshotDescriptionUtils { } /** - * Write the snapshot description into the working directory of a snapshot - * @param snapshot description of the snapshot being taken - * @param workingDir working directory of the snapshot - * @param fs {@link FileSystem} on which the snapshot should be taken - * @throws IOException if we can't reach the filesystem and the file cannot be cleaned up on - * failure - */ - public static void writeSnapshotInfo(SnapshotDescription snapshot, Path workingDir, FileSystem fs) - throws IOException { - FsPermission perms = FSUtils.getFilePermissions(fs, fs.getConf(), - HConstants.DATA_FILE_UMASK_KEY); - Path snapshotInfo = new Path(workingDir, SnapshotDescriptionUtils.SNAPSHOTINFO_FILE); - try { - FSDataOutputStream out = FSUtils.create(fs, snapshotInfo, perms, true); - try { - snapshot.writeTo(out); - } finally { - out.close(); - } - } catch (IOException e) { - // if we get an exception, try to remove the snapshot info - if (!fs.delete(snapshotInfo, false)) { - String msg = "Couldn't delete snapshot info file: " + snapshotInfo; - LOG.error(msg); - throw new IOException(msg); - } - } - } - - /** - * Create in-progress tag under .tmp of in-progress snapshot - * */ - public static void createInProgressTag(Path workingDir, FileSystem fs) throws IOException { - FsPermission perms = FSUtils.getFilePermissions(fs, fs.getConf(), - HConstants.DATA_FILE_UMASK_KEY); - Path snapshot_in_progress = new Path(workingDir, SnapshotDescriptionUtils.SNAPSHOT_IN_PROGRESS); - FSUtils.create(fs, snapshot_in_progress, perms, true); - } - - /** - * Read in the {@link org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription} stored for the snapshot in the passed directory - * @param fs filesystem where the snapshot was taken - * @param snapshotDir directory where the snapshot was stored - * @return the stored snapshot description - * @throws CorruptedSnapshotException if the - * snapshot cannot be read - */ - public static SnapshotDescription readSnapshotInfo(FileSystem fs, Path snapshotDir) - throws CorruptedSnapshotException { - Path snapshotInfo = new Path(snapshotDir, SNAPSHOTINFO_FILE); - try { - FSDataInputStream in = null; - try { - in = fs.open(snapshotInfo); - SnapshotDescription desc = SnapshotDescription.parseFrom(in); - return desc; - } finally { - if (in != null) in.close(); - } - } catch (IOException e) { - throw new CorruptedSnapshotException("Couldn't read snapshot info from:" + snapshotInfo, e); - } - } - - /** - * Move the finished snapshot to its final, publicly visible directory - this marks the snapshot - * as 'complete'. - * @param snapshot description of the snapshot being tabken - * @param rootdir root directory of the hbase installation - * @param workingDir directory where the in progress snapshot was built - * @param fs {@link FileSystem} where the snapshot was built - * @throws org.apache.hadoop.hbase.snapshot.SnapshotCreationException if the - * snapshot could not be moved - * @throws IOException the filesystem could not be reached - */ - public static void completeSnapshot(SnapshotDescription snapshot, Path rootdir, Path workingDir, - FileSystem fs) throws SnapshotCreationException, IOException { - Path finishedDir = getCompletedSnapshotDir(snapshot, rootdir); - LOG.debug("Snapshot is done, just moving the snapshot from " + workingDir + " to " - + finishedDir); - if (!fs.rename(workingDir, finishedDir)) { - throw new SnapshotCreationException( - "Failed to move working directory(" + workingDir + ") to completed directory(" - + finishedDir + ").", ProtobufUtil.createSnapshotDesc(snapshot)); - } - } - - /** * Check if the user is this table snapshot's owner * @param snapshot the table snapshot description * @param user the user diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java index 8050d93..3e3828a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotInfo.java @@ -36,14 +36,20 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.SnapshotDescription; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.fs.MasterStorage; +import org.apache.hadoop.hbase.fs.StorageContext; +import org.apache.hadoop.hbase.fs.StorageIdentifier; +import org.apache.hadoop.hbase.fs.legacy.LegacyPathIdentifier; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifest; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; @@ -53,7 +59,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.WALLink; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; import org.apache.hadoop.hbase.util.FSUtils; @@ -135,24 +140,21 @@ public final class SnapshotInfo extends Configured implements Tool { private final HBaseProtos.SnapshotDescription snapshot; private final TableName snapshotTable; - private final Configuration conf; - private final FileSystem fs; + private final MasterStorage masterStorage; - SnapshotStats(final Configuration conf, final FileSystem fs, + SnapshotStats(final MasterStorage masterStorage, final SnapshotDescription snapshot) { this.snapshot = ProtobufUtil.createHBaseProtosSnapshotDesc(snapshot); this.snapshotTable = TableName.valueOf(snapshot.getTable()); - this.conf = conf; - this.fs = fs; + this.masterStorage = masterStorage; } - SnapshotStats(final Configuration conf, final FileSystem fs, + SnapshotStats(final MasterStorage masterStorage, final HBaseProtos.SnapshotDescription snapshot) { this.snapshot = snapshot; this.snapshotTable = TableName.valueOf(snapshot.getTable()); - this.conf = conf; - this.fs = fs; + this.masterStorage = masterStorage; } @@ -225,7 +227,7 @@ public final class SnapshotInfo extends Configured implements Tool { * with other snapshots and tables * * This is only calculated when - * {@link #getSnapshotStats(Configuration, HBaseProtos.SnapshotDescription, Map)} + * {@link #getSnapshotStats(Configuration, HBaseProtos.SnapshotDescription, Map)} * is called with a non-null Map */ public long getNonSharedArchivedStoreFilesSize() { @@ -265,7 +267,7 @@ public final class SnapshotInfo extends Configured implements Tool { Path parentDir = filePath.getParent(); Path backRefDir = HFileLink.getBackReferencesDir(parentDir, filePath.getName()); try { - if (FSUtils.listStatus(fs, backRefDir) == null) { + if (FSUtils.listStatus(masterStorage.getFileSystem(), backRefDir) == null) { return false; } } catch (IOException e) { @@ -287,6 +289,8 @@ public final class SnapshotInfo extends Configured implements Tool { FileInfo addStoreFile(final HRegionInfo region, final String family, final SnapshotRegionManifest.StoreFile storeFile, final Map filesMap) throws IOException { + Configuration conf = masterStorage.getConfiguration(); + FileSystem fs = masterStorage.getFileSystem(); HFileLink link = HFileLink.build(conf, snapshotTable, region.getEncodedName(), family, storeFile.getName()); boolean isCorrupted = false; @@ -328,10 +332,10 @@ public final class SnapshotInfo extends Configured implements Tool { * @return the log information */ FileInfo addLogFile(final String server, final String logfile) throws IOException { - WALLink logLink = new WALLink(conf, server, logfile); + WALLink logLink = new WALLink(masterStorage.getConfiguration(), server, logfile); long size = -1; try { - size = logLink.getFileStatus(fs).getLen(); + size = logLink.getFileStatus(masterStorage.getFileSystem()).getLen(); logSize.addAndGet(size); logsCount.incrementAndGet(); } catch (FileNotFoundException e) { @@ -342,10 +346,10 @@ public final class SnapshotInfo extends Configured implements Tool { } private boolean printSizeInBytes = false; - private FileSystem fs; - private Path rootDir; + private MasterStorage masterStorage; - private SnapshotManifest snapshotManifest; + private HBaseProtos.SnapshotDescription snapshotDesc; + private HTableDescriptor snapshotTable; @Override @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION", @@ -391,11 +395,14 @@ public final class SnapshotInfo extends Configured implements Tool { } } + // instantiate MasterStorage + masterStorage = MasterStorage.open(conf, false); + // List Available Snapshots if (listSnapshots) { SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss"); System.out.printf("%-20s | %-20s | %s%n", "SNAPSHOT", "CREATION TIME", "TABLE NAME"); - for (SnapshotDescription desc: getSnapshotList(conf)) { + for (SnapshotDescription desc: getSnapshotList(masterStorage)) { System.out.printf("%-20s | %20s | %s%n", desc.getName(), df.format(new Date(desc.getCreationTime())), @@ -410,9 +417,8 @@ public final class SnapshotInfo extends Configured implements Tool { return 1; } - rootDir = FSUtils.getRootDir(conf); - fs = FileSystem.get(rootDir.toUri(), conf); - LOG.debug("fs=" + fs.getUri().toString() + " root=" + rootDir); + LOG.debug("fs=" + masterStorage.getFileSystem().getUri().toString() + " root=" + + ((LegacyPathIdentifier)masterStorage.getRootContainer()).path); // Load snapshot information if (!loadSnapshotInfo(snapshotName)) { @@ -433,15 +439,8 @@ public final class SnapshotInfo extends Configured implements Tool { * @return false if snapshot is not found */ private boolean loadSnapshotInfo(final String snapshotName) throws IOException { - Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir); - if (!fs.exists(snapshotDir)) { - LOG.warn("Snapshot '" + snapshotName + "' not found in: " + snapshotDir); - return false; - } - - HBaseProtos.SnapshotDescription snapshotDesc = - SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir); - snapshotManifest = SnapshotManifest.open(getConf(), fs, snapshotDir, snapshotDesc); + snapshotDesc = masterStorage.getSnapshot(snapshotName); + snapshotTable = masterStorage.getTableDescriptorForSnapshot(snapshotDesc); return true; } @@ -449,7 +448,6 @@ public final class SnapshotInfo extends Configured implements Tool { * Dump the {@link SnapshotDescription} */ private void printInfo() { - HBaseProtos.SnapshotDescription snapshotDesc = snapshotManifest.getSnapshotDescription(); SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss"); System.out.println("Snapshot Info"); System.out.println("----------------------------------------"); @@ -468,7 +466,7 @@ public final class SnapshotInfo extends Configured implements Tool { private void printSchema() { System.out.println("Table Descriptor"); System.out.println("----------------------------------------"); - System.out.println(snapshotManifest.getTableDescriptor().toString()); + System.out.println(snapshotTable.toString()); System.out.println(); } @@ -483,30 +481,30 @@ public final class SnapshotInfo extends Configured implements Tool { } // Collect information about hfiles and logs in the snapshot - final HBaseProtos.SnapshotDescription snapshotDesc = snapshotManifest.getSnapshotDescription(); final String table = snapshotDesc.getTable(); SnapshotDescription desc = new SnapshotDescription(snapshotDesc.getName(), snapshotDesc.getTable(), ProtobufUtil.createSnapshotType(snapshotDesc.getType()), snapshotDesc.getOwner(), snapshotDesc.getCreationTime(), snapshotDesc.getVersion()); - final SnapshotStats stats = new SnapshotStats(this.getConf(), this.fs, desc); - SnapshotReferenceUtil.concurrentVisitReferencedFiles(getConf(), fs, snapshotManifest, - "SnapshotInfo", - new SnapshotReferenceUtil.SnapshotVisitor() { - @Override - public void storeFile(final HRegionInfo regionInfo, final String family, - final SnapshotRegionManifest.StoreFile storeFile) throws IOException { - if (storeFile.hasReference()) return; - - SnapshotStats.FileInfo info = stats.addStoreFile(regionInfo, family, storeFile, null); - if (showFiles) { - String state = info.getStateToString(); - System.out.printf("%8s %s/%s/%s/%s %s%n", - (info.isMissing() ? "-" : fileSizeToString(info.getSize())), - table, regionInfo.getEncodedName(), family, storeFile.getName(), - state == null ? "" : "(" + state + ")"); + final SnapshotStats stats = new SnapshotStats(masterStorage, desc); + // TODO: add concurrent API to MasterStorage + masterStorage.visitSnapshotStoreFiles(snapshotDesc, StorageContext.DATA, + new MasterStorage.SnapshotStoreFileVisitor() { + @Override + public void visitSnapshotStoreFile(HBaseProtos.SnapshotDescription snapshot, + StorageContext ctx, HRegionInfo hri, String familyName, + SnapshotRegionManifest.StoreFile storeFile) throws IOException { + if (storeFile.hasReference()) return; + + SnapshotStats.FileInfo info = stats.addStoreFile(hri, familyName, storeFile, null); + if (showFiles) { + String state = info.getStateToString(); + System.out.printf("%8s %s/%s/%s/%s %s%n", + (info.isMissing() ? "-" : fileSizeToString(info.getSize())), + table, hri.getEncodedName(), familyName, storeFile.getName(), + state == null ? "" : "(" + state + ")"); + } } - } - }); + }); // Dump the stats System.out.println(); @@ -569,7 +567,8 @@ public final class SnapshotInfo extends Configured implements Tool { HBaseProtos.SnapshotDescription snapshotDesc = ProtobufUtil.createHBaseProtosSnapshotDesc( snapshot); - return getSnapshotStats(conf, snapshotDesc, null); + MasterStorage masterStorage = MasterStorage.open(conf, false); + return getSnapshotStats(masterStorage.getConfiguration(), snapshotDesc, null); } /** @@ -579,53 +578,43 @@ public final class SnapshotInfo extends Configured implements Tool { * @param filesMap {@link Map} store files map for all snapshots, it may be null * @return the snapshot stats */ - public static SnapshotStats getSnapshotStats(final Configuration conf, - final HBaseProtos.SnapshotDescription snapshotDesc, + public static SnapshotStats getSnapshotStats( + final Configuration conf, final HBaseProtos.SnapshotDescription snapshotDesc, final Map filesMap) throws IOException { - Path rootDir = FSUtils.getRootDir(conf); - FileSystem fs = FileSystem.get(rootDir.toUri(), conf); - Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotDesc, rootDir); - SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc); - final SnapshotStats stats = new SnapshotStats(conf, fs, snapshotDesc); - SnapshotReferenceUtil.concurrentVisitReferencedFiles(conf, fs, manifest, - "SnapshotsStatsAggregation", new SnapshotReferenceUtil.SnapshotVisitor() { + MasterStorage masterStorage = MasterStorage.open(conf, false); + final SnapshotStats stats = new SnapshotStats(masterStorage, snapshotDesc); + masterStorage.visitSnapshotStoreFiles(snapshotDesc, StorageContext.DATA, + new MasterStorage.SnapshotStoreFileVisitor() { @Override - public void storeFile(final HRegionInfo regionInfo, final String family, - final SnapshotRegionManifest.StoreFile storeFile) throws IOException { + public void visitSnapshotStoreFile(HBaseProtos.SnapshotDescription snapshot, + StorageContext ctx, HRegionInfo hri, String familyName, + SnapshotRegionManifest.StoreFile storeFile) throws IOException { if (!storeFile.hasReference()) { - stats.addStoreFile(regionInfo, family, storeFile, filesMap); + stats.addStoreFile(hri, familyName, storeFile, filesMap); } - }}); + } + }); return stats; } /** * Returns the list of available snapshots in the specified location - * @param conf the {@link Configuration} to use + * @param masterStorage the {@link MasterStorage} to use * @return the list of snapshots */ - public static List getSnapshotList(final Configuration conf) - throws IOException { - Path rootDir = FSUtils.getRootDir(conf); - FileSystem fs = FileSystem.get(rootDir.toUri(), conf); - Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir); - FileStatus[] snapshots = fs.listStatus(snapshotDir, - new SnapshotDescriptionUtils.CompletedSnaphotDirectoriesFilter(fs)); - List snapshotLists = - new ArrayList(snapshots.length); - for (FileStatus snapshotDirStat: snapshots) { - HBaseProtos.SnapshotDescription snapshotDesc = - SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDirStat.getPath()); - snapshotLists.add(new SnapshotDescription(snapshotDesc.getName(), - snapshotDesc.getTable(), ProtobufUtil.createSnapshotType(snapshotDesc.getType()), - snapshotDesc.getOwner(), snapshotDesc.getCreationTime(), snapshotDesc.getVersion())); + public static List getSnapshotList( + final MasterStorage masterStorage) throws IOException { + List descriptions = masterStorage.getSnapshots(); + List snapshotLists = new ArrayList(descriptions.size()); + for (HBaseProtos.SnapshotDescription desc: masterStorage.getSnapshots()) { + snapshotLists.add(SnapshotDescription.convert(desc)); } return snapshotLists; } /** * Gets the store files map for snapshot - * @param conf the {@link Configuration} to use + * @param masterStorage the {@link MasterStorage} to use * @param snapshot {@link SnapshotDescription} to get stats from * @param exec the {@link ExecutorService} to use * @param filesMap {@link Map} the map to put the mapping entries @@ -634,33 +623,32 @@ public final class SnapshotInfo extends Configured implements Tool { * @param uniqueHFilesMobSize {@link AtomicLong} the accumulated mob store file size shared * @return the snapshot stats */ - private static void getSnapshotFilesMap(final Configuration conf, + private static void getSnapshotFilesMap( + final MasterStorage masterStorage, final SnapshotDescription snapshot, final ExecutorService exec, final ConcurrentHashMap filesMap, final AtomicLong uniqueHFilesArchiveSize, final AtomicLong uniqueHFilesSize, final AtomicLong uniqueHFilesMobSize) throws IOException { HBaseProtos.SnapshotDescription snapshotDesc = ProtobufUtil.createHBaseProtosSnapshotDesc( snapshot); - Path rootDir = FSUtils.getRootDir(conf); - final FileSystem fs = FileSystem.get(rootDir.toUri(), conf); - - Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotDesc, rootDir); - SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc); - SnapshotReferenceUtil.concurrentVisitReferencedFiles(conf, fs, manifest, exec, - new SnapshotReferenceUtil.SnapshotVisitor() { - @Override public void storeFile(final HRegionInfo regionInfo, final String family, - final SnapshotRegionManifest.StoreFile storeFile) throws IOException { + masterStorage.visitSnapshotStoreFiles(snapshotDesc, StorageContext.DATA, + new MasterStorage.SnapshotStoreFileVisitor() { + @Override + public void visitSnapshotStoreFile(HBaseProtos.SnapshotDescription snapshot, + StorageContext ctx, HRegionInfo hri, String familyName, + SnapshotRegionManifest.StoreFile storeFile) throws IOException { + Configuration conf = masterStorage.getConfiguration(); + FileSystem fs = masterStorage.getFileSystem(); if (!storeFile.hasReference()) { - HFileLink link = HFileLink - .build(conf, TableName.valueOf(snapshot.getTable()), regionInfo.getEncodedName(), - family, storeFile.getName()); + HFileLink link = HFileLink.build(conf, TableName.valueOf(snapshot.getTable()), + hri.getEncodedName(), familyName, storeFile.getName()); long size; Integer count; Path p; AtomicLong al; int c = 0; - if (fs.exists(link.getArchivePath())) { + if (masterStorage.getFileSystem().exists(link.getArchivePath())) { p = link.getArchivePath(); al = uniqueHFilesArchiveSize; size = fs.getFileStatus(p).getLen(); @@ -685,7 +673,7 @@ public final class SnapshotInfo extends Configured implements Tool { filesMap.put(p, ++c); } } - }); + }); } /** @@ -699,7 +687,8 @@ public final class SnapshotInfo extends Configured implements Tool { public static Map getSnapshotsFilesMap(final Configuration conf, AtomicLong uniqueHFilesArchiveSize, AtomicLong uniqueHFilesSize, AtomicLong uniqueHFilesMobSize) throws IOException { - List snapshotList = getSnapshotList(conf); + MasterStorage masterStorage = MasterStorage.open(conf, false); + List snapshotList = getSnapshotList(masterStorage); if (snapshotList.size() == 0) { @@ -712,7 +701,7 @@ public final class SnapshotInfo extends Configured implements Tool { try { for (final SnapshotDescription snapshot : snapshotList) { - getSnapshotFilesMap(conf, snapshot, exec, fileMap, uniqueHFilesArchiveSize, + getSnapshotFilesMap(masterStorage, snapshot, exec, fileMap, uniqueHFilesArchiveSize, uniqueHFilesSize, uniqueHFilesMobSize); } } finally { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java deleted file mode 100644 index 572bc04..0000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java +++ /dev/null @@ -1,570 +0,0 @@ -/** - * 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.CodedInputStream; -import com.google.protobuf.InvalidProtocolBufferException; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; -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.HTableDescriptor; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.fs.RegionStorage; -import org.apache.hadoop.hbase.fs.legacy.LegacyPathIdentifier; -import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor; -import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -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.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; -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.Threads; -import org.apache.zookeeper.server.persistence.SnapShot; - -/** - * Utility class to help read/write the Snapshot Manifest. - * - * The snapshot format is transparent for the users of this class, - * once the snapshot is written, it will never be modified. - * On open() the snapshot will be loaded to the current in-memory format. - */ -@InterfaceAudience.Private -public final class SnapshotManifest { - private static final Log LOG = LogFactory.getLog(SnapshotManifest.class); - - 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 regionManifests; - private SnapshotDescription desc; - private HTableDescriptor htd; - - private final ForeignExceptionSnare monitor; - 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, - final ForeignExceptionSnare monitor) { - this.monitor = monitor; - this.desc = desc; - this.workingDir = workingDir; - this.conf = conf; - this.fs = fs; - - this.manifestSizeLimit = conf.getInt(SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY, 64 * 1024 * 1024); - } - - /** - * Return a SnapshotManifest instance, used for writing a snapshot. - * - * There are two usage pattern: - * - The Master will create a manifest, add the descriptor, offline regions - * and consolidate the snapshot by writing all the pending stuff on-disk. - * manifest = SnapshotManifest.create(...) - * manifest.addRegion(tableDir, hri) - * manifest.consolidate() - * - The RegionServer will create a single region manifest - * manifest = SnapshotManifest.create(...) - * manifest.addRegion(region) - */ - public static SnapshotManifest create(final Configuration conf, final FileSystem fs, - final Path workingDir, final SnapshotDescription desc, - final ForeignExceptionSnare monitor) { - return new SnapshotManifest(conf, fs, workingDir, desc, monitor); - } - - /** - * Return a SnapshotManifest instance with the information already loaded in-memory. - * SnapshotManifest manifest = SnapshotManifest.open(...) - * HTableDescriptor htd = manifest.getTableDescriptor() - * for (SnapshotRegionManifest regionManifest: manifest.getRegionManifests()) - * hri = regionManifest.getRegionInfo() - * for (regionManifest.getFamilyFiles()) - * ... - */ - public static SnapshotManifest open(final Configuration conf, final FileSystem fs, - final Path workingDir, final SnapshotDescription desc) throws IOException { - SnapshotManifest manifest = new SnapshotManifest(conf, fs, workingDir, desc, null); - manifest.load(); - return manifest; - } - - public static SnapshotManifest open(final Configuration conf, final SnapshotDescription desc) - throws IOException { - Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(desc, - FSUtils.getRootDir(conf)); - return open(conf, snapshotDir.getFileSystem(conf), snapshotDir, desc); - } - - /** - * Add the table descriptor to the snapshot manifest - */ - public void addTableDescriptor(final HTableDescriptor htd) throws IOException { - this.htd = htd; - } - - interface RegionVisitor { - TRegion regionOpen(final HRegionInfo regionInfo) throws IOException; - void regionClose(final TRegion region) throws IOException; - - TFamily familyOpen(final TRegion region, final byte[] familyName) throws IOException; - void familyClose(final TRegion region, final TFamily family) throws IOException; - - void storeFile(final TRegion region, final TFamily family, final StoreFileInfo storeFile) - throws IOException; - } - - private RegionVisitor createRegionVisitor(final SnapshotDescription desc) throws IOException { - switch (getSnapshotFormat(desc)) { - case SnapshotManifestV1.DESCRIPTOR_VERSION: - return new SnapshotManifestV1.ManifestBuilder(conf, fs, workingDir); - case SnapshotManifestV2.DESCRIPTOR_VERSION: - return new SnapshotManifestV2.ManifestBuilder(conf, fs, workingDir); - default: - throw new CorruptedSnapshotException("Invalid Snapshot version: " + desc.getVersion(), - ProtobufUtil.createSnapshotDesc(desc)); - } - } - - public void addMobRegion(HRegionInfo regionInfo, HColumnDescriptor[] hcds) throws IOException { - // 0. Get the ManifestBuilder/RegionVisitor - RegionVisitor visitor = createRegionVisitor(desc); - - // 1. dump region meta info into the snapshot directory - LOG.debug("Storing mob region '" + regionInfo + "' region-info for snapshot."); - Object regionData = visitor.regionOpen(regionInfo); - monitor.rethrowException(); - - // 2. iterate through all the stores in the region - LOG.debug("Creating references for mob files"); - - Path mobRegionPath = MobUtils.getMobRegionPath(conf, regionInfo.getTable()); - for (HColumnDescriptor hcd : hcds) { - // 2.1. build the snapshot reference for the store if it's a mob store - if (!hcd.isMobEnabled()) { - continue; - } - Object familyData = visitor.familyOpen(regionData, hcd.getName()); - monitor.rethrowException(); - - Path storePath = MobUtils.getMobFamilyPath(mobRegionPath, hcd.getNameAsString()); - List storeFiles = getStoreFiles(storePath); - if (storeFiles == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("No mob files under family: " + hcd.getNameAsString()); - } - continue; - } - - addReferenceFiles(visitor, regionData, familyData, storeFiles, true); - - visitor.familyClose(regionData, familyData); - } - visitor.regionClose(regionData); - } - - /** - * Creates a 'manifest' for the specified region, by reading directly from the HRegion object. - * This is used by the "online snapshot" when the table is enabled. - */ - public void addRegion(final HRegion region) throws IOException { - // 0. Get the ManifestBuilder/RegionVisitor - RegionVisitor visitor = createRegionVisitor(desc); - - // 1. dump region meta info into the snapshot directory - LOG.debug("Storing '" + region + "' region-info for snapshot."); - Object regionData = visitor.regionOpen(region.getRegionInfo()); - monitor.rethrowException(); - - // 2. iterate through all the stores in the region - LOG.debug("Creating references for hfiles"); - - for (Store store : region.getStores()) { - // 2.1. build the snapshot reference for the store - Object familyData = visitor.familyOpen(regionData, store.getFamily().getName()); - monitor.rethrowException(); - - List storeFiles = new ArrayList(store.getStorefiles()); - if (LOG.isDebugEnabled()) { - LOG.debug("Adding snapshot references for " + storeFiles + " hfiles"); - } - - // 2.2. iterate through all the store's files and create "references". - for (int i = 0, sz = storeFiles.size(); i < sz; i++) { - StoreFile storeFile = storeFiles.get(i); - monitor.rethrowException(); - - // create "reference" to this store file. - LOG.debug("Adding reference for file (" + (i+1) + "/" + sz + "): " + storeFile.getPath()); - visitor.storeFile(regionData, familyData, storeFile.getFileInfo()); - } - visitor.familyClose(regionData, familyData); - } - visitor.regionClose(regionData); - } - - /** - * Creates a 'manifest' for the specified region, by reading directly from the disk. - * This is used by the "offline snapshot" when the table is disabled. - */ - public void addRegion(final Path tableDir, final HRegionInfo regionInfo) throws IOException { - // 0. Get the ManifestBuilder/RegionVisitor - RegionVisitor visitor = createRegionVisitor(desc); - - boolean isMobRegion = MobUtils.isMobRegionInfo(regionInfo); - try { - // Open the RegionFS - RegionStorage regionFs = RegionStorage.open(conf, regionInfo, false); - monitor.rethrowException(); - - // 1. dump region meta info into the snapshot directory - LOG.debug("Storing region-info for snapshot."); - Object regionData = visitor.regionOpen(regionInfo); - monitor.rethrowException(); - - // 2. iterate through all the stores in the region - LOG.debug("Creating references for hfiles"); - - // This ensures that we have an atomic view of the directory as long as we have < ls limit - // (batch size of the files in a directory) on the namenode. Otherwise, we get back the files - // in batches and may miss files being added/deleted. This could be more robust (iteratively - // checking to see if we have all the files until we are sure), but the limit is currently - // 1000 files/batch, far more than the number of store files under a single column family. - Collection familyNames = regionFs.getFamilies(); - if (familyNames != null) { - for (String familyName: familyNames) { - Object familyData = visitor.familyOpen(regionData, Bytes.toBytes(familyName)); - monitor.rethrowException(); - - Collection storeFiles = null; - if (isMobRegion) { - Path regionPath = MobUtils.getMobRegionPath(conf, regionInfo.getTable()); - Path storePath = MobUtils.getMobFamilyPath(regionPath, familyName); - storeFiles = getStoreFiles(storePath); - } else { - storeFiles = regionFs.getStoreFiles(familyName); - } - - if (storeFiles == null) { - if (LOG.isDebugEnabled()) { - LOG.debug("No files under family: " + familyName); - } - continue; - } - - // 2.1. build the snapshot reference for the store - // iterate through all the store's files and create "references". - addReferenceFiles(visitor, regionData, familyData, storeFiles, false); - - visitor.familyClose(regionData, familyData); - } - } - visitor.regionClose(regionData); - } catch (IOException e) { - // the mob directory might not be created yet, so do nothing when it is a mob region - if (!isMobRegion) { - throw e; - } - } - } - - private List getStoreFiles(Path storeDir) throws IOException { - FileStatus[] stats = FSUtils.listStatus(fs, storeDir); - if (stats == null) return null; - - ArrayList storeFiles = new ArrayList(stats.length); - for (int i = 0; i < stats.length; ++i) { - storeFiles.add(new StoreFileInfo(conf, fs, stats[i])); - } - return storeFiles; - } - - private void addReferenceFiles(RegionVisitor visitor, Object regionData, Object familyData, - Collection storeFiles, boolean isMob) throws IOException { - final String fileType = isMob ? "mob file" : "hfile"; - - if (LOG.isDebugEnabled()) { - LOG.debug(String.format("Adding snapshot references for %s %ss", storeFiles, fileType)); - } - - int i = 0; - int sz = storeFiles.size(); - for (StoreFileInfo storeFile: storeFiles) { - monitor.rethrowException(); - - LOG.debug(String.format("Adding reference for %s (%d/%d): %s", - fileType, ++i, sz, storeFile.getPath())); - - // create "reference" to this store file. - visitor.storeFile(regionData, familyData, storeFile); - } - } - - /** - * Load the information in the SnapshotManifest. Called by SnapshotManifest.open() - * - * If the format is v2 and there is no data-manifest, means that we are loading an - * in-progress snapshot. Since we support rolling-upgrades, we loook for v1 and v2 - * regions format. - */ - private void load() throws IOException { - switch (getSnapshotFormat(desc)) { - case SnapshotManifestV1.DESCRIPTOR_VERSION: { - this.htd = LegacyTableDescriptor.getTableDescriptorFromFs(fs, workingDir); - ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader"); - try { - this.regionManifests = - SnapshotManifestV1.loadRegionManifests(conf, tpool, fs, workingDir, desc); - } finally { - tpool.shutdown(); - } - break; - } - case SnapshotManifestV2.DESCRIPTOR_VERSION: { - SnapshotDataManifest dataManifest = readDataManifest(); - if (dataManifest != null) { - htd = ProtobufUtil.convertToHTableDesc(dataManifest.getTableSchema()); - regionManifests = dataManifest.getRegionManifestsList(); - } else { - // Compatibility, load the v1 regions - // This happens only when the snapshot is in-progress and the cache wants to refresh. - List v1Regions, v2Regions; - ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader"); - try { - v1Regions = SnapshotManifestV1.loadRegionManifests(conf, tpool, fs, workingDir, desc); - v2Regions = SnapshotManifestV2.loadRegionManifests(conf, tpool, fs, workingDir, desc); - } catch (InvalidProtocolBufferException e) { - throw new CorruptedSnapshotException("unable to parse region manifest " + - e.getMessage(), e); - } finally { - tpool.shutdown(); - } - if (v1Regions != null && v2Regions != null) { - regionManifests = - new ArrayList(v1Regions.size() + v2Regions.size()); - regionManifests.addAll(v1Regions); - regionManifests.addAll(v2Regions); - } else if (v1Regions != null) { - regionManifests = v1Regions; - } else /* if (v2Regions != null) */ { - regionManifests = v2Regions; - } - } - break; - } - default: - throw new CorruptedSnapshotException("Invalid Snapshot version: " + desc.getVersion(), - ProtobufUtil.createSnapshotDesc(desc)); - } - } - - /** - * Get the current snapshot working dir - */ - public Path getSnapshotDir() { - return this.workingDir; - } - - /** - * Get the SnapshotDescription - */ - public SnapshotDescription getSnapshotDescription() { - return this.desc; - } - - /** - * Get the table descriptor from the Snapshot - */ - public HTableDescriptor getTableDescriptor() { - return this.htd; - } - - /** - * Get all the Region Manifest from the snapshot - */ - public List getRegionManifests() { - return this.regionManifests; - } - - /** - * Get all the Region Manifest from the snapshot. - * This is an helper to get a map with the region encoded name - */ - public Map getRegionManifestsMap() { - if (regionManifests == null || regionManifests.size() == 0) return null; - - HashMap regionsMap = - new HashMap(regionManifests.size()); - for (SnapshotRegionManifest manifest: regionManifests) { - String regionName = getRegionNameFromManifest(manifest); - regionsMap.put(regionName, manifest); - } - return regionsMap; - } - - public void consolidate() throws IOException { - if (getSnapshotFormat(desc) == SnapshotManifestV1.DESCRIPTOR_VERSION) { - Path rootDir = FSUtils.getRootDir(conf); - LOG.info("Using old Snapshot Format"); - // write a copy of descriptor to the snapshot directory - LegacyTableDescriptor.createTableDescriptor(fs, workingDir, htd, false); - } else { - LOG.debug("Convert to Single Snapshot Manifest"); - convertToV2SingleManifest(); - } - } - - /* - * In case of rolling-upgrade, we try to read all the formats and build - * the snapshot with the latest format. - */ - private void convertToV2SingleManifest() throws IOException { - // Try to load v1 and v2 regions - List v1Regions, v2Regions; - ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader"); - try { - v1Regions = SnapshotManifestV1.loadRegionManifests(conf, tpool, fs, workingDir, desc); - v2Regions = SnapshotManifestV2.loadRegionManifests(conf, tpool, fs, workingDir, desc); - } finally { - tpool.shutdown(); - } - - SnapshotDataManifest.Builder dataManifestBuilder = SnapshotDataManifest.newBuilder(); - dataManifestBuilder.setTableSchema(ProtobufUtil.convertToTableSchema(htd)); - - if (v1Regions != null && v1Regions.size() > 0) { - dataManifestBuilder.addAllRegionManifests(v1Regions); - } - if (v2Regions != null && v2Regions.size() > 0) { - dataManifestBuilder.addAllRegionManifests(v2Regions); - } - - // Write the v2 Data Manifest. - // Once the data-manifest is written, the snapshot can be considered complete. - // Currently snapshots are written in a "temporary" directory and later - // moved to the "complated" snapshot directory. - SnapshotDataManifest dataManifest = dataManifestBuilder.build(); - writeDataManifest(dataManifest); - this.regionManifests = dataManifest.getRegionManifestsList(); - - // Remove the region manifests. Everything is now in the data-manifest. - // The delete operation is "relaxed", unless we get an exception we keep going. - // The extra files in the snapshot directory will not give any problem, - // since they have the same content as the data manifest, and even by re-reading - // them we will get the same information. - if (v1Regions != null && v1Regions.size() > 0) { - for (SnapshotRegionManifest regionManifest: v1Regions) { - SnapshotManifestV1.deleteRegionManifest(fs, workingDir, regionManifest); - } - } - if (v2Regions != null && v2Regions.size() > 0) { - for (SnapshotRegionManifest regionManifest: v2Regions) { - SnapshotManifestV2.deleteRegionManifest(fs, workingDir, regionManifest); - } - } - } - - /* - * Write the SnapshotDataManifest file - */ - private void writeDataManifest(final SnapshotDataManifest manifest) - throws IOException { - FSDataOutputStream stream = fs.create(new Path(workingDir, DATA_MANIFEST_NAME)); - try { - manifest.writeTo(stream); - } finally { - stream.close(); - } - } - - /* - * Read the SnapshotDataManifest file - */ - private SnapshotDataManifest readDataManifest() throws IOException { - FSDataInputStream in = null; - try { - in = fs.open(new Path(workingDir, DATA_MANIFEST_NAME)); - CodedInputStream cin = CodedInputStream.newInstance(in); - cin.setSizeLimit(manifestSizeLimit); - return SnapshotDataManifest.parseFrom(cin); - } catch (FileNotFoundException e) { - return null; - } catch (InvalidProtocolBufferException e) { - throw new CorruptedSnapshotException("unable to parse data manifest " + e.getMessage(), e); - } finally { - if (in != null) in.close(); - } - } - - private ThreadPoolExecutor createExecutor(final String name) { - return createExecutor(conf, name); - } - - public static ThreadPoolExecutor createExecutor(final Configuration conf, final String name) { - int maxThreads = conf.getInt("hbase.snapshot.thread.pool.max", 8); - return Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS, - Threads.getNamedThreadFactory(name)); - } - - /** - * Extract the region encoded name from the region manifest - */ - static String getRegionNameFromManifest(final SnapshotRegionManifest manifest) { - byte[] regionName = HRegionInfo.createRegionName( - ProtobufUtil.toTableName(manifest.getRegionInfo().getTableName()), - manifest.getRegionInfo().getStartKey().toByteArray(), - manifest.getRegionInfo().getRegionId(), true); - return HRegionInfo.encodeRegionName(regionName); - } - - /* - * Return the snapshot format - */ - private static int getSnapshotFormat(final SnapshotDescription desc) { - return desc.hasVersion() ? desc.getVersion() : SnapshotManifestV1.DESCRIPTOR_VERSION; - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java deleted file mode 100644 index 3ca48fe..0000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV1.java +++ /dev/null @@ -1,209 +0,0 @@ -/** - * 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.InterruptedIOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.Executor; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorCompletionService; - -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.fs.RegionStorage; -import org.apache.hadoop.hbase.fs.StorageIdentifier; -import org.apache.hadoop.hbase.fs.legacy.LegacyPathIdentifier; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; -import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; -import org.apache.hadoop.hbase.regionserver.StoreFileInfo; -import org.apache.hadoop.hbase.util.ByteStringer; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSUtils; - -/** - * DO NOT USE DIRECTLY. USE {@link SnapshotManifest}. - * - * Snapshot v1 layout format - * - Each region in the table is represented by a directory with the .hregioninfo file - * /snapshotName/regionName/.hregioninfo - * - Each file present in the table is represented by an empty file - * /snapshotName/regionName/familyName/fileName - */ -@InterfaceAudience.Private -public final class SnapshotManifestV1 { - private static final Log LOG = LogFactory.getLog(SnapshotManifestV1.class); - - public static final int DESCRIPTOR_VERSION = 0; - - private SnapshotManifestV1() { - } - - // TODO update for RegionStorage - static class ManifestBuilder implements SnapshotManifest.RegionVisitor { - private final Configuration conf; - private final StorageIdentifier snapshotDir; - private final FileSystem fs; - - public ManifestBuilder(final Configuration conf, final FileSystem fs, final Path snapshotDir) { - this.snapshotDir = new LegacyPathIdentifier(snapshotDir); - this.conf = conf; - this.fs = fs; - } - - public RegionStorage regionOpen(final HRegionInfo regionInfo) throws IOException { - RegionStorage snapshotRegionFs = RegionStorage.open(conf, fs, - snapshotDir, regionInfo, true); - return snapshotRegionFs; - } - - public void regionClose(final RegionStorage region) { - } - - public Path familyOpen(final RegionStorage snapshotRegionFs, final byte[] familyName) { - Path familyDir = ((LegacyPathIdentifier)snapshotRegionFs.getStoreContainer(Bytes.toString(familyName))).path; - return familyDir; - } - - public void familyClose(final RegionStorage region, final Path family) { - } - - public void storeFile(final RegionStorage region, final Path familyDir, - final StoreFileInfo storeFile) throws IOException { - Path referenceFile = new Path(familyDir, storeFile.getPath().getName()); - boolean success = true; - if (storeFile.isReference()) { - // write the Reference object to the snapshot - storeFile.getReference().write(fs, 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.createNewFile(referenceFile); - } - if (!success) { - throw new IOException("Failed to create reference file:" + referenceFile); - } - } - } - - static List loadRegionManifests(final Configuration conf, - final Executor executor,final FileSystem fs, final Path snapshotDir, - final SnapshotDescription desc) throws IOException { - FileStatus[] regions = FSUtils.listStatus(fs, snapshotDir, new FSUtils.RegionDirFilter(fs)); - if (regions == null) { - LOG.debug("No regions under directory:" + snapshotDir); - return null; - } - - final ExecutorCompletionService completionService = - new ExecutorCompletionService(executor); - for (final FileStatus region: regions) { - completionService.submit(new Callable() { - @Override - public SnapshotRegionManifest call() throws IOException { - final RegionStorage rs = RegionStorage.open(conf, new LegacyPathIdentifier(region.getPath()), true); - return buildManifestFromDisk(conf, fs, snapshotDir, rs); - } - }); - } - - ArrayList regionsManifest = - new ArrayList(regions.length); - try { - for (int i = 0; i < regions.length; ++i) { - regionsManifest.add(completionService.take().get()); - } - } catch (InterruptedException e) { - throw new InterruptedIOException(e.getMessage()); - } catch (ExecutionException e) { - IOException ex = new IOException(); - ex.initCause(e.getCause()); - throw ex; - } - return regionsManifest; - } - - static void deleteRegionManifest(final FileSystem fs, final Path snapshotDir, - final SnapshotRegionManifest manifest) throws IOException { - String regionName = SnapshotManifest.getRegionNameFromManifest(manifest); - fs.delete(new Path(snapshotDir, regionName), true); - } - - static SnapshotRegionManifest buildManifestFromDisk(final Configuration conf, - final FileSystem fs, final Path tableDir, final RegionStorage regionFs) throws IOException { - SnapshotRegionManifest.Builder manifest = SnapshotRegionManifest.newBuilder(); - - // 1. dump region meta info into the snapshot directory - LOG.debug("Storing region-info for snapshot."); - manifest.setRegionInfo(HRegionInfo.convert(regionFs.getRegionInfo())); - - // 2. iterate through all the stores in the region - LOG.debug("Creating references for hfiles"); - - // This ensures that we have an atomic view of the directory as long as we have < ls limit - // (batch size of the files in a directory) on the namenode. Otherwise, we get back the files in - // batches and may miss files being added/deleted. This could be more robust (iteratively - // checking to see if we have all the files until we are sure), but the limit is currently 1000 - // files/batch, far more than the number of store files under a single column family. - Collection familyNames = regionFs.getFamilies(); - if (familyNames != null) { - for (String familyName: familyNames) { - Collection storeFiles = regionFs.getStoreFiles(familyName, false); - if (storeFiles == null) { - LOG.debug("No files under family: " + familyName); - continue; - } - - // 2.1. build the snapshot reference for the store - SnapshotRegionManifest.FamilyFiles.Builder family = - SnapshotRegionManifest.FamilyFiles.newBuilder(); - family.setFamilyName(ByteStringer.wrap(Bytes.toBytes(familyName))); - - if (LOG.isDebugEnabled()) { - LOG.debug("Adding snapshot references for " + storeFiles + " hfiles"); - } - - // 2.2. iterate through all the store's files and create "references". - int i = 0; - int sz = storeFiles.size(); - for (StoreFileInfo storeFile: storeFiles) { - // create "reference" to this store file. - LOG.debug("Adding reference for file ("+ (++i) +"/" + sz + "): " + storeFile.getPath()); - SnapshotRegionManifest.StoreFile.Builder sfManifest = - SnapshotRegionManifest.StoreFile.newBuilder(); - sfManifest.setName(storeFile.getPath().getName()); - family.addStoreFiles(sfManifest.build()); - } - manifest.addFamilyFiles(family.build()); - } - } - return manifest.build(); - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java deleted file mode 100644 index df5dcd3..0000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifestV2.java +++ /dev/null @@ -1,187 +0,0 @@ -/** - * 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 java.io.IOException; -import java.io.InterruptedIOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.Executor; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorCompletionService; - -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; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathFilter; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; -import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; -import org.apache.hadoop.hbase.regionserver.StoreFileInfo; -import org.apache.hadoop.hbase.util.ByteStringer; -import org.apache.hadoop.hbase.util.FSUtils; - -/** - * DO NOT USE DIRECTLY. USE {@link SnapshotManifest}. - * - * Snapshot v2 layout format - * - Single Manifest file containing all the information of regions - * - In the online-snapshot case each region will write a "region manifest" - * /snapshotName/manifest.regionName - */ -@InterfaceAudience.Private -public final class SnapshotManifestV2 { - private static final Log LOG = LogFactory.getLog(SnapshotManifestV2.class); - - public static final int DESCRIPTOR_VERSION = 2; - - public static final String SNAPSHOT_MANIFEST_PREFIX = "region-manifest."; - - private SnapshotManifestV2() {} - - static class ManifestBuilder implements SnapshotManifest.RegionVisitor< - SnapshotRegionManifest.Builder, SnapshotRegionManifest.FamilyFiles.Builder> { - private final Configuration conf; - private final Path snapshotDir; - private final FileSystem fs; - - public ManifestBuilder(final Configuration conf, final FileSystem fs, final Path snapshotDir) { - this.snapshotDir = snapshotDir; - this.conf = conf; - this.fs = fs; - } - - public SnapshotRegionManifest.Builder regionOpen(final HRegionInfo regionInfo) { - SnapshotRegionManifest.Builder manifest = SnapshotRegionManifest.newBuilder(); - manifest.setRegionInfo(HRegionInfo.convert(regionInfo)); - return manifest; - } - - public void regionClose(final SnapshotRegionManifest.Builder region) throws IOException { - // we should ensure the snapshot dir exist, maybe it has been deleted by master - // see HBASE-16464 - if (fs.exists(snapshotDir)) { - SnapshotRegionManifest manifest = region.build(); - FSDataOutputStream stream = fs.create(getRegionManifestPath(snapshotDir, manifest)); - try { - manifest.writeTo(stream); - } finally { - stream.close(); - } - } else { - LOG.warn("can't write manifest without parent dir, maybe it has been deleted by master?"); - } - } - - public SnapshotRegionManifest.FamilyFiles.Builder familyOpen( - final SnapshotRegionManifest.Builder region, final byte[] familyName) { - SnapshotRegionManifest.FamilyFiles.Builder family = - SnapshotRegionManifest.FamilyFiles.newBuilder(); - family.setFamilyName(ByteStringer.wrap(familyName)); - return family; - } - - public void familyClose(final SnapshotRegionManifest.Builder region, - final SnapshotRegionManifest.FamilyFiles.Builder family) { - region.addFamilyFiles(family.build()); - } - - public void storeFile(final SnapshotRegionManifest.Builder region, - final SnapshotRegionManifest.FamilyFiles.Builder family, final StoreFileInfo storeFile) - throws IOException { - SnapshotRegionManifest.StoreFile.Builder sfManifest = - SnapshotRegionManifest.StoreFile.newBuilder(); - sfManifest.setName(storeFile.getPath().getName()); - if (storeFile.isReference()) { - sfManifest.setReference(storeFile.getReference().convert()); - } - sfManifest.setFileSize(storeFile.getReferencedFileStatus(fs).getLen()); - family.addStoreFiles(sfManifest.build()); - } - } - - static List loadRegionManifests(final Configuration conf, - final Executor executor,final FileSystem fs, final Path snapshotDir, - final SnapshotDescription desc) throws IOException { - FileStatus[] manifestFiles = FSUtils.listStatus(fs, snapshotDir, new PathFilter() { - @Override - public boolean accept(Path path) { - return path.getName().startsWith(SNAPSHOT_MANIFEST_PREFIX); - } - }); - - if (manifestFiles == null || manifestFiles.length == 0) return null; - - final ExecutorCompletionService completionService = - new ExecutorCompletionService(executor); - for (final FileStatus st: manifestFiles) { - completionService.submit(new Callable() { - @Override - public SnapshotRegionManifest call() throws IOException { - FSDataInputStream stream = fs.open(st.getPath()); - try { - return SnapshotRegionManifest.parseFrom(stream); - } finally { - stream.close(); - } - } - }); - } - - ArrayList regionsManifest = - new ArrayList(manifestFiles.length); - try { - for (int i = 0; i < manifestFiles.length; ++i) { - regionsManifest.add(completionService.take().get()); - } - } catch (InterruptedException e) { - throw new InterruptedIOException(e.getMessage()); - } catch (ExecutionException e) { - Throwable t = e.getCause(); - - if(t instanceof InvalidProtocolBufferException) { - throw (InvalidProtocolBufferException)t; - } else { - IOException ex = new IOException("ExecutionException"); - ex.initCause(e.getCause()); - throw ex; - } - } - return regionsManifest; - } - - static void deleteRegionManifest(final FileSystem fs, final Path snapshotDir, - final SnapshotRegionManifest manifest) throws IOException { - fs.delete(getRegionManifestPath(snapshotDir, manifest), true); - } - - private static Path getRegionManifestPath(final Path snapshotDir, - final SnapshotRegionManifest manifest) { - String regionName = SnapshotManifest.getRegionNameFromManifest(manifest); - return new Path(snapshotDir, SNAPSHOT_MANIFEST_PREFIX + regionName); - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java index c655dd8..5068d2e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotReferenceUtil.java @@ -38,10 +38,16 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.fs.MasterStorage; +import org.apache.hadoop.hbase.fs.StorageContext; +import org.apache.hadoop.hbase.fs.StorageIdentifier; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifest; import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; +import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos; import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.util.HFileArchiveUtil; @@ -53,207 +59,110 @@ import org.apache.hadoop.hbase.util.HFileArchiveUtil; public final class SnapshotReferenceUtil { private static final Log LOG = LogFactory.getLog(SnapshotReferenceUtil.class); - public interface StoreFileVisitor { - void storeFile(final HRegionInfo regionInfo, final String familyName, - final SnapshotRegionManifest.StoreFile storeFile) throws IOException; - } - - public interface SnapshotVisitor extends StoreFileVisitor { - } - private SnapshotReferenceUtil() { // private constructor for utility class } /** - * Iterate over the snapshot store files - * - * @param conf The current {@link Configuration} instance. - * @param fs {@link FileSystem} - * @param snapshotDir {@link Path} to the Snapshot directory - * @param visitor callback object to get the referenced files - * @throws IOException if an error occurred while scanning the directory - */ - public static void visitReferencedFiles(final Configuration conf, final FileSystem fs, - final Path snapshotDir, final SnapshotVisitor visitor) - throws IOException { - SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir); - visitReferencedFiles(conf, fs, snapshotDir, desc, visitor); - } - - /** - * Iterate over the snapshot store files, restored.edits and logs - * - * @param conf The current {@link Configuration} instance. - * @param fs {@link FileSystem} - * @param snapshotDir {@link Path} to the Snapshot directory - * @param desc the {@link SnapshotDescription} of the snapshot to verify - * @param visitor callback object to get the referenced files - * @throws IOException if an error occurred while scanning the directory - */ - public static void visitReferencedFiles(final Configuration conf, final FileSystem fs, - final Path snapshotDir, final SnapshotDescription desc, final SnapshotVisitor visitor) - throws IOException { - visitTableStoreFiles(conf, fs, snapshotDir, desc, visitor); - } - - /**© - * Iterate over the snapshot store files - * - * @param conf The current {@link Configuration} instance. - * @param fs {@link FileSystem} - * @param snapshotDir {@link Path} to the Snapshot directory - * @param desc the {@link SnapshotDescription} of the snapshot to verify - * @param visitor callback object to get the store files - * @throws IOException if an error occurred while scanning the directory - */ - static void visitTableStoreFiles(final Configuration conf, final FileSystem fs, - final Path snapshotDir, final SnapshotDescription desc, final StoreFileVisitor visitor) - throws IOException { - SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, desc); - List regionManifests = manifest.getRegionManifests(); - if (regionManifests == null || regionManifests.size() == 0) { - LOG.debug("No manifest files present: " + snapshotDir); - return; - } - - for (SnapshotRegionManifest regionManifest: regionManifests) { - visitRegionStoreFiles(regionManifest, visitor); - } - } - - /** - * Iterate over the snapshot store files in the specified region - * - * @param manifest snapshot manifest to inspect - * @param visitor callback object to get the store files - * @throws IOException if an error occurred while scanning the directory - */ - static void visitRegionStoreFiles(final SnapshotRegionManifest manifest, - final StoreFileVisitor visitor) throws IOException { - HRegionInfo regionInfo = HRegionInfo.convert(manifest.getRegionInfo()); - for (SnapshotRegionManifest.FamilyFiles familyFiles: manifest.getFamilyFilesList()) { - String familyName = familyFiles.getFamilyName().toStringUtf8(); - for (SnapshotRegionManifest.StoreFile storeFile: familyFiles.getStoreFilesList()) { - visitor.storeFile(regionInfo, familyName, storeFile); - } - } - } - - /** - * Verify the validity of the snapshot - * - * @param conf The current {@link Configuration} instance. - * @param fs {@link FileSystem} - * @param snapshotDir {@link Path} to the Snapshot directory of the snapshot to verify - * @param snapshotDesc the {@link SnapshotDescription} of the snapshot to verify - * @throws CorruptedSnapshotException if the snapshot is corrupted - * @throws IOException if an error occurred while scanning the directory - */ - public static void verifySnapshot(final Configuration conf, final FileSystem fs, - final Path snapshotDir, final SnapshotDescription snapshotDesc) throws IOException { - SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc); - verifySnapshot(conf, fs, manifest); - } - - /** * Verify the validity of the snapshot * - * @param conf The current {@link Configuration} instance. - * @param fs {@link FileSystem} - * @param manifest snapshot manifest to inspect + * @param masterStorage {@link MasterStorage} for a snapshot + * @param snapshot the {@link SnapshotDescription} of the snapshot to verify + * @param ctx {@link StorageContext} of a snapshot * @throws CorruptedSnapshotException if the snapshot is corrupted * @throws IOException if an error occurred while scanning the directory */ - public static void verifySnapshot(final Configuration conf, final FileSystem fs, - final SnapshotManifest manifest) throws IOException { - final SnapshotDescription snapshotDesc = manifest.getSnapshotDescription(); - final Path snapshotDir = manifest.getSnapshotDir(); - concurrentVisitReferencedFiles(conf, fs, manifest, "VerifySnapshot", new StoreFileVisitor() { - @Override - public void storeFile(final HRegionInfo regionInfo, final String family, - final SnapshotRegionManifest.StoreFile storeFile) throws IOException { - verifyStoreFile(conf, fs, snapshotDir, snapshotDesc, regionInfo, family, storeFile); - } + public static void verifySnapshot(final MasterStorage masterStorage, + final SnapshotDescription snapshot, StorageContext ctx) throws IOException { + masterStorage.visitSnapshotStoreFiles(snapshot, ctx, + new MasterStorage.SnapshotStoreFileVisitor() { + @Override + public void visitSnapshotStoreFile(SnapshotDescription snapshot, StorageContext ctx, + HRegionInfo hri, String familyName, SnapshotRegionManifest.StoreFile storeFile) + throws IOException { + verifyStoreFile(masterStorage, snapshot, ctx, hri, familyName, storeFile); + } }); } - public static void concurrentVisitReferencedFiles(final Configuration conf, final FileSystem fs, - final SnapshotManifest manifest, final String desc, final StoreFileVisitor visitor) - throws IOException { - - final Path snapshotDir = manifest.getSnapshotDir(); - List regionManifests = manifest.getRegionManifests(); - if (regionManifests == null || regionManifests.size() == 0) { - LOG.debug("No manifest files present: " + snapshotDir); - return; - } + // TODO modify master storage to use ExecutorService and run concurrent queries to HDFS NN and + // remove the commented out code below. - ExecutorService exec = SnapshotManifest.createExecutor(conf, desc); - - try { - concurrentVisitReferencedFiles(conf, fs, manifest, exec, visitor); - } finally { - exec.shutdown(); - } - } - - public static void concurrentVisitReferencedFiles(final Configuration conf, final FileSystem fs, - final SnapshotManifest manifest, final ExecutorService exec, final StoreFileVisitor visitor) - throws IOException { - final SnapshotDescription snapshotDesc = manifest.getSnapshotDescription(); - final Path snapshotDir = manifest.getSnapshotDir(); - - List regionManifests = manifest.getRegionManifests(); - if (regionManifests == null || regionManifests.size() == 0) { - LOG.debug("No manifest files present: " + snapshotDir); - return; - } - - final ExecutorCompletionService completionService = - new ExecutorCompletionService(exec); - - for (final SnapshotRegionManifest regionManifest : regionManifests) { - completionService.submit(new Callable() { - @Override public Void call() throws IOException { - visitRegionStoreFiles(regionManifest, visitor); - return null; - } - }); - } - try { - for (int i = 0; i < regionManifests.size(); ++i) { - completionService.take().get(); - } - } catch (InterruptedException e) { - throw new InterruptedIOException(e.getMessage()); - } catch (ExecutionException e) { - if (e.getCause() instanceof CorruptedSnapshotException) { - throw new CorruptedSnapshotException(e.getCause().getMessage(), - ProtobufUtil.createSnapshotDesc(snapshotDesc)); - } else { - IOException ex = new IOException(); - ex.initCause(e.getCause()); - throw ex; - } - } - } +// public static void concurrentVisitReferencedFiles(final Configuration conf, final FileSystem fs, +// final SnapshotManifest manifest, final String desc, final StoreFileVisitor visitor) +// throws IOException { +// +// final Path snapshotDir = manifest.getSnapshotDir(); +// List regionManifests = manifest.getRegionManifests(); +// if (regionManifests == null || regionManifests.size() == 0) { +// LOG.debug("No manifest files present: " + snapshotDir); +// return; +// } +// +// ExecutorService exec = SnapshotManifest.createExecutor(conf, desc); +// +// try { +// concurrentVisitReferencedFiles(conf, fs, manifest, exec, visitor); +// } finally { +// exec.shutdown(); +// } +// } +// +// public static void concurrentVisitReferencedFiles(final Configuration conf, final FileSystem fs, +// final SnapshotManifest manifest, final ExecutorService exec, final StoreFileVisitor visitor) +// throws IOException { +// final SnapshotDescription snapshotDesc = manifest.getSnapshotDescription(); +// final Path snapshotDir = manifest.getSnapshotDir(); +// +// List regionManifests = manifest.getRegionManifests(); +// if (regionManifests == null || regionManifests.size() == 0) { +// LOG.debug("No manifest files present: " + snapshotDir); +// return; +// } +// +// final ExecutorCompletionService completionService = +// new ExecutorCompletionService(exec); +// +// for (final SnapshotRegionManifest regionManifest : regionManifests) { +// completionService.submit(new Callable() { +// @Override public Void call() throws IOException { +// visitRegionStoreFiles(regionManifest, visitor); +// return null; +// } +// }); +// } +// try { +// for (int i = 0; i < regionManifests.size(); ++i) { +// completionService.take().get(); +// } +// } catch (InterruptedException e) { +// throw new InterruptedIOException(e.getMessage()); +// } catch (ExecutionException e) { +// if (e.getCause() instanceof CorruptedSnapshotException) { +// throw new CorruptedSnapshotException(e.getCause().getMessage(), +// ProtobufUtil.createSnapshotDesc(snapshotDesc)); +// } else { +// IOException ex = new IOException(); +// ex.initCause(e.getCause()); +// throw ex; +// } +// } +// } /** * Verify the validity of the snapshot store file * - * @param conf The current {@link Configuration} instance. - * @param fs {@link FileSystem} - * @param snapshotDir {@link Path} to the Snapshot directory of the snapshot to verify + * @param masterStorage (@link MasterStorage} for the snapshot * @param snapshot the {@link SnapshotDescription} of the snapshot to verify + * @param ctx {@link StorageContext} of a snapshot * @param regionInfo {@link HRegionInfo} of the region that contains the store file * @param family family that contains the store file * @param storeFile the store file to verify * @throws CorruptedSnapshotException if the snapshot is corrupted * @throws IOException if an error occurred while scanning the directory */ - private static void verifyStoreFile(final Configuration conf, final FileSystem fs, - final Path snapshotDir, final SnapshotDescription snapshot, final HRegionInfo regionInfo, + private static void verifyStoreFile(final MasterStorage masterStorage, + final SnapshotDescription snapshot, StorageContext ctx, final HRegionInfo regionInfo, final String family, final SnapshotRegionManifest.StoreFile storeFile) throws IOException { TableName table = TableName.valueOf(snapshot.getTable()); String fileName = storeFile.getName(); @@ -265,7 +174,8 @@ public final class SnapshotReferenceUtil { refPath = StoreFileInfo.getReferredToFile(refPath); String refRegion = refPath.getParent().getParent().getName(); refPath = HFileLink.createPath(table, refRegion, family, refPath.getName()); - if (!HFileLink.buildFromHFileLinkPattern(conf, refPath).exists(fs)) { + if (!HFileLink.buildFromHFileLinkPattern(masterStorage.getConfiguration(), refPath) + .exists(masterStorage.getFileSystem())) { throw new CorruptedSnapshotException( "Missing parent hfile for: " + fileName + " path=" + refPath, ProtobufUtil.createSnapshotDesc(snapshot)); @@ -292,14 +202,15 @@ public final class SnapshotReferenceUtil { HFileLink link = null; if (MobUtils.isMobRegionInfo(regionInfo)) { // for mob region - link = HFileLink.buildFromHFileLinkPattern(MobUtils.getQualifiedMobRootDir(conf), - HFileArchiveUtil.getArchivePath(conf), linkPath); + link = HFileLink.buildFromHFileLinkPattern(MobUtils.getQualifiedMobRootDir( + masterStorage.getConfiguration()), HFileArchiveUtil.getArchivePath( + masterStorage.getConfiguration()), linkPath); } else { // not mob region - link = HFileLink.buildFromHFileLinkPattern(conf, linkPath); + link = HFileLink.buildFromHFileLinkPattern(masterStorage.getConfiguration(), linkPath); } try { - FileStatus fstat = link.getFileStatus(fs); + FileStatus fstat = link.getFileStatus(masterStorage.getFileSystem()); if (storeFile.hasFileSize() && storeFile.getFileSize() != fstat.getLen()) { String msg = "hfile: " + fileName + " size does not match with the expected one. " + " found=" + fstat.getLen() + " expected=" + storeFile.getFileSize(); @@ -320,43 +231,47 @@ public final class SnapshotReferenceUtil { /** * Returns the store file names in the snapshot. * - * @param conf The current {@link Configuration} instance. - * @param fs {@link FileSystem} - * @param snapshotDir {@link Path} to the Snapshot directory + * @param masterStorage {@link MasterStorage} for a snapshot. + * @param snapshotName Name of the snapshot * @throws IOException if an error occurred while scanning the directory * @return the names of hfiles in the specified snaphot */ - public static Set getHFileNames(final Configuration conf, final FileSystem fs, - final Path snapshotDir) throws IOException { - SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir); - return getHFileNames(conf, fs, snapshotDir, desc); + public static Set getHFileNames(final MasterStorage + masterStorage, final String snapshotName) throws IOException { + return getHFileNames(masterStorage, snapshotName, StorageContext.DATA); + } + + public static Set getHFileNames(final MasterStorage + masterStorage, final String snapshotName, StorageContext ctx) throws IOException { + SnapshotDescription desc = masterStorage.getSnapshot(snapshotName, ctx); + return getHFileNames(masterStorage, desc, ctx); } /** * Returns the store file names in the snapshot. * - * @param conf The current {@link Configuration} instance. - * @param fs {@link FileSystem} - * @param snapshotDir {@link Path} to the Snapshot directory - * @param snapshotDesc the {@link SnapshotDescription} of the snapshot to inspect + * @param masterStorage {@link MasterStorage} for a snapshot + * @param snapshot the {@link SnapshotDescription} of the snapshot to inspect + * @param ctx {@link StorageContext} for a snapshot * @throws IOException if an error occurred while scanning the directory * @return the names of hfiles in the specified snaphot */ - private static Set getHFileNames(final Configuration conf, final FileSystem fs, - final Path snapshotDir, final SnapshotDescription snapshotDesc) - throws IOException { + private static Set getHFileNames(final MasterStorage + masterStorage,final SnapshotDescription snapshot, StorageContext ctx) throws IOException { final Set names = new HashSet(); - visitTableStoreFiles(conf, fs, snapshotDir, snapshotDesc, new StoreFileVisitor() { - @Override - public void storeFile(final HRegionInfo regionInfo, final String family, - final SnapshotRegionManifest.StoreFile storeFile) throws IOException { - String hfile = storeFile.getName(); - if (HFileLink.isHFileLink(hfile)) { - names.add(HFileLink.getReferencedHFileName(hfile)); - } else { - names.add(hfile); - } - } + masterStorage.visitSnapshotStoreFiles(snapshot, ctx, + new MasterStorage.SnapshotStoreFileVisitor() { + @Override + public void visitSnapshotStoreFile(SnapshotDescription snapshot, StorageContext ctx, + HRegionInfo hri, String familyName, SnapshotRegionManifest.StoreFile storeFile) + throws IOException { + String hfile = storeFile.getName(); + if (HFileLink.isHFileLink(hfile)) { + names.add(HFileLink.getReferencedHFileName(hfile)); + } else { + names.add(hfile); + } + } }); return names; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotRestoreMetaChanges.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotRestoreMetaChanges.java new file mode 100644 index 0000000..22899d5 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotRestoreMetaChanges.java @@ -0,0 +1,157 @@ +package org.apache.hadoop.hbase.snapshot; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.util.Pair; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +/** + * Describe the set of operations needed to update hbase:meta after restore. + */ +public class SnapshotRestoreMetaChanges { + private static final Log LOG = LogFactory.getLog(SnapshotRestoreMetaChanges.class); + + private final Map > parentsMap; + private final HTableDescriptor htd; + + private List regionsToRestore = null; + private List regionsToRemove = null; + private List regionsToAdd = null; + + public SnapshotRestoreMetaChanges(HTableDescriptor htd, Map > parentsMap) { + this.parentsMap = parentsMap; + this.htd = htd; + } + + public HTableDescriptor getTableDescriptor() { + return htd; + } + + /** + * Returns the map of parent-children_pair. + * @return the map + */ + public Map> getParentToChildrenPairMap() { + return this.parentsMap; + } + + /** + * @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; + } + + public void setNewRegions(final HRegionInfo[] hris) { + if (hris != null) { + regionsToAdd = Arrays.asList(hris); + } else { + regionsToAdd = null; + } + } + + public void addRegionToRemove(final HRegionInfo hri) { + if (regionsToRemove == null) { + regionsToRemove = new LinkedList(); + } + regionsToRemove.add(hri); + } + + public 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())); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java index fa61be0..219e4af 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java @@ -28,24 +28,19 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; -import org.apache.hadoop.hbase.snapshot.SnapshotManifestV1; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSUtils; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -53,8 +48,6 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.google.common.collect.Lists; - /** * Test create/using/deleting snapshots from the client *

    diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestExportSnapshot.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestExportSnapshot.java new file mode 100644 index 0000000..ef3d13c --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestExportSnapshot.java @@ -0,0 +1,384 @@ +/** + * 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.fs.legacy.snapshot; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +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.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.fs.MasterStorage; +import org.apache.hadoop.hbase.fs.StorageContext; +import org.apache.hadoop.hbase.fs.StorageIdentifier; +import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; +import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos; +import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestRule; + +/** + * Test Export Snapshot Tool + */ +@Category({VerySlowMapReduceTests.class, LargeTests.class}) +public class TestExportSnapshot { + @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). + withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + private static final Log LOG = LogFactory.getLog(TestExportSnapshot.class); + + protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + protected final static byte[] FAMILY = Bytes.toBytes("cf"); + + protected TableName tableName; + private byte[] emptySnapshotName; + private byte[] snapshotName; + private int tableNumFiles; + private Admin admin; + + public static void setUpBaseConf(Configuration conf) { + conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true); + conf.setInt("hbase.regionserver.msginterval", 100); + conf.setInt("hbase.client.pause", 250); + conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6); + conf.setBoolean("hbase.master.enabletable.roundrobin", true); + conf.setInt("mapreduce.map.maxattempts", 10); + } + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + setUpBaseConf(TEST_UTIL.getConfiguration()); + TEST_UTIL.startMiniCluster(3); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + /** + * Create a table and take a snapshot of the table used by the export test. + */ + @Before + public void setUp() throws Exception { + this.admin = TEST_UTIL.getHBaseAdmin(); + + long tid = System.currentTimeMillis(); + tableName = TableName.valueOf("testtb-" + tid); + snapshotName = Bytes.toBytes("snaptb0-" + tid); + emptySnapshotName = Bytes.toBytes("emptySnaptb0-" + tid); + + // create Table + createTable(); + + // Take an empty snapshot + admin.snapshot(emptySnapshotName, tableName); + + // Add some rows + SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 50, FAMILY); + tableNumFiles = admin.getTableRegions(tableName).size(); + + // take a snapshot + admin.snapshot(snapshotName, tableName); + } + + protected void createTable() throws Exception { + SnapshotTestingUtils.createPreSplitTable(TEST_UTIL, tableName, 2, FAMILY); + } + + protected interface RegionPredicate { + boolean evaluate(final HRegionInfo regionInfo); + } + + protected RegionPredicate getBypassRegionPredicate() { + return null; + } + + @After + public void tearDown() throws Exception { + TEST_UTIL.deleteTable(tableName); + SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin()); + SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL); + } + + /** + * Verify if exported snapshot and copied files matches the original one. + */ + @Test + public void testExportFileSystemState() throws Exception { + testExportFileSystemState(tableName, snapshotName, snapshotName, tableNumFiles); + } + + @Test + public void testExportFileSystemStateWithSkipTmp() throws Exception { + TEST_UTIL.getConfiguration().setBoolean(ExportSnapshot.CONF_SKIP_TMP, true); + try { + testExportFileSystemState(tableName, snapshotName, snapshotName, tableNumFiles); + } finally { + TEST_UTIL.getConfiguration().setBoolean(ExportSnapshot.CONF_SKIP_TMP, false); + } + } + + @Test + public void testEmptyExportFileSystemState() throws Exception { + testExportFileSystemState(tableName, emptySnapshotName, emptySnapshotName, 0); + } + + @Test + public void testConsecutiveExports() throws Exception { + Path copyDir = getLocalDestinationDir(); + testExportFileSystemState(tableName, snapshotName, snapshotName, tableNumFiles, copyDir, false); + testExportFileSystemState(tableName, snapshotName, snapshotName, tableNumFiles, copyDir, true); + removeExportDir(copyDir); + } + + @Test + public void testExportWithTargetName() throws Exception { + final byte[] targetName = Bytes.toBytes("testExportWithTargetName"); + testExportFileSystemState(tableName, snapshotName, targetName, tableNumFiles); + } + + private void testExportFileSystemState(final TableName tableName, final byte[] snapshotName, + final byte[] targetName, int filesExpected) throws Exception { + testExportFileSystemState(tableName, snapshotName, targetName, + filesExpected, getHdfsDestinationDir(), false); + } + + protected void testExportFileSystemState(final TableName tableName, + final byte[] snapshotName, final byte[] targetName, int filesExpected, + Path copyDir, boolean overwrite) throws Exception { + testExportFileSystemState(TEST_UTIL.getConfiguration(), tableName, snapshotName, targetName, + filesExpected, TEST_UTIL.getDefaultRootDirPath(), copyDir, + overwrite, getBypassRegionPredicate()); + } + + /** + * Test ExportSnapshot + */ + protected static void testExportFileSystemState(final Configuration conf, final TableName tableName, + final byte[] snapshotName, final byte[] targetName, final int filesExpected, + final Path sourceDir, Path copyDir, final boolean overwrite, + final RegionPredicate bypassregionPredicate) throws Exception { + URI hdfsUri = FileSystem.get(conf).getUri(); + FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration()); + copyDir = copyDir.makeQualified(fs); + + List opts = new ArrayList(); + opts.add("-snapshot"); + opts.add(Bytes.toString(snapshotName)); + opts.add("-copy-to"); + opts.add(copyDir.toString()); + if (targetName != snapshotName) { + opts.add("-target"); + opts.add(Bytes.toString(targetName)); + } + if (overwrite) opts.add("-overwrite"); + + // Export Snapshot + int res = ExportSnapshot.innerMain(conf, opts.toArray(new String[opts.size()])); + assertEquals(0, res); + + // Verify File-System state + FileStatus[] rootFiles = fs.listStatus(copyDir); + assertEquals(filesExpected > 0 ? 2 : 1, rootFiles.length); + for (FileStatus fileStatus: rootFiles) { + String name = fileStatus.getPath().getName(); + assertTrue(fileStatus.isDirectory()); + assertTrue(name.equals(HConstants.SNAPSHOT_DIR_NAME) || + name.equals(HConstants.HFILE_ARCHIVE_DIRECTORY)); + } + + // compare the snapshot metadata and verify the hfiles + final FileSystem hdfs = FileSystem.get(hdfsUri, conf); + final Path snapshotDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(snapshotName)); + final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(targetName)); + verifySnapshotDir(hdfs, new Path(sourceDir, snapshotDir), + fs, new Path(copyDir, targetDir)); + Set snapshotFiles = verifySnapshot(conf, fs, copyDir, tableName, + Bytes.toString(targetName), bypassregionPredicate); + assertEquals(filesExpected, snapshotFiles.size()); + } + + /** + * Check that ExportSnapshot will return a failure if something fails. + */ + @Test + public void testExportFailure() throws Exception { + assertEquals(1, runExportAndInjectFailures(snapshotName, false)); + } + + /** + * Check that ExportSnapshot will succede if something fails but the retry succede. + */ + @Test + public void testExportRetry() throws Exception { + assertEquals(0, runExportAndInjectFailures(snapshotName, true)); + } + + /* + * Execute the ExportSnapshot job injecting failures + */ + private int runExportAndInjectFailures(final byte[] snapshotName, boolean retry) + throws Exception { + Path copyDir = getLocalDestinationDir(); + URI hdfsUri = FileSystem.get(TEST_UTIL.getConfiguration()).getUri(); + FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration()); + copyDir = copyDir.makeQualified(fs); + + Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); + conf.setBoolean(ExportSnapshot.CONF_TEST_FAILURE, true); + conf.setBoolean(ExportSnapshot.CONF_TEST_RETRY, retry); + if (!retry) { + conf.setInt("mapreduce.map.maxattempts", 3); + } + // Export Snapshot +// Path sourceDir = TEST_UTIL.getHBaseCluster().getMaster().getMasterStorage().getRootDir(); + Path sourceDir = null; + int res = ExportSnapshot.innerMain(conf, new String[] { + "-snapshot", Bytes.toString(snapshotName), + "-copy-from", sourceDir.toString(), + "-copy-to", copyDir.toString() + }); + return res; + } + + /* + * verify if the snapshot folder on file-system 1 match the one on file-system 2 + */ + protected static void verifySnapshotDir(final FileSystem fs1, final Path root1, + final FileSystem fs2, final Path root2) throws IOException { + assertEquals(listFiles(fs1, root1, root1), listFiles(fs2, root2, root2)); + } + + protected Set verifySnapshot(final FileSystem fs, final Path rootDir, + final TableName tableName, final String snapshotName) throws IOException { + return verifySnapshot(TEST_UTIL.getConfiguration(), fs, rootDir, tableName, + snapshotName, getBypassRegionPredicate()); + } + + /* + * Verify if the files exists + */ + protected static Set verifySnapshot(final Configuration conf, final FileSystem fs, + final Path rootDir, final TableName tableName, final String snapshotName, + final RegionPredicate bypassregionPredicate) throws IOException { + final Path exportedSnapshot = new Path(rootDir, + new Path(HConstants.SNAPSHOT_DIR_NAME, snapshotName)); + final Set snapshotFiles = new HashSet(); + final Path exportedArchive = new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY); + MasterStorage masterStorage = MasterStorage.open(conf, false); + SnapshotDescription desc = masterStorage.getSnapshot(snapshotName); + masterStorage.visitSnapshotStoreFiles(desc, StorageContext.DATA, + new MasterStorage.SnapshotStoreFileVisitor() { + @Override + public void visitSnapshotStoreFile(SnapshotDescription snapshot, StorageContext ctx, + HRegionInfo hri, String familyName, SnapshotRegionManifest.StoreFile storeFile) + throws IOException { + if (bypassregionPredicate != null && bypassregionPredicate.evaluate(hri)) + return; + + String hfile = storeFile.getName(); + snapshotFiles.add(hfile); + if (storeFile.hasReference()) { + // Nothing to do here, we have already the reference embedded + } else { + verifyNonEmptyFile(new Path(exportedArchive, + new Path(FSUtils.getTableDir(new Path("./"), tableName), + new Path(hri.getEncodedName(), new Path(familyName, hfile))))); + } + } + + private void verifyNonEmptyFile(final Path path) throws IOException { + assertTrue(path + " should exists", fs.exists(path)); + assertTrue(path + " should not be empty", fs.getFileStatus(path).getLen() > 0); + } + }); + + // Verify Snapshot description + assertTrue(desc.getName().equals(snapshotName)); + assertTrue(desc.getTable().equals(tableName.getNameAsString())); + return snapshotFiles; + } + + private static Set listFiles(final FileSystem fs, final Path root, final Path dir) + throws IOException { + Set files = new HashSet(); + int rootPrefix = root.makeQualified(fs).toString().length(); + FileStatus[] list = FSUtils.listStatus(fs, dir); + if (list != null) { + for (FileStatus fstat: list) { + LOG.debug(fstat.getPath()); + if (fstat.isDirectory()) { + files.addAll(listFiles(fs, root, fstat.getPath())); + } else { + files.add(fstat.getPath().makeQualified(fs).toString().substring(rootPrefix)); + } + } + } + return files; + } + + private Path getHdfsDestinationDir() { +// Path rootDir = TEST_UTIL.getHBaseCluster().getMaster().getMasterStorage().getRootDir(); + Path rootDir = null; + Path path = new Path(new Path(rootDir, "export-test"), "export-" + System.currentTimeMillis()); + LOG.info("HDFS export destination path: " + path); + return path; + } + + private Path getLocalDestinationDir() { + Path path = TEST_UTIL.getDataTestDir("local-export-" + System.currentTimeMillis()); + LOG.info("Local export destination path: " + path); + return path; + } + + private static void removeExportDir(final Path path) throws IOException { + FileSystem fs = FileSystem.get(path.toUri(), new Configuration()); + fs.delete(path, true); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestExportSnapshotHelpers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestExportSnapshotHelpers.java new file mode 100644 index 0000000..d70e8c0 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestExportSnapshotHelpers.java @@ -0,0 +1,96 @@ +/** + * 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.fs.legacy.snapshot; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo; +import org.apache.hadoop.hbase.util.Pair; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Test Export Snapshot Tool helpers + */ +@Category({RegionServerTests.class, SmallTests.class}) +public class TestExportSnapshotHelpers { + private static final Log LOG = LogFactory.getLog(TestExportSnapshotHelpers.class); + + /** + * Verfy the result of getBalanceSplits() method. + * The result are groups of files, used as input list for the "export" mappers. + * All the groups should have similar amount of data. + * + * The input list is a pair of file path and length. + * The getBalanceSplits() function sort it by length, + * and assign to each group a file, going back and forth through the groups. + */ + @Test + public void testBalanceSplit() throws Exception { + // Create a list of files + List> files = new ArrayList>(); + for (long i = 0; i <= 20; i++) { + SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder() + .setType(SnapshotFileInfo.Type.HFILE) + .setHfile("file-" + i) + .build(); + files.add(new Pair(fileInfo, i)); + } + + // Create 5 groups (total size 210) + // group 0: 20, 11, 10, 1 (total size: 42) + // group 1: 19, 12, 9, 2 (total size: 42) + // group 2: 18, 13, 8, 3 (total size: 42) + // group 3: 17, 12, 7, 4 (total size: 42) + // group 4: 16, 11, 6, 5 (total size: 42) + List>> splits = ExportSnapshot.getBalancedSplits(files, 5); + assertEquals(5, splits.size()); + + String[] split0 = new String[] {"file-20", "file-11", "file-10", "file-1", "file-0"}; + verifyBalanceSplit(splits.get(0), split0, 42); + String[] split1 = new String[] {"file-19", "file-12", "file-9", "file-2"}; + verifyBalanceSplit(splits.get(1), split1, 42); + String[] split2 = new String[] {"file-18", "file-13", "file-8", "file-3"}; + verifyBalanceSplit(splits.get(2), split2, 42); + String[] split3 = new String[] {"file-17", "file-14", "file-7", "file-4"}; + verifyBalanceSplit(splits.get(3), split3, 42); + String[] split4 = new String[] {"file-16", "file-15", "file-6", "file-5"}; + verifyBalanceSplit(splits.get(4), split4, 42); + } + + private void verifyBalanceSplit(final List> split, + final String[] expected, final long expectedSize) { + assertEquals(expected.length, split.size()); + long totalSize = 0; + for (int i = 0; i < expected.length; ++i) { + Pair fileInfo = split.get(i); + assertEquals(expected[i], fileInfo.getFirst().getHfile()); + totalSize += fileInfo.getSecond(); + } + assertEquals(expectedSize, totalSize); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestRestoreSnapshotHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestRestoreSnapshotHelper.java new file mode 100644 index 0000000..258fa0d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestRestoreSnapshotHelper.java @@ -0,0 +1,181 @@ +/** + * 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifest; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; +import org.apache.hadoop.hbase.io.HFileLink; +import org.apache.hadoop.hbase.monitoring.MonitoredTask; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; +import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils.SnapshotMock; +import org.apache.hadoop.hbase.util.FSTableDescriptors; +import org.apache.hadoop.hbase.util.FSUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.Mockito; + +/** + * Test the restore/clone operation from a file-system point of view. + */ +@Category({RegionServerTests.class, SmallTests.class}) +public class TestRestoreSnapshotHelper { + private static final Log LOG = LogFactory.getLog(TestRestoreSnapshotHelper.class); + + protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + protected final static String TEST_HFILE = "abc"; + + protected Configuration conf; + protected Path archiveDir; + protected FileSystem fs; + protected Path rootDir; + + protected void setupConf(Configuration conf) { + } + + @Before + public void setup() throws Exception { + rootDir = TEST_UTIL.getDataTestDir("testRestore"); + archiveDir = new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY); + fs = TEST_UTIL.getTestFileSystem(); + conf = TEST_UTIL.getConfiguration(); + setupConf(conf); + FSUtils.setRootDir(conf, rootDir); + } + + @After + public void tearDown() throws Exception { + fs.delete(TEST_UTIL.getDataTestDir(), true); + } + + protected SnapshotMock createSnapshotMock() throws IOException { + return new SnapshotMock(TEST_UTIL.getConfiguration(), fs, rootDir); + } + + @Test + public void testRestore() throws IOException { + restoreAndVerify("snapshot", "testRestore"); + } + + @Test + public void testRestoreWithNamespace() throws IOException { + restoreAndVerify("snapshot", "namespace1:testRestoreWithNamespace"); + } + + private void restoreAndVerify(final String snapshotName, final String tableName) throws IOException { + // Test Rolling-Upgrade like Snapshot. + // half machines writing using v1 and the others using v2 format. + SnapshotMock snapshotMock = createSnapshotMock(); + SnapshotMock.SnapshotBuilder builder = snapshotMock.createSnapshotV2("snapshot", tableName); + builder.addRegionV1(); + builder.addRegionV2(); + builder.addRegionV2(); + builder.addRegionV1(); + Path snapshotDir = builder.commit(); + HTableDescriptor htd = builder.getTableDescriptor(); + SnapshotDescription desc = builder.getSnapshotDescription(); + + // Test clone a snapshot + HTableDescriptor htdClone = snapshotMock.createHtd("testtb-clone"); + testRestore(snapshotDir, desc, htdClone); + verifyRestore(rootDir, htd, htdClone); + + // Test clone a clone ("link to link") + SnapshotDescription cloneDesc = SnapshotDescription.newBuilder() + .setName("cloneSnapshot") + .setTable("testtb-clone") + .build(); + Path cloneDir = FSUtils.getTableDir(rootDir, htdClone.getTableName()); + HTableDescriptor htdClone2 = snapshotMock.createHtd("testtb-clone2"); + testRestore(cloneDir, cloneDesc, htdClone2); + verifyRestore(rootDir, htd, htdClone2); + } + + private void verifyRestore(final Path rootDir, final HTableDescriptor sourceHtd, + final HTableDescriptor htdClone) throws IOException { + List files = SnapshotTestingUtils.listHFileNames(fs, + FSUtils.getTableDir(rootDir, htdClone.getTableName())); + assertEquals(12, files.size()); + for (int i = 0; i < files.size(); i += 2) { + String linkFile = files.get(i); + String refFile = files.get(i+1); + assertTrue(linkFile + " should be a HFileLink", HFileLink.isHFileLink(linkFile)); + assertTrue(refFile + " should be a Referene", StoreFileInfo.isReference(refFile)); + assertEquals(sourceHtd.getTableName(), HFileLink.getReferencedTableName(linkFile)); + Path refPath = getReferredToFile(refFile); + LOG.debug("get reference name for file " + refFile + " = " + refPath); + assertTrue(refPath.getName() + " should be a HFileLink", HFileLink.isHFileLink(refPath.getName())); + assertEquals(linkFile, refPath.getName()); + } + } + + /** + * Execute the restore operation + * @param snapshotDir The snapshot directory to use as "restore source" + * @param sd The snapshot descriptor + * @param htdClone The HTableDescriptor of the table to restore/clone. + */ + private void testRestore(final Path snapshotDir, final SnapshotDescription sd, + final HTableDescriptor htdClone) throws IOException { + LOG.debug("pre-restore table=" + htdClone.getTableName() + " snapshot=" + snapshotDir); + FSUtils.logFileSystemState(fs, rootDir, LOG); + + new FSTableDescriptors(conf).createTableDescriptor(htdClone); + RestoreSnapshotHelper helper = getRestoreHelper(rootDir, snapshotDir, sd, htdClone); + helper.restoreStorageRegions(); + + LOG.debug("post-restore table=" + htdClone.getTableName() + " snapshot=" + snapshotDir); + FSUtils.logFileSystemState(fs, rootDir, LOG); + } + + /** + * Initialize the restore helper, based on the snapshot and table information provided. + */ + private RestoreSnapshotHelper getRestoreHelper(final Path rootDir, final Path snapshotDir, + final SnapshotDescription sd, final HTableDescriptor htdClone) throws IOException { + ForeignExceptionDispatcher monitor = Mockito.mock(ForeignExceptionDispatcher.class); + MonitoredTask status = Mockito.mock(MonitoredTask.class); + + SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, sd); + return new RestoreSnapshotHelper(conf, manifest, + htdClone, monitor, status); + } + + private Path getReferredToFile(final String referenceName) { + Path fakeBasePath = new Path(new Path("table", "region"), "cf"); + return StoreFileInfo.getReferredToFile(new Path(fakeBasePath, referenceName)); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestSnapshotFileCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestSnapshotFileCache.java index a92a5bc..64ea9d8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestSnapshotFileCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestSnapshotFileCache.java @@ -41,7 +41,12 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.fs.MasterStorage; +import org.apache.hadoop.hbase.fs.StorageContext; +import org.apache.hadoop.hbase.fs.StorageIdentifier; +import org.apache.hadoop.hbase.fs.legacy.LegacyLayout; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; +import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MasterTests; @@ -66,12 +71,14 @@ public class TestSnapshotFileCache { private static long sequenceId = 0; private static FileSystem fs; private static Path rootDir; + private static MasterStorage masterStorage; @BeforeClass public static void startCluster() throws Exception { UTIL.startMiniDFSCluster(1); fs = UTIL.getDFSCluster().getFileSystem(); rootDir = UTIL.getDefaultRootDirPath(); + masterStorage = MasterStorage.open(UTIL.getConfiguration(), false); } @AfterClass @@ -82,7 +89,7 @@ public class TestSnapshotFileCache { @After public void cleanupFiles() throws Exception { // cleanup the snapshot directory - Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir); + Path snapshotDir = LegacyLayout.getSnapshotDir(rootDir); fs.delete(snapshotDir, true); } @@ -183,15 +190,16 @@ public class TestSnapshotFileCache { private List getStoreFilesForSnapshot(SnapshotMock.SnapshotBuilder builder) throws IOException { final List allStoreFiles = Lists.newArrayList(); - SnapshotReferenceUtil - .visitReferencedFiles(UTIL.getConfiguration(), fs, builder.getSnapshotsDir(), - new SnapshotReferenceUtil.SnapshotVisitor() { - @Override public void storeFile(HRegionInfo regionInfo, String familyName, - SnapshotProtos.SnapshotRegionManifest.StoreFile storeFile) throws IOException { - FileStatus status = mockStoreFile(storeFile.getName()); - allStoreFiles.add(status); - } - }); + masterStorage.visitSnapshotStoreFiles(builder.getSnapshotDescription(), StorageContext.DATA, + new MasterStorage.SnapshotStoreFileVisitor() { + @Override + public void visitSnapshotStoreFile(HBaseProtos.SnapshotDescription snapshot, + StorageContext ctx, HRegionInfo hri, String familyName, + SnapshotProtos.SnapshotRegionManifest.StoreFile storeFile) throws IOException { + FileStatus status = mockStoreFile(storeFile.getName()); + allStoreFiles.add(status); + } + }); return allStoreFiles; } @@ -206,7 +214,7 @@ public class TestSnapshotFileCache { class SnapshotFiles implements SnapshotFileCache.SnapshotFileInspector { public Collection filesUnderSnapshot(final Path snapshotDir) throws IOException { Collection files = new HashSet(); - files.addAll(SnapshotReferenceUtil.getHFileNames(UTIL.getConfiguration(), fs, snapshotDir)); + files.addAll(SnapshotReferenceUtil.getHFileNames(masterStorage, snapshotDir.getName())); return files; } }; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestSnapshotManifest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestSnapshotManifest.java new file mode 100644 index 0000000..0112d5e8 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/legacy/snapshot/TestSnapshotManifest.java @@ -0,0 +1,146 @@ +/** + * 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.fs.legacy.snapshot; + +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.ProtobufUtil; +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.snapshot.CorruptedSnapshotException; +import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; +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(ProtobufUtil.convertToTableSchema(builder.getTableDescriptor())); + + 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 (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."); + } + } + } + + 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(); + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java index 540f6b1..920e04b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java @@ -42,10 +42,8 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.fs.RegionStorage; -import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.CacheConfig; @@ -60,7 +58,6 @@ import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ChecksumType; -import org.apache.hadoop.hbase.util.FSUtils; import org.junit.After; import org.junit.Before; import org.junit.Test; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java index 40bd961..1515d06 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/SnapshotTestingUtils.java @@ -41,7 +41,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; @@ -56,8 +55,10 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.fs.MasterStorage; -import org.apache.hadoop.hbase.fs.RegionStorage; import org.apache.hadoop.hbase.fs.legacy.LegacyTableDescriptor; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifest; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifestV1; +import org.apache.hadoop.hbase.fs.legacy.snapshot.SnapshotManifestV2; import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.mob.MobUtils; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java deleted file mode 100644 index 6d7d4e1..0000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshot.java +++ /dev/null @@ -1,376 +0,0 @@ -/** - * 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 static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.net.URI; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -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.CategoryBasedTimeout; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; -import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSUtils; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.junit.rules.TestRule; - -/** - * Test Export Snapshot Tool - */ -@Category({VerySlowMapReduceTests.class, LargeTests.class}) -public class TestExportSnapshot { - @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). - withTimeout(this.getClass()).withLookingForStuckThread(true).build(); - private static final Log LOG = LogFactory.getLog(TestExportSnapshot.class); - - protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - - protected final static byte[] FAMILY = Bytes.toBytes("cf"); - - protected TableName tableName; - private byte[] emptySnapshotName; - private byte[] snapshotName; - private int tableNumFiles; - private Admin admin; - - public static void setUpBaseConf(Configuration conf) { - conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true); - conf.setInt("hbase.regionserver.msginterval", 100); - conf.setInt("hbase.client.pause", 250); - conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 6); - conf.setBoolean("hbase.master.enabletable.roundrobin", true); - conf.setInt("mapreduce.map.maxattempts", 10); - } - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - setUpBaseConf(TEST_UTIL.getConfiguration()); - TEST_UTIL.startMiniCluster(3); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - } - - /** - * Create a table and take a snapshot of the table used by the export test. - */ - @Before - public void setUp() throws Exception { - this.admin = TEST_UTIL.getHBaseAdmin(); - - long tid = System.currentTimeMillis(); - tableName = TableName.valueOf("testtb-" + tid); - snapshotName = Bytes.toBytes("snaptb0-" + tid); - emptySnapshotName = Bytes.toBytes("emptySnaptb0-" + tid); - - // create Table - createTable(); - - // Take an empty snapshot - admin.snapshot(emptySnapshotName, tableName); - - // Add some rows - SnapshotTestingUtils.loadData(TEST_UTIL, tableName, 50, FAMILY); - tableNumFiles = admin.getTableRegions(tableName).size(); - - // take a snapshot - admin.snapshot(snapshotName, tableName); - } - - protected void createTable() throws Exception { - SnapshotTestingUtils.createPreSplitTable(TEST_UTIL, tableName, 2, FAMILY); - } - - protected interface RegionPredicate { - boolean evaluate(final HRegionInfo regionInfo); - } - - protected RegionPredicate getBypassRegionPredicate() { - return null; - } - - @After - public void tearDown() throws Exception { - TEST_UTIL.deleteTable(tableName); - SnapshotTestingUtils.deleteAllSnapshots(TEST_UTIL.getHBaseAdmin()); - SnapshotTestingUtils.deleteArchiveDirectory(TEST_UTIL); - } - - /** - * Verify if exported snapshot and copied files matches the original one. - */ - @Test - public void testExportFileSystemState() throws Exception { - testExportFileSystemState(tableName, snapshotName, snapshotName, tableNumFiles); - } - - @Test - public void testExportFileSystemStateWithSkipTmp() throws Exception { - TEST_UTIL.getConfiguration().setBoolean(ExportSnapshot.CONF_SKIP_TMP, true); - try { - testExportFileSystemState(tableName, snapshotName, snapshotName, tableNumFiles); - } finally { - TEST_UTIL.getConfiguration().setBoolean(ExportSnapshot.CONF_SKIP_TMP, false); - } - } - - @Test - public void testEmptyExportFileSystemState() throws Exception { - testExportFileSystemState(tableName, emptySnapshotName, emptySnapshotName, 0); - } - - @Test - public void testConsecutiveExports() throws Exception { - Path copyDir = getLocalDestinationDir(); - testExportFileSystemState(tableName, snapshotName, snapshotName, tableNumFiles, copyDir, false); - testExportFileSystemState(tableName, snapshotName, snapshotName, tableNumFiles, copyDir, true); - removeExportDir(copyDir); - } - - @Test - public void testExportWithTargetName() throws Exception { - final byte[] targetName = Bytes.toBytes("testExportWithTargetName"); - testExportFileSystemState(tableName, snapshotName, targetName, tableNumFiles); - } - - private void testExportFileSystemState(final TableName tableName, final byte[] snapshotName, - final byte[] targetName, int filesExpected) throws Exception { - testExportFileSystemState(tableName, snapshotName, targetName, - filesExpected, getHdfsDestinationDir(), false); - } - - protected void testExportFileSystemState(final TableName tableName, - final byte[] snapshotName, final byte[] targetName, int filesExpected, - Path copyDir, boolean overwrite) throws Exception { - testExportFileSystemState(TEST_UTIL.getConfiguration(), tableName, snapshotName, targetName, - filesExpected, TEST_UTIL.getDefaultRootDirPath(), copyDir, - overwrite, getBypassRegionPredicate()); - } - - /** - * Test ExportSnapshot - */ - protected static void testExportFileSystemState(final Configuration conf, final TableName tableName, - final byte[] snapshotName, final byte[] targetName, final int filesExpected, - final Path sourceDir, Path copyDir, final boolean overwrite, - final RegionPredicate bypassregionPredicate) throws Exception { - URI hdfsUri = FileSystem.get(conf).getUri(); - FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration()); - copyDir = copyDir.makeQualified(fs); - - List opts = new ArrayList(); - opts.add("-snapshot"); - opts.add(Bytes.toString(snapshotName)); - opts.add("-copy-to"); - opts.add(copyDir.toString()); - if (targetName != snapshotName) { - opts.add("-target"); - opts.add(Bytes.toString(targetName)); - } - if (overwrite) opts.add("-overwrite"); - - // Export Snapshot - int res = ExportSnapshot.innerMain(conf, opts.toArray(new String[opts.size()])); - assertEquals(0, res); - - // Verify File-System state - FileStatus[] rootFiles = fs.listStatus(copyDir); - assertEquals(filesExpected > 0 ? 2 : 1, rootFiles.length); - for (FileStatus fileStatus: rootFiles) { - String name = fileStatus.getPath().getName(); - assertTrue(fileStatus.isDirectory()); - assertTrue(name.equals(HConstants.SNAPSHOT_DIR_NAME) || - name.equals(HConstants.HFILE_ARCHIVE_DIRECTORY)); - } - - // compare the snapshot metadata and verify the hfiles - final FileSystem hdfs = FileSystem.get(hdfsUri, conf); - final Path snapshotDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(snapshotName)); - final Path targetDir = new Path(HConstants.SNAPSHOT_DIR_NAME, Bytes.toString(targetName)); - verifySnapshotDir(hdfs, new Path(sourceDir, snapshotDir), - fs, new Path(copyDir, targetDir)); - Set snapshotFiles = verifySnapshot(conf, fs, copyDir, tableName, - Bytes.toString(targetName), bypassregionPredicate); - assertEquals(filesExpected, snapshotFiles.size()); - } - - /** - * Check that ExportSnapshot will return a failure if something fails. - */ - @Test - public void testExportFailure() throws Exception { - assertEquals(1, runExportAndInjectFailures(snapshotName, false)); - } - - /** - * Check that ExportSnapshot will succede if something fails but the retry succede. - */ - @Test - public void testExportRetry() throws Exception { - assertEquals(0, runExportAndInjectFailures(snapshotName, true)); - } - - /* - * Execute the ExportSnapshot job injecting failures - */ - private int runExportAndInjectFailures(final byte[] snapshotName, boolean retry) - throws Exception { - Path copyDir = getLocalDestinationDir(); - URI hdfsUri = FileSystem.get(TEST_UTIL.getConfiguration()).getUri(); - FileSystem fs = FileSystem.get(copyDir.toUri(), new Configuration()); - copyDir = copyDir.makeQualified(fs); - - Configuration conf = new Configuration(TEST_UTIL.getConfiguration()); - conf.setBoolean(ExportSnapshot.CONF_TEST_FAILURE, true); - conf.setBoolean(ExportSnapshot.CONF_TEST_RETRY, retry); - if (!retry) { - conf.setInt("mapreduce.map.maxattempts", 3); - } - // Export Snapshot -// Path sourceDir = TEST_UTIL.getHBaseCluster().getMaster().getMasterStorage().getRootDir(); - Path sourceDir = null; - int res = ExportSnapshot.innerMain(conf, new String[] { - "-snapshot", Bytes.toString(snapshotName), - "-copy-from", sourceDir.toString(), - "-copy-to", copyDir.toString() - }); - return res; - } - - /* - * verify if the snapshot folder on file-system 1 match the one on file-system 2 - */ - protected static void verifySnapshotDir(final FileSystem fs1, final Path root1, - final FileSystem fs2, final Path root2) throws IOException { - assertEquals(listFiles(fs1, root1, root1), listFiles(fs2, root2, root2)); - } - - protected Set verifySnapshot(final FileSystem fs, final Path rootDir, - final TableName tableName, final String snapshotName) throws IOException { - return verifySnapshot(TEST_UTIL.getConfiguration(), fs, rootDir, tableName, - snapshotName, getBypassRegionPredicate()); - } - - /* - * Verify if the files exists - */ - protected static Set verifySnapshot(final Configuration conf, final FileSystem fs, - final Path rootDir, final TableName tableName, final String snapshotName, - final RegionPredicate bypassregionPredicate) throws IOException { - final Path exportedSnapshot = new Path(rootDir, - new Path(HConstants.SNAPSHOT_DIR_NAME, snapshotName)); - final Set snapshotFiles = new HashSet(); - final Path exportedArchive = new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY); - SnapshotReferenceUtil.visitReferencedFiles(conf, fs, exportedSnapshot, - new SnapshotReferenceUtil.SnapshotVisitor() { - @Override - public void storeFile(final HRegionInfo regionInfo, final String family, - final SnapshotRegionManifest.StoreFile storeFile) throws IOException { - if (bypassregionPredicate != null && bypassregionPredicate.evaluate(regionInfo)) - return; - - String hfile = storeFile.getName(); - snapshotFiles.add(hfile); - if (storeFile.hasReference()) { - // Nothing to do here, we have already the reference embedded - } else { - verifyNonEmptyFile(new Path(exportedArchive, - new Path(FSUtils.getTableDir(new Path("./"), tableName), - new Path(regionInfo.getEncodedName(), new Path(family, hfile))))); - } - } - - private void verifyNonEmptyFile(final Path path) throws IOException { - assertTrue(path + " should exists", fs.exists(path)); - assertTrue(path + " should not be empty", fs.getFileStatus(path).getLen() > 0); - } - }); - - // Verify Snapshot description - SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, exportedSnapshot); - assertTrue(desc.getName().equals(snapshotName)); - assertTrue(desc.getTable().equals(tableName.getNameAsString())); - return snapshotFiles; - } - - private static Set listFiles(final FileSystem fs, final Path root, final Path dir) - throws IOException { - Set files = new HashSet(); - int rootPrefix = root.makeQualified(fs).toString().length(); - FileStatus[] list = FSUtils.listStatus(fs, dir); - if (list != null) { - for (FileStatus fstat: list) { - LOG.debug(fstat.getPath()); - if (fstat.isDirectory()) { - files.addAll(listFiles(fs, root, fstat.getPath())); - } else { - files.add(fstat.getPath().makeQualified(fs).toString().substring(rootPrefix)); - } - } - } - return files; - } - - private Path getHdfsDestinationDir() { -// Path rootDir = TEST_UTIL.getHBaseCluster().getMaster().getMasterStorage().getRootDir(); - Path rootDir = null; - Path path = new Path(new Path(rootDir, "export-test"), "export-" + System.currentTimeMillis()); - LOG.info("HDFS export destination path: " + path); - return path; - } - - private Path getLocalDestinationDir() { - Path path = TEST_UTIL.getDataTestDir("local-export-" + System.currentTimeMillis()); - LOG.info("Local export destination path: " + path); - return path; - } - - private static void removeExportDir(final Path path) throws IOException { - FileSystem fs = FileSystem.get(path.toUri(), new Configuration()); - fs.delete(path, true); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotHelpers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotHelpers.java deleted file mode 100644 index 2d0088b..0000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestExportSnapshotHelpers.java +++ /dev/null @@ -1,96 +0,0 @@ -/** - * 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 static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotFileInfo; -import org.apache.hadoop.hbase.util.Pair; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -/** - * Test Export Snapshot Tool helpers - */ -@Category({RegionServerTests.class, SmallTests.class}) -public class TestExportSnapshotHelpers { - private static final Log LOG = LogFactory.getLog(TestExportSnapshotHelpers.class); - - /** - * Verfy the result of getBalanceSplits() method. - * The result are groups of files, used as input list for the "export" mappers. - * All the groups should have similar amount of data. - * - * The input list is a pair of file path and length. - * The getBalanceSplits() function sort it by length, - * and assign to each group a file, going back and forth through the groups. - */ - @Test - public void testBalanceSplit() throws Exception { - // Create a list of files - List> files = new ArrayList>(); - for (long i = 0; i <= 20; i++) { - SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder() - .setType(SnapshotFileInfo.Type.HFILE) - .setHfile("file-" + i) - .build(); - files.add(new Pair(fileInfo, i)); - } - - // Create 5 groups (total size 210) - // group 0: 20, 11, 10, 1 (total size: 42) - // group 1: 19, 12, 9, 2 (total size: 42) - // group 2: 18, 13, 8, 3 (total size: 42) - // group 3: 17, 12, 7, 4 (total size: 42) - // group 4: 16, 11, 6, 5 (total size: 42) - List>> splits = ExportSnapshot.getBalancedSplits(files, 5); - assertEquals(5, splits.size()); - - String[] split0 = new String[] {"file-20", "file-11", "file-10", "file-1", "file-0"}; - verifyBalanceSplit(splits.get(0), split0, 42); - String[] split1 = new String[] {"file-19", "file-12", "file-9", "file-2"}; - verifyBalanceSplit(splits.get(1), split1, 42); - String[] split2 = new String[] {"file-18", "file-13", "file-8", "file-3"}; - verifyBalanceSplit(splits.get(2), split2, 42); - String[] split3 = new String[] {"file-17", "file-14", "file-7", "file-4"}; - verifyBalanceSplit(splits.get(3), split3, 42); - String[] split4 = new String[] {"file-16", "file-15", "file-6", "file-5"}; - verifyBalanceSplit(splits.get(4), split4, 42); - } - - private void verifyBalanceSplit(final List> split, - final String[] expected, final long expectedSize) { - assertEquals(expected.length, split.size()); - long totalSize = 0; - for (int i = 0; i < expected.length; ++i) { - Pair fileInfo = split.get(i); - assertEquals(expected[i], fileInfo.getFirst().getHfile()); - totalSize += fileInfo.getSecond(); - } - assertEquals(expectedSize, totalSize); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java deleted file mode 100644 index 4011af5..0000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRestoreSnapshotHelper.java +++ /dev/null @@ -1,180 +0,0 @@ -/** - * 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 static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.util.List; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; -import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher; -import org.apache.hadoop.hbase.io.HFileLink; -import org.apache.hadoop.hbase.monitoring.MonitoredTask; -import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; -import org.apache.hadoop.hbase.regionserver.StoreFileInfo; -import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils.SnapshotMock; -import org.apache.hadoop.hbase.util.FSTableDescriptors; -import org.apache.hadoop.hbase.util.FSUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.experimental.categories.Category; -import org.mockito.Mockito; - -/** - * Test the restore/clone operation from a file-system point of view. - */ -@Category({RegionServerTests.class, SmallTests.class}) -public class TestRestoreSnapshotHelper { - private static final Log LOG = LogFactory.getLog(TestRestoreSnapshotHelper.class); - - protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - protected final static String TEST_HFILE = "abc"; - - protected Configuration conf; - protected Path archiveDir; - protected FileSystem fs; - protected Path rootDir; - - protected void setupConf(Configuration conf) { - } - - @Before - public void setup() throws Exception { - rootDir = TEST_UTIL.getDataTestDir("testRestore"); - archiveDir = new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY); - fs = TEST_UTIL.getTestFileSystem(); - conf = TEST_UTIL.getConfiguration(); - setupConf(conf); - FSUtils.setRootDir(conf, rootDir); - } - - @After - public void tearDown() throws Exception { - fs.delete(TEST_UTIL.getDataTestDir(), true); - } - - protected SnapshotMock createSnapshotMock() throws IOException { - return new SnapshotMock(TEST_UTIL.getConfiguration(), fs, rootDir); - } - - @Test - public void testRestore() throws IOException { - restoreAndVerify("snapshot", "testRestore"); - } - - @Test - public void testRestoreWithNamespace() throws IOException { - restoreAndVerify("snapshot", "namespace1:testRestoreWithNamespace"); - } - - private void restoreAndVerify(final String snapshotName, final String tableName) throws IOException { - // Test Rolling-Upgrade like Snapshot. - // half machines writing using v1 and the others using v2 format. - SnapshotMock snapshotMock = createSnapshotMock(); - SnapshotMock.SnapshotBuilder builder = snapshotMock.createSnapshotV2("snapshot", tableName); - builder.addRegionV1(); - builder.addRegionV2(); - builder.addRegionV2(); - builder.addRegionV1(); - Path snapshotDir = builder.commit(); - HTableDescriptor htd = builder.getTableDescriptor(); - SnapshotDescription desc = builder.getSnapshotDescription(); - - // Test clone a snapshot - HTableDescriptor htdClone = snapshotMock.createHtd("testtb-clone"); - testRestore(snapshotDir, desc, htdClone); - verifyRestore(rootDir, htd, htdClone); - - // Test clone a clone ("link to link") - SnapshotDescription cloneDesc = SnapshotDescription.newBuilder() - .setName("cloneSnapshot") - .setTable("testtb-clone") - .build(); - Path cloneDir = FSUtils.getTableDir(rootDir, htdClone.getTableName()); - HTableDescriptor htdClone2 = snapshotMock.createHtd("testtb-clone2"); - testRestore(cloneDir, cloneDesc, htdClone2); - verifyRestore(rootDir, htd, htdClone2); - } - - private void verifyRestore(final Path rootDir, final HTableDescriptor sourceHtd, - final HTableDescriptor htdClone) throws IOException { - List files = SnapshotTestingUtils.listHFileNames(fs, - FSUtils.getTableDir(rootDir, htdClone.getTableName())); - assertEquals(12, files.size()); - for (int i = 0; i < files.size(); i += 2) { - String linkFile = files.get(i); - String refFile = files.get(i+1); - assertTrue(linkFile + " should be a HFileLink", HFileLink.isHFileLink(linkFile)); - assertTrue(refFile + " should be a Referene", StoreFileInfo.isReference(refFile)); - assertEquals(sourceHtd.getTableName(), HFileLink.getReferencedTableName(linkFile)); - Path refPath = getReferredToFile(refFile); - LOG.debug("get reference name for file " + refFile + " = " + refPath); - assertTrue(refPath.getName() + " should be a HFileLink", HFileLink.isHFileLink(refPath.getName())); - assertEquals(linkFile, refPath.getName()); - } - } - - /** - * Execute the restore operation - * @param snapshotDir The snapshot directory to use as "restore source" - * @param sd The snapshot descriptor - * @param htdClone The HTableDescriptor of the table to restore/clone. - */ - private void testRestore(final Path snapshotDir, final SnapshotDescription sd, - final HTableDescriptor htdClone) throws IOException { - LOG.debug("pre-restore table=" + htdClone.getTableName() + " snapshot=" + snapshotDir); - FSUtils.logFileSystemState(fs, rootDir, LOG); - - new FSTableDescriptors(conf).createTableDescriptor(htdClone); - RestoreSnapshotHelper helper = getRestoreHelper(rootDir, snapshotDir, sd, htdClone); - helper.restoreStorageRegions(); - - LOG.debug("post-restore table=" + htdClone.getTableName() + " snapshot=" + snapshotDir); - FSUtils.logFileSystemState(fs, rootDir, LOG); - } - - /** - * Initialize the restore helper, based on the snapshot and table information provided. - */ - private RestoreSnapshotHelper getRestoreHelper(final Path rootDir, final Path snapshotDir, - final SnapshotDescription sd, final HTableDescriptor htdClone) throws IOException { - ForeignExceptionDispatcher monitor = Mockito.mock(ForeignExceptionDispatcher.class); - MonitoredTask status = Mockito.mock(MonitoredTask.class); - - SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, sd); - return new RestoreSnapshotHelper(conf, manifest, - htdClone, monitor, status); - } - - private Path getReferredToFile(final String referenceName) { - Path fakeBasePath = new Path(new Path("table", "region"), "cf"); - return StoreFileInfo.getReferredToFile(new Path(fakeBasePath, referenceName)); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotManifest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotManifest.java deleted file mode 100644 index 835f92e..0000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotManifest.java +++ /dev/null @@ -1,145 +0,0 @@ -/** - * 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.ProtobufUtil; -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(ProtobufUtil.convertToTableSchema(builder.getTableDescriptor())); - - 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 (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."); - } - } - } - - 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(); - } - } -} diff --git a/src/main/asciidoc/_chapters/ops_mgt.adoc b/src/main/asciidoc/_chapters/ops_mgt.adoc index c1db106..dde1779 100644 --- a/src/main/asciidoc/_chapters/ops_mgt.adoc +++ b/src/main/asciidoc/_chapters/ops_mgt.adoc @@ -2063,7 +2063,7 @@ To copy a snapshot called MySnapshot to an HBase cluster srv2 (hdfs:///srv2:8082 [source,bourne] ---- -$ bin/hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase -mappers 16 +$ bin/hbase org.apache.hadoop.hbase.fs.legacy.snapshot.ExportSnapshot -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase -mappers 16 ---- .Limiting Bandwidth Consumption @@ -2072,7 +2072,7 @@ The following example limits the above example to 200 MB/sec. [source,bourne] ---- -$ bin/hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase -mappers 16 -bandwidth 200 +$ bin/hbase org.apache.hadoop.hbase.fs.legacy.snapshot.ExportSnapshot -snapshot MySnapshot -copy-to hdfs://srv2:8082/hbase -mappers 16 -bandwidth 200 ---- [[snapshots_s3]] @@ -2091,12 +2091,12 @@ and `s3://` protocols have various limitations and do not use the Amazon AWS SDK the commands to export and restore the snapshot. After you have fulfilled the prerequisites, take the snapshot like you normally would. -Afterward, you can export it using the `org.apache.hadoop.hbase.snapshot.ExportSnapshot` +Afterward, you can export it using the `org.apache.hadoop.hbase.fs.legacy.snapshot.ExportSnapshot` command like the one below, substituting your own `s3a://` path in the `copy-from` or `copy-to` directive and substituting or modifying other options as required: ---- -$ hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot \ +$ hbase org.apache.hadoop.hbase.fs.legacy.snapshot.ExportSnapshot \ -snapshot MySnapshot \ -copy-from hdfs://srv2:8082/hbase \ -copy-to s3a:////hbase \ @@ -2107,7 +2107,7 @@ $ hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot \ ---- ---- -$ hbase org.apache.hadoop.hbase.snapshot.ExportSnapshot \ +$ hbase org.apache.hadoop.hbase.fs.legacy.snapshot.ExportSnapshot \ -snapshot MySnapshot -copy-from s3a:////hbase \ -copy-to hdfs://srv2:8082/hbase \ -- 2.7.4 (Apple Git-66)