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