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}