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