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 java.util.concurrent.CountDownLatch;
022import org.apache.hadoop.hbase.TableName;
023import org.apache.hadoop.hbase.client.RegionInfoBuilder;
024import org.apache.hadoop.hbase.master.assignment.AssignProcedure;
025import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
026import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
027import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
028import org.apache.yetus.audience.InterfaceAudience;
029
030import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.InitMetaState;
031import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.InitMetaStateData;
032
033/**
034 * This procedure is used to initialize meta table for a new hbase deploy. It will just schedule an
035 * {@link AssignProcedure} to assign meta.
036 */
037@InterfaceAudience.Private
038public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMetaState> {
039
040  private CountDownLatch latch = new CountDownLatch(1);
041
042  @Override
043  public TableName getTableName() {
044    return TableName.META_TABLE_NAME;
045  }
046
047  @Override
048  public TableOperationType getTableOperationType() {
049    return TableOperationType.CREATE;
050  }
051
052  @Override
053  protected Flow executeFromState(MasterProcedureEnv env, InitMetaState state)
054      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
055    switch (state) {
056      case INIT_META_ASSIGN_META:
057        addChildProcedure(env.getAssignmentManager()
058          .createAssignProcedure(RegionInfoBuilder.FIRST_META_REGIONINFO));
059        return Flow.NO_MORE_STATE;
060      default:
061        throw new UnsupportedOperationException("unhandled state=" + state);
062    }
063  }
064
065  @Override
066  protected boolean waitInitialized(MasterProcedureEnv env) {
067    // we do not need to wait for master initialized, we are part of the initialization.
068    return false;
069  }
070
071  @Override
072  protected LockState acquireLock(MasterProcedureEnv env) {
073    if (env.getProcedureScheduler().waitTableExclusiveLock(this, getTableName())) {
074      return LockState.LOCK_EVENT_WAIT;
075    }
076    return LockState.LOCK_ACQUIRED;
077  }
078
079  @Override
080  protected void rollbackState(MasterProcedureEnv env, InitMetaState state)
081      throws IOException, InterruptedException {
082    throw new UnsupportedOperationException();
083  }
084
085  @Override
086  protected InitMetaState getState(int stateId) {
087    return InitMetaState.forNumber(stateId);
088  }
089
090  @Override
091  protected int getStateId(InitMetaState state) {
092    return state.getNumber();
093  }
094
095  @Override
096  protected InitMetaState getInitialState() {
097    return InitMetaState.INIT_META_ASSIGN_META;
098  }
099
100  @Override
101  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
102    super.serializeStateData(serializer);
103    serializer.serialize(InitMetaStateData.getDefaultInstance());
104  }
105
106  @Override
107  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
108    super.deserializeStateData(serializer);
109    serializer.deserialize(InitMetaStateData.class);
110  }
111
112  @Override
113  protected void completionCleanup(MasterProcedureEnv env) {
114    latch.countDown();
115  }
116
117  public void await() throws InterruptedException {
118    latch.await();
119  }
120}