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}