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