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.regionserver.handler;
019
020import java.io.IOException;
021import org.apache.hadoop.hbase.HConstants;
022import org.apache.hadoop.hbase.Server;
023import org.apache.hadoop.hbase.ServerName;
024import org.apache.hadoop.hbase.client.RegionInfo;
025import org.apache.hadoop.hbase.executor.EventHandler;
026import org.apache.hadoop.hbase.executor.EventType;
027import org.apache.hadoop.hbase.procedure2.Procedure;
028import org.apache.hadoop.hbase.regionserver.HRegion;
029import org.apache.hadoop.hbase.regionserver.RegionServerServices;
030import org.apache.hadoop.hbase.regionserver.RegionServerServices.RegionStateTransitionContext;
031import org.apache.yetus.audience.InterfaceAudience;
032import org.slf4j.Logger;
033import org.slf4j.LoggerFactory;
034
035import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
036
037/**
038 * Handles closing of a region on a region server.
039 * <p/>
040 * In normal operation, we use {@link UnassignRegionHandler} closing Regions but when shutting down
041 * the region server and closing out Regions, we use this handler instead; it does not expect to be
042 * able to communicate the close back to the Master.
043 * <p>
044 * Expects that the close *has* been registered in the hosting RegionServer before submitting this
045 * Handler; i.e. <code>rss.getRegionsInTransitionInRS().putIfAbsent(
046 * this.regionInfo.getEncodedNameAsBytes(), Boolean.FALSE);</code> has been called first. In here
047 * when done, we do the deregister.
048 * </p>
049 * @see UnassignRegionHandler
050 */
051@InterfaceAudience.Private
052public class CloseRegionHandler extends EventHandler {
053  // NOTE on priorities shutting down. There are none for close. There are some
054  // for open. I think that is right. On shutdown, we want the meta to close
055  // after the user regions have closed. What
056  // about the case where master tells us to shutdown a catalog region and we
057  // have a running queue of user regions to close?
058  private static final Logger LOG = LoggerFactory.getLogger(CloseRegionHandler.class);
059
060  private final RegionServerServices rsServices;
061  private final RegionInfo regionInfo;
062
063  // If true, the hosting server is aborting. Region close process is different
064  // when we are aborting.
065  private final boolean abort;
066  private ServerName destination;
067
068  /**
069   * This method used internally by the RegionServer to close out regions.
070   * @param abort If the regionserver is aborting.
071   */
072  public CloseRegionHandler(final Server server, final RegionServerServices rsServices,
073    final RegionInfo regionInfo, final boolean abort, ServerName destination) {
074    this(server, rsServices, regionInfo, abort, EventType.M_RS_CLOSE_REGION, destination);
075  }
076
077  protected CloseRegionHandler(final Server server, final RegionServerServices rsServices,
078    RegionInfo regionInfo, boolean abort, EventType eventType, ServerName destination) {
079    super(server, eventType);
080    this.server = server;
081    this.rsServices = rsServices;
082    this.regionInfo = regionInfo;
083    this.abort = abort;
084    this.destination = destination;
085  }
086
087  public RegionInfo getRegionInfo() {
088    return regionInfo;
089  }
090
091  @Override
092  public void process() throws IOException {
093    String name = regionInfo.getEncodedName();
094    LOG.trace("Processing close of {}", name);
095    // Check that this region is being served here
096    HRegion region = (HRegion) rsServices.getRegion(name);
097    try {
098      if (region == null) {
099        LOG.warn("Received CLOSE for region {} but currently not serving - ignoring", name);
100        // TODO: do better than a simple warning
101        return;
102      }
103
104      // Close the region
105      if (region.close(abort, false, true) == null) {
106        // This region has already been closed. Should not happen (A unit test makes this
107        // happen as a side effect, TestRegionObserverInterface.testPreWALAppendNotCalledOnMetaEdit)
108        LOG.warn("Can't close {}; already closed", name);
109        return;
110      }
111
112      this.rsServices.removeRegion(region, destination);
113      rsServices.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.CLOSED,
114        HConstants.NO_SEQNUM, Procedure.NO_PROC_ID, -1, regionInfo));
115
116      // Done! Region is closed on this RS
117      LOG.debug("Closed {}", region.getRegionInfo().getRegionNameAsString());
118    } finally {
119      // Clear any reference in getServer().getRegionsInTransitionInRS() on success or failure,
120      // since a reference was added before this CRH was invoked. If we don't clear it, it can
121      // hold up regionserver abort on cluster shutdown. HBASE-23984.
122      this.rsServices.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
123    }
124  }
125
126  @Override
127  protected void handleException(Throwable t) {
128    server.abort("Unrecoverable exception while closing " + this.regionInfo.getRegionNameAsString(),
129      t);
130  }
131}