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.assignment;
019
020import java.io.IOException;
021import org.apache.hadoop.hbase.ServerName;
022import org.apache.hadoop.hbase.client.RegionInfo;
023import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
024import org.apache.hadoop.hbase.master.RegionState.State;
025import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
026import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.RegionCloseOperation;
027import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
028import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
029import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
030import org.apache.yetus.audience.InterfaceAudience;
031
032import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
033import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloseRegionProcedureStateData;
034import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
035
036/**
037 * The remote procedure used to close a region.
038 */
039@InterfaceAudience.Private
040public class CloseRegionProcedure extends RegionRemoteProcedureBase {
041
042  // For a region move operation, we will assign the region after we unassign it, this is the target
043  // server for the subsequent assign. We will send this value to RS, and RS will record the region
044  // in a Map to tell client that where the region has been moved to. Can be null. And also, can be
045  // wrong(but do not make it wrong intentionally). The client can handle this error.
046  private ServerName assignCandidate;
047
048  private boolean evictCache;
049
050  public CloseRegionProcedure() {
051    super();
052  }
053
054  public CloseRegionProcedure(TransitRegionStateProcedure parent, RegionInfo region,
055    ServerName targetServer, ServerName assignCandidate, boolean evictCache) {
056    super(parent, region, targetServer);
057    this.assignCandidate = assignCandidate;
058    this.evictCache = evictCache;
059  }
060
061  @Override
062  public TableOperationType getTableOperationType() {
063    return TableOperationType.REGION_UNASSIGN;
064  }
065
066  @Override
067  public RemoteOperation newRemoteOperation() {
068    return new RegionCloseOperation(this, region, getProcId(), assignCandidate, evictCache);
069  }
070
071  @Override
072  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
073    super.serializeStateData(serializer);
074    CloseRegionProcedureStateData.Builder builder = CloseRegionProcedureStateData.newBuilder();
075    if (assignCandidate != null) {
076      builder.setAssignCandidate(ProtobufUtil.toServerName(assignCandidate));
077    }
078    builder.setEvictCache(evictCache);
079    serializer.serialize(builder.build());
080  }
081
082  @Override
083  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
084    super.deserializeStateData(serializer);
085    CloseRegionProcedureStateData data =
086      serializer.deserialize(CloseRegionProcedureStateData.class);
087    if (data.hasAssignCandidate()) {
088      assignCandidate = ProtobufUtil.toServerName(data.getAssignCandidate());
089    }
090    evictCache = data.getEvictCache();
091  }
092
093  @Override
094  protected ProcedureMetrics getProcedureMetrics(MasterProcedureEnv env) {
095    return env.getAssignmentManager().getAssignmentManagerMetrics().getCloseProcMetrics();
096  }
097
098  @Override
099  protected void checkTransition(RegionStateNode regionNode, TransitionCode transitionCode,
100    long seqId) throws UnexpectedStateException {
101    if (transitionCode != TransitionCode.CLOSED) {
102      throw new UnexpectedStateException("Received report unexpected " + transitionCode
103        + " transition, " + regionNode.toShortString() + ", " + this + ", expected CLOSED.");
104    }
105  }
106
107  @Override
108  protected void updateTransitionWithoutPersistingToMeta(MasterProcedureEnv env,
109    RegionStateNode regionNode, TransitionCode transitionCode, long seqId) throws IOException {
110    assert transitionCode == TransitionCode.CLOSED;
111    env.getAssignmentManager().regionClosedWithoutPersistingToMeta(regionNode);
112  }
113
114  @Override
115  protected void restoreSucceedState(AssignmentManager am, RegionStateNode regionNode, long seqId)
116    throws IOException {
117    if (regionNode.getState() == State.CLOSED) {
118      // should have already been persisted, ignore
119      return;
120    }
121    am.regionClosedWithoutPersistingToMeta(regionNode);
122  }
123}