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.janitor; 019 020import java.io.IOException; 021import java.util.ArrayList; 022import java.util.Collection; 023import java.util.Collections; 024import java.util.HashSet; 025import java.util.List; 026import java.util.Map; 027import java.util.Optional; 028import java.util.Set; 029import java.util.SortedSet; 030import java.util.TreeSet; 031import java.util.stream.Collectors; 032import org.apache.hadoop.hbase.HBaseIOException; 033import org.apache.hadoop.hbase.HConstants; 034import org.apache.hadoop.hbase.MetaTableAccessor; 035import org.apache.hadoop.hbase.TableName; 036import org.apache.hadoop.hbase.client.RegionInfo; 037import org.apache.hadoop.hbase.client.RegionInfoBuilder; 038import org.apache.hadoop.hbase.client.RegionReplicaUtil; 039import org.apache.hadoop.hbase.client.TableDescriptor; 040import org.apache.hadoop.hbase.exceptions.MergeRegionException; 041import org.apache.hadoop.hbase.master.MasterServices; 042import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure; 043import org.apache.hadoop.hbase.replication.ReplicationException; 044import org.apache.hadoop.hbase.util.Bytes; 045import org.apache.hadoop.hbase.util.Pair; 046import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; 047import org.apache.yetus.audience.InterfaceAudience; 048import org.slf4j.Logger; 049import org.slf4j.LoggerFactory; 050 051import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap; 052import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap; 053 054/** 055 * Server-side fixing of bad or inconsistent state in hbase:meta. Distinct from MetaTableAccessor 056 * because {@link MetaTableAccessor} is about low-level manipulations driven by the Master. This 057 * class MetaFixer is employed by the Master and it 'knows' about holes and orphans and encapsulates 058 * their fixing on behalf of the Master. 059 */ 060@InterfaceAudience.Private 061public class MetaFixer { 062 private static final Logger LOG = LoggerFactory.getLogger(MetaFixer.class); 063 private static final String MAX_MERGE_COUNT_KEY = "hbase.master.metafixer.max.merge.count"; 064 private static final int MAX_MERGE_COUNT_DEFAULT = 64; 065 066 private final MasterServices masterServices; 067 /** 068 * Maximum for many regions to merge at a time. 069 */ 070 private final int maxMergeCount; 071 072 public MetaFixer(MasterServices masterServices) { 073 this.masterServices = masterServices; 074 this.maxMergeCount = 075 this.masterServices.getConfiguration().getInt(MAX_MERGE_COUNT_KEY, MAX_MERGE_COUNT_DEFAULT); 076 } 077 078 public void fix() throws IOException { 079 CatalogJanitorReport report = this.masterServices.getCatalogJanitor().getLastReport(); 080 if (report == null) { 081 LOG.info("CatalogJanitor has not generated a report yet; run 'catalogjanitor_run' in " 082 + "shell or wait until CatalogJanitor chore runs."); 083 return; 084 } 085 fixHoles(report); 086 fixOverlaps(report); 087 // Run the ReplicationBarrierCleaner here; it may clear out rep_barrier rows which 088 // can help cleaning up damaged hbase:meta. 089 this.masterServices.runReplicationBarrierCleaner(); 090 } 091 092 /** 093 * If hole, it papers it over by adding a region in the filesystem and to hbase:meta. Does not 094 * assign. 095 */ 096 void fixHoles(CatalogJanitorReport report) { 097 final List<Pair<RegionInfo, RegionInfo>> holes = report.getHoles(); 098 if (holes.isEmpty()) { 099 LOG.info("CatalogJanitor Report contains no holes to fix. Skipping."); 100 return; 101 } 102 103 LOG.info("Identified {} region holes to fix. Detailed fixup progress logged at DEBUG.", 104 holes.size()); 105 106 final List<RegionInfo> newRegionInfos = createRegionInfosForHoles(holes); 107 final List<RegionInfo> newMetaEntries = createMetaEntries(masterServices, newRegionInfos); 108 final TransitRegionStateProcedure[] assignProcedures = 109 masterServices.getAssignmentManager().createRoundRobinAssignProcedures(newMetaEntries); 110 111 masterServices.getMasterProcedureExecutor().submitProcedures(assignProcedures); 112 LOG.info("Scheduled {}/{} new regions for assignment.", assignProcedures.length, holes.size()); 113 } 114 115 /** 116 * Create a new {@link RegionInfo} corresponding to each provided "hole" pair. 117 */ 118 private static List<RegionInfo> 119 createRegionInfosForHoles(final List<Pair<RegionInfo, RegionInfo>> holes) { 120 final List<RegionInfo> newRegionInfos = holes.stream().map(MetaFixer::getHoleCover) 121 .filter(Optional::isPresent).map(Optional::get).collect(Collectors.toList()); 122 LOG.debug("Constructed {}/{} RegionInfo descriptors corresponding to identified holes.", 123 newRegionInfos.size(), holes.size()); 124 return newRegionInfos; 125 } 126 127 /** 128 * @return Attempts to calculate a new {@link RegionInfo} that covers the region range described 129 * in {@code hole}. 130 */ 131 private static Optional<RegionInfo> getHoleCover(Pair<RegionInfo, RegionInfo> hole) { 132 final RegionInfo left = hole.getFirst(); 133 final RegionInfo right = hole.getSecond(); 134 135 if (left.getTable().equals(right.getTable())) { 136 // Simple case. 137 if (Bytes.compareTo(left.getEndKey(), right.getStartKey()) >= 0) { 138 LOG.warn("Skipping hole fix; left-side endKey is not less than right-side startKey;" 139 + " left=<{}>, right=<{}>", left, right); 140 return Optional.empty(); 141 } 142 return Optional.of(buildRegionInfo(left.getTable(), left.getEndKey(), right.getStartKey())); 143 } 144 145 final boolean leftUndefined = left.equals(RegionInfoBuilder.UNDEFINED); 146 final boolean rightUndefined = right.equals(RegionInfoBuilder.UNDEFINED); 147 final boolean last = left.isLast(); 148 final boolean first = right.isFirst(); 149 if (leftUndefined && rightUndefined) { 150 LOG.warn("Skipping hole fix; both the hole left-side and right-side RegionInfos are " 151 + "UNDEFINED; left=<{}>, right=<{}>", left, right); 152 return Optional.empty(); 153 } 154 if (leftUndefined || last) { 155 return Optional 156 .of(buildRegionInfo(right.getTable(), HConstants.EMPTY_START_ROW, right.getStartKey())); 157 } 158 if (rightUndefined || first) { 159 return Optional 160 .of(buildRegionInfo(left.getTable(), left.getEndKey(), HConstants.EMPTY_END_ROW)); 161 } 162 LOG.warn("Skipping hole fix; don't know what to do with left=<{}>, right=<{}>", left, right); 163 return Optional.empty(); 164 } 165 166 private static RegionInfo buildRegionInfo(TableName tn, byte[] start, byte[] end) { 167 return RegionInfoBuilder.newBuilder(tn).setStartKey(start).setEndKey(end).build(); 168 } 169 170 /** 171 * Create entries in the {@code hbase:meta} for each provided {@link RegionInfo}. Best effort. 172 * @param masterServices used to connect to {@code hbase:meta} 173 * @param newRegionInfos the new {@link RegionInfo} entries to add to the filesystem 174 * @return a list of {@link RegionInfo} entries for which {@code hbase:meta} entries were 175 * successfully created 176 */ 177 private static List<RegionInfo> createMetaEntries(final MasterServices masterServices, 178 final List<RegionInfo> newRegionInfos) { 179 180 final List<Either<List<RegionInfo>, IOException>> addMetaEntriesResults = 181 newRegionInfos.stream().map(regionInfo -> { 182 try { 183 TableDescriptor td = masterServices.getTableDescriptors().get(regionInfo.getTable()); 184 185 // Add replicas if needed 186 // we need to create regions with replicaIds starting from 1 187 List<RegionInfo> newRegions = RegionReplicaUtil 188 .addReplicas(Collections.singletonList(regionInfo), 1, td.getRegionReplication()); 189 190 // Add regions to META 191 MetaTableAccessor.addRegionsToMeta(masterServices.getConnection(), newRegions, 192 td.getRegionReplication()); 193 194 // Setup replication for region replicas if needed 195 if (td.getRegionReplication() > 1) { 196 ServerRegionReplicaUtil.setupRegionReplicaReplication(masterServices); 197 } 198 return Either.<List<RegionInfo>, IOException> ofLeft(newRegions); 199 } catch (IOException e) { 200 return Either.<List<RegionInfo>, IOException> ofRight(e); 201 } catch (ReplicationException e) { 202 return Either.<List<RegionInfo>, IOException> ofRight(new HBaseIOException(e)); 203 } 204 }).collect(Collectors.toList()); 205 final List<RegionInfo> createMetaEntriesSuccesses = 206 addMetaEntriesResults.stream().filter(Either::hasLeft).map(Either::getLeft) 207 .flatMap(List::stream).collect(Collectors.toList()); 208 final List<IOException> createMetaEntriesFailures = addMetaEntriesResults.stream() 209 .filter(Either::hasRight).map(Either::getRight).collect(Collectors.toList()); 210 LOG.debug("Added {}/{} entries to hbase:meta", createMetaEntriesSuccesses.size(), 211 newRegionInfos.size()); 212 213 if (!createMetaEntriesFailures.isEmpty()) { 214 LOG.warn( 215 "Failed to create entries in hbase:meta for {}/{} RegionInfo descriptors. First" 216 + " failure message included; full list of failures with accompanying stack traces is" 217 + " available at log level DEBUG. message={}", 218 createMetaEntriesFailures.size(), addMetaEntriesResults.size(), 219 createMetaEntriesFailures.get(0).getMessage()); 220 if (LOG.isDebugEnabled()) { 221 createMetaEntriesFailures 222 .forEach(ioe -> LOG.debug("Attempt to fix region hole in hbase:meta failed.", ioe)); 223 } 224 } 225 226 return createMetaEntriesSuccesses; 227 } 228 229 /** 230 * Fix overlaps noted in CJ consistency report. 231 */ 232 List<Long> fixOverlaps(CatalogJanitorReport report) throws IOException { 233 List<Long> pidList = new ArrayList<>(); 234 for (Set<RegionInfo> regions : calculateMerges(maxMergeCount, report.getOverlaps())) { 235 RegionInfo[] regionsArray = regions.toArray(new RegionInfo[] {}); 236 try { 237 pidList.add(this.masterServices.mergeRegions(regionsArray, true, HConstants.NO_NONCE, 238 HConstants.NO_NONCE)); 239 } catch (MergeRegionException mre) { 240 LOG.warn("Failed overlap fix of {}", regionsArray, mre); 241 } 242 } 243 return pidList; 244 } 245 246 /** 247 * Run through <code>overlaps</code> and return a list of merges to run. Presumes overlaps are 248 * ordered (which they are coming out of the CatalogJanitor consistency report). 249 * @param maxMergeCount Maximum regions to merge at a time (avoid merging 100k regions in one go!) 250 */ 251 static List<SortedSet<RegionInfo>> calculateMerges(int maxMergeCount, 252 List<Pair<RegionInfo, RegionInfo>> overlaps) { 253 if (overlaps.isEmpty()) { 254 LOG.debug("No overlaps."); 255 return Collections.emptyList(); 256 } 257 List<SortedSet<RegionInfo>> merges = new ArrayList<>(); 258 // First group overlaps by table then calculate merge table by table. 259 ListMultimap<TableName, Pair<RegionInfo, RegionInfo>> overlapGroups = 260 ArrayListMultimap.create(); 261 for (Pair<RegionInfo, RegionInfo> pair : overlaps) { 262 overlapGroups.put(pair.getFirst().getTable(), pair); 263 } 264 for (Map.Entry<TableName, Collection<Pair<RegionInfo, RegionInfo>>> entry : overlapGroups 265 .asMap().entrySet()) { 266 calculateTableMerges(maxMergeCount, merges, entry.getValue()); 267 } 268 return merges; 269 } 270 271 private static void calculateTableMerges(int maxMergeCount, List<SortedSet<RegionInfo>> merges, 272 Collection<Pair<RegionInfo, RegionInfo>> overlaps) { 273 SortedSet<RegionInfo> currentMergeSet = new TreeSet<>(); 274 HashSet<RegionInfo> regionsInMergeSet = new HashSet<>(); 275 RegionInfo regionInfoWithlargestEndKey = null; 276 for (Pair<RegionInfo, RegionInfo> pair : overlaps) { 277 if (regionInfoWithlargestEndKey != null) { 278 if ( 279 !isOverlap(regionInfoWithlargestEndKey, pair) || currentMergeSet.size() >= maxMergeCount 280 ) { 281 // Log when we cut-off-merge because we hit the configured maximum merge limit. 282 if (currentMergeSet.size() >= maxMergeCount) { 283 LOG.warn("Ran into maximum-at-a-time merges limit={}", maxMergeCount); 284 } 285 286 // In the case of the merge set contains only 1 region or empty, it does not need to 287 // submit this merge request as no merge is going to happen. currentMergeSet can be 288 // reused in this case. 289 if (currentMergeSet.size() <= 1) { 290 for (RegionInfo ri : currentMergeSet) { 291 regionsInMergeSet.remove(ri); 292 } 293 currentMergeSet.clear(); 294 } else { 295 merges.add(currentMergeSet); 296 currentMergeSet = new TreeSet<>(); 297 } 298 } 299 } 300 301 // Do not add the same region into multiple merge set, this will fail 302 // the second merge request. 303 if (!regionsInMergeSet.contains(pair.getFirst())) { 304 currentMergeSet.add(pair.getFirst()); 305 regionsInMergeSet.add(pair.getFirst()); 306 } 307 if (!regionsInMergeSet.contains(pair.getSecond())) { 308 currentMergeSet.add(pair.getSecond()); 309 regionsInMergeSet.add(pair.getSecond()); 310 } 311 312 regionInfoWithlargestEndKey = getRegionInfoWithLargestEndKey( 313 getRegionInfoWithLargestEndKey(pair.getFirst(), pair.getSecond()), 314 regionInfoWithlargestEndKey); 315 } 316 merges.add(currentMergeSet); 317 } 318 319 /** 320 * @return Either <code>a</code> or <code>b</code>, whichever has the endkey that is furthest 321 * along in the Table. 322 */ 323 static RegionInfo getRegionInfoWithLargestEndKey(RegionInfo a, RegionInfo b) { 324 if (a == null) { 325 // b may be null. 326 return b; 327 } 328 if (b == null) { 329 // Both are null. The return is not-defined. 330 return a; 331 } 332 if (!a.getTable().equals(b.getTable())) { 333 // This is an odd one. This should be the right answer. 334 return b; 335 } 336 if (a.isLast()) { 337 return a; 338 } 339 if (b.isLast()) { 340 return b; 341 } 342 int compare = Bytes.compareTo(a.getEndKey(), b.getEndKey()); 343 return compare == 0 || compare > 0 ? a : b; 344 } 345 346 /** 347 * @return True if an overlap found between passed in <code>ri</code> and the <code>pair</code>. 348 * Does NOT check the pairs themselves overlap. 349 */ 350 static boolean isOverlap(RegionInfo ri, Pair<RegionInfo, RegionInfo> pair) { 351 if (ri == null || pair == null) { 352 // Can't be an overlap in either of these cases. 353 return false; 354 } 355 return ri.isOverlap(pair.getFirst()) || ri.isOverlap(pair.getSecond()); 356 } 357 358 /** 359 * A union over {@link L} and {@link R}. 360 */ 361 private static class Either<L, R> { 362 private final L left; 363 private final R right; 364 365 public static <L, R> Either<L, R> ofLeft(L left) { 366 return new Either<>(left, null); 367 } 368 369 public static <L, R> Either<L, R> ofRight(R right) { 370 return new Either<>(null, right); 371 } 372 373 Either(L left, R right) { 374 this.left = left; 375 this.right = right; 376 } 377 378 public boolean hasLeft() { 379 return left != null; 380 } 381 382 public L getLeft() { 383 if (!hasLeft()) { 384 throw new IllegalStateException("Either contains no left."); 385 } 386 return left; 387 } 388 389 public boolean hasRight() { 390 return right != null; 391 } 392 393 public R getRight() { 394 if (!hasRight()) { 395 throw new IllegalStateException("Either contains no right."); 396 } 397 return right; 398 } 399 } 400}