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 */
018package org.apache.hadoop.hbase.master.snapshot;
019
020import java.io.IOException;
021import java.util.List;
022import java.util.Map;
023import java.util.Set;
024import org.apache.hadoop.fs.FileSystem;
025import org.apache.hadoop.fs.Path;
026import org.apache.hadoop.hbase.MetaTableAccessor;
027import org.apache.hadoop.hbase.TableName;
028import org.apache.hadoop.hbase.client.RegionInfo;
029import org.apache.hadoop.hbase.client.RegionReplicaUtil;
030import org.apache.hadoop.hbase.client.TableDescriptor;
031import org.apache.hadoop.hbase.master.MasterServices;
032import org.apache.hadoop.hbase.mob.MobUtils;
033import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
034import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
035import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
036import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
037import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
038import org.apache.hadoop.hbase.util.FSUtils;
039import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
040import org.apache.yetus.audience.InterfaceAudience;
041import org.apache.yetus.audience.InterfaceStability;
042import org.slf4j.Logger;
043import org.slf4j.LoggerFactory;
044
045import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
046import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
047import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
048
049/**
050 * General snapshot verification on the master.
051 * <p>
052 * This is a light-weight verification mechanism for all the files in a snapshot. It doesn't
053 * attempt to verify that the files are exact copies (that would be paramount to taking the
054 * snapshot again!), but instead just attempts to ensure that the files match the expected
055 * files and are the same length.
056 * <p>
057 * Taking an online snapshots can race against other operations and this is an last line of
058 * defense.  For example, if meta changes between when snapshots are taken not all regions of a
059 * table may be present.  This can be caused by a region split (daughters present on this scan,
060 * but snapshot took parent), or move (snapshots only checks lists of region servers, a move could
061 * have caused a region to be skipped or done twice).
062 * <p>
063 * Current snapshot files checked:
064 * <ol>
065 * <li>SnapshotDescription is readable</li>
066 * <li>Table info is readable</li>
067 * <li>Regions</li>
068 * </ol>
069 * <ul>
070 * <li>Matching regions in the snapshot as currently in the table</li>
071 * <li>{@link RegionInfo} matches the current and stored regions</li>
072 * <li>All referenced hfiles have valid names</li>
073 * <li>All the hfiles are present (either in .archive directory in the region)</li>
074 * <li>All recovered.edits files are present (by name) and have the correct file size</li>
075 * </ul>
076 */
077@InterfaceAudience.Private
078@InterfaceStability.Unstable
079public final class MasterSnapshotVerifier {
080  private static final Logger LOG = LoggerFactory.getLogger(MasterSnapshotVerifier.class);
081
082  private SnapshotDescription snapshot;
083  private FileSystem workingDirFs;
084  private TableName tableName;
085  private MasterServices services;
086
087  /**
088   * @param services services for the master
089   * @param snapshot snapshot to check
090   * @param workingDirFs the file system containing the temporary snapshot information
091   */
092  public MasterSnapshotVerifier(MasterServices services,
093      SnapshotDescription snapshot, FileSystem workingDirFs) {
094    this.workingDirFs = workingDirFs;
095    this.services = services;
096    this.snapshot = snapshot;
097    this.tableName = TableName.valueOf(snapshot.getTable());
098  }
099
100  /**
101   * Verify that the snapshot in the directory is a valid snapshot
102   * @param snapshotDir snapshot directory to check
103   * @param snapshotServers {@link org.apache.hadoop.hbase.ServerName} of the servers
104   *        that are involved in the snapshot
105   * @throws CorruptedSnapshotException if the snapshot is invalid
106   * @throws IOException if there is an unexpected connection issue to the filesystem
107   */
108  public void verifySnapshot(Path snapshotDir, Set<String> snapshotServers)
109      throws CorruptedSnapshotException, IOException {
110    SnapshotManifest manifest = SnapshotManifest.open(services.getConfiguration(), workingDirFs,
111                                                      snapshotDir, snapshot);
112    // verify snapshot info matches
113    verifySnapshotDescription(snapshotDir);
114
115    // check that tableinfo is a valid table description
116    verifyTableInfo(manifest);
117
118    // check that each region is valid
119    verifyRegions(manifest);
120  }
121
122  /**
123   * Check that the snapshot description written in the filesystem matches the current snapshot
124   * @param snapshotDir snapshot directory to check
125   */
126  private void verifySnapshotDescription(Path snapshotDir) throws CorruptedSnapshotException {
127    SnapshotDescription found = SnapshotDescriptionUtils.readSnapshotInfo(workingDirFs,
128        snapshotDir);
129    if (!this.snapshot.equals(found)) {
130      throw new CorruptedSnapshotException(
131          "Snapshot read (" + found + ") doesn't equal snapshot we ran (" + snapshot + ").",
132          ProtobufUtil.createSnapshotDesc(snapshot));
133    }
134  }
135
136  /**
137   * Check that the table descriptor for the snapshot is a valid table descriptor
138   * @param manifest snapshot manifest to inspect
139   */
140  private void verifyTableInfo(final SnapshotManifest manifest) throws IOException {
141    TableDescriptor htd = manifest.getTableDescriptor();
142    if (htd == null) {
143      throw new CorruptedSnapshotException("Missing Table Descriptor",
144        ProtobufUtil.createSnapshotDesc(snapshot));
145    }
146
147    if (!htd.getTableName().getNameAsString().equals(snapshot.getTable())) {
148      throw new CorruptedSnapshotException(
149          "Invalid Table Descriptor. Expected " + snapshot.getTable() + " name, got "
150              + htd.getTableName().getNameAsString(), ProtobufUtil.createSnapshotDesc(snapshot));
151    }
152  }
153
154  /**
155   * Check that all the regions in the snapshot are valid, and accounted for.
156   * @param manifest snapshot manifest to inspect
157   * @throws IOException if we can't reach hbase:meta or read the files from the FS
158   */
159  private void verifyRegions(final SnapshotManifest manifest) throws IOException {
160    List<RegionInfo> regions;
161    if (TableName.META_TABLE_NAME.equals(tableName)) {
162      regions = MetaTableLocator.getMetaRegions(services.getZooKeeper());
163    } else {
164      regions = MetaTableAccessor.getTableRegions(services.getConnection(), tableName);
165    }
166    // Remove the non-default regions
167    RegionReplicaUtil.removeNonDefaultRegions(regions);
168
169    Map<String, SnapshotRegionManifest> regionManifests = manifest.getRegionManifestsMap();
170    if (regionManifests == null) {
171      String msg = "Snapshot " + ClientSnapshotDescriptionUtils.toString(snapshot) + " looks empty";
172      LOG.error(msg);
173      throw new CorruptedSnapshotException(msg);
174    }
175
176    String errorMsg = "";
177    boolean hasMobStore = false;
178    // the mob region is a dummy region, it's not a real region in HBase.
179    // the mob region has a special name, it could be found by the region name.
180    if (regionManifests.get(MobUtils.getMobRegionInfo(tableName).getEncodedName()) != null) {
181      hasMobStore = true;
182    }
183    int realRegionCount = hasMobStore ? regionManifests.size() - 1 : regionManifests.size();
184    if (realRegionCount != regions.size()) {
185      errorMsg = "Regions moved during the snapshot '" +
186                   ClientSnapshotDescriptionUtils.toString(snapshot) + "'. expected=" +
187                   regions.size() + " snapshotted=" + realRegionCount + ".";
188      LOG.error(errorMsg);
189    }
190
191    // Verify RegionInfo
192    for (RegionInfo region : regions) {
193      SnapshotRegionManifest regionManifest = regionManifests.get(region.getEncodedName());
194      if (regionManifest == null) {
195        // could happen due to a move or split race.
196        String mesg = " No snapshot region directory found for region:" + region;
197        if (errorMsg.isEmpty()) errorMsg = mesg;
198        LOG.error(mesg);
199        continue;
200      }
201
202      verifyRegionInfo(region, regionManifest);
203    }
204
205    if (!errorMsg.isEmpty()) {
206      throw new CorruptedSnapshotException(errorMsg);
207    }
208
209    // Verify Snapshot HFiles
210    // Requires the root directory file system as HFiles are stored in the root directory
211    SnapshotReferenceUtil.verifySnapshot(services.getConfiguration(),
212        FSUtils.getRootDirFileSystem(services.getConfiguration()), manifest);
213  }
214
215  /**
216   * Verify that the regionInfo is valid
217   * @param region the region to check
218   * @param manifest snapshot manifest to inspect
219   */
220  private void verifyRegionInfo(final RegionInfo region,
221      final SnapshotRegionManifest manifest) throws IOException {
222    RegionInfo manifestRegionInfo = ProtobufUtil.toRegionInfo(manifest.getRegionInfo());
223    if (RegionInfo.COMPARATOR.compare(region, manifestRegionInfo) != 0) {
224      String msg = "Manifest region info " + manifestRegionInfo +
225                   "doesn't match expected region:" + region;
226      throw new CorruptedSnapshotException(msg, ProtobufUtil.createSnapshotDesc(snapshot));
227    }
228  }
229}