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