View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.snapshot;
20  
21  import java.io.IOException;
22  import java.io.InputStream;
23  import java.io.OutputStream;
24  import java.util.ArrayList;
25  import java.util.Arrays;
26  import java.util.HashMap;
27  import java.util.HashSet;
28  import java.util.LinkedList;
29  import java.util.List;
30  import java.util.Map;
31  import java.util.Set;
32  import java.util.TreeMap;
33  import java.util.concurrent.ThreadPoolExecutor;
34  
35  import org.apache.commons.logging.Log;
36  import org.apache.commons.logging.LogFactory;
37  import org.apache.hadoop.hbase.classification.InterfaceAudience;
38  import org.apache.hadoop.conf.Configuration;
39  import org.apache.hadoop.fs.FileStatus;
40  import org.apache.hadoop.fs.FileSystem;
41  import org.apache.hadoop.fs.Path;
42  import org.apache.hadoop.hbase.HColumnDescriptor;
43  import org.apache.hadoop.hbase.HRegionInfo;
44  import org.apache.hadoop.hbase.HTableDescriptor;
45  import org.apache.hadoop.hbase.TableName;
46  import org.apache.hadoop.hbase.backup.HFileArchiver;
47  import org.apache.hadoop.hbase.MetaTableAccessor;
48  import org.apache.hadoop.hbase.client.Connection;
49  import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
50  import org.apache.hadoop.hbase.io.HFileLink;
51  import org.apache.hadoop.hbase.io.Reference;
52  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
53  import org.apache.hadoop.hbase.mob.MobUtils;
54  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
55  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
56  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
57  import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
58  import org.apache.hadoop.hbase.regionserver.HRegion;
59  import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
60  import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
61  import org.apache.hadoop.hbase.util.Bytes;
62  import org.apache.hadoop.hbase.util.FSUtils;
63  import org.apache.hadoop.hbase.util.ModifyRegionUtils;
64  import org.apache.hadoop.hbase.util.Pair;
65  import org.apache.hadoop.io.IOUtils;
66  
67  /**
68   * Helper to Restore/Clone a Snapshot
69   *
70   * <p>The helper assumes that a table is already created, and by calling restore()
71   * the content present in the snapshot will be restored as the new content of the table.
72   *
73   * <p>Clone from Snapshot: If the target table is empty, the restore operation
74   * is just a "clone operation", where the only operations are:
75   * <ul>
76   *  <li>for each region in the snapshot create a new region
77   *    (note that the region will have a different name, since the encoding contains the table name)
78   *  <li>for each file in the region create a new HFileLink to point to the original file.
79   *  <li>restore the logs, if any
80   * </ul>
81   *
82   * <p>Restore from Snapshot:
83   * <ul>
84   *  <li>for each region in the table verify which are available in the snapshot and which are not
85   *    <ul>
86   *    <li>if the region is not present in the snapshot, remove it.
87   *    <li>if the region is present in the snapshot
88   *      <ul>
89   *      <li>for each file in the table region verify which are available in the snapshot
90   *        <ul>
91   *          <li>if the hfile is not present in the snapshot, remove it
92   *          <li>if the hfile is present, keep it (nothing to do)
93   *        </ul>
94   *      <li>for each file in the snapshot region but not in the table
95   *        <ul>
96   *          <li>create a new HFileLink that point to the original file
97   *        </ul>
98   *      </ul>
99   *    </ul>
100  *  <li>for each region in the snapshot not present in the current table state
101  *    <ul>
102  *    <li>create a new region and for each file in the region create a new HFileLink
103  *      (This is the same as the clone operation)
104  *    </ul>
105  *  <li>restore the logs, if any
106  * </ul>
107  */
108 @InterfaceAudience.Private
109 public class RestoreSnapshotHelper {
110   private static final Log LOG = LogFactory.getLog(RestoreSnapshotHelper.class);
111 
112   private final Map<byte[], byte[]> regionsMap =
113         new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
114 
115   private final Map<String, Pair<String, String> > parentsMap =
116       new HashMap<String, Pair<String, String> >();
117 
118   private final ForeignExceptionDispatcher monitor;
119   private final MonitoredTask status;
120 
121   private final SnapshotManifest snapshotManifest;
122   private final SnapshotDescription snapshotDesc;
123   private final TableName snapshotTable;
124 
125   private final HTableDescriptor tableDesc;
126   private final Path rootDir;
127   private final Path tableDir;
128 
129   private final Configuration conf;
130   private final FileSystem fs;
131   private final boolean createBackRefs;
132 
133   public RestoreSnapshotHelper(final Configuration conf,
134       final FileSystem fs,
135       final SnapshotManifest manifest,
136       final HTableDescriptor tableDescriptor,
137       final Path rootDir,
138       final ForeignExceptionDispatcher monitor,
139       final MonitoredTask status) {
140     this(conf, fs, manifest, tableDescriptor, rootDir, monitor, status, true);
141   }
142 
143   public RestoreSnapshotHelper(final Configuration conf,
144       final FileSystem fs,
145       final SnapshotManifest manifest,
146       final HTableDescriptor tableDescriptor,
147       final Path rootDir,
148       final ForeignExceptionDispatcher monitor,
149       final MonitoredTask status,
150       final boolean createBackRefs)
151   {
152     this.fs = fs;
153     this.conf = conf;
154     this.snapshotManifest = manifest;
155     this.snapshotDesc = manifest.getSnapshotDescription();
156     this.snapshotTable = TableName.valueOf(snapshotDesc.getTable());
157     this.tableDesc = tableDescriptor;
158     this.rootDir = rootDir;
159     this.tableDir = FSUtils.getTableDir(rootDir, tableDesc.getTableName());
160     this.monitor = monitor;
161     this.status = status;
162     this.createBackRefs = createBackRefs;
163   }
164 
165   /**
166    * Restore the on-disk table to a specified snapshot state.
167    * @return the set of regions touched by the restore operation
168    */
169   public RestoreMetaChanges restoreHdfsRegions() throws IOException {
170     ThreadPoolExecutor exec = SnapshotManifest.createExecutor(conf, "RestoreSnapshot");
171     try {
172       return restoreHdfsRegions(exec);
173     } finally {
174       exec.shutdown();
175     }
176   }
177 
178   private RestoreMetaChanges restoreHdfsRegions(final ThreadPoolExecutor exec) throws IOException {
179     LOG.debug("starting restore");
180 
181     Map<String, SnapshotRegionManifest> regionManifests = snapshotManifest.getRegionManifestsMap();
182     if (regionManifests == null) {
183       LOG.warn("Nothing to restore. Snapshot " + snapshotDesc + " looks empty");
184       return null;
185     }
186 
187     RestoreMetaChanges metaChanges = new RestoreMetaChanges(parentsMap);
188 
189     // Take a copy of the manifest.keySet() since we are going to modify
190     // this instance, by removing the regions already present in the restore dir.
191     Set<String> regionNames = new HashSet<String>(regionManifests.keySet());
192 
193     HRegionInfo mobRegion = MobUtils.getMobRegionInfo(snapshotManifest.getTableDescriptor()
194         .getTableName());
195     // Identify which region are still available and which not.
196     // NOTE: we rely upon the region name as: "table name, start key, end key"
197     List<HRegionInfo> tableRegions = getTableRegions();
198     if (tableRegions != null) {
199       monitor.rethrowException();
200       for (HRegionInfo regionInfo: tableRegions) {
201         String regionName = regionInfo.getEncodedName();
202         if (regionNames.contains(regionName)) {
203           LOG.info("region to restore: " + regionName);
204           regionNames.remove(regionName);
205           metaChanges.addRegionToRestore(regionInfo);
206         } else {
207           LOG.info("region to remove: " + regionName);
208           metaChanges.addRegionToRemove(regionInfo);
209         }
210       }
211 
212       // Restore regions using the snapshot data
213       monitor.rethrowException();
214       status.setStatus("Restoring table regions...");
215       if (regionNames.contains(mobRegion.getEncodedName())) {
216         // restore the mob region in case
217         List<HRegionInfo> mobRegions = new ArrayList<HRegionInfo>(1);
218         mobRegions.add(mobRegion);
219         restoreHdfsMobRegions(exec, regionManifests, mobRegions);
220         regionNames.remove(mobRegion.getEncodedName());
221       }
222       restoreHdfsRegions(exec, regionManifests, metaChanges.getRegionsToRestore());
223       status.setStatus("Finished restoring all table regions.");
224 
225       // Remove regions from the current table
226       monitor.rethrowException();
227       status.setStatus("Starting to delete excess regions from table");
228       removeHdfsRegions(exec, metaChanges.getRegionsToRemove());
229       status.setStatus("Finished deleting excess regions from table.");
230     }
231 
232     // Regions to Add: present in the snapshot but not in the current table
233     if (regionNames.size() > 0) {
234       List<HRegionInfo> regionsToAdd = new ArrayList<HRegionInfo>(regionNames.size());
235 
236       monitor.rethrowException();
237       // add the mob region
238       if (regionNames.contains(mobRegion.getEncodedName())) {
239         cloneHdfsMobRegion(regionManifests, mobRegion);
240         regionNames.remove(mobRegion.getEncodedName());
241       }
242       for (String regionName: regionNames) {
243         LOG.info("region to add: " + regionName);
244         regionsToAdd.add(HRegionInfo.convert(regionManifests.get(regionName).getRegionInfo()));
245       }
246 
247       // Create new regions cloning from the snapshot
248       monitor.rethrowException();
249       status.setStatus("Cloning regions...");
250       HRegionInfo[] clonedRegions = cloneHdfsRegions(exec, regionManifests, regionsToAdd);
251       metaChanges.setNewRegions(clonedRegions);
252       status.setStatus("Finished cloning regions.");
253     }
254 
255     return metaChanges;
256   }
257 
258   /**
259    * Describe the set of operations needed to update hbase:meta after restore.
260    */
261   public static class RestoreMetaChanges {
262     private final Map<String, Pair<String, String> > parentsMap;
263 
264     private List<HRegionInfo> regionsToRestore = null;
265     private List<HRegionInfo> regionsToRemove = null;
266     private List<HRegionInfo> regionsToAdd = null;
267 
268     RestoreMetaChanges(final Map<String, Pair<String, String> > parentsMap) {
269       this.parentsMap = parentsMap;
270     }
271 
272     /**
273      * @return true if there're new regions
274      */
275     public boolean hasRegionsToAdd() {
276       return this.regionsToAdd != null && this.regionsToAdd.size() > 0;
277     }
278 
279     /**
280      * Returns the list of new regions added during the on-disk restore.
281      * The caller is responsible to add the regions to META.
282      * e.g MetaTableAccessor.addRegionsToMeta(...)
283      * @return the list of regions to add to META
284      */
285     public List<HRegionInfo> getRegionsToAdd() {
286       return this.regionsToAdd;
287     }
288 
289     /**
290      * @return true if there're regions to restore
291      */
292     public boolean hasRegionsToRestore() {
293       return this.regionsToRestore != null && this.regionsToRestore.size() > 0;
294     }
295 
296     /**
297      * Returns the list of 'restored regions' during the on-disk restore.
298      * The caller is responsible to add the regions to hbase:meta if not present.
299      * @return the list of regions restored
300      */
301     public List<HRegionInfo> getRegionsToRestore() {
302       return this.regionsToRestore;
303     }
304 
305     /**
306      * @return true if there're regions to remove
307      */
308     public boolean hasRegionsToRemove() {
309       return this.regionsToRemove != null && this.regionsToRemove.size() > 0;
310     }
311 
312     /**
313      * Returns the list of regions removed during the on-disk restore.
314      * The caller is responsible to remove the regions from META.
315      * e.g. MetaTableAccessor.deleteRegions(...)
316      * @return the list of regions to remove from META
317      */
318     public List<HRegionInfo> getRegionsToRemove() {
319       return this.regionsToRemove;
320     }
321 
322     void setNewRegions(final HRegionInfo[] hris) {
323       if (hris != null) {
324         regionsToAdd = Arrays.asList(hris);
325       } else {
326         regionsToAdd = null;
327       }
328     }
329 
330     void addRegionToRemove(final HRegionInfo hri) {
331       if (regionsToRemove == null) {
332         regionsToRemove = new LinkedList<HRegionInfo>();
333       }
334       regionsToRemove.add(hri);
335     }
336 
337     void addRegionToRestore(final HRegionInfo hri) {
338       if (regionsToRestore == null) {
339         regionsToRestore = new LinkedList<HRegionInfo>();
340       }
341       regionsToRestore.add(hri);
342     }
343 
344     public void updateMetaParentRegions(Connection connection,
345         final List<HRegionInfo> regionInfos) throws IOException {
346       if (regionInfos == null || parentsMap.isEmpty()) return;
347 
348       // Extract region names and offlined regions
349       Map<String, HRegionInfo> regionsByName = new HashMap<String, HRegionInfo>(regionInfos.size());
350       List<HRegionInfo> parentRegions = new LinkedList<>();
351       for (HRegionInfo regionInfo: regionInfos) {
352         if (regionInfo.isSplitParent()) {
353           parentRegions.add(regionInfo);
354         } else {
355           regionsByName.put(regionInfo.getEncodedName(), regionInfo);
356         }
357       }
358 
359       // Update Offline parents
360       for (HRegionInfo regionInfo: parentRegions) {
361         Pair<String, String> daughters = parentsMap.get(regionInfo.getEncodedName());
362         if (daughters == null) {
363           // The snapshot contains an unreferenced region.
364           // It will be removed by the CatalogJanitor.
365           LOG.warn("Skip update of unreferenced offline parent: " + regionInfo);
366           continue;
367         }
368 
369         // One side of the split is already compacted
370         if (daughters.getSecond() == null) {
371           daughters.setSecond(daughters.getFirst());
372         }
373 
374         LOG.debug("Update splits parent " + regionInfo.getEncodedName() + " -> " + daughters);
375         MetaTableAccessor.addRegionToMeta(connection, regionInfo,
376           regionsByName.get(daughters.getFirst()),
377           regionsByName.get(daughters.getSecond()));
378       }
379     }
380   }
381 
382   /**
383    * Remove specified regions from the file-system, using the archiver.
384    */
385   private void removeHdfsRegions(final ThreadPoolExecutor exec, final List<HRegionInfo> regions)
386       throws IOException {
387     if (regions == null || regions.size() == 0) return;
388     ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() {
389       @Override
390       public void editRegion(final HRegionInfo hri) throws IOException {
391         HFileArchiver.archiveRegion(conf, fs, hri);
392       }
393     });
394   }
395 
396   /**
397    * Restore specified regions by restoring content to the snapshot state.
398    */
399   private void restoreHdfsRegions(final ThreadPoolExecutor exec,
400       final Map<String, SnapshotRegionManifest> regionManifests,
401       final List<HRegionInfo> regions) throws IOException {
402     if (regions == null || regions.size() == 0) return;
403     ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() {
404       @Override
405       public void editRegion(final HRegionInfo hri) throws IOException {
406         restoreRegion(hri, regionManifests.get(hri.getEncodedName()));
407       }
408     });
409   }
410 
411   /**
412    * Restore specified mob regions by restoring content to the snapshot state.
413    */
414   private void restoreHdfsMobRegions(final ThreadPoolExecutor exec,
415       final Map<String, SnapshotRegionManifest> regionManifests,
416       final List<HRegionInfo> regions) throws IOException {
417     if (regions == null || regions.size() == 0) return;
418     ModifyRegionUtils.editRegions(exec, regions, new ModifyRegionUtils.RegionEditTask() {
419       @Override
420       public void editRegion(final HRegionInfo hri) throws IOException {
421         restoreMobRegion(hri, regionManifests.get(hri.getEncodedName()));
422       }
423     });
424   }
425 
426   private Map<String, List<SnapshotRegionManifest.StoreFile>> getRegionHFileReferences(
427       final SnapshotRegionManifest manifest) {
428     Map<String, List<SnapshotRegionManifest.StoreFile>> familyMap =
429       new HashMap<String, List<SnapshotRegionManifest.StoreFile>>(manifest.getFamilyFilesCount());
430     for (SnapshotRegionManifest.FamilyFiles familyFiles: manifest.getFamilyFilesList()) {
431       familyMap.put(familyFiles.getFamilyName().toStringUtf8(),
432         new ArrayList<SnapshotRegionManifest.StoreFile>(familyFiles.getStoreFilesList()));
433     }
434     return familyMap;
435   }
436 
437   /**
438    * Restore region by removing files not in the snapshot
439    * and adding the missing ones from the snapshot.
440    */
441   private void restoreRegion(final HRegionInfo regionInfo,
442       final SnapshotRegionManifest regionManifest) throws IOException {
443     restoreRegion(regionInfo, regionManifest, new Path(tableDir, regionInfo.getEncodedName()));
444   }
445 
446   /**
447    * Restore mob region by removing files not in the snapshot
448    * and adding the missing ones from the snapshot.
449    */
450   private void restoreMobRegion(final HRegionInfo regionInfo,
451       final SnapshotRegionManifest regionManifest) throws IOException {
452     if (regionManifest == null) {
453       return;
454     }
455     restoreRegion(regionInfo, regionManifest,
456       MobUtils.getMobRegionPath(conf, tableDesc.getTableName()));
457   }
458 
459   /**
460    * Restore region by removing files not in the snapshot
461    * and adding the missing ones from the snapshot.
462    */
463   private void restoreRegion(final HRegionInfo regionInfo,
464       final SnapshotRegionManifest regionManifest, Path regionDir) throws IOException {
465     Map<String, List<SnapshotRegionManifest.StoreFile>> snapshotFiles =
466                 getRegionHFileReferences(regionManifest);
467 
468     String tableName = tableDesc.getTableName().getNameAsString();
469 
470     // Restore families present in the table
471     for (Path familyDir: FSUtils.getFamilyDirs(fs, regionDir)) {
472       byte[] family = Bytes.toBytes(familyDir.getName());
473       Set<String> familyFiles = getTableRegionFamilyFiles(familyDir);
474       List<SnapshotRegionManifest.StoreFile> snapshotFamilyFiles =
475           snapshotFiles.remove(familyDir.getName());
476       if (snapshotFamilyFiles != null) {
477         List<SnapshotRegionManifest.StoreFile> hfilesToAdd =
478             new ArrayList<SnapshotRegionManifest.StoreFile>();
479         for (SnapshotRegionManifest.StoreFile storeFile: snapshotFamilyFiles) {
480           if (familyFiles.contains(storeFile.getName())) {
481             // HFile already present
482             familyFiles.remove(storeFile.getName());
483           } else {
484             // HFile missing
485             hfilesToAdd.add(storeFile);
486           }
487         }
488 
489         // Remove hfiles not present in the snapshot
490         for (String hfileName: familyFiles) {
491           Path hfile = new Path(familyDir, hfileName);
492           LOG.trace("Removing hfile=" + hfileName +
493             " from region=" + regionInfo.getEncodedName() + " table=" + tableName);
494           HFileArchiver.archiveStoreFile(conf, fs, regionInfo, tableDir, family, hfile);
495         }
496 
497         // Restore Missing files
498         for (SnapshotRegionManifest.StoreFile storeFile: hfilesToAdd) {
499           LOG.debug("Adding HFileLink " + storeFile.getName() +
500             " to region=" + regionInfo.getEncodedName() + " table=" + tableName);
501           restoreStoreFile(familyDir, regionInfo, storeFile, createBackRefs);
502         }
503       } else {
504         // Family doesn't exists in the snapshot
505         LOG.trace("Removing family=" + Bytes.toString(family) +
506           " from region=" + regionInfo.getEncodedName() + " table=" + tableName);
507         HFileArchiver.archiveFamily(fs, conf, regionInfo, tableDir, family);
508         fs.delete(familyDir, true);
509       }
510     }
511 
512     // Add families not present in the table
513     for (Map.Entry<String, List<SnapshotRegionManifest.StoreFile>> familyEntry:
514                                                                       snapshotFiles.entrySet()) {
515       Path familyDir = new Path(regionDir, familyEntry.getKey());
516       if (!fs.mkdirs(familyDir)) {
517         throw new IOException("Unable to create familyDir=" + familyDir);
518       }
519 
520       for (SnapshotRegionManifest.StoreFile storeFile: familyEntry.getValue()) {
521         LOG.trace("Adding HFileLink " + storeFile.getName() + " to table=" + tableName);
522         restoreStoreFile(familyDir, regionInfo, storeFile, createBackRefs);
523       }
524     }
525   }
526 
527   /**
528    * @return The set of files in the specified family directory.
529    */
530   private Set<String> getTableRegionFamilyFiles(final Path familyDir) throws IOException {
531     Set<String> familyFiles = new HashSet<String>();
532 
533     FileStatus[] hfiles = FSUtils.listStatus(fs, familyDir);
534     if (hfiles == null) return familyFiles;
535 
536     for (FileStatus hfileRef: hfiles) {
537       String hfileName = hfileRef.getPath().getName();
538       familyFiles.add(hfileName);
539     }
540 
541     return familyFiles;
542   }
543 
544   /**
545    * Clone specified regions. For each region create a new region
546    * and create a HFileLink for each hfile.
547    */
548   private HRegionInfo[] cloneHdfsRegions(final ThreadPoolExecutor exec,
549       final Map<String, SnapshotRegionManifest> regionManifests,
550       final List<HRegionInfo> regions) throws IOException {
551     if (regions == null || regions.size() == 0) return null;
552 
553     final Map<String, HRegionInfo> snapshotRegions =
554       new HashMap<String, HRegionInfo>(regions.size());
555 
556     // clone region info (change embedded tableName with the new one)
557     HRegionInfo[] clonedRegionsInfo = new HRegionInfo[regions.size()];
558     for (int i = 0; i < clonedRegionsInfo.length; ++i) {
559       // clone the region info from the snapshot region info
560       HRegionInfo snapshotRegionInfo = regions.get(i);
561       clonedRegionsInfo[i] = cloneRegionInfo(snapshotRegionInfo);
562 
563       // add the region name mapping between snapshot and cloned
564       String snapshotRegionName = snapshotRegionInfo.getEncodedName();
565       String clonedRegionName = clonedRegionsInfo[i].getEncodedName();
566       regionsMap.put(Bytes.toBytes(snapshotRegionName), Bytes.toBytes(clonedRegionName));
567       LOG.info("clone region=" + snapshotRegionName + " as " + clonedRegionName);
568 
569       // Add mapping between cloned region name and snapshot region info
570       snapshotRegions.put(clonedRegionName, snapshotRegionInfo);
571     }
572 
573     // create the regions on disk
574     ModifyRegionUtils.createRegions(exec, conf, rootDir, tableDir,
575       tableDesc, clonedRegionsInfo, new ModifyRegionUtils.RegionFillTask() {
576         @Override
577         public void fillRegion(final HRegion region) throws IOException {
578           HRegionInfo snapshotHri = snapshotRegions.get(region.getRegionInfo().getEncodedName());
579           cloneRegion(region, snapshotHri, regionManifests.get(snapshotHri.getEncodedName()));
580         }
581       });
582 
583     return clonedRegionsInfo;
584   }
585 
586   /**
587    * Clone the mob region. For the region create a new region
588    * and create a HFileLink for each hfile.
589    */
590   private void cloneHdfsMobRegion(final Map<String, SnapshotRegionManifest> regionManifests,
591       final HRegionInfo region) throws IOException {
592     // clone region info (change embedded tableName with the new one)
593     Path clonedRegionPath = MobUtils.getMobRegionPath(conf, tableDesc.getTableName());
594     cloneRegion(clonedRegionPath, region, regionManifests.get(region.getEncodedName()));
595   }
596 
597   /**
598    * Clone region directory content from the snapshot info.
599    *
600    * Each region is encoded with the table name, so the cloned region will have
601    * a different region name.
602    *
603    * Instead of copying the hfiles a HFileLink is created.
604    *
605    * @param regionDir {@link Path} cloned dir
606    * @param snapshotRegionInfo
607    */
608   private void cloneRegion(final Path regionDir, final HRegionInfo snapshotRegionInfo,
609       final SnapshotRegionManifest manifest) throws IOException {
610     final String tableName = tableDesc.getTableName().getNameAsString();
611     for (SnapshotRegionManifest.FamilyFiles familyFiles: manifest.getFamilyFilesList()) {
612       Path familyDir = new Path(regionDir, familyFiles.getFamilyName().toStringUtf8());
613       for (SnapshotRegionManifest.StoreFile storeFile: familyFiles.getStoreFilesList()) {
614         LOG.info("Adding HFileLink " + storeFile.getName() + " to table=" + tableName);
615         restoreStoreFile(familyDir, snapshotRegionInfo, storeFile, createBackRefs);
616       }
617     }
618   }
619 
620   /**
621    * Clone region directory content from the snapshot info.
622    *
623    * Each region is encoded with the table name, so the cloned region will have
624    * a different region name.
625    *
626    * Instead of copying the hfiles a HFileLink is created.
627    *
628    * @param region {@link HRegion} cloned
629    * @param snapshotRegionInfo
630    */
631   private void cloneRegion(final HRegion region, final HRegionInfo snapshotRegionInfo,
632       final SnapshotRegionManifest manifest) throws IOException {
633     cloneRegion(new Path(tableDir, region.getRegionInfo().getEncodedName()), snapshotRegionInfo,
634       manifest);
635   }
636 
637   /**
638    * Create a new {@link HFileLink} to reference the store file.
639    * <p>The store file in the snapshot can be a simple hfile, an HFileLink or a reference.
640    * <ul>
641    *   <li>hfile: abc -> table=region-abc
642    *   <li>reference: abc.1234 -> table=region-abc.1234
643    *   <li>hfilelink: table=region-hfile -> table=region-hfile
644    * </ul>
645    * @param familyDir destination directory for the store file
646    * @param regionInfo destination region info for the table
647    * @param storeFile store file name (can be a Reference, HFileLink or simple HFile)
648    * @param createBackRef - Whether back reference should be created. Defaults to true.
649    */
650   private void restoreStoreFile(final Path familyDir, final HRegionInfo regionInfo,
651       final SnapshotRegionManifest.StoreFile storeFile, final boolean createBackRef)
652           throws IOException {
653     String hfileName = storeFile.getName();
654     if (HFileLink.isHFileLink(hfileName)) {
655       HFileLink.createFromHFileLink(conf, fs, familyDir, hfileName, createBackRef);
656     } else if (StoreFileInfo.isReference(hfileName)) {
657       restoreReferenceFile(familyDir, regionInfo, storeFile);
658     } else {
659       HFileLink.create(conf, fs, familyDir, regionInfo, hfileName, createBackRef);
660     }
661   }
662 
663   /**
664    * Create a new {@link Reference} as copy of the source one.
665    * <p><blockquote><pre>
666    * The source table looks like:
667    *    1234/abc      (original file)
668    *    5678/abc.1234 (reference file)
669    *
670    * After the clone operation looks like:
671    *   wxyz/table=1234-abc
672    *   stuv/table=1234-abc.wxyz
673    *
674    * NOTE that the region name in the clone changes (md5 of regioninfo)
675    * and the reference should reflect that change.
676    * </pre></blockquote>
677    * @param familyDir destination directory for the store file
678    * @param regionInfo destination region info for the table
679    * @param hfileName reference file name
680    */
681   private void restoreReferenceFile(final Path familyDir, final HRegionInfo regionInfo,
682       final SnapshotRegionManifest.StoreFile storeFile) throws IOException {
683     String hfileName = storeFile.getName();
684 
685     // Extract the referred information (hfile name and parent region)
686     Path refPath =
687         StoreFileInfo.getReferredToFile(new Path(new Path(new Path(new Path(snapshotTable
688             .getNamespaceAsString(), snapshotTable.getQualifierAsString()), regionInfo
689             .getEncodedName()), familyDir.getName()), hfileName)); 
690     String snapshotRegionName = refPath.getParent().getParent().getName();
691     String fileName = refPath.getName();
692 
693     // The new reference should have the cloned region name as parent, if it is a clone.
694     String clonedRegionName = Bytes.toString(regionsMap.get(Bytes.toBytes(snapshotRegionName)));
695     if (clonedRegionName == null) clonedRegionName = snapshotRegionName;
696 
697     // The output file should be a reference link table=snapshotRegion-fileName.clonedRegionName
698     Path linkPath = null;
699     String refLink = fileName;
700     if (!HFileLink.isHFileLink(fileName)) {
701       refLink = HFileLink.createHFileLinkName(snapshotTable, snapshotRegionName, fileName);
702       linkPath = new Path(familyDir,
703         HFileLink.createHFileLinkName(snapshotTable, regionInfo.getEncodedName(), hfileName));
704     }
705 
706     Path outPath = new Path(familyDir, refLink + '.' + clonedRegionName);
707 
708     // Create the new reference
709     if (storeFile.hasReference()) {
710       Reference reference = Reference.convert(storeFile.getReference());
711       reference.write(fs, outPath);
712     } else {
713       InputStream in;
714       if (linkPath != null) {
715         in = HFileLink.buildFromHFileLinkPattern(conf, linkPath).open(fs);
716       } else {
717         linkPath = new Path(new Path(HRegion.getRegionDir(snapshotManifest.getSnapshotDir(),
718                         regionInfo.getEncodedName()), familyDir.getName()), hfileName);
719         in = fs.open(linkPath);
720       }
721       OutputStream out = fs.create(outPath);
722       IOUtils.copyBytes(in, out, conf);
723     }
724 
725     // Add the daughter region to the map
726     String regionName = Bytes.toString(regionsMap.get(regionInfo.getEncodedNameAsBytes()));
727     LOG.debug("Restore reference " + regionName + " to " + clonedRegionName);
728     synchronized (parentsMap) {
729       Pair<String, String> daughters = parentsMap.get(clonedRegionName);
730       if (daughters == null) {
731         daughters = new Pair<String, String>(regionName, null);
732         parentsMap.put(clonedRegionName, daughters);
733       } else if (!regionName.equals(daughters.getFirst())) {
734         daughters.setSecond(regionName);
735       }
736     }
737   }
738 
739   /**
740    * Create a new {@link HRegionInfo} from the snapshot region info.
741    * Keep the same startKey, endKey, regionId and split information but change
742    * the table name.
743    *
744    * @param snapshotRegionInfo Info for region to clone.
745    * @return the new HRegion instance
746    */
747   public HRegionInfo cloneRegionInfo(final HRegionInfo snapshotRegionInfo) {
748     HRegionInfo regionInfo = new HRegionInfo(tableDesc.getTableName(),
749                       snapshotRegionInfo.getStartKey(), snapshotRegionInfo.getEndKey(),
750                       snapshotRegionInfo.isSplit(), snapshotRegionInfo.getRegionId());
751     regionInfo.setOffline(snapshotRegionInfo.isOffline());
752     return regionInfo;
753   }
754 
755   /**
756    * @return the set of the regions contained in the table
757    */
758   private List<HRegionInfo> getTableRegions() throws IOException {
759     LOG.debug("get table regions: " + tableDir);
760     FileStatus[] regionDirs = FSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs));
761     if (regionDirs == null) return null;
762 
763     List<HRegionInfo> regions = new LinkedList<HRegionInfo>();
764     for (FileStatus regionDir: regionDirs) {
765       HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir.getPath());
766       regions.add(hri);
767     }
768     LOG.debug("found " + regions.size() + " regions for table=" +
769         tableDesc.getTableName().getNameAsString());
770     return regions;
771   }
772 
773   /**
774    * Create a new table descriptor cloning the snapshot table schema.
775    *
776    * @param snapshotTableDescriptor
777    * @param tableName
778    * @return cloned table descriptor
779    * @throws IOException
780    */
781   public static HTableDescriptor cloneTableSchema(final HTableDescriptor snapshotTableDescriptor,
782       final TableName tableName) throws IOException {
783     HTableDescriptor htd = new HTableDescriptor(tableName);
784     for (HColumnDescriptor hcd: snapshotTableDescriptor.getColumnFamilies()) {
785       htd.addFamily(hcd);
786     }
787     for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
788         snapshotTableDescriptor.getValues().entrySet()) {
789       htd.setValue(e.getKey(), e.getValue());
790     }
791     for (Map.Entry<String, String> e: snapshotTableDescriptor.getConfiguration().entrySet()) {
792       htd.setConfiguration(e.getKey(), e.getValue());
793     }
794     return htd;
795   }
796 
797   /**
798    * Copy the snapshot files for a snapshot scanner, discards meta changes.
799    * @param conf
800    * @param fs
801    * @param rootDir
802    * @param restoreDir
803    * @param snapshotName
804    * @throws IOException
805    */
806   public static void copySnapshotForScanner(Configuration conf, FileSystem fs, Path rootDir,
807       Path restoreDir, String snapshotName) throws IOException {
808     // ensure that restore dir is not under root dir
809     if (!restoreDir.getFileSystem(conf).getUri().equals(rootDir.getFileSystem(conf).getUri())) {
810       throw new IllegalArgumentException("Filesystems for restore directory and HBase root directory " +
811           "should be the same");
812     }
813     if (restoreDir.toUri().getPath().startsWith(rootDir.toUri().getPath())) {
814       throw new IllegalArgumentException("Restore directory cannot be a sub directory of HBase " +
815           "root directory. RootDir: " + rootDir + ", restoreDir: " + restoreDir);
816     }
817 
818     Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
819     SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
820     SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc);
821 
822     MonitoredTask status = TaskMonitor.get().createStatus(
823         "Restoring  snapshot '" + snapshotName + "' to directory " + restoreDir);
824     ForeignExceptionDispatcher monitor = new ForeignExceptionDispatcher();
825 
826     // we send createBackRefs=false so that restored hfiles do not create back reference links
827     // in the base hbase root dir.
828     RestoreSnapshotHelper helper = new RestoreSnapshotHelper(conf, fs,
829       manifest, manifest.getTableDescriptor(), restoreDir, monitor, status, false);
830     helper.restoreHdfsRegions(); // TODO: parallelize.
831 
832     if (LOG.isDebugEnabled()) {
833       LOG.debug("Restored table dir:" + restoreDir);
834       FSUtils.logFileSystemState(fs, restoreDir, LOG);
835     }
836   }
837 }