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