001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.snapshot;
020
021import java.io.FileNotFoundException;
022import java.io.IOException;
023import java.io.InterruptedIOException;
024import java.util.ArrayList;
025import java.util.Collection;
026import java.util.HashMap;
027import java.util.List;
028import java.util.Map;
029import java.util.concurrent.ExecutionException;
030import java.util.concurrent.ExecutorCompletionService;
031import java.util.concurrent.ThreadPoolExecutor;
032import java.util.concurrent.TimeUnit;
033import org.apache.hadoop.conf.Configuration;
034import org.apache.hadoop.fs.FSDataInputStream;
035import org.apache.hadoop.fs.FSDataOutputStream;
036import org.apache.hadoop.fs.FileStatus;
037import org.apache.hadoop.fs.FileSystem;
038import org.apache.hadoop.fs.Path;
039import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
040import org.apache.hadoop.hbase.client.RegionInfo;
041import org.apache.hadoop.hbase.client.TableDescriptor;
042import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
043import org.apache.hadoop.hbase.mob.MobUtils;
044import org.apache.hadoop.hbase.monitoring.MonitoredTask;
045import org.apache.hadoop.hbase.regionserver.HRegion;
046import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
047import org.apache.hadoop.hbase.regionserver.HStore;
048import org.apache.hadoop.hbase.regionserver.HStoreFile;
049import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
050import org.apache.hadoop.hbase.util.Bytes;
051import org.apache.hadoop.hbase.util.CommonFSUtils;
052import org.apache.hadoop.hbase.util.FSTableDescriptors;
053import org.apache.hadoop.hbase.util.Threads;
054import org.apache.yetus.audience.InterfaceAudience;
055import org.slf4j.Logger;
056import org.slf4j.LoggerFactory;
057
058import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
059import org.apache.hbase.thirdparty.com.google.protobuf.CodedInputStream;
060import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
061
062import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
063import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
064import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
065import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
066
067/**
068 * Utility class to help read/write the Snapshot Manifest.
069 *
070 * The snapshot format is transparent for the users of this class,
071 * once the snapshot is written, it will never be modified.
072 * On open() the snapshot will be loaded to the current in-memory format.
073 */
074@InterfaceAudience.Private
075public final class SnapshotManifest {
076  private static final Logger LOG = LoggerFactory.getLogger(SnapshotManifest.class);
077
078  public static final String SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY = "snapshot.manifest.size.limit";
079
080  public static final String DATA_MANIFEST_NAME = "data.manifest";
081
082  private List<SnapshotRegionManifest> regionManifests;
083  private SnapshotDescription desc;
084  private TableDescriptor htd;
085
086  private final ForeignExceptionSnare monitor;
087  private final Configuration conf;
088  private final Path workingDir;
089  private final FileSystem rootFs;
090  private final FileSystem workingDirFs;
091  private int manifestSizeLimit;
092  private final MonitoredTask statusTask;
093
094  /**
095   *
096   * @param conf configuration file for HBase setup
097   * @param rootFs root filesystem containing HFiles
098   * @param workingDir file path of where the manifest should be located
099   * @param desc description of snapshot being taken
100   * @param monitor monitor of foreign exceptions
101   * @throws IOException if the working directory file system cannot be
102   *                     determined from the config file
103   */
104  private SnapshotManifest(final Configuration conf, final FileSystem rootFs,
105      final Path workingDir, final SnapshotDescription desc,
106      final ForeignExceptionSnare monitor, final MonitoredTask statusTask) throws IOException {
107    this.monitor = monitor;
108    this.desc = desc;
109    this.workingDir = workingDir;
110    this.conf = conf;
111    this.rootFs = rootFs;
112    this.statusTask = statusTask;
113    this.workingDirFs = this.workingDir.getFileSystem(this.conf);
114    this.manifestSizeLimit = conf.getInt(SNAPSHOT_MANIFEST_SIZE_LIMIT_CONF_KEY, 64 * 1024 * 1024);
115  }
116
117  /**
118   * Return a SnapshotManifest instance, used for writing a snapshot.
119   *
120   * There are two usage pattern:
121   *  - The Master will create a manifest, add the descriptor, offline regions
122   *    and consolidate the snapshot by writing all the pending stuff on-disk.
123   *      manifest = SnapshotManifest.create(...)
124   *      manifest.addRegion(tableDir, hri)
125   *      manifest.consolidate()
126   *  - The RegionServer will create a single region manifest
127   *      manifest = SnapshotManifest.create(...)
128   *      manifest.addRegion(region)
129   */
130  public static SnapshotManifest create(final Configuration conf, final FileSystem fs,
131      final Path workingDir, final SnapshotDescription desc,
132      final ForeignExceptionSnare monitor) throws IOException {
133    return create(conf, fs, workingDir, desc, monitor, null);
134
135  }
136
137  public static SnapshotManifest create(final Configuration conf, final FileSystem fs,
138      final Path workingDir, final SnapshotDescription desc, final ForeignExceptionSnare monitor,
139      final MonitoredTask statusTask) throws IOException {
140    return new SnapshotManifest(conf, fs, workingDir, desc, monitor, statusTask);
141
142  }
143
144  /**
145   * Return a SnapshotManifest instance with the information already loaded in-memory.
146   *    SnapshotManifest manifest = SnapshotManifest.open(...)
147   *    TableDescriptor htd = manifest.getTableDescriptor()
148   *    for (SnapshotRegionManifest regionManifest: manifest.getRegionManifests())
149   *      hri = regionManifest.getRegionInfo()
150   *      for (regionManifest.getFamilyFiles())
151   *        ...
152   */
153  public static SnapshotManifest open(final Configuration conf, final FileSystem fs,
154      final Path workingDir, final SnapshotDescription desc) throws IOException {
155    SnapshotManifest manifest = new SnapshotManifest(conf, fs, workingDir, desc, null, null);
156    manifest.load();
157    return manifest;
158  }
159
160
161  /**
162   * Add the table descriptor to the snapshot manifest
163   */
164  public void addTableDescriptor(final TableDescriptor htd) throws IOException {
165    this.htd = htd;
166  }
167
168  interface RegionVisitor<TRegion, TFamily> {
169    TRegion regionOpen(final RegionInfo regionInfo) throws IOException;
170    void regionClose(final TRegion region) throws IOException;
171
172    TFamily familyOpen(final TRegion region, final byte[] familyName) throws IOException;
173    void familyClose(final TRegion region, final TFamily family) throws IOException;
174
175    void storeFile(final TRegion region, final TFamily family, final StoreFileInfo storeFile)
176      throws IOException;
177  }
178
179  private RegionVisitor createRegionVisitor(final SnapshotDescription desc) throws IOException {
180    switch (getSnapshotFormat(desc)) {
181      case SnapshotManifestV1.DESCRIPTOR_VERSION:
182        return new SnapshotManifestV1.ManifestBuilder(conf, rootFs, workingDir);
183      case SnapshotManifestV2.DESCRIPTOR_VERSION:
184        return new SnapshotManifestV2.ManifestBuilder(conf, rootFs, workingDir);
185      default:
186      throw new CorruptedSnapshotException("Invalid Snapshot version: " + desc.getVersion(),
187        ProtobufUtil.createSnapshotDesc(desc));
188    }
189  }
190
191  public void addMobRegion(RegionInfo regionInfo) throws IOException {
192    // Get the ManifestBuilder/RegionVisitor
193    RegionVisitor visitor = createRegionVisitor(desc);
194
195    // Visit the region and add it to the manifest
196    addMobRegion(regionInfo, visitor);
197  }
198
199  protected void addMobRegion(RegionInfo regionInfo, RegionVisitor visitor) throws IOException {
200    // 1. dump region meta info into the snapshot directory
201    final String snapshotName = desc.getName();
202    LOG.debug("Storing mob region '" + regionInfo + "' region-info for snapshot=" + snapshotName);
203    Object regionData = visitor.regionOpen(regionInfo);
204    monitor.rethrowException();
205
206    // 2. iterate through all the stores in the region
207    LOG.debug("Creating references for mob files");
208
209    Path mobRegionPath = MobUtils.getMobRegionPath(conf, regionInfo.getTable());
210    for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
211      // 2.1. build the snapshot reference for the store if it's a mob store
212      if (!hcd.isMobEnabled()) {
213        continue;
214      }
215      Object familyData = visitor.familyOpen(regionData, hcd.getName());
216      monitor.rethrowException();
217
218      Path storePath = MobUtils.getMobFamilyPath(mobRegionPath, hcd.getNameAsString());
219      List<StoreFileInfo> storeFiles = getStoreFiles(storePath);
220      if (storeFiles == null) {
221        if (LOG.isDebugEnabled()) {
222          LOG.debug("No mob files under family: " + hcd.getNameAsString());
223        }
224        continue;
225      }
226
227      addReferenceFiles(visitor, regionData, familyData, storeFiles, true);
228
229      visitor.familyClose(regionData, familyData);
230    }
231    visitor.regionClose(regionData);
232  }
233
234  /**
235   * Creates a 'manifest' for the specified region, by reading directly from the HRegion object.
236   * This is used by the "online snapshot" when the table is enabled.
237   */
238  public void addRegion(final HRegion region) throws IOException {
239    // Get the ManifestBuilder/RegionVisitor
240    RegionVisitor visitor = createRegionVisitor(desc);
241
242    // Visit the region and add it to the manifest
243    addRegion(region, visitor);
244  }
245
246  protected void addRegion(final HRegion region, RegionVisitor visitor) throws IOException {
247    // 1. dump region meta info into the snapshot directory
248    final String snapshotName = desc.getName();
249    LOG.debug("Storing '" + region + "' region-info for snapshot=" + snapshotName);
250    Object regionData = visitor.regionOpen(region.getRegionInfo());
251    monitor.rethrowException();
252
253    // 2. iterate through all the stores in the region
254    LOG.debug("Creating references for hfiles");
255
256    for (HStore store : region.getStores()) {
257      // 2.1. build the snapshot reference for the store
258      Object familyData = visitor.familyOpen(regionData,
259          store.getColumnFamilyDescriptor().getName());
260      monitor.rethrowException();
261
262      List<HStoreFile> storeFiles = new ArrayList<>(store.getStorefiles());
263      if (LOG.isDebugEnabled()) {
264        LOG.debug("Adding snapshot references for " + storeFiles  + " hfiles");
265      }
266
267      // 2.2. iterate through all the store's files and create "references".
268      for (int i = 0, sz = storeFiles.size(); i < sz; i++) {
269        HStoreFile storeFile = storeFiles.get(i);
270        monitor.rethrowException();
271
272        // create "reference" to this store file.
273        LOG.debug("Adding reference for file (" + (i+1) + "/" + sz + "): " + storeFile.getPath() +
274                " for snapshot=" + snapshotName);
275        visitor.storeFile(regionData, familyData, storeFile.getFileInfo());
276      }
277      visitor.familyClose(regionData, familyData);
278    }
279    visitor.regionClose(regionData);
280  }
281
282  /**
283   * Creates a 'manifest' for the specified region, by reading directly from the disk.
284   * This is used by the "offline snapshot" when the table is disabled.
285   */
286  public void addRegion(final Path tableDir, final RegionInfo regionInfo) throws IOException {
287    // Get the ManifestBuilder/RegionVisitor
288    RegionVisitor visitor = createRegionVisitor(desc);
289
290    // Visit the region and add it to the manifest
291    addRegion(tableDir, regionInfo, visitor);
292  }
293
294  protected void addRegion(final Path tableDir, final RegionInfo regionInfo, RegionVisitor visitor)
295      throws IOException {
296    boolean isMobRegion = MobUtils.isMobRegionInfo(regionInfo);
297    try {
298      Path baseDir = tableDir;
299      // Open the RegionFS
300      if (isMobRegion) {
301        baseDir = CommonFSUtils.getTableDir(MobUtils.getMobHome(conf), regionInfo.getTable());
302      }
303      HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, rootFs,
304        baseDir, regionInfo, true);
305      monitor.rethrowException();
306
307      // 1. dump region meta info into the snapshot directory
308      LOG.debug("Storing region-info for snapshot.");
309      Object regionData = visitor.regionOpen(regionInfo);
310      monitor.rethrowException();
311
312      // 2. iterate through all the stores in the region
313      LOG.debug("Creating references for hfiles");
314
315      // This ensures that we have an atomic view of the directory as long as we have < ls limit
316      // (batch size of the files in a directory) on the namenode. Otherwise, we get back the files
317      // in batches and may miss files being added/deleted. This could be more robust (iteratively
318      // checking to see if we have all the files until we are sure), but the limit is currently
319      // 1000 files/batch, far more than the number of store files under a single column family.
320      Collection<String> familyNames = regionFs.getFamilies();
321      if (familyNames != null) {
322        for (String familyName: familyNames) {
323          Object familyData = visitor.familyOpen(regionData, Bytes.toBytes(familyName));
324          monitor.rethrowException();
325
326          Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(familyName);
327          if (storeFiles == null) {
328            if (LOG.isDebugEnabled()) {
329              LOG.debug("No files under family: " + familyName);
330            }
331            continue;
332          }
333
334          // 2.1. build the snapshot reference for the store
335          // iterate through all the store's files and create "references".
336          addReferenceFiles(visitor, regionData, familyData, storeFiles, false);
337
338          visitor.familyClose(regionData, familyData);
339        }
340      }
341      visitor.regionClose(regionData);
342    } catch (IOException e) {
343      // the mob directory might not be created yet, so do nothing when it is a mob region
344      if (!isMobRegion) {
345        throw e;
346      }
347    }
348  }
349
350  private List<StoreFileInfo> getStoreFiles(Path storeDir) throws IOException {
351    FileStatus[] stats = CommonFSUtils.listStatus(rootFs, storeDir);
352    if (stats == null) return null;
353
354    ArrayList<StoreFileInfo> storeFiles = new ArrayList<>(stats.length);
355    for (int i = 0; i < stats.length; ++i) {
356      storeFiles.add(new StoreFileInfo(conf, rootFs, stats[i]));
357    }
358    return storeFiles;
359  }
360
361  private void addReferenceFiles(RegionVisitor visitor, Object regionData, Object familyData,
362      Collection<StoreFileInfo> storeFiles, boolean isMob) throws IOException {
363    final String fileType = isMob ? "mob file" : "hfile";
364
365    if (LOG.isDebugEnabled()) {
366      LOG.debug(String.format("Adding snapshot references for %s %ss", storeFiles, fileType));
367    }
368
369    int i = 0;
370    int sz = storeFiles.size();
371    for (StoreFileInfo storeFile: storeFiles) {
372      monitor.rethrowException();
373
374      LOG.debug(String.format("Adding reference for %s (%d/%d): %s",
375          fileType, ++i, sz, storeFile.getPath()));
376
377      // create "reference" to this store file.
378      visitor.storeFile(regionData, familyData, storeFile);
379    }
380  }
381
382  /**
383   * Load the information in the SnapshotManifest. Called by SnapshotManifest.open()
384   *
385   * If the format is v2 and there is no data-manifest, means that we are loading an
386   * in-progress snapshot. Since we support rolling-upgrades, we loook for v1 and v2
387   * regions format.
388   */
389  private void load() throws IOException {
390    switch (getSnapshotFormat(desc)) {
391      case SnapshotManifestV1.DESCRIPTOR_VERSION: {
392        this.htd = FSTableDescriptors.getTableDescriptorFromFs(workingDirFs, workingDir);
393        ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader");
394        try {
395          this.regionManifests =
396            SnapshotManifestV1.loadRegionManifests(conf, tpool, rootFs, workingDir, desc);
397        } finally {
398          tpool.shutdown();
399        }
400        break;
401      }
402      case SnapshotManifestV2.DESCRIPTOR_VERSION: {
403        SnapshotDataManifest dataManifest = readDataManifest();
404        if (dataManifest != null) {
405          htd = ProtobufUtil.toTableDescriptor(dataManifest.getTableSchema());
406          regionManifests = dataManifest.getRegionManifestsList();
407        } else {
408          // Compatibility, load the v1 regions
409          // This happens only when the snapshot is in-progress and the cache wants to refresh.
410          List<SnapshotRegionManifest> v1Regions, v2Regions;
411          ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader");
412          try {
413            v1Regions = SnapshotManifestV1.loadRegionManifests(conf, tpool, rootFs,
414                workingDir, desc);
415            v2Regions = SnapshotManifestV2.loadRegionManifests(conf, tpool, rootFs,
416                workingDir, desc, manifestSizeLimit);
417          } catch (InvalidProtocolBufferException e) {
418            throw new CorruptedSnapshotException("unable to parse region manifest " +
419                e.getMessage(), e);
420          } finally {
421            tpool.shutdown();
422          }
423          if (v1Regions != null && v2Regions != null) {
424            regionManifests = new ArrayList<>(v1Regions.size() + v2Regions.size());
425            regionManifests.addAll(v1Regions);
426            regionManifests.addAll(v2Regions);
427          } else if (v1Regions != null) {
428            regionManifests = v1Regions;
429          } else /* if (v2Regions != null) */ {
430            regionManifests = v2Regions;
431          }
432        }
433        break;
434      }
435      default:
436      throw new CorruptedSnapshotException("Invalid Snapshot version: " + desc.getVersion(),
437        ProtobufUtil.createSnapshotDesc(desc));
438    }
439  }
440
441  /**
442   * Get the current snapshot working dir
443   */
444  public Path getSnapshotDir() {
445    return this.workingDir;
446  }
447
448  /**
449   * Get the SnapshotDescription
450   */
451  public SnapshotDescription getSnapshotDescription() {
452    return this.desc;
453  }
454
455  /**
456   * Get the table descriptor from the Snapshot
457   */
458  public TableDescriptor getTableDescriptor() {
459    return this.htd;
460  }
461
462  /**
463   * Get all the Region Manifest from the snapshot
464   */
465  public List<SnapshotRegionManifest> getRegionManifests() {
466    return this.regionManifests;
467  }
468
469  private void setStatusMsg(String msg) {
470    if (this.statusTask != null) {
471      statusTask.setStatus(msg);
472    }
473  }
474
475  /**
476   * Get all the Region Manifest from the snapshot.
477   * This is an helper to get a map with the region encoded name
478   */
479  public Map<String, SnapshotRegionManifest> getRegionManifestsMap() {
480    if (regionManifests == null || regionManifests.isEmpty()) return null;
481
482    HashMap<String, SnapshotRegionManifest> regionsMap = new HashMap<>(regionManifests.size());
483    for (SnapshotRegionManifest manifest: regionManifests) {
484      String regionName = getRegionNameFromManifest(manifest);
485      regionsMap.put(regionName, manifest);
486    }
487    return regionsMap;
488  }
489
490  public void consolidate() throws IOException {
491    if (getSnapshotFormat(desc) == SnapshotManifestV1.DESCRIPTOR_VERSION) {
492      LOG.info("Using old Snapshot Format");
493      // write a copy of descriptor to the snapshot directory
494      FSTableDescriptors.createTableDescriptorForTableDirectory(workingDirFs, workingDir, htd,
495          false);
496    } else {
497      LOG.debug("Convert to Single Snapshot Manifest for {}", this.desc.getName());
498      convertToV2SingleManifest();
499    }
500  }
501
502  /*
503   * In case of rolling-upgrade, we try to read all the formats and build
504   * the snapshot with the latest format.
505   */
506  private void convertToV2SingleManifest() throws IOException {
507    // Try to load v1 and v2 regions
508    List<SnapshotRegionManifest> v1Regions, v2Regions;
509    ThreadPoolExecutor tpool = createExecutor("SnapshotManifestLoader");
510    setStatusMsg("Loading Region manifests for " + this.desc.getName());
511    try {
512      v1Regions = SnapshotManifestV1.loadRegionManifests(conf, tpool, workingDirFs,
513          workingDir, desc);
514      v2Regions = SnapshotManifestV2.loadRegionManifests(conf, tpool, workingDirFs,
515          workingDir, desc, manifestSizeLimit);
516
517      SnapshotDataManifest.Builder dataManifestBuilder = SnapshotDataManifest.newBuilder();
518      dataManifestBuilder.setTableSchema(ProtobufUtil.toTableSchema(htd));
519
520      if (v1Regions != null && v1Regions.size() > 0) {
521        dataManifestBuilder.addAllRegionManifests(v1Regions);
522      }
523      if (v2Regions != null && v2Regions.size() > 0) {
524        dataManifestBuilder.addAllRegionManifests(v2Regions);
525      }
526
527      // Write the v2 Data Manifest.
528      // Once the data-manifest is written, the snapshot can be considered complete.
529      // Currently snapshots are written in a "temporary" directory and later
530      // moved to the "complated" snapshot directory.
531      setStatusMsg("Writing data manifest for " + this.desc.getName());
532      SnapshotDataManifest dataManifest = dataManifestBuilder.build();
533      writeDataManifest(dataManifest);
534      this.regionManifests = dataManifest.getRegionManifestsList();
535
536      // Remove the region manifests. Everything is now in the data-manifest.
537      // The delete operation is "relaxed", unless we get an exception we keep going.
538      // The extra files in the snapshot directory will not give any problem,
539      // since they have the same content as the data manifest, and even by re-reading
540      // them we will get the same information.
541      int totalDeletes = 0;
542      ExecutorCompletionService<Void> completionService = new ExecutorCompletionService<>(tpool);
543      if (v1Regions != null) {
544        for (SnapshotRegionManifest regionManifest: v1Regions) {
545          ++totalDeletes;
546          completionService.submit(() -> {
547            SnapshotManifestV1.deleteRegionManifest(workingDirFs, workingDir, regionManifest);
548            return null;
549          });
550        }
551      }
552      if (v2Regions != null) {
553        for (SnapshotRegionManifest regionManifest: v2Regions) {
554          ++totalDeletes;
555          completionService.submit(() -> {
556            SnapshotManifestV2.deleteRegionManifest(workingDirFs, workingDir, regionManifest);
557            return null;
558          });
559        }
560      }
561      // Wait for the deletes to finish.
562      for (int i = 0; i < totalDeletes; i++) {
563        try {
564          completionService.take().get();
565        } catch (InterruptedException ie) {
566          throw new InterruptedIOException(ie.getMessage());
567        } catch (ExecutionException e) {
568          throw new IOException("Error deleting region manifests", e.getCause());
569        }
570      }
571    } finally {
572      tpool.shutdown();
573    }
574  }
575
576  /*
577   * Write the SnapshotDataManifest file
578   */
579  private void writeDataManifest(final SnapshotDataManifest manifest)
580      throws IOException {
581    try (FSDataOutputStream stream = workingDirFs.create(new Path(workingDir, DATA_MANIFEST_NAME))) {
582      manifest.writeTo(stream);
583    }
584  }
585
586  /*
587   * Read the SnapshotDataManifest file
588   */
589  private SnapshotDataManifest readDataManifest() throws IOException {
590    try (FSDataInputStream in = workingDirFs.open(new Path(workingDir, DATA_MANIFEST_NAME))) {
591      CodedInputStream cin = CodedInputStream.newInstance(in);
592      cin.setSizeLimit(manifestSizeLimit);
593      return SnapshotDataManifest.parseFrom(cin);
594    } catch (FileNotFoundException e) {
595      return null;
596    } catch (InvalidProtocolBufferException e) {
597      throw new CorruptedSnapshotException("unable to parse data manifest " + e.getMessage(), e);
598    }
599  }
600
601  private ThreadPoolExecutor createExecutor(final String name) {
602    return createExecutor(conf, name);
603  }
604
605  public static ThreadPoolExecutor createExecutor(final Configuration conf, final String name) {
606    int maxThreads = conf.getInt("hbase.snapshot.thread.pool.max", 8);
607    return Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
608      new ThreadFactoryBuilder().setNameFormat(name + "-pool-%d").setDaemon(true)
609        .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build());
610  }
611
612  /**
613   * Extract the region encoded name from the region manifest
614   */
615  static String getRegionNameFromManifest(final SnapshotRegionManifest manifest) {
616    byte[] regionName = RegionInfo.createRegionName(
617            ProtobufUtil.toTableName(manifest.getRegionInfo().getTableName()),
618            manifest.getRegionInfo().getStartKey().toByteArray(),
619            manifest.getRegionInfo().getRegionId(), true);
620    return RegionInfo.encodeRegionName(regionName);
621  }
622
623  /*
624   * Return the snapshot format
625   */
626  private static int getSnapshotFormat(final SnapshotDescription desc) {
627    return desc.hasVersion() ? desc.getVersion() : SnapshotManifestV1.DESCRIPTOR_VERSION;
628  }
629}