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.InputStream;
22  import java.io.IOException;
23  import java.io.OutputStream;
24  import java.util.Arrays;
25  import java.util.HashMap;
26  import java.util.HashSet;
27  import java.util.LinkedList;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.Set;
31  import java.util.TreeMap;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.classification.InterfaceAudience;
36  import org.apache.hadoop.conf.Configuration;
37  import org.apache.hadoop.fs.FileStatus;
38  import org.apache.hadoop.fs.FileSystem;
39  import org.apache.hadoop.fs.Path;
40  import org.apache.hadoop.hbase.TableName;
41  import org.apache.hadoop.hbase.HColumnDescriptor;
42  import org.apache.hadoop.hbase.HRegionInfo;
43  import org.apache.hadoop.hbase.HTableDescriptor;
44  import org.apache.hadoop.hbase.backup.HFileArchiver;
45  import org.apache.hadoop.hbase.catalog.CatalogTracker;
46  import org.apache.hadoop.hbase.catalog.MetaEditor;
47  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
48  import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
49  import org.apache.hadoop.hbase.io.HFileLink;
50  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
51  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
52  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
53  import org.apache.hadoop.hbase.regionserver.HRegion;
54  import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
55  import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
56  import org.apache.hadoop.hbase.util.Bytes;
57  import org.apache.hadoop.hbase.util.FSUtils;
58  import org.apache.hadoop.hbase.util.FSVisitor;
59  import org.apache.hadoop.hbase.util.ModifyRegionUtils;
60  import org.apache.hadoop.hbase.util.Pair;
61  import org.apache.hadoop.io.IOUtils;
62  
63  /**
64   * Helper to Restore/Clone a Snapshot
65   *
66   * <p>The helper assumes that a table is already created, and by calling restore()
67   * the content present in the snapshot will be restored as the new content of the table.
68   *
69   * <p>Clone from Snapshot: If the target table is empty, the restore operation
70   * is just a "clone operation", where the only operations are:
71   * <ul>
72   *  <li>for each region in the snapshot create a new region
73   *    (note that the region will have a different name, since the encoding contains the table name)
74   *  <li>for each file in the region create a new HFileLink to point to the original file.
75   *  <li>restore the logs, if any
76   * </ul>
77   *
78   * <p>Restore from Snapshot:
79   * <ul>
80   *  <li>for each region in the table verify which are available in the snapshot and which are not
81   *    <ul>
82   *    <li>if the region is not present in the snapshot, remove it.
83   *    <li>if the region is present in the snapshot
84   *      <ul>
85   *      <li>for each file in the table region verify which are available in the snapshot
86   *        <ul>
87   *          <li>if the hfile is not present in the snapshot, remove it
88   *          <li>if the hfile is present, keep it (nothing to do)
89   *        </ul>
90   *      <li>for each file in the snapshot region but not in the table
91   *        <ul>
92   *          <li>create a new HFileLink that point to the original file
93   *        </ul>
94   *      </ul>
95   *    </ul>
96   *  <li>for each region in the snapshot not present in the current table state
97   *    <ul>
98   *    <li>create a new region and for each file in the region create a new HFileLink
99   *      (This is the same as the clone operation)
100  *    </ul>
101  *  <li>restore the logs, if any
102  * </ul>
103  */
104 @InterfaceAudience.Private
105 public class RestoreSnapshotHelper {
106   private static final Log LOG = LogFactory.getLog(RestoreSnapshotHelper.class);
107 
108   private final Map<byte[], byte[]> regionsMap =
109         new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
110 
111   private final Map<String, Pair<String, String> > parentsMap =
112       new HashMap<String, Pair<String, String> >();
113 
114   private final ForeignExceptionDispatcher monitor;
115   private final MonitoredTask status;
116 
117   private final SnapshotDescription snapshotDesc;
118   private final TableName snapshotTable;
119   private final Path snapshotDir;
120 
121   private final HTableDescriptor tableDesc;
122   private final Path rootDir;
123   private final Path tableDir;
124 
125   private final Configuration conf;
126   private final FileSystem fs;
127 
128   public RestoreSnapshotHelper(final Configuration conf,
129       final FileSystem fs,
130       final SnapshotDescription snapshotDescription,
131       final Path snapshotDir,
132       final HTableDescriptor tableDescriptor,
133       final Path rootDir,
134       final ForeignExceptionDispatcher monitor,
135       final MonitoredTask status)
136   {
137     this.fs = fs;
138     this.conf = conf;
139     this.snapshotDesc = snapshotDescription;
140     this.snapshotTable = TableName.valueOf(snapshotDescription.getTable());
141     this.snapshotDir = snapshotDir;
142     this.tableDesc = tableDescriptor;
143     this.rootDir = rootDir;
144     this.tableDir = FSUtils.getTableDir(rootDir, tableDesc.getTableName());
145     this.monitor = monitor;
146     this.status = status;
147   }
148 
149   /**
150    * Restore the on-disk table to a specified snapshot state.
151    * @return the set of regions touched by the restore operation
152    */
153   public RestoreMetaChanges restoreHdfsRegions() throws IOException {
154     LOG.debug("starting restore");
155     Set<String> snapshotRegionNames = SnapshotReferenceUtil.getSnapshotRegionNames(fs, snapshotDir);
156     if (snapshotRegionNames == null) {
157       LOG.warn("Nothing to restore. Snapshot " + snapshotDesc + " looks empty");
158       return null;
159     }
160 
161     RestoreMetaChanges metaChanges = new RestoreMetaChanges(parentsMap);
162 
163     // Identify which region are still available and which not.
164     // NOTE: we rely upon the region name as: "table name, start key, end key"
165     List<HRegionInfo> tableRegions = getTableRegions();
166     if (tableRegions != null) {
167       monitor.rethrowException();
168       for (HRegionInfo regionInfo: tableRegions) {
169         String regionName = regionInfo.getEncodedName();
170         if (snapshotRegionNames.contains(regionName)) {
171           LOG.info("region to restore: " + regionName);
172           snapshotRegionNames.remove(regionName);
173           metaChanges.addRegionToRestore(regionInfo);
174         } else {
175           LOG.info("region to remove: " + regionName);
176           metaChanges.addRegionToRemove(regionInfo);
177         }
178       }
179 
180       // Restore regions using the snapshot data
181       monitor.rethrowException();
182       status.setStatus("Restoring table regions...");
183       restoreHdfsRegions(metaChanges.getRegionsToRestore());
184       status.setStatus("Finished restoring all table regions.");
185 
186       // Remove regions from the current table
187       monitor.rethrowException();
188       status.setStatus("Starting to delete excess regions from table");
189       removeHdfsRegions(metaChanges.getRegionsToRemove());
190       status.setStatus("Finished deleting excess regions from table.");
191     }
192 
193     // Regions to Add: present in the snapshot but not in the current table
194     if (snapshotRegionNames.size() > 0) {
195       List<HRegionInfo> regionsToAdd = new LinkedList<HRegionInfo>();
196 
197       monitor.rethrowException();
198       for (String regionName: snapshotRegionNames) {
199         LOG.info("region to add: " + regionName);
200         Path regionDir = new Path(snapshotDir, regionName);
201         regionsToAdd.add(HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir));
202       }
203 
204       // Create new regions cloning from the snapshot
205       monitor.rethrowException();
206       status.setStatus("Cloning regions...");
207       HRegionInfo[] clonedRegions = cloneHdfsRegions(regionsToAdd);
208       metaChanges.setNewRegions(clonedRegions);
209       status.setStatus("Finished cloning regions.");
210     }
211 
212     // Restore WALs
213     monitor.rethrowException();
214     status.setStatus("Restoring WALs to table...");
215     restoreWALs();
216     status.setStatus("Finished restoring WALs to table.");
217 
218     return metaChanges;
219   }
220 
221   /**
222    * Describe the set of operations needed to update hbase:meta after restore.
223    */
224   public static class RestoreMetaChanges {
225     private final Map<String, Pair<String, String> > parentsMap;
226 
227     private List<HRegionInfo> regionsToRestore = null;
228     private List<HRegionInfo> regionsToRemove = null;
229     private List<HRegionInfo> regionsToAdd = null;
230 
231     RestoreMetaChanges(final Map<String, Pair<String, String> > parentsMap) {
232       this.parentsMap = parentsMap;
233     }
234 
235     /**
236      * @return true if there're new regions
237      */
238     public boolean hasRegionsToAdd() {
239       return this.regionsToAdd != null && this.regionsToAdd.size() > 0;
240     }
241 
242     /**
243      * Returns the list of new regions added during the on-disk restore.
244      * The caller is responsible to add the regions to META.
245      * e.g MetaEditor.addRegionsToMeta(...)
246      * @return the list of regions to add to META
247      */
248     public List<HRegionInfo> getRegionsToAdd() {
249       return this.regionsToAdd;
250     }
251 
252     /**
253      * @return true if there're regions to restore
254      */
255     public boolean hasRegionsToRestore() {
256       return this.regionsToRestore != null && this.regionsToRestore.size() > 0;
257     }
258 
259     /**
260      * Returns the list of 'restored regions' during the on-disk restore.
261      * The caller is responsible to add the regions to hbase:meta if not present.
262      * @return the list of regions restored
263      */
264     public List<HRegionInfo> getRegionsToRestore() {
265       return this.regionsToRestore;
266     }
267 
268     /**
269      * @return true if there're regions to remove
270      */
271     public boolean hasRegionsToRemove() {
272       return this.regionsToRemove != null && this.regionsToRemove.size() > 0;
273     }
274 
275     /**
276      * Returns the list of regions removed during the on-disk restore.
277      * The caller is responsible to remove the regions from META.
278      * e.g. MetaEditor.deleteRegions(...)
279      * @return the list of regions to remove from META
280      */
281     public List<HRegionInfo> getRegionsToRemove() {
282       return this.regionsToRemove;
283     }
284 
285     void setNewRegions(final HRegionInfo[] hris) {
286       if (hris != null) {
287         regionsToAdd = Arrays.asList(hris);
288       } else {
289         regionsToAdd = null;
290       }
291     }
292 
293     void addRegionToRemove(final HRegionInfo hri) {
294       if (regionsToRemove == null) {
295         regionsToRemove = new LinkedList<HRegionInfo>();
296       }
297       regionsToRemove.add(hri);
298     }
299 
300     void addRegionToRestore(final HRegionInfo hri) {
301       if (regionsToRestore == null) {
302         regionsToRestore = new LinkedList<HRegionInfo>();
303       }
304       regionsToRestore.add(hri);
305     }
306 
307     public void updateMetaParentRegions(final CatalogTracker catalogTracker,
308         final List<HRegionInfo> regionInfos) throws IOException {
309       if (regionInfos == null || parentsMap.isEmpty()) return;
310 
311       // Extract region names and offlined regions
312       Map<String, HRegionInfo> regionsByName = new HashMap<String, HRegionInfo>(regionInfos.size());
313       List<HRegionInfo> parentRegions = new LinkedList();
314       for (HRegionInfo regionInfo: regionInfos) {
315         if (regionInfo.isSplitParent()) {
316           parentRegions.add(regionInfo);
317         } else {
318           regionsByName.put(regionInfo.getEncodedName(), regionInfo);
319         }
320       }
321 
322       // Update Offline parents
323       for (HRegionInfo regionInfo: parentRegions) {
324         Pair<String, String> daughters = parentsMap.get(regionInfo.getEncodedName());
325         if (daughters == null) {
326           // The snapshot contains an unreferenced region.
327           // It will be removed by the CatalogJanitor.
328           LOG.warn("Skip update of unreferenced offline parent: " + regionInfo);
329           continue;
330         }
331 
332         // One side of the split is already compacted
333         if (daughters.getSecond() == null) {
334           daughters.setSecond(daughters.getFirst());
335         }
336 
337         LOG.debug("Update splits parent " + regionInfo.getEncodedName() + " -> " + daughters);
338         MetaEditor.addRegionToMeta(catalogTracker, regionInfo,
339             regionsByName.get(daughters.getFirst()),
340             regionsByName.get(daughters.getSecond()));
341       }
342     }
343   }
344 
345   /**
346    * Remove specified regions from the file-system, using the archiver.
347    */
348   private void removeHdfsRegions(final List<HRegionInfo> regions) throws IOException {
349     if (regions != null && regions.size() > 0) {
350       for (HRegionInfo hri: regions) {
351         HFileArchiver.archiveRegion(conf, fs, hri);
352       }
353     }
354   }
355 
356   /**
357    * Restore specified regions by restoring content to the snapshot state.
358    */
359   private void restoreHdfsRegions(final List<HRegionInfo> regions) throws IOException {
360     if (regions == null || regions.size() == 0) return;
361     for (HRegionInfo hri: regions) restoreRegion(hri);
362   }
363 
364   /**
365    * Restore region by removing files not in the snapshot
366    * and adding the missing ones from the snapshot.
367    */
368   private void restoreRegion(HRegionInfo regionInfo) throws IOException {
369     Path snapshotRegionDir = new Path(snapshotDir, regionInfo.getEncodedName());
370     Map<String, List<String>> snapshotFiles =
371                 SnapshotReferenceUtil.getRegionHFileReferences(fs, snapshotRegionDir);
372     Path regionDir = new Path(tableDir, regionInfo.getEncodedName());
373     String tableName = tableDesc.getTableName().getNameAsString();
374 
375     // Restore families present in the table
376     for (Path familyDir: FSUtils.getFamilyDirs(fs, regionDir)) {
377       byte[] family = Bytes.toBytes(familyDir.getName());
378       Set<String> familyFiles = getTableRegionFamilyFiles(familyDir);
379       List<String> snapshotFamilyFiles = snapshotFiles.remove(familyDir.getName());
380       if (snapshotFamilyFiles != null) {
381         List<String> hfilesToAdd = new LinkedList<String>();
382         for (String hfileName: snapshotFamilyFiles) {
383           if (familyFiles.contains(hfileName)) {
384             // HFile already present
385             familyFiles.remove(hfileName);
386           } else {
387             // HFile missing
388             hfilesToAdd.add(hfileName);
389           }
390         }
391 
392         // Remove hfiles not present in the snapshot
393         for (String hfileName: familyFiles) {
394           Path hfile = new Path(familyDir, hfileName);
395           LOG.trace("Removing hfile=" + hfile +
396             " from region=" + regionInfo.getEncodedName() + " table=" + tableName);
397           HFileArchiver.archiveStoreFile(conf, fs, regionInfo, tableDir, family, hfile);
398         }
399 
400         // Restore Missing files
401         for (String hfileName: hfilesToAdd) {
402           LOG.trace("Adding HFileLink " + hfileName +
403             " to region=" + regionInfo.getEncodedName() + " table=" + tableName);
404           restoreStoreFile(familyDir, regionInfo, hfileName);
405         }
406       } else {
407         // Family doesn't exists in the snapshot
408         LOG.trace("Removing family=" + Bytes.toString(family) +
409           " from region=" + regionInfo.getEncodedName() + " table=" + tableName);
410         HFileArchiver.archiveFamily(fs, conf, regionInfo, tableDir, family);
411         fs.delete(familyDir, true);
412       }
413     }
414 
415     // Add families not present in the table
416     for (Map.Entry<String, List<String>> familyEntry: snapshotFiles.entrySet()) {
417       Path familyDir = new Path(regionDir, familyEntry.getKey());
418       if (!fs.mkdirs(familyDir)) {
419         throw new IOException("Unable to create familyDir=" + familyDir);
420       }
421 
422       for (String hfileName: familyEntry.getValue()) {
423         LOG.trace("Adding HFileLink " + hfileName + " to table=" + tableName);
424         restoreStoreFile(familyDir, regionInfo, hfileName);
425       }
426     }
427   }
428 
429   /**
430    * @return The set of files in the specified family directory.
431    */
432   private Set<String> getTableRegionFamilyFiles(final Path familyDir) throws IOException {
433     Set<String> familyFiles = new HashSet<String>();
434 
435     FileStatus[] hfiles = FSUtils.listStatus(fs, familyDir);
436     if (hfiles == null) return familyFiles;
437 
438     for (FileStatus hfileRef: hfiles) {
439       String hfileName = hfileRef.getPath().getName();
440       familyFiles.add(hfileName);
441     }
442 
443     return familyFiles;
444   }
445 
446   /**
447    * Clone specified regions. For each region create a new region
448    * and create a HFileLink for each hfile.
449    */
450   private HRegionInfo[] cloneHdfsRegions(final List<HRegionInfo> regions) throws IOException {
451     if (regions == null || regions.size() == 0) return null;
452 
453     final Map<String, HRegionInfo> snapshotRegions =
454       new HashMap<String, HRegionInfo>(regions.size());
455 
456     // clone region info (change embedded tableName with the new one)
457     HRegionInfo[] clonedRegionsInfo = new HRegionInfo[regions.size()];
458     for (int i = 0; i < clonedRegionsInfo.length; ++i) {
459       // clone the region info from the snapshot region info
460       HRegionInfo snapshotRegionInfo = regions.get(i);
461       clonedRegionsInfo[i] = cloneRegionInfo(snapshotRegionInfo);
462 
463       // add the region name mapping between snapshot and cloned
464       String snapshotRegionName = snapshotRegionInfo.getEncodedName();
465       String clonedRegionName = clonedRegionsInfo[i].getEncodedName();
466       regionsMap.put(Bytes.toBytes(snapshotRegionName), Bytes.toBytes(clonedRegionName));
467       LOG.info("clone region=" + snapshotRegionName + " as " + clonedRegionName);
468 
469       // Add mapping between cloned region name and snapshot region info
470       snapshotRegions.put(clonedRegionName, snapshotRegionInfo);
471     }
472 
473     // create the regions on disk
474     ModifyRegionUtils.createRegions(conf, rootDir,
475       tableDesc, clonedRegionsInfo, new ModifyRegionUtils.RegionFillTask() {
476         public void fillRegion(final HRegion region) throws IOException {
477           cloneRegion(region, snapshotRegions.get(region.getRegionInfo().getEncodedName()));
478         }
479       });
480 
481     return clonedRegionsInfo;
482   }
483 
484   /**
485    * Clone region directory content from the snapshot info.
486    *
487    * Each region is encoded with the table name, so the cloned region will have
488    * a different region name.
489    *
490    * Instead of copying the hfiles a HFileLink is created.
491    *
492    * @param region {@link HRegion} cloned
493    * @param snapshotRegionInfo
494    */
495   private void cloneRegion(final HRegion region, final HRegionInfo snapshotRegionInfo)
496       throws IOException {
497     final Path snapshotRegionDir = new Path(snapshotDir, snapshotRegionInfo.getEncodedName());
498     final Path regionDir = new Path(tableDir, region.getRegionInfo().getEncodedName());
499     final String tableName = tableDesc.getTableName().getNameAsString();
500     SnapshotReferenceUtil.visitRegionStoreFiles(fs, snapshotRegionDir,
501       new FSVisitor.StoreFileVisitor() {
502         public void storeFile (final String region, final String family, final String hfile)
503             throws IOException {
504           LOG.info("Adding HFileLink " + hfile + " to table=" + tableName);
505           Path familyDir = new Path(regionDir, family);
506           restoreStoreFile(familyDir, snapshotRegionInfo, hfile);
507         }
508     });
509   }
510 
511   /**
512    * Create a new {@link HFileLink} to reference the store file.
513    * <p>The store file in the snapshot can be a simple hfile, an HFileLink or a reference.
514    * <ul>
515    *   <li>hfile: abc -> table=region-abc
516    *   <li>reference: abc.1234 -> table=region-abc.1234
517    *   <li>hfilelink: table=region-hfile -> table=region-hfile
518    * </ul>
519    * @param familyDir destination directory for the store file
520    * @param regionInfo destination region info for the table
521    * @param hfileName store file name (can be a Reference, HFileLink or simple HFile)
522    */
523   private void restoreStoreFile(final Path familyDir, final HRegionInfo regionInfo,
524       final String hfileName) throws IOException {
525     if (HFileLink.isHFileLink(hfileName)) {
526       HFileLink.createFromHFileLink(conf, fs, familyDir, hfileName);
527     } else if (StoreFileInfo.isReference(hfileName)) {
528       restoreReferenceFile(familyDir, regionInfo, hfileName);
529     } else {
530       HFileLink.create(conf, fs, familyDir, regionInfo, hfileName);
531     }
532   }
533 
534   /**
535    * Create a new {@link Reference} as copy of the source one.
536    * <p><blockquote><pre>
537    * The source table looks like:
538    *    1234/abc      (original file)
539    *    5678/abc.1234 (reference file)
540    *
541    * After the clone operation looks like:
542    *   wxyz/table=1234-abc
543    *   stuv/table=1234-abc.wxyz
544    *
545    * NOTE that the region name in the clone changes (md5 of regioninfo)
546    * and the reference should reflect that change.
547    * </pre></blockquote>
548    * @param familyDir destination directory for the store file
549    * @param regionInfo destination region info for the table
550    * @param hfileName reference file name
551    */
552   private void restoreReferenceFile(final Path familyDir, final HRegionInfo regionInfo,
553       final String hfileName) throws IOException {
554     // Extract the referred information (hfile name and parent region)
555     Path refPath = StoreFileInfo.getReferredToFile(new Path(new Path(new Path(
556         snapshotTable.getNameAsString(), regionInfo.getEncodedName()), familyDir.getName()),
557         hfileName));
558     String snapshotRegionName = refPath.getParent().getParent().getName();
559     String fileName = refPath.getName();
560 
561     // The new reference should have the cloned region name as parent, if it is a clone.
562     String clonedRegionName = Bytes.toString(regionsMap.get(Bytes.toBytes(snapshotRegionName)));
563     if (clonedRegionName == null) clonedRegionName = snapshotRegionName;
564 
565     // The output file should be a reference link table=snapshotRegion-fileName.clonedRegionName
566     Path linkPath = null;
567     String refLink = fileName;
568     if (!HFileLink.isHFileLink(fileName)) {
569       refLink = HFileLink.createHFileLinkName(snapshotTable, snapshotRegionName, fileName);
570       linkPath = new Path(familyDir,
571         HFileLink.createHFileLinkName(snapshotTable, regionInfo.getEncodedName(), hfileName));
572     }
573 
574     Path outPath = new Path(familyDir, refLink + '.' + clonedRegionName);
575 
576     // Create the new reference
577     InputStream in;
578     if (linkPath != null) {
579       in = new HFileLink(conf, linkPath).open(fs);
580     } else {
581       linkPath = new Path(new Path(HRegion.getRegionDir(snapshotDir, regionInfo.getEncodedName()),
582                       familyDir.getName()), hfileName);
583       in = fs.open(linkPath);
584     }
585     OutputStream out = fs.create(outPath);
586     IOUtils.copyBytes(in, out, conf);
587 
588     // Add the daughter region to the map
589     String regionName = Bytes.toString(regionsMap.get(regionInfo.getEncodedNameAsBytes()));
590     LOG.debug("Restore reference " + regionName + " to " + clonedRegionName);
591     synchronized (parentsMap) {
592       Pair<String, String> daughters = parentsMap.get(clonedRegionName);
593       if (daughters == null) {
594         daughters = new Pair<String, String>(regionName, null);
595         parentsMap.put(clonedRegionName, daughters);
596       } else if (!regionName.equals(daughters.getFirst())) {
597         daughters.setSecond(regionName);
598       }
599     }
600   }
601 
602   /**
603    * Create a new {@link HRegionInfo} from the snapshot region info.
604    * Keep the same startKey, endKey, regionId and split information but change
605    * the table name.
606    *
607    * @param snapshotRegionInfo Info for region to clone.
608    * @return the new HRegion instance
609    */
610   public HRegionInfo cloneRegionInfo(final HRegionInfo snapshotRegionInfo) {
611     HRegionInfo regionInfo = new HRegionInfo(tableDesc.getTableName(),
612                       snapshotRegionInfo.getStartKey(), snapshotRegionInfo.getEndKey(),
613                       snapshotRegionInfo.isSplit(), snapshotRegionInfo.getRegionId());
614     regionInfo.setOffline(snapshotRegionInfo.isOffline());
615     return regionInfo;
616   }
617 
618   /**
619    * Restore snapshot WALs.
620    *
621    * Global Snapshot keep a reference to region servers logs present during the snapshot.
622    * (/hbase/.snapshot/snapshotName/.logs/hostName/logName)
623    *
624    * Since each log contains different tables data, logs must be split to
625    * extract the table that we are interested in.
626    */
627   private void restoreWALs() throws IOException {
628     final SnapshotLogSplitter logSplitter = new SnapshotLogSplitter(conf, fs, tableDir,
629         snapshotTable, regionsMap);
630     try {
631       // Recover.Edits
632       SnapshotReferenceUtil.visitRecoveredEdits(fs, snapshotDir,
633           new FSVisitor.RecoveredEditsVisitor() {
634         public void recoveredEdits (final String region, final String logfile) throws IOException {
635           Path path = SnapshotReferenceUtil.getRecoveredEdits(snapshotDir, region, logfile);
636           logSplitter.splitRecoveredEdit(path);
637         }
638       });
639 
640       // Region Server Logs
641       SnapshotReferenceUtil.visitLogFiles(fs, snapshotDir, new FSVisitor.LogFileVisitor() {
642         public void logFile (final String server, final String logfile) throws IOException {
643           logSplitter.splitLog(server, logfile);
644         }
645       });
646     } finally {
647       logSplitter.close();
648     }
649   }
650 
651   /**
652    * @return the set of the regions contained in the table
653    */
654   private List<HRegionInfo> getTableRegions() throws IOException {
655     LOG.debug("get table regions: " + tableDir);
656     FileStatus[] regionDirs = FSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs));
657     if (regionDirs == null) return null;
658 
659     List<HRegionInfo> regions = new LinkedList<HRegionInfo>();
660     for (FileStatus regionDir: regionDirs) {
661       HRegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir.getPath());
662       regions.add(hri);
663     }
664     LOG.debug("found " + regions.size() + " regions for table=" +
665         tableDesc.getTableName().getNameAsString());
666     return regions;
667   }
668 
669   /**
670    * Create a new table descriptor cloning the snapshot table schema.
671    *
672    * @param snapshotTableDescriptor
673    * @param tableName
674    * @return cloned table descriptor
675    * @throws IOException
676    */
677   public static HTableDescriptor cloneTableSchema(final HTableDescriptor snapshotTableDescriptor,
678       final TableName tableName) throws IOException {
679     HTableDescriptor htd = new HTableDescriptor(tableName);
680     for (HColumnDescriptor hcd: snapshotTableDescriptor.getColumnFamilies()) {
681       htd.addFamily(hcd);
682     }
683     for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
684         snapshotTableDescriptor.getValues().entrySet()) {
685       htd.setValue(e.getKey(), e.getValue());
686     }
687     for (Map.Entry<String, String> e: snapshotTableDescriptor.getConfiguration().entrySet()) {
688       htd.setConfiguration(e.getKey(), e.getValue());
689     }
690     return htd;
691   }
692 }