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}