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 static org.apache.hadoop.hbase.NamespaceDescriptor.DEFAULT_NAMESPACE;
021import static org.apache.hadoop.hbase.NamespaceDescriptor.SYSTEM_NAMESPACE;
022import static org.apache.hadoop.hbase.master.TableNamespaceManager.insertNamespaceToMeta;
023import static org.apache.hadoop.hbase.master.procedure.AbstractStateMachineNamespaceProcedure.createDirectory;
024
025import java.io.IOException;
026import java.util.Arrays;
027import java.util.concurrent.CountDownLatch;
028import org.apache.hadoop.hbase.TableName;
029import org.apache.hadoop.hbase.client.RegionInfoBuilder;
030import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
031import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
032import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
033import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
034import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
035import org.apache.hadoop.hbase.util.RetryCounter;
036import org.apache.yetus.audience.InterfaceAudience;
037import org.slf4j.Logger;
038import org.slf4j.LoggerFactory;
039
040import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.InitMetaState;
041import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.InitMetaStateData;
042import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
043
044/**
045 * This procedure is used to initialize meta table for a new hbase deploy. It will just schedule an
046 * {@link TransitRegionStateProcedure} to assign meta.
047 */
048@InterfaceAudience.Private
049public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMetaState> {
050
051  private static final Logger LOG = LoggerFactory.getLogger(InitMetaProcedure.class);
052
053  private CountDownLatch latch = new CountDownLatch(1);
054
055  private RetryCounter retryCounter;
056
057  @Override
058  public TableName getTableName() {
059    return TableName.META_TABLE_NAME;
060  }
061
062  @Override
063  public TableOperationType getTableOperationType() {
064    return TableOperationType.CREATE;
065  }
066
067  @Override
068  protected Flow executeFromState(MasterProcedureEnv env, InitMetaState state)
069      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
070    LOG.debug("Execute {}", this);
071    switch (state) {
072      case INIT_META_ASSIGN_META:
073        LOG.info("Going to assign meta");
074        addChildProcedure(env.getAssignmentManager()
075          .createAssignProcedures(Arrays.asList(RegionInfoBuilder.FIRST_META_REGIONINFO)));
076        setNextState(InitMetaState.INIT_META_CREATE_NAMESPACES);
077        return Flow.HAS_MORE_STATE;
078      case INIT_META_CREATE_NAMESPACES:
079        LOG.info("Going to create {} and {} namespaces", DEFAULT_NAMESPACE, SYSTEM_NAMESPACE);
080        try {
081          createDirectory(env, DEFAULT_NAMESPACE);
082          createDirectory(env, SYSTEM_NAMESPACE);
083          // here the TableNamespaceManager has not been initialized yet, so we have to insert the
084          // record directly into meta table, later the TableNamespaceManager will load these two
085          // namespaces when starting.
086          insertNamespaceToMeta(env.getMasterServices().getConnection(), DEFAULT_NAMESPACE);
087          insertNamespaceToMeta(env.getMasterServices().getConnection(), SYSTEM_NAMESPACE);
088        } catch (IOException e) {
089          if (retryCounter == null) {
090            retryCounter = ProcedureUtil.createRetryCounter(env.getMasterConfiguration());
091          }
092          long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
093          LOG.warn("Failed to init default and system namespaces, suspend {}secs", backoff, e);
094          setTimeout(Math.toIntExact(backoff));
095          setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
096          skipPersistence();
097          throw new ProcedureSuspendedException();
098        }
099        return Flow.NO_MORE_STATE;
100      default:
101        throw new UnsupportedOperationException("unhandled state=" + state);
102    }
103  }
104
105  @Override
106  protected boolean waitInitialized(MasterProcedureEnv env) {
107    // we do not need to wait for master initialized, we are part of the initialization.
108    return false;
109  }
110
111  @Override
112  protected synchronized boolean setTimeoutFailure(MasterProcedureEnv env) {
113    setState(ProcedureProtos.ProcedureState.RUNNABLE);
114    env.getProcedureScheduler().addFront(this);
115    return false;
116  }
117
118  @Override
119  protected LockState acquireLock(MasterProcedureEnv env) {
120    if (env.getProcedureScheduler().waitTableExclusiveLock(this, getTableName())) {
121      return LockState.LOCK_EVENT_WAIT;
122    }
123    return LockState.LOCK_ACQUIRED;
124  }
125
126  @Override
127  protected void rollbackState(MasterProcedureEnv env, InitMetaState state)
128      throws IOException, InterruptedException {
129    throw new UnsupportedOperationException();
130  }
131
132  @Override
133  protected InitMetaState getState(int stateId) {
134    return InitMetaState.forNumber(stateId);
135  }
136
137  @Override
138  protected int getStateId(InitMetaState state) {
139    return state.getNumber();
140  }
141
142  @Override
143  protected InitMetaState getInitialState() {
144    return InitMetaState.INIT_META_ASSIGN_META;
145  }
146
147  @Override
148  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
149    super.serializeStateData(serializer);
150    serializer.serialize(InitMetaStateData.getDefaultInstance());
151  }
152
153  @Override
154  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
155    super.deserializeStateData(serializer);
156    serializer.deserialize(InitMetaStateData.class);
157  }
158
159  @Override
160  protected void completionCleanup(MasterProcedureEnv env) {
161    latch.countDown();
162  }
163
164  public void await() throws InterruptedException {
165    latch.await();
166  }
167}