|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.hadoop.hbase.master.procedure; |
| 19 | + |
| 20 | +import java.io.IOException; |
| 21 | +import org.apache.commons.lang3.mutable.MutableBoolean; |
| 22 | +import org.apache.hadoop.hbase.TableName; |
| 23 | +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; |
| 24 | +import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; |
| 25 | +import org.apache.hadoop.hbase.procedure2.ProcedureUtil; |
| 26 | +import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; |
| 27 | +import org.apache.hadoop.hbase.util.RetryCounter; |
| 28 | +import org.apache.yetus.audience.InterfaceAudience; |
| 29 | +import org.slf4j.Logger; |
| 30 | +import org.slf4j.LoggerFactory; |
| 31 | + |
| 32 | +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; |
| 33 | +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloseExcessRegionReplicasProcedureState; |
| 34 | +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CloseExcessRegionReplicasProcedureStateData; |
| 35 | +import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos; |
| 36 | + |
| 37 | +/** |
| 38 | + * Procedure for close excess region replicas. |
| 39 | + */ |
| 40 | +@InterfaceAudience.Private |
| 41 | +public class CloseExcessRegionReplicasProcedure |
| 42 | + extends AbstractStateMachineTableProcedure<CloseExcessRegionReplicasProcedureState> { |
| 43 | + |
| 44 | + private static final Logger LOG = |
| 45 | + LoggerFactory.getLogger(CloseExcessRegionReplicasProcedure.class); |
| 46 | + |
| 47 | + private TableName tableName; |
| 48 | + private int newReplicaCount; |
| 49 | + |
| 50 | + private RetryCounter retryCounter; |
| 51 | + |
| 52 | + public CloseExcessRegionReplicasProcedure() { |
| 53 | + } |
| 54 | + |
| 55 | + public CloseExcessRegionReplicasProcedure(TableName tableName, int newReplicaCount) { |
| 56 | + this.tableName = tableName; |
| 57 | + this.newReplicaCount = newReplicaCount; |
| 58 | + } |
| 59 | + |
| 60 | + @Override |
| 61 | + public TableName getTableName() { |
| 62 | + return tableName; |
| 63 | + } |
| 64 | + |
| 65 | + @Override |
| 66 | + public TableOperationType getTableOperationType() { |
| 67 | + return TableOperationType.REGION_EDIT; |
| 68 | + } |
| 69 | + |
| 70 | + @Override |
| 71 | + protected Flow executeFromState(MasterProcedureEnv env, |
| 72 | + CloseExcessRegionReplicasProcedureState state) |
| 73 | + throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { |
| 74 | + LOG.trace("{} execute state={}", this, state); |
| 75 | + switch (state) { |
| 76 | + case CLOSE_EXCESS_REGION_REPLICAS_SCHEDULE: |
| 77 | + MutableBoolean submitted = new MutableBoolean(false); |
| 78 | + int inTransitionCount = env.getAssignmentManager() |
| 79 | + .submitUnassignProcedureForClosingExcessRegionReplicas(tableName, newReplicaCount, p -> { |
| 80 | + submitted.setTrue(); |
| 81 | + addChildProcedure(p); |
| 82 | + }); |
| 83 | + if (inTransitionCount > 0 && submitted.isFalse()) { |
| 84 | + // we haven't scheduled any unassign procedures and there are still regions in |
| 85 | + // transition, sleep for a while and try again |
| 86 | + if (retryCounter == null) { |
| 87 | + retryCounter = ProcedureUtil.createRetryCounter(env.getMasterConfiguration()); |
| 88 | + } |
| 89 | + long backoffMillis = retryCounter.getBackoffTimeAndIncrementAttempts(); |
| 90 | + LOG.info( |
| 91 | + "There are still {} region(s) in transition for table {} when closing excess" |
| 92 | + + " region replicas, suspend {}secs and try again later", |
| 93 | + inTransitionCount, tableName, backoffMillis / 1000); |
| 94 | + suspend((int) backoffMillis, true); |
| 95 | + } |
| 96 | + setNextState(CloseExcessRegionReplicasProcedureState.CLOSE_EXCESS_REGION_REPLICAS_CONFIRM); |
| 97 | + return Flow.HAS_MORE_STATE; |
| 98 | + case CLOSE_EXCESS_REGION_REPLICAS_CONFIRM: |
| 99 | + int unclosedCount = env.getAssignmentManager() |
| 100 | + .numberOfUnclosedExcessRegionReplicas(tableName, newReplicaCount); |
| 101 | + if (unclosedCount > 0) { |
| 102 | + LOG.info("There are still {} unclosed region(s) for table {} when closing excess" |
| 103 | + + " region replicas, continue..."); |
| 104 | + setNextState( |
| 105 | + CloseExcessRegionReplicasProcedureState.CLOSE_EXCESS_REGION_REPLICAS_SCHEDULE); |
| 106 | + return Flow.HAS_MORE_STATE; |
| 107 | + } else { |
| 108 | + return Flow.NO_MORE_STATE; |
| 109 | + } |
| 110 | + default: |
| 111 | + throw new UnsupportedOperationException("unhandled state=" + state); |
| 112 | + } |
| 113 | + } |
| 114 | + |
| 115 | + @Override |
| 116 | + protected synchronized boolean setTimeoutFailure(MasterProcedureEnv env) { |
| 117 | + setState(ProcedureProtos.ProcedureState.RUNNABLE); |
| 118 | + env.getProcedureScheduler().addFront(this); |
| 119 | + return false; |
| 120 | + } |
| 121 | + |
| 122 | + @Override |
| 123 | + protected void rollbackState(MasterProcedureEnv env, |
| 124 | + CloseExcessRegionReplicasProcedureState state) throws IOException, InterruptedException { |
| 125 | + throw new UnsupportedOperationException(); |
| 126 | + } |
| 127 | + |
| 128 | + @Override |
| 129 | + protected CloseExcessRegionReplicasProcedureState getState(int stateId) { |
| 130 | + return CloseExcessRegionReplicasProcedureState.forNumber(stateId); |
| 131 | + } |
| 132 | + |
| 133 | + @Override |
| 134 | + protected int getStateId(CloseExcessRegionReplicasProcedureState state) { |
| 135 | + return state.getNumber(); |
| 136 | + } |
| 137 | + |
| 138 | + @Override |
| 139 | + protected CloseExcessRegionReplicasProcedureState getInitialState() { |
| 140 | + return CloseExcessRegionReplicasProcedureState.CLOSE_EXCESS_REGION_REPLICAS_SCHEDULE; |
| 141 | + } |
| 142 | + |
| 143 | + @Override |
| 144 | + protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { |
| 145 | + CloseExcessRegionReplicasProcedureStateData data = CloseExcessRegionReplicasProcedureStateData |
| 146 | + .newBuilder().setTableName(ProtobufUtil.toProtoTableName(tableName)) |
| 147 | + .setNewReplicaCount(newReplicaCount).build(); |
| 148 | + serializer.serialize(data); |
| 149 | + } |
| 150 | + |
| 151 | + @Override |
| 152 | + protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { |
| 153 | + CloseExcessRegionReplicasProcedureStateData data = |
| 154 | + serializer.deserialize(CloseExcessRegionReplicasProcedureStateData.class); |
| 155 | + tableName = ProtobufUtil.toTableName(data.getTableName()); |
| 156 | + newReplicaCount = data.getNewReplicaCount(); |
| 157 | + } |
| 158 | + |
| 159 | +} |
0 commit comments