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.procedure;
019
020import java.io.IOException;
021import org.apache.hadoop.hbase.HBaseIOException;
022import org.apache.hadoop.hbase.HConstants;
023import org.apache.hadoop.hbase.TableName;
024import org.apache.hadoop.hbase.TableNotEnabledException;
025import org.apache.hadoop.hbase.TableNotFoundException;
026import org.apache.hadoop.hbase.client.BufferedMutator;
027import org.apache.hadoop.hbase.client.RegionInfo;
028import org.apache.hadoop.hbase.client.TableState;
029import org.apache.hadoop.hbase.constraint.ConstraintException;
030import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
031import org.apache.hadoop.hbase.master.MasterFileSystem;
032import org.apache.hadoop.hbase.master.TableStateManager;
033import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
034import org.apache.hadoop.hbase.replication.ReplicationBarrierFamilyFormat;
035import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
036import org.apache.hadoop.hbase.wal.WALSplitUtil;
037import org.apache.yetus.audience.InterfaceAudience;
038import org.slf4j.Logger;
039import org.slf4j.LoggerFactory;
040
041import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
042import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
043import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisableTableState;
044
045@InterfaceAudience.Private
046public class DisableTableProcedure extends AbstractStateMachineTableProcedure<DisableTableState> {
047  private static final Logger LOG = LoggerFactory.getLogger(DisableTableProcedure.class);
048
049  private TableName tableName;
050  private boolean skipTableStateCheck;
051
052  public DisableTableProcedure() {
053    super();
054  }
055
056  /**
057   * Constructor
058   * @param env                 MasterProcedureEnv
059   * @param tableName           the table to operate on
060   * @param skipTableStateCheck whether to check table state
061   */
062  public DisableTableProcedure(final MasterProcedureEnv env, final TableName tableName,
063    final boolean skipTableStateCheck) throws HBaseIOException {
064    this(env, tableName, skipTableStateCheck, null);
065  }
066
067  /**
068   * Constructor
069   * @param env                 MasterProcedureEnv
070   * @param tableName           the table to operate on
071   * @param skipTableStateCheck whether to check table state
072   */
073  public DisableTableProcedure(final MasterProcedureEnv env, final TableName tableName,
074    final boolean skipTableStateCheck, final ProcedurePrepareLatch syncLatch)
075    throws HBaseIOException {
076    super(env, syncLatch);
077    this.tableName = tableName;
078    preflightChecks(env, true);
079    this.skipTableStateCheck = skipTableStateCheck;
080  }
081
082  @Override
083  protected Flow executeFromState(final MasterProcedureEnv env, final DisableTableState state)
084    throws InterruptedException {
085    LOG.trace("{} execute state={}", this, state);
086    try {
087      switch (state) {
088        case DISABLE_TABLE_PREPARE:
089          if (prepareDisable(env)) {
090            setNextState(DisableTableState.DISABLE_TABLE_PRE_OPERATION);
091          } else {
092            assert isFailed() : "disable should have an exception here";
093            return Flow.NO_MORE_STATE;
094          }
095          break;
096        case DISABLE_TABLE_PRE_OPERATION:
097          preDisable(env, state);
098          setNextState(DisableTableState.DISABLE_TABLE_SET_DISABLING_TABLE_STATE);
099          break;
100        case DISABLE_TABLE_SET_DISABLING_TABLE_STATE:
101          setTableStateToDisabling(env, tableName);
102          setNextState(DisableTableState.DISABLE_TABLE_MARK_REGIONS_OFFLINE);
103          break;
104        case DISABLE_TABLE_MARK_REGIONS_OFFLINE:
105          addChildProcedure(new CloseTableRegionsProcedure(tableName));
106          setNextState(DisableTableState.DISABLE_TABLE_ADD_REPLICATION_BARRIER);
107          break;
108        case DISABLE_TABLE_ADD_REPLICATION_BARRIER:
109          if (
110            env.getMasterServices().getTableDescriptors().get(tableName).hasGlobalReplicationScope()
111          ) {
112            MasterFileSystem fs = env.getMasterFileSystem();
113            try (BufferedMutator mutator = env.getMasterServices().getConnection()
114              .getBufferedMutator(TableName.META_TABLE_NAME)) {
115              for (RegionInfo region : env.getAssignmentManager().getRegionStates()
116                .getRegionsOfTable(tableName)) {
117                long maxSequenceId = WALSplitUtil.getMaxRegionSequenceId(
118                  env.getMasterConfiguration(), region, fs::getFileSystem, fs::getWALFileSystem);
119                long openSeqNum = maxSequenceId > 0 ? maxSequenceId + 1 : HConstants.NO_SEQNUM;
120                mutator.mutate(ReplicationBarrierFamilyFormat.makePutForReplicationBarrier(region,
121                  openSeqNum, EnvironmentEdgeManager.currentTime()));
122              }
123            }
124          }
125          setNextState(DisableTableState.DISABLE_TABLE_SET_DISABLED_TABLE_STATE);
126          break;
127        case DISABLE_TABLE_SET_DISABLED_TABLE_STATE:
128          setTableStateToDisabled(env, tableName);
129          setNextState(DisableTableState.DISABLE_TABLE_POST_OPERATION);
130          break;
131        case DISABLE_TABLE_POST_OPERATION:
132          postDisable(env, state);
133          return Flow.NO_MORE_STATE;
134        default:
135          throw new UnsupportedOperationException("Unhandled state=" + state);
136      }
137    } catch (IOException e) {
138      if (isRollbackSupported(state)) {
139        setFailure("master-disable-table", e);
140      } else {
141        LOG.warn("Retryable error in {}", this, e);
142      }
143    }
144    return Flow.HAS_MORE_STATE;
145  }
146
147  @Override
148  protected void rollbackState(final MasterProcedureEnv env, final DisableTableState state)
149    throws IOException {
150    // nothing to rollback, prepare-disable is just table-state checks.
151    // We can fail if the table does not exist or is not disabled.
152    switch (state) {
153      case DISABLE_TABLE_PRE_OPERATION:
154        return;
155      case DISABLE_TABLE_PREPARE:
156        releaseSyncLatch();
157        return;
158      default:
159        break;
160    }
161
162    // The delete doesn't have a rollback. The execution will succeed, at some point.
163    throw new UnsupportedOperationException("Unhandled state=" + state);
164  }
165
166  @Override
167  protected boolean isRollbackSupported(final DisableTableState state) {
168    switch (state) {
169      case DISABLE_TABLE_PREPARE:
170      case DISABLE_TABLE_PRE_OPERATION:
171        return true;
172      default:
173        return false;
174    }
175  }
176
177  @Override
178  protected DisableTableState getState(final int stateId) {
179    return DisableTableState.forNumber(stateId);
180  }
181
182  @Override
183  protected int getStateId(final DisableTableState state) {
184    return state.getNumber();
185  }
186
187  @Override
188  protected DisableTableState getInitialState() {
189    return DisableTableState.DISABLE_TABLE_PREPARE;
190  }
191
192  @Override
193  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
194    super.serializeStateData(serializer);
195
196    MasterProcedureProtos.DisableTableStateData.Builder disableTableMsg =
197      MasterProcedureProtos.DisableTableStateData.newBuilder()
198        .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
199        .setTableName(ProtobufUtil.toProtoTableName(tableName))
200        .setSkipTableStateCheck(skipTableStateCheck);
201
202    serializer.serialize(disableTableMsg.build());
203  }
204
205  @Override
206  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
207    super.deserializeStateData(serializer);
208
209    MasterProcedureProtos.DisableTableStateData disableTableMsg =
210      serializer.deserialize(MasterProcedureProtos.DisableTableStateData.class);
211    setUser(MasterProcedureUtil.toUserInfo(disableTableMsg.getUserInfo()));
212    tableName = ProtobufUtil.toTableName(disableTableMsg.getTableName());
213    skipTableStateCheck = disableTableMsg.getSkipTableStateCheck();
214  }
215
216  @Override
217  public TableName getTableName() {
218    return tableName;
219  }
220
221  @Override
222  public TableOperationType getTableOperationType() {
223    return TableOperationType.DISABLE;
224  }
225
226  /**
227   * Action before any real action of disabling table. Set the exception in the procedure instead of
228   * throwing it. This approach is to deal with backward compatible with 1.0.
229   * @param env MasterProcedureEnv
230   */
231  private boolean prepareDisable(final MasterProcedureEnv env) throws IOException {
232    boolean canTableBeDisabled = true;
233    if (tableName.equals(TableName.META_TABLE_NAME)) {
234      setFailure("master-disable-table",
235        new ConstraintException("Cannot disable " + this.tableName));
236      canTableBeDisabled = false;
237    } else if (!env.getMasterServices().getTableDescriptors().exists(tableName)) {
238      setFailure("master-disable-table", new TableNotFoundException(tableName));
239      canTableBeDisabled = false;
240    } else if (!skipTableStateCheck) {
241      // There could be multiple client requests trying to disable or enable
242      // the table at the same time. Ensure only the first request is honored
243      // After that, no other requests can be accepted until the table reaches
244      // DISABLED or ENABLED.
245      //
246      // Note: in 1.0 release, we called TableStateManager.setTableStateIfInStates() to set
247      // the state to DISABLING from ENABLED. The implementation was done before table lock
248      // was implemented. With table lock, there is no need to set the state here (it will
249      // set the state later on). A quick state check should be enough for us to move forward.
250      TableStateManager tsm = env.getMasterServices().getTableStateManager();
251      TableState ts = tsm.getTableState(tableName);
252      if (!ts.isEnabled()) {
253        LOG.info("Not ENABLED, state={}, skipping disable; {}", ts.getState(), this);
254        setFailure("master-disable-table", new TableNotEnabledException(ts.toString()));
255        canTableBeDisabled = false;
256      }
257    }
258
259    // We are done the check. Future actions in this procedure could be done asynchronously.
260    releaseSyncLatch();
261
262    return canTableBeDisabled;
263  }
264
265  /**
266   * Action before disabling table.
267   * @param env   MasterProcedureEnv
268   * @param state the procedure state
269   */
270  protected void preDisable(final MasterProcedureEnv env, final DisableTableState state)
271    throws IOException, InterruptedException {
272    runCoprocessorAction(env, state);
273  }
274
275  /**
276   * Mark table state to Disabling
277   * @param env MasterProcedureEnv
278   */
279  private static void setTableStateToDisabling(final MasterProcedureEnv env,
280    final TableName tableName) throws IOException {
281    // Set table disabling flag up in zk.
282    env.getMasterServices().getTableStateManager().setTableState(tableName,
283      TableState.State.DISABLING);
284    LOG.info("Set {} to state={}", tableName, TableState.State.DISABLING);
285  }
286
287  /**
288   * Mark table state to Disabled
289   * @param env MasterProcedureEnv
290   */
291  protected static void setTableStateToDisabled(final MasterProcedureEnv env,
292    final TableName tableName) throws IOException {
293    // Flip the table to disabled
294    env.getMasterServices().getTableStateManager().setTableState(tableName,
295      TableState.State.DISABLED);
296    LOG.info("Set {} to state={}", tableName, TableState.State.DISABLED);
297  }
298
299  /**
300   * Action after disabling table.
301   * @param env   MasterProcedureEnv
302   * @param state the procedure state
303   */
304  protected void postDisable(final MasterProcedureEnv env, final DisableTableState state)
305    throws IOException, InterruptedException {
306    runCoprocessorAction(env, state);
307  }
308
309  /**
310   * Coprocessor Action.
311   * @param env   MasterProcedureEnv
312   * @param state the procedure state
313   */
314  private void runCoprocessorAction(final MasterProcedureEnv env, final DisableTableState state)
315    throws IOException, InterruptedException {
316    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
317    if (cpHost != null) {
318      switch (state) {
319        case DISABLE_TABLE_PRE_OPERATION:
320          cpHost.preDisableTableAction(tableName, getUser());
321          break;
322        case DISABLE_TABLE_POST_OPERATION:
323          cpHost.postCompletedDisableTableAction(tableName, getUser());
324          break;
325        default:
326          throw new UnsupportedOperationException(this + " unhandled state=" + state);
327      }
328    }
329  }
330}