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.fs.Path;
022import org.apache.hadoop.hbase.HBaseIOException;
023import org.apache.hadoop.hbase.TableName;
024import org.apache.hadoop.hbase.TableNotDisabledException;
025import org.apache.hadoop.hbase.TableNotEnabledException;
026import org.apache.hadoop.hbase.TableNotFoundException;
027import org.apache.hadoop.hbase.UnknownRegionException;
028import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
029import org.apache.hadoop.hbase.client.RegionInfo;
030import org.apache.hadoop.hbase.client.TableState;
031import org.apache.hadoop.hbase.master.MasterServices;
032import org.apache.hadoop.hbase.master.TableStateManager;
033import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
034import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
035import org.apache.hadoop.hbase.security.User;
036import org.apache.hadoop.hbase.util.CommonFSUtils;
037import org.apache.yetus.audience.InterfaceAudience;
038
039/**
040 * Base class for all the Table procedures that want to use a StateMachineProcedure. It provides
041 * helpers like basic locking, sync latch, and toStringClassDetails().
042 */
043@InterfaceAudience.Private
044public abstract class AbstractStateMachineTableProcedure<TState>
045  extends StateMachineProcedure<MasterProcedureEnv, TState> implements TableProcedureInterface {
046
047  // used for compatibility with old clients
048  private final ProcedurePrepareLatch syncLatch;
049
050  private User user;
051
052  protected AbstractStateMachineTableProcedure() {
053    // Required by the Procedure framework to create the procedure on replay
054    syncLatch = null;
055  }
056
057  protected AbstractStateMachineTableProcedure(final MasterProcedureEnv env) {
058    this(env, null);
059  }
060
061  /**
062   * @param env Uses this to set Procedure Owner at least.
063   */
064  protected AbstractStateMachineTableProcedure(final MasterProcedureEnv env,
065    final ProcedurePrepareLatch latch) {
066    if (env != null) {
067      this.user = env.getRequestUser();
068      this.setOwner(user);
069    }
070    // used for compatibility with clients without procedures
071    // they need a sync TableExistsException, TableNotFoundException, TableNotDisabledException, ...
072    this.syncLatch = latch;
073  }
074
075  @Override
076  public abstract TableName getTableName();
077
078  @Override
079  public abstract TableOperationType getTableOperationType();
080
081  @Override
082  public void toStringClassDetails(final StringBuilder sb) {
083    sb.append(getClass().getSimpleName());
084    sb.append(" table=");
085    sb.append(getTableName());
086  }
087
088  @Override
089  protected boolean waitInitialized(MasterProcedureEnv env) {
090    return env.waitInitialized(this);
091  }
092
093  @Override
094  protected LockState acquireLock(final MasterProcedureEnv env) {
095    if (env.getProcedureScheduler().waitTableExclusiveLock(this, getTableName())) {
096      return LockState.LOCK_EVENT_WAIT;
097    }
098    return LockState.LOCK_ACQUIRED;
099  }
100
101  @Override
102  protected void releaseLock(final MasterProcedureEnv env) {
103    env.getProcedureScheduler().wakeTableExclusiveLock(this, getTableName());
104  }
105
106  protected User getUser() {
107    return user;
108  }
109
110  protected void setUser(final User user) {
111    this.user = user;
112  }
113
114  protected void releaseSyncLatch() {
115    ProcedurePrepareLatch.releaseLatch(syncLatch, this);
116  }
117
118  /**
119   * Check whether a table is modifiable - exists and either offline or online with config set
120   * @param env MasterProcedureEnv
121   */
122  protected void checkTableModifiable(final MasterProcedureEnv env) throws IOException {
123    // Checks whether the table exists
124    if (!env.getMasterServices().getTableDescriptors().exists(getTableName())) {
125      throw new TableNotFoundException(getTableName());
126    }
127  }
128
129  protected final Path getWALRegionDir(MasterProcedureEnv env, RegionInfo region)
130    throws IOException {
131    return CommonFSUtils.getWALRegionDir(env.getMasterConfiguration(), region.getTable(),
132      region.getEncodedName());
133  }
134
135  /**
136   * Check that cluster is up and master is running. Check table is modifiable. If
137   * <code>enabled</code>, check table is enabled else check it is disabled. Call in Procedure
138   * constructor so can pass any exception to caller.
139   * @param enabled If true, check table is enabled and throw exception if not. If false, do the
140   *                inverse. If null, do no table checks.
141   */
142  protected void preflightChecks(MasterProcedureEnv env, Boolean enabled) throws HBaseIOException {
143    MasterServices master = env.getMasterServices();
144    if (!master.isClusterUp()) {
145      throw new HBaseIOException("Cluster not up!");
146    }
147    if (master.isStopping() || master.isStopped()) {
148      throw new HBaseIOException(
149        "Master stopping=" + master.isStopping() + ", stopped=" + master.isStopped());
150    }
151    if (enabled == null) {
152      // Don't do any table checks.
153      return;
154    }
155    try {
156      // Checks table exists and is modifiable.
157      checkTableModifiable(env);
158      TableName tn = getTableName();
159      TableStateManager tsm = master.getTableStateManager();
160      TableState ts = tsm.getTableState(tn);
161      if (enabled) {
162        if (!ts.isEnabledOrEnabling()) {
163          throw new TableNotEnabledException(tn);
164        }
165      } else {
166        if (!ts.isDisabledOrDisabling()) {
167          throw new TableNotDisabledException(tn);
168        }
169      }
170    } catch (IOException ioe) {
171      if (ioe instanceof HBaseIOException) {
172        throw (HBaseIOException) ioe;
173      }
174      throw new HBaseIOException(ioe);
175    }
176  }
177
178  protected boolean isTableEnabled(MasterProcedureEnv env) {
179    return env.getMasterServices().getTableStateManager().isTableState(getTableName(),
180      TableState.State.ENABLED);
181  }
182
183  /**
184   * Check region is online.
185   */
186  protected static void checkOnline(MasterProcedureEnv env, RegionInfo ri)
187    throws DoNotRetryRegionException {
188    RegionStateNode regionNode =
189      env.getAssignmentManager().getRegionStates().getRegionStateNode(ri);
190    if (regionNode == null) {
191      throw new UnknownRegionException("No RegionState found for " + ri.getEncodedName());
192    }
193    regionNode.checkOnline();
194  }
195}