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