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 edu.umd.cs.findbugs.annotations.Nullable;
021import java.io.IOException;
022import java.util.concurrent.TimeUnit;
023import org.apache.hadoop.conf.Configuration;
024import org.apache.hadoop.hbase.HConstants;
025import org.apache.hadoop.hbase.client.RegionInfo;
026import org.apache.hadoop.hbase.client.TableDescriptor;
027import org.apache.hadoop.hbase.executor.EventHandler;
028import org.apache.hadoop.hbase.executor.EventType;
029import org.apache.hadoop.hbase.regionserver.HRegion;
030import org.apache.hadoop.hbase.regionserver.HRegionServer;
031import org.apache.hadoop.hbase.regionserver.Region;
032import org.apache.hadoop.hbase.regionserver.RegionServerServices.PostOpenDeployContext;
033import org.apache.hadoop.hbase.regionserver.RegionServerServices.RegionStateTransitionContext;
034import org.apache.hadoop.hbase.util.RetryCounter;
035import org.apache.yetus.audience.InterfaceAudience;
036import org.slf4j.Logger;
037import org.slf4j.LoggerFactory;
038import org.slf4j.MDC;
039
040import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
041
042/**
043 * Handles opening of a region on a region server.
044 * <p/>
045 * Just done the same thing with the old {@link OpenRegionHandler}, with some modifications on
046 * fencing and retrying. But we need to keep the {@link OpenRegionHandler} as is to keep compatible
047 * with the zk less assignment for 1.x, otherwise it is not possible to do rolling upgrade.
048 */
049@InterfaceAudience.Private
050public class AssignRegionHandler extends EventHandler {
051
052  private static final Logger LOG = LoggerFactory.getLogger(AssignRegionHandler.class);
053
054  private final RegionInfo regionInfo;
055
056  private final long openProcId;
057
058  private final TableDescriptor tableDesc;
059
060  private final long masterSystemTime;
061
062  // active time of the master that sent this assign request, used for fencing
063  private final long initiatingMasterActiveTime;
064
065  private final RetryCounter retryCounter;
066
067  public AssignRegionHandler(HRegionServer server, RegionInfo regionInfo, long openProcId,
068    @Nullable TableDescriptor tableDesc, long masterSystemTime, long initiatingMasterActiveTime,
069    EventType eventType) {
070    super(server, eventType);
071    this.regionInfo = regionInfo;
072    this.openProcId = openProcId;
073    this.tableDesc = tableDesc;
074    this.masterSystemTime = masterSystemTime;
075    this.initiatingMasterActiveTime = initiatingMasterActiveTime;
076    this.retryCounter = HandlerUtil.getRetryCounter();
077  }
078
079  private HRegionServer getServer() {
080    return (HRegionServer) server;
081  }
082
083  private void cleanUpAndReportFailure(IOException error) throws IOException {
084    LOG.warn("Failed to open region {}, will report to master", regionInfo.getRegionNameAsString(),
085      error);
086    HRegionServer rs = getServer();
087    rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes(), Boolean.TRUE);
088    if (
089      !rs.reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.FAILED_OPEN,
090        HConstants.NO_SEQNUM, openProcId, masterSystemTime, regionInfo, initiatingMasterActiveTime))
091    ) {
092      throw new IOException(
093        "Failed to report failed open to master: " + regionInfo.getRegionNameAsString());
094    }
095  }
096
097  @Override
098  public void process() throws IOException {
099    MDC.put("pid", Long.toString(openProcId));
100    HRegionServer rs = getServer();
101    String encodedName = regionInfo.getEncodedName();
102    byte[] encodedNameBytes = regionInfo.getEncodedNameAsBytes();
103    String regionName = regionInfo.getRegionNameAsString();
104    Region onlineRegion = rs.getRegion(encodedName);
105    if (onlineRegion != null) {
106      LOG.warn("Received OPEN for {} which is already online", regionName);
107      // Just follow the old behavior, do we need to call reportRegionStateTransition? Maybe not?
108      // For normal case, it could happen that the rpc call to schedule this handler is succeeded,
109      // but before returning to master the connection is broken. And when master tries again, we
110      // have already finished the opening. For this case we do not need to call
111      // reportRegionStateTransition any more.
112      return;
113    }
114    Boolean previous = rs.getRegionsInTransitionInRS().putIfAbsent(encodedNameBytes, Boolean.TRUE);
115    if (previous != null) {
116      if (previous) {
117        // The region is opening and this maybe a retry on the rpc call, it is safe to ignore it.
118        LOG.info("Receiving OPEN for {} which we are already trying to OPEN"
119          + " - ignoring this new request for this region.", regionName);
120      } else {
121        // The region is closing. This is possible as we will update the region state to CLOSED when
122        // calling reportRegionStateTransition, so the HMaster will think the region is offline,
123        // before we actually close the region, as reportRegionStateTransition is part of the
124        // closing process.
125        long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
126        LOG.info("Receiving OPEN for {} which we are trying to close, try again after {}ms",
127          regionName, backoff);
128        rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
129      }
130      return;
131    }
132    LOG.info("Open {}", regionName);
133    HRegion region;
134    try {
135      TableDescriptor htd =
136        tableDesc != null ? tableDesc : rs.getTableDescriptors().get(regionInfo.getTable());
137      if (htd == null) {
138        throw new IOException("Missing table descriptor for " + regionName);
139      }
140      // pass null for the last parameter, which used to be a CancelableProgressable, as now the
141      // opening can not be interrupted by a close request any more.
142      Configuration conf = rs.getConfiguration();
143      region = HRegion.openHRegion(regionInfo, htd, rs.getWAL(regionInfo), conf, rs, null);
144    } catch (IOException e) {
145      cleanUpAndReportFailure(e);
146      return;
147    }
148    // From here on out, this is PONR. We can not revert back. The only way to address an
149    // exception from here on out is to abort the region server.
150    rs.postOpenDeployTasks(
151      new PostOpenDeployContext(region, openProcId, masterSystemTime, initiatingMasterActiveTime));
152    rs.addRegion(region);
153    LOG.info("Opened {}", regionName);
154    // Cache the open region procedure id after report region transition succeed.
155    rs.finishRegionProcedure(openProcId);
156    Boolean current = rs.getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
157    if (current == null) {
158      // Should NEVER happen, but let's be paranoid.
159      LOG.error("Bad state: we've just opened {} which was NOT in transition", regionName);
160    } else if (!current) {
161      // Should NEVER happen, but let's be paranoid.
162      LOG.error("Bad state: we've just opened {} which was closing", regionName);
163    }
164  }
165
166  @Override
167  protected void handleException(Throwable t) {
168    LOG.warn("Fatal error occurred while opening region {}, aborting...",
169      regionInfo.getRegionNameAsString(), t);
170    // Clear any reference in getServer().getRegionsInTransitionInRS() otherwise can hold up
171    // regionserver abort on cluster shutdown. HBASE-23984.
172    getServer().getRegionsInTransitionInRS().remove(regionInfo.getEncodedNameAsBytes());
173    getServer().abort(
174      "Failed to open region " + regionInfo.getRegionNameAsString() + " and can not recover", t);
175  }
176
177  public static AssignRegionHandler create(HRegionServer server, RegionInfo regionInfo,
178    long openProcId, TableDescriptor tableDesc, long masterSystemTime,
179    long initiatingMasterActiveTime) {
180    EventType eventType;
181    if (regionInfo.isMetaRegion()) {
182      eventType = EventType.M_RS_OPEN_META;
183    } else if (
184      regionInfo.getTable().isSystemTable()
185        || (tableDesc != null && tableDesc.getPriority() >= HConstants.ADMIN_QOS)
186    ) {
187      eventType = EventType.M_RS_OPEN_PRIORITY_REGION;
188    } else {
189      eventType = EventType.M_RS_OPEN_REGION;
190    }
191    return new AssignRegionHandler(server, regionInfo, openProcId, tableDesc, masterSystemTime,
192      initiatingMasterActiveTime, eventType);
193  }
194}