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.ArrayList;
022import java.util.List;
023import org.apache.hadoop.fs.Path;
024import org.apache.hadoop.hbase.DoNotRetryIOException;
025import org.apache.hadoop.hbase.HBaseIOException;
026import org.apache.hadoop.hbase.MetaTableAccessor;
027import org.apache.hadoop.hbase.TableExistsException;
028import org.apache.hadoop.hbase.TableName;
029import org.apache.hadoop.hbase.client.RegionInfo;
030import org.apache.hadoop.hbase.client.RegionReplicaUtil;
031import org.apache.hadoop.hbase.client.TableDescriptor;
032import org.apache.hadoop.hbase.client.TableState;
033import org.apache.hadoop.hbase.fs.ErasureCodingUtils;
034import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
035import org.apache.hadoop.hbase.master.MasterFileSystem;
036import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
037import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
038import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
039import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
040import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerValidationUtils;
041import org.apache.hadoop.hbase.replication.ReplicationException;
042import org.apache.hadoop.hbase.util.CommonFSUtils;
043import org.apache.hadoop.hbase.util.FSTableDescriptors;
044import org.apache.hadoop.hbase.util.ModifyRegionUtils;
045import org.apache.hadoop.hbase.util.RetryCounter;
046import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
047import org.apache.yetus.audience.InterfaceAudience;
048import org.slf4j.Logger;
049import org.slf4j.LoggerFactory;
050
051import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
052
053import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
054import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
055import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
056import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.CreateTableState;
057import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
058
059@InterfaceAudience.Private
060public class CreateTableProcedure extends AbstractStateMachineTableProcedure<CreateTableState> {
061  private static final Logger LOG = LoggerFactory.getLogger(CreateTableProcedure.class);
062
063  private static final int MAX_REGION_REPLICATION = 0x10000;
064
065  private TableDescriptor tableDescriptor;
066  private List<RegionInfo> newRegions;
067  private RetryCounter retryCounter;
068
069  public CreateTableProcedure() {
070    // Required by the Procedure framework to create the procedure on replay
071    super();
072  }
073
074  public CreateTableProcedure(final MasterProcedureEnv env, final TableDescriptor tableDescriptor,
075    final RegionInfo[] newRegions) {
076    this(env, tableDescriptor, newRegions, null);
077  }
078
079  public CreateTableProcedure(final MasterProcedureEnv env, final TableDescriptor tableDescriptor,
080    final RegionInfo[] newRegions, final ProcedurePrepareLatch syncLatch) {
081    super(env, syncLatch);
082    this.tableDescriptor = tableDescriptor;
083    this.newRegions = newRegions != null ? Lists.newArrayList(newRegions) : null;
084  }
085
086  @Override
087  protected Flow executeFromState(final MasterProcedureEnv env, final CreateTableState state)
088    throws InterruptedException, ProcedureSuspendedException {
089    LOG.info("{} execute state={}", this, state);
090    try {
091      switch (state) {
092        case CREATE_TABLE_PRE_OPERATION:
093          // Verify if we can create the table
094          boolean success = prepareCreate(env);
095          releaseSyncLatch();
096
097          if (!success) {
098            assert isFailed() : "the delete should have an exception here";
099            return Flow.NO_MORE_STATE;
100          }
101
102          preCreate(env);
103          setNextState(CreateTableState.CREATE_TABLE_WRITE_FS_LAYOUT);
104          break;
105        case CREATE_TABLE_WRITE_FS_LAYOUT:
106          DeleteTableProcedure.deleteFromFs(env, getTableName(), newRegions, true);
107          newRegions = createFsLayout(env, tableDescriptor, newRegions);
108          env.getMasterServices().getTableDescriptors().update(tableDescriptor, true);
109          if (tableDescriptor.getErasureCodingPolicy() != null) {
110            setNextState(CreateTableState.CREATE_TABLE_SET_ERASURE_CODING_POLICY);
111          } else {
112            setNextState(CreateTableState.CREATE_TABLE_ADD_TO_META);
113          }
114          break;
115        case CREATE_TABLE_SET_ERASURE_CODING_POLICY:
116          ErasureCodingUtils.setPolicy(env.getMasterFileSystem().getFileSystem(),
117            env.getMasterFileSystem().getRootDir(), getTableName(),
118            tableDescriptor.getErasureCodingPolicy());
119          setNextState(CreateTableState.CREATE_TABLE_ADD_TO_META);
120          break;
121        case CREATE_TABLE_ADD_TO_META:
122          newRegions = addTableToMeta(env, tableDescriptor, newRegions);
123          setNextState(CreateTableState.CREATE_TABLE_ASSIGN_REGIONS);
124          break;
125        case CREATE_TABLE_ASSIGN_REGIONS:
126          setEnablingState(env, getTableName());
127          addChildProcedure(
128            env.getAssignmentManager().createRoundRobinAssignProcedures(newRegions));
129          setNextState(CreateTableState.CREATE_TABLE_UPDATE_DESC_CACHE);
130          break;
131        case CREATE_TABLE_UPDATE_DESC_CACHE:
132          // XXX: this stage should be named as set table enabled, as now we will cache the
133          // descriptor after writing fs layout.
134          setEnabledState(env, getTableName());
135          setNextState(CreateTableState.CREATE_TABLE_POST_OPERATION);
136          break;
137        case CREATE_TABLE_POST_OPERATION:
138          postCreate(env);
139          retryCounter = null;
140          return Flow.NO_MORE_STATE;
141        default:
142          throw new UnsupportedOperationException("unhandled state=" + state);
143      }
144    } catch (IOException e) {
145      if (isRollbackSupported(state)) {
146        setFailure("master-create-table", e);
147      } else {
148        if (retryCounter == null) {
149          retryCounter = ProcedureUtil.createRetryCounter(env.getMasterConfiguration());
150        }
151        long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
152        LOG.warn("Retriable error trying to create table={},state={},suspend {}secs.",
153          getTableName(), state, backoff / 1000, e);
154        throw suspend(Math.toIntExact(backoff), true);
155      }
156    }
157    retryCounter = null;
158    return Flow.HAS_MORE_STATE;
159  }
160
161  @Override
162  protected synchronized boolean setTimeoutFailure(MasterProcedureEnv env) {
163    setState(ProcedureProtos.ProcedureState.RUNNABLE);
164    env.getProcedureScheduler().addFront(this);
165    return false;
166  }
167
168  @Override
169  protected void rollbackState(final MasterProcedureEnv env, final CreateTableState state)
170    throws IOException {
171    if (state == CreateTableState.CREATE_TABLE_PRE_OPERATION) {
172      // nothing to rollback, pre-create is just table-state checks.
173      // We can fail if the table does exist or the descriptor is malformed.
174      // TODO: coprocessor rollback semantic is still undefined.
175      if (
176        hasException()
177          /* avoid NPE */ && getException().getCause().getClass() != TableExistsException.class
178      ) {
179        DeleteTableProcedure.deleteTableStates(env, getTableName());
180
181        final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
182        if (cpHost != null) {
183          cpHost.postDeleteTable(getTableName());
184        }
185      }
186
187      releaseSyncLatch();
188      return;
189    }
190
191    // The procedure doesn't have a rollback. The execution will succeed, at some point.
192    throw new UnsupportedOperationException("unhandled state=" + state);
193  }
194
195  @Override
196  protected boolean isRollbackSupported(final CreateTableState state) {
197    switch (state) {
198      case CREATE_TABLE_PRE_OPERATION:
199        return true;
200      default:
201        return false;
202    }
203  }
204
205  @Override
206  protected CreateTableState getState(final int stateId) {
207    return CreateTableState.forNumber(stateId);
208  }
209
210  @Override
211  protected int getStateId(final CreateTableState state) {
212    return state.getNumber();
213  }
214
215  @Override
216  protected CreateTableState getInitialState() {
217    return CreateTableState.CREATE_TABLE_PRE_OPERATION;
218  }
219
220  @Override
221  public TableName getTableName() {
222    return tableDescriptor.getTableName();
223  }
224
225  @Override
226  public TableOperationType getTableOperationType() {
227    return TableOperationType.CREATE;
228  }
229
230  @Override
231  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
232    super.serializeStateData(serializer);
233
234    MasterProcedureProtos.CreateTableStateData.Builder state =
235      MasterProcedureProtos.CreateTableStateData.newBuilder()
236        .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
237        .setTableSchema(ProtobufUtil.toTableSchema(tableDescriptor));
238    if (newRegions != null) {
239      for (RegionInfo hri : newRegions) {
240        state.addRegionInfo(ProtobufUtil.toRegionInfo(hri));
241      }
242    }
243    serializer.serialize(state.build());
244  }
245
246  @Override
247  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
248    super.deserializeStateData(serializer);
249
250    MasterProcedureProtos.CreateTableStateData state =
251      serializer.deserialize(MasterProcedureProtos.CreateTableStateData.class);
252    setUser(MasterProcedureUtil.toUserInfo(state.getUserInfo()));
253    tableDescriptor = ProtobufUtil.toTableDescriptor(state.getTableSchema());
254    if (state.getRegionInfoCount() == 0) {
255      newRegions = null;
256    } else {
257      newRegions = new ArrayList<>(state.getRegionInfoCount());
258      for (HBaseProtos.RegionInfo hri : state.getRegionInfoList()) {
259        newRegions.add(ProtobufUtil.toRegionInfo(hri));
260      }
261    }
262  }
263
264  @Override
265  protected boolean waitInitialized(MasterProcedureEnv env) {
266    if (getTableName().isSystemTable()) {
267      // Creating system table is part of the initialization, so only wait for meta loaded instead
268      // of waiting for master fully initialized.
269      return env.getAssignmentManager().waitMetaLoaded(this);
270    }
271    return super.waitInitialized(env);
272  }
273
274  private boolean prepareCreate(final MasterProcedureEnv env) throws IOException {
275    final TableName tableName = getTableName();
276    if (env.getMasterServices().getTableDescriptors().exists(tableName)) {
277      setFailure("master-create-table", new TableExistsException(getTableName()));
278      return false;
279    }
280
281    // check that we have at least 1 CF
282    if (tableDescriptor.getColumnFamilyCount() == 0) {
283      setFailure("master-create-table", new DoNotRetryIOException(
284        "Table " + getTableName().toString() + " should have at least one column family."));
285      return false;
286    }
287
288    int regionReplicationCount = tableDescriptor.getRegionReplication();
289    if (regionReplicationCount > MAX_REGION_REPLICATION) {
290      setFailure("master-create-table", new IllegalArgumentException(
291        "Region Replication cannot exceed " + MAX_REGION_REPLICATION + "."));
292      return false;
293    }
294
295    // check for store file tracker configurations
296    StoreFileTrackerValidationUtils.checkForCreateTable(env.getMasterConfiguration(),
297      tableDescriptor);
298
299    return true;
300  }
301
302  private void preCreate(final MasterProcedureEnv env) throws IOException, InterruptedException {
303    if (!getTableName().isSystemTable()) {
304      ProcedureSyncWait.getMasterQuotaManager(env).checkNamespaceTableAndRegionQuota(getTableName(),
305        (newRegions != null ? newRegions.size() : 0));
306    }
307
308    tableDescriptor = StoreFileTrackerFactory.updateWithTrackerConfigs(env.getMasterConfiguration(),
309      tableDescriptor);
310
311    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
312    if (cpHost != null) {
313      final RegionInfo[] regions =
314        newRegions == null ? null : newRegions.toArray(new RegionInfo[newRegions.size()]);
315      cpHost.preCreateTableAction(tableDescriptor, regions, getUser());
316    }
317  }
318
319  private void postCreate(final MasterProcedureEnv env) throws IOException, InterruptedException {
320    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
321    if (cpHost != null) {
322      final RegionInfo[] regions =
323        (newRegions == null) ? null : newRegions.toArray(new RegionInfo[newRegions.size()]);
324      cpHost.postCompletedCreateTableAction(tableDescriptor, regions, getUser());
325    }
326  }
327
328  protected interface CreateHdfsRegions {
329    List<RegionInfo> createHdfsRegions(final MasterProcedureEnv env, final Path tableRootDir,
330      final TableName tableName, final List<RegionInfo> newRegions) throws IOException;
331  }
332
333  protected static List<RegionInfo> createFsLayout(final MasterProcedureEnv env,
334    final TableDescriptor tableDescriptor, final List<RegionInfo> newRegions) throws IOException {
335    return createFsLayout(env, tableDescriptor, newRegions, new CreateHdfsRegions() {
336      @Override
337      public List<RegionInfo> createHdfsRegions(final MasterProcedureEnv env,
338        final Path tableRootDir, final TableName tableName, final List<RegionInfo> newRegions)
339        throws IOException {
340        RegionInfo[] regions =
341          newRegions != null ? newRegions.toArray(new RegionInfo[newRegions.size()]) : null;
342        return ModifyRegionUtils.createRegions(env.getMasterConfiguration(), tableRootDir,
343          tableDescriptor, regions, null);
344      }
345    });
346  }
347
348  protected static List<RegionInfo> createFsLayout(final MasterProcedureEnv env,
349    final TableDescriptor tableDescriptor, List<RegionInfo> newRegions,
350    final CreateHdfsRegions hdfsRegionHandler) throws IOException {
351    final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
352
353    // 1. Create Table Descriptor
354    // using a copy of descriptor, table will be created enabling first
355    final Path tableDir =
356      CommonFSUtils.getTableDir(mfs.getRootDir(), tableDescriptor.getTableName());
357    ((FSTableDescriptors) (env.getMasterServices().getTableDescriptors()))
358      .createTableDescriptorForTableDirectory(tableDir, tableDescriptor, false);
359
360    // 2. Create Regions
361    newRegions = hdfsRegionHandler.createHdfsRegions(env, mfs.getRootDir(),
362      tableDescriptor.getTableName(), newRegions);
363
364    return newRegions;
365  }
366
367  protected static List<RegionInfo> addTableToMeta(final MasterProcedureEnv env,
368    final TableDescriptor tableDescriptor, final List<RegionInfo> regions) throws IOException {
369    assert (regions != null && regions.size() > 0) : "expected at least 1 region, got " + regions;
370
371    ProcedureSyncWait.waitMetaRegions(env);
372
373    // Add replicas if needed
374    // we need to create regions with replicaIds starting from 1
375    List<RegionInfo> newRegions =
376      RegionReplicaUtil.addReplicas(regions, 1, tableDescriptor.getRegionReplication());
377
378    // Add regions to META
379    addRegionsToMeta(env, tableDescriptor, newRegions);
380
381    // Setup replication for region replicas if needed
382    if (tableDescriptor.getRegionReplication() > 1) {
383      try {
384        ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterServices());
385      } catch (ReplicationException e) {
386        throw new HBaseIOException(e);
387      }
388    }
389    return newRegions;
390  }
391
392  protected static void setEnablingState(final MasterProcedureEnv env, final TableName tableName)
393    throws IOException {
394    // Mark the table as Enabling
395    env.getMasterServices().getTableStateManager().setTableState(tableName,
396      TableState.State.ENABLING);
397  }
398
399  protected static void setEnabledState(final MasterProcedureEnv env, final TableName tableName)
400    throws IOException {
401    // Enable table
402    env.getMasterServices().getTableStateManager().setTableState(tableName,
403      TableState.State.ENABLED);
404  }
405
406  /**
407   * Add the specified set of regions to the hbase:meta table.
408   */
409  private static void addRegionsToMeta(final MasterProcedureEnv env,
410    final TableDescriptor tableDescriptor, final List<RegionInfo> regionInfos) throws IOException {
411    MetaTableAccessor.addRegionsToMeta(env.getMasterServices().getConnection(), regionInfos,
412      tableDescriptor.getRegionReplication());
413  }
414
415  @Override
416  protected boolean shouldWaitClientAck(MasterProcedureEnv env) {
417    // system tables are created on bootstrap internally by the system
418    // the client does not know about this procedures.
419    return !getTableName().isSystemTable();
420  }
421
422  RegionInfo getFirstRegionInfo() {
423    if (newRegions == null || newRegions.isEmpty()) {
424      return null;
425    }
426    return newRegions.get(0);
427  }
428}