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 org.apache.hadoop.hbase.TableName;
021import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
022import org.apache.yetus.audience.InterfaceAudience;
023
024/**
025 * Base class for all the Namespace procedures that want to use a StateMachineProcedure. It provide
026 * some basic helpers like basic locking and basic toStringClassDetails().
027 */
028@InterfaceAudience.Private
029public abstract class AbstractStateMachineNamespaceProcedure<TState>
030  extends StateMachineProcedure<MasterProcedureEnv, TState> implements TableProcedureInterface {
031
032  private final ProcedurePrepareLatch syncLatch;
033
034  protected AbstractStateMachineNamespaceProcedure() {
035    // Required by the Procedure framework to create the procedure on replay
036    syncLatch = null;
037  }
038
039  protected AbstractStateMachineNamespaceProcedure(final MasterProcedureEnv env) {
040    this(env, null);
041  }
042
043  protected AbstractStateMachineNamespaceProcedure(final MasterProcedureEnv env,
044    final ProcedurePrepareLatch latch) {
045    this.setOwner(env.getRequestUser());
046    this.syncLatch = latch;
047  }
048
049  protected abstract String getNamespaceName();
050
051  @Override
052  public TableName getTableName() {
053    return TableName.NAMESPACE_TABLE_NAME;
054  }
055
056  @Override
057  public abstract TableOperationType getTableOperationType();
058
059  @Override
060  public void toStringClassDetails(final StringBuilder sb) {
061    sb.append(getClass().getSimpleName());
062    sb.append(", namespace=");
063    sb.append(getNamespaceName());
064  }
065
066  @Override
067  protected boolean waitInitialized(MasterProcedureEnv env) {
068    return env.waitInitialized(this);
069  }
070
071  @Override
072  protected LockState acquireLock(final MasterProcedureEnv env) {
073    if (env.getProcedureScheduler().waitNamespaceExclusiveLock(this, getNamespaceName())) {
074      return LockState.LOCK_EVENT_WAIT;
075    }
076    return LockState.LOCK_ACQUIRED;
077  }
078
079  @Override
080  protected void releaseLock(final MasterProcedureEnv env) {
081    env.getProcedureScheduler().wakeNamespaceExclusiveLock(this, getNamespaceName());
082  }
083
084  protected void releaseSyncLatch() {
085    ProcedurePrepareLatch.releaseLatch(syncLatch, this);
086  }
087}