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.hbck; 019 020import java.io.IOException; 021import java.time.Instant; 022import java.util.HashMap; 023import java.util.HashSet; 024import java.util.LinkedList; 025import java.util.List; 026import java.util.Map; 027import java.util.Set; 028import org.apache.hadoop.fs.FileSystem; 029import org.apache.hadoop.fs.Path; 030import org.apache.hadoop.hbase.MetaTableAccessor; 031import org.apache.hadoop.hbase.ScheduledChore; 032import org.apache.hadoop.hbase.ServerName; 033import org.apache.hadoop.hbase.client.RegionInfo; 034import org.apache.hadoop.hbase.client.TableState; 035import org.apache.hadoop.hbase.master.MasterServices; 036import org.apache.hadoop.hbase.master.RegionState; 037import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 038import org.apache.hadoop.hbase.util.FSUtils; 039import org.apache.hadoop.hbase.util.HbckRegionInfo; 040import org.apache.hadoop.hbase.util.Pair; 041import org.apache.yetus.audience.InterfaceAudience; 042import org.apache.yetus.audience.InterfaceStability; 043import org.slf4j.Logger; 044import org.slf4j.LoggerFactory; 045 046/** 047 * Used to do the hbck checking job at master side. 048 */ 049@InterfaceAudience.Private 050@InterfaceStability.Evolving 051public class HbckChore extends ScheduledChore { 052 private static final Logger LOG = LoggerFactory.getLogger(HbckChore.class.getName()); 053 054 private static final String HBCK_CHORE_INTERVAL = "hbase.master.hbck.chore.interval"; 055 private static final int DEFAULT_HBCK_CHORE_INTERVAL = 60 * 60 * 1000; 056 057 private final MasterServices master; 058 059 /** 060 * Saved report from last time this chore ran. Check its date. 061 */ 062 private volatile HbckReport lastReport = null; 063 064 /** 065 * When running, the "snapshot" may be changed when this round's checking finish. 066 */ 067 private volatile boolean running = false; 068 069 private boolean disabled = false; 070 071 public HbckChore(MasterServices master) { 072 super("HbckChore-", master, 073 master.getConfiguration().getInt(HBCK_CHORE_INTERVAL, DEFAULT_HBCK_CHORE_INTERVAL)); 074 this.master = master; 075 int interval = 076 master.getConfiguration().getInt(HBCK_CHORE_INTERVAL, DEFAULT_HBCK_CHORE_INTERVAL); 077 if (interval <= 0) { 078 LOG.warn(HBCK_CHORE_INTERVAL + " is <=0 hence disabling hbck chore"); 079 disableChore(); 080 } 081 } 082 083 /** 084 * Returns Returns last published Report that comes of last successful execution of this chore. 085 */ 086 public HbckReport getLastReport() { 087 return lastReport; 088 } 089 090 @Override 091 protected synchronized void chore() { 092 if (isDisabled() || isRunning()) { 093 LOG.warn("hbckChore is either disabled or is already running. Can't run the chore"); 094 return; 095 } 096 running = true; 097 final HbckReport report = new HbckReport(); 098 report.setCheckingStartTimestamp(Instant.ofEpochMilli(EnvironmentEdgeManager.currentTime())); 099 try { 100 loadRegionsFromInMemoryState(report); 101 loadRegionsFromRSReport(report); 102 try { 103 loadRegionsFromFS(scanForMergedParentRegions(), report); 104 } catch (IOException e) { 105 LOG.warn("Failed to load the regions from filesystem", e); 106 } 107 } catch (Throwable t) { 108 LOG.warn("Unexpected", t); 109 } 110 report.setCheckingEndTimestamp(Instant.ofEpochMilli(EnvironmentEdgeManager.currentTime())); 111 this.lastReport = report; 112 running = false; 113 updateAssignmentManagerMetrics(report); 114 } 115 116 /** 117 * Request execution of this chore's action. 118 * @return {@code true} if the chore was executed, {@code false} if the chore is disabled or 119 * already running. 120 */ 121 public boolean runChore() { 122 // This function does the sanity checks of making sure the chore is not run when it is 123 // disabled or when it's already running. It returns whether the chore was actually run or not. 124 if (isDisabled() || isRunning()) { 125 if (isDisabled()) { 126 LOG.warn("hbck chore is disabled! Set " + HBCK_CHORE_INTERVAL + " > 0 to enable it."); 127 } else { 128 LOG.warn("hbck chore already running. Can't run till it finishes."); 129 } 130 return false; 131 } 132 chore(); 133 return true; 134 } 135 136 private void disableChore() { 137 this.disabled = true; 138 } 139 140 public boolean isDisabled() { 141 return this.disabled; 142 } 143 144 /** 145 * Scan hbase:meta to get set of merged parent regions, this is a very heavy scan. 146 * @return Return generated {@link HashSet} 147 */ 148 private HashSet<String> scanForMergedParentRegions() throws IOException { 149 HashSet<String> mergedParentRegions = new HashSet<>(); 150 // Null tablename means scan all of meta. 151 MetaTableAccessor.scanMetaForTableRegions(this.master.getConnection(), r -> { 152 List<RegionInfo> mergeParents = MetaTableAccessor.getMergeRegions(r.rawCells()); 153 if (mergeParents != null) { 154 for (RegionInfo mergeRegion : mergeParents) { 155 if (mergeRegion != null) { 156 // This region is already being merged 157 mergedParentRegions.add(mergeRegion.getEncodedName()); 158 } 159 } 160 } 161 return true; 162 }, null); 163 return mergedParentRegions; 164 } 165 166 private void loadRegionsFromInMemoryState(final HbckReport report) { 167 List<RegionState> regionStates = 168 master.getAssignmentManager().getRegionStates().getRegionStates(); 169 for (RegionState regionState : regionStates) { 170 RegionInfo regionInfo = regionState.getRegion(); 171 if ( 172 master.getTableStateManager().isTableState(regionInfo.getTable(), TableState.State.DISABLED) 173 ) { 174 report.getDisabledTableRegions().add(regionInfo.getRegionNameAsString()); 175 } 176 // Check both state and regioninfo for split status, see HBASE-26383 177 if (regionState.isSplit() || regionInfo.isSplit()) { 178 report.getSplitParentRegions().add(regionInfo.getRegionNameAsString()); 179 } 180 HbckRegionInfo.MetaEntry metaEntry = new HbckRegionInfo.MetaEntry(regionInfo, 181 regionState.getServerName(), regionState.getStamp()); 182 report.getRegionInfoMap().put(regionInfo.getEncodedName(), new HbckRegionInfo(metaEntry)); 183 } 184 LOG.info("Loaded {} regions ({} disabled, {} split parents) from in-memory state", 185 regionStates.size(), report.getDisabledTableRegions().size(), 186 report.getSplitParentRegions().size()); 187 if (LOG.isDebugEnabled()) { 188 Map<RegionState.State, Integer> stateCountMap = new HashMap<>(); 189 for (RegionState regionState : regionStates) { 190 stateCountMap.compute(regionState.getState(), (k, v) -> (v == null) ? 1 : v + 1); 191 } 192 StringBuffer sb = new StringBuffer(); 193 sb.append("Regions by state: "); 194 stateCountMap.entrySet().forEach(e -> { 195 sb.append(e.getKey()); 196 sb.append('='); 197 sb.append(e.getValue()); 198 sb.append(' '); 199 }); 200 LOG.debug(sb.toString()); 201 } 202 if (LOG.isTraceEnabled()) { 203 for (RegionState regionState : regionStates) { 204 LOG.trace("{}: {}, serverName={}", regionState.getRegion(), regionState.getState(), 205 regionState.getServerName()); 206 } 207 } 208 } 209 210 private void loadRegionsFromRSReport(final HbckReport report) { 211 int numRegions = 0; 212 Map<ServerName, Set<byte[]>> rsReports = master.getAssignmentManager().getRSReports(); 213 for (Map.Entry<ServerName, Set<byte[]>> entry : rsReports.entrySet()) { 214 ServerName serverName = entry.getKey(); 215 for (byte[] regionName : entry.getValue()) { 216 String encodedRegionName = RegionInfo.encodeRegionName(regionName); 217 HbckRegionInfo hri = report.getRegionInfoMap().get(encodedRegionName); 218 if (hri == null) { 219 report.getOrphanRegionsOnRS().put(RegionInfo.getRegionNameAsString(regionName), 220 serverName); 221 continue; 222 } 223 hri.addServer(hri.getMetaEntry(), serverName); 224 } 225 numRegions += entry.getValue().size(); 226 } 227 LOG.info("Loaded {} regions from {} regionservers' reports and found {} orphan regions", 228 numRegions, rsReports.size(), report.getOrphanRegionsOnRS().size()); 229 230 for (Map.Entry<String, HbckRegionInfo> entry : report.getRegionInfoMap().entrySet()) { 231 HbckRegionInfo hri = entry.getValue(); 232 ServerName locationInMeta = hri.getMetaEntry().getRegionServer(); // can be null 233 if (hri.getDeployedOn().size() == 0) { 234 // skip the offline region which belong to disabled table. 235 if (report.getDisabledTableRegions().contains(hri.getRegionNameAsString())) { 236 continue; 237 } 238 // skip the split parent regions 239 if (report.getSplitParentRegions().contains(hri.getRegionNameAsString())) { 240 continue; 241 } 242 // Master thought this region opened, but no regionserver reported it. 243 report.getInconsistentRegions().put(hri.getRegionNameAsString(), 244 new Pair<>(locationInMeta, new LinkedList<>())); 245 } else if (hri.getDeployedOn().size() > 1) { 246 // More than one regionserver reported opened this region 247 report.getInconsistentRegions().put(hri.getRegionNameAsString(), 248 new Pair<>(locationInMeta, hri.getDeployedOn())); 249 } else if (!hri.getDeployedOn().get(0).equals(locationInMeta)) { 250 // Master thought this region opened on Server1, but regionserver reported Server2 251 report.getInconsistentRegions().put(hri.getRegionNameAsString(), 252 new Pair<>(locationInMeta, hri.getDeployedOn())); 253 } 254 } 255 } 256 257 private void loadRegionsFromFS(final HashSet<String> mergedParentRegions, final HbckReport report) 258 throws IOException { 259 Path rootDir = master.getMasterFileSystem().getRootDir(); 260 FileSystem fs = master.getMasterFileSystem().getFileSystem(); 261 262 int numRegions = 0; 263 List<Path> tableDirs = FSUtils.getTableDirs(fs, rootDir); 264 for (Path tableDir : tableDirs) { 265 List<Path> regionDirs = FSUtils.getRegionDirs(fs, tableDir); 266 for (Path regionDir : regionDirs) { 267 String encodedRegionName = regionDir.getName(); 268 if (encodedRegionName == null) { 269 LOG.warn("Failed get of encoded name from {}", regionDir); 270 continue; 271 } 272 HbckRegionInfo hri = report.getRegionInfoMap().get(encodedRegionName); 273 // If it is not in in-memory database and not a merged region, 274 // report it as an orphan region. 275 if (hri == null && !mergedParentRegions.contains(encodedRegionName)) { 276 report.getOrphanRegionsOnFS().put(encodedRegionName, regionDir); 277 continue; 278 } 279 } 280 numRegions += regionDirs.size(); 281 } 282 LOG.info("Loaded {} tables {} regions from filesystem and found {} orphan regions", 283 tableDirs.size(), numRegions, report.getOrphanRegionsOnFS().size()); 284 } 285 286 private void updateAssignmentManagerMetrics(final HbckReport report) { 287 master.getAssignmentManager().getAssignmentManagerMetrics() 288 .updateOrphanRegionsOnRs(report.getOrphanRegionsOnRS().size()); 289 master.getAssignmentManager().getAssignmentManagerMetrics() 290 .updateOrphanRegionsOnFs(report.getOrphanRegionsOnFS().size()); 291 master.getAssignmentManager().getAssignmentManagerMetrics() 292 .updateInconsistentRegions(report.getInconsistentRegions().size()); 293 } 294 295 /** 296 * When running, the HBCK report may be changed later. 297 */ 298 public boolean isRunning() { 299 return running; 300 } 301}