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