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.procedure;
019
020import java.io.IOException;
021import java.util.ArrayList;
022import java.util.List;
023import java.util.stream.Collectors;
024import org.apache.hadoop.hbase.HRegionLocation;
025import org.apache.hadoop.hbase.MetaTableAccessor;
026import org.apache.hadoop.hbase.RegionLocations;
027import org.apache.hadoop.hbase.ServerName;
028import org.apache.hadoop.hbase.client.Connection;
029import org.apache.hadoop.hbase.client.RegionInfo;
030import org.apache.hadoop.hbase.client.Result;
031import org.apache.hadoop.hbase.master.RegionState;
032import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
033import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
034import org.apache.yetus.audience.InterfaceAudience;
035import org.slf4j.Logger;
036import org.slf4j.LoggerFactory;
037
038/**
039 * Acts like the super class in all cases except when no Regions found in the current Master
040 * in-memory context. In this latter case, when the call to super#getRegionsOnCrashedServer returns
041 * nothing, this SCP will scan hbase:meta for references to the passed ServerName. If any found,
042 * we'll clean them up.
043 * <p>
044 * This version of SCP is for external invocation as part of fix-up (e.g. HBCK2's
045 * scheduleRecoveries); the super class is used during normal recovery operations. It is for the
046 * case where meta has references to 'Unknown Servers', servers that are in hbase:meta but not in
047 * live-server or dead-server lists; i.e. Master and hbase:meta content have deviated. It should
048 * never happen in normal running cluster but if we do drop accounting of servers, we need a means
049 * of fix-up. Eventually, as part of normal CatalogJanitor task, rather than just identify these
050 * 'Unknown Servers', it would make repair, queuing something like this HBCKSCP to do cleanup,
051 * reassigning them so Master and hbase:meta are aligned again.
052 * <p>
053 * NOTE that this SCP is costly to run; does a full scan of hbase:meta.
054 * </p>
055 */
056@InterfaceAudience.Private
057public class HBCKServerCrashProcedure extends ServerCrashProcedure {
058  private static final Logger LOG = LoggerFactory.getLogger(HBCKServerCrashProcedure.class);
059
060  /**
061   * @param serverName     Name of the crashed server.
062   * @param shouldSplitWal True if we should split WALs as part of crashed server processing.
063   * @param carryingMeta   True if carrying hbase:meta table region.
064   */
065  public HBCKServerCrashProcedure(final MasterProcedureEnv env, final ServerName serverName,
066    final boolean shouldSplitWal, final boolean carryingMeta) {
067    super(env, serverName, shouldSplitWal, carryingMeta);
068  }
069
070  /**
071   * Used when deserializing from a procedure store; we'll construct one of these then call
072   * #deserializeStateData(InputStream). Do not use directly.
073   */
074  public HBCKServerCrashProcedure() {
075  }
076
077  /**
078   * If no Regions found in Master context, then we will search hbase:meta for references to the
079   * passed server. Operator may have passed ServerName because they have found references to
080   * 'Unknown Servers'. They are using HBCKSCP to clear them out.
081   */
082  @Override
083  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NP_NULL_ON_SOME_PATH_EXCEPTION",
084      justification = "FindBugs seems confused on ps in below.")
085  List<RegionInfo> getRegionsOnCrashedServer(MasterProcedureEnv env) {
086    // Super will return an immutable list (empty if nothing on this server).
087    List<RegionInfo> ris = super.getRegionsOnCrashedServer(env);
088    if (!ris.isEmpty()) {
089      return ris;
090    }
091    // Nothing in in-master context. Check for Unknown Server! in hbase:meta.
092    // If super list is empty, then allow that an operator scheduled an SCP because they are trying
093    // to purge 'Unknown Servers' -- servers that are neither online nor in dead servers
094    // list but that ARE in hbase:meta and so showing as unknown in places like 'HBCK Report'.
095    // This mis-accounting does not happen in normal circumstance but may arise in-extremis
096    // when cluster has been damaged in operation.
097    UnknownServerVisitor visitor =
098      new UnknownServerVisitor(env.getMasterServices().getConnection(), getServerName());
099    try {
100      MetaTableAccessor.scanMetaForTableRegions(env.getMasterServices().getConnection(), visitor,
101        null);
102    } catch (IOException ioe) {
103      LOG.warn("Failed scan of hbase:meta for 'Unknown Servers'", ioe);
104      return ris;
105    }
106    LOG.info("Found {} mentions of {} in hbase:meta of OPEN/OPENING Regions: {}",
107      visitor.getReassigns().size(), getServerName(), visitor.getReassigns().stream()
108        .map(RegionInfo::getEncodedName).collect(Collectors.joining(",")));
109    return visitor.getReassigns();
110  }
111
112  /**
113   * Visitor for hbase:meta that 'fixes' Unknown Server issues. Collects a List of Regions to
114   * reassign as 'result'.
115   */
116  private static class UnknownServerVisitor implements MetaTableAccessor.Visitor {
117    private final List<RegionInfo> reassigns = new ArrayList<>();
118    private final ServerName unknownServerName;
119    private final Connection connection;
120
121    private UnknownServerVisitor(Connection connection, ServerName unknownServerName) {
122      this.connection = connection;
123      this.unknownServerName = unknownServerName;
124    }
125
126    @Override
127    public boolean visit(Result result) throws IOException {
128      RegionLocations rls = MetaTableAccessor.getRegionLocations(result);
129      if (rls == null) {
130        return true;
131      }
132      for (HRegionLocation hrl : rls.getRegionLocations()) {
133        if (hrl == null) {
134          continue;
135        }
136        if (hrl.getRegion() == null) {
137          continue;
138        }
139        if (hrl.getServerName() == null) {
140          continue;
141        }
142        if (!hrl.getServerName().equals(this.unknownServerName)) {
143          continue;
144        }
145        RegionState.State state = RegionStateStore.getRegionState(result, hrl.getRegion());
146        RegionState rs = new RegionState(hrl.getRegion(), state, hrl.getServerName());
147        if (rs.isClosing()) {
148          // Move region to CLOSED in hbase:meta.
149          LOG.info("Moving {} from CLOSING to CLOSED in hbase:meta",
150            hrl.getRegion().getRegionNameAsString());
151          try {
152            MetaTableAccessor.updateRegionState(this.connection, hrl.getRegion(),
153              RegionState.State.CLOSED);
154          } catch (IOException ioe) {
155            LOG.warn("Failed moving {} from CLOSING to CLOSED",
156              hrl.getRegion().getRegionNameAsString(), ioe);
157          }
158        } else if (rs.isOpening() || rs.isOpened()) {
159          this.reassigns.add(hrl.getRegion());
160        } else {
161          LOG.info("Passing {}", rs);
162        }
163      }
164      return true;
165    }
166
167    private List<RegionInfo> getReassigns() {
168      return this.reassigns;
169    }
170  }
171
172  /**
173   * The RegionStateNode will not have a location if a confirm of an OPEN fails. On fail, the
174   * RegionStateNode regionLocation is set to null. This is 'looser' than the test done in the
175   * superclass. The HBCKSCP has been scheduled by an operator via hbck2 probably at the behest of a
176   * report of an 'Unknown Server' in the 'HBCK Report'. Let the operators operation succeed even in
177   * case where the region location in the RegionStateNode is null.
178   */
179  @Override
180  protected boolean isMatchingRegionLocation(RegionStateNode rsn) {
181    return super.isMatchingRegionLocation(rsn) || rsn.getRegionLocation() == null;
182  }
183}