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.Collections;
023import java.util.HashSet;
024import java.util.List;
025import java.util.Set;
026import java.util.function.Supplier;
027import java.util.stream.Collectors;
028import java.util.stream.IntStream;
029import org.apache.hadoop.hbase.ConcurrentTableModificationException;
030import org.apache.hadoop.hbase.DoNotRetryIOException;
031import org.apache.hadoop.hbase.HBaseIOException;
032import org.apache.hadoop.hbase.HConstants;
033import org.apache.hadoop.hbase.TableName;
034import org.apache.hadoop.hbase.TableNotFoundException;
035import org.apache.hadoop.hbase.client.RegionInfo;
036import org.apache.hadoop.hbase.client.RegionReplicaUtil;
037import org.apache.hadoop.hbase.client.TableDescriptor;
038import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
039import org.apache.hadoop.hbase.fs.ErasureCodingUtils;
040import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
041import org.apache.hadoop.hbase.master.zksyncer.MetaLocationSyncer;
042import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
043import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerValidationUtils;
044import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
045import org.apache.hadoop.hbase.util.Bytes;
046import org.apache.yetus.audience.InterfaceAudience;
047import org.slf4j.Logger;
048import org.slf4j.LoggerFactory;
049
050import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
051import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
052import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableState;
053
054@InterfaceAudience.Private
055public class ModifyTableProcedure extends AbstractStateMachineTableProcedure<ModifyTableState> {
056  private static final Logger LOG = LoggerFactory.getLogger(ModifyTableProcedure.class);
057
058  private TableDescriptor unmodifiedTableDescriptor = null;
059  private TableDescriptor modifiedTableDescriptor;
060  private boolean deleteColumnFamilyInModify;
061  private boolean shouldCheckDescriptor;
062  private boolean reopenRegions;
063  /**
064   * List of column families that cannot be deleted from the hbase:meta table. They are critical to
065   * cluster operation. This is a bit of an odd place to keep this list but then this is the tooling
066   * that does add/remove. Keeping it local!
067   */
068  private static final List<byte[]> UNDELETABLE_META_COLUMNFAMILIES =
069    Collections.unmodifiableList(Arrays.asList(HConstants.CATALOG_FAMILY, HConstants.TABLE_FAMILY,
070      HConstants.REPLICATION_BARRIER_FAMILY, HConstants.NAMESPACE_FAMILY));
071
072  public ModifyTableProcedure() {
073    super();
074    initialize(null, false);
075  }
076
077  public ModifyTableProcedure(final MasterProcedureEnv env, final TableDescriptor htd)
078    throws HBaseIOException {
079    this(env, htd, null);
080  }
081
082  public ModifyTableProcedure(final MasterProcedureEnv env, final TableDescriptor htd,
083    final ProcedurePrepareLatch latch) throws HBaseIOException {
084    this(env, htd, latch, null, false, true);
085  }
086
087  public ModifyTableProcedure(final MasterProcedureEnv env,
088    final TableDescriptor newTableDescriptor, final ProcedurePrepareLatch latch,
089    final TableDescriptor oldTableDescriptor, final boolean shouldCheckDescriptor,
090    final boolean reopenRegions) throws HBaseIOException {
091    super(env, latch);
092    this.reopenRegions = reopenRegions;
093    initialize(oldTableDescriptor, shouldCheckDescriptor);
094    this.modifiedTableDescriptor = newTableDescriptor;
095    preflightChecks(env, null/* No table checks; if changing peers, table can be online */);
096  }
097
098  @Override
099  protected void preflightChecks(MasterProcedureEnv env, Boolean enabled) throws HBaseIOException {
100    super.preflightChecks(env, enabled);
101    if (this.modifiedTableDescriptor.isMetaTable()) {
102      // If we are modifying the hbase:meta table, make sure we are not deleting critical
103      // column families else we'll damage the cluster.
104      Set<byte[]> cfs = this.modifiedTableDescriptor.getColumnFamilyNames();
105      for (byte[] family : UNDELETABLE_META_COLUMNFAMILIES) {
106        if (!cfs.contains(family)) {
107          throw new HBaseIOException(
108            "Delete of hbase:meta column family " + Bytes.toString(family));
109        }
110      }
111    }
112
113    if (!reopenRegions) {
114      if (this.unmodifiedTableDescriptor == null) {
115        throw new HBaseIOException(
116          "unmodifiedTableDescriptor cannot be null when this table modification won't reopen regions");
117      }
118      if (
119        !this.unmodifiedTableDescriptor.getTableName()
120          .equals(this.modifiedTableDescriptor.getTableName())
121      ) {
122        throw new HBaseIOException(
123          "Cannot change the table name when this modification won't " + "reopen regions.");
124      }
125      if (
126        this.unmodifiedTableDescriptor.getColumnFamilyCount()
127            != this.modifiedTableDescriptor.getColumnFamilyCount()
128      ) {
129        throw new HBaseIOException(
130          "Cannot add or remove column families when this modification " + "won't reopen regions.");
131      }
132      if (
133        this.unmodifiedTableDescriptor.getCoprocessorDescriptors().hashCode()
134            != this.modifiedTableDescriptor.getCoprocessorDescriptors().hashCode()
135      ) {
136        throw new HBaseIOException(
137          "Can not modify Coprocessor when table modification won't reopen regions");
138      }
139      final Set<String> s = new HashSet<>(Arrays.asList(TableDescriptorBuilder.REGION_REPLICATION,
140        TableDescriptorBuilder.REGION_MEMSTORE_REPLICATION, RSGroupInfo.TABLE_DESC_PROP_GROUP));
141      for (String k : s) {
142        if (
143          isTablePropertyModified(this.unmodifiedTableDescriptor, this.modifiedTableDescriptor, k)
144        ) {
145          throw new HBaseIOException(
146            "Can not modify " + k + " of a table when modification won't reopen regions");
147        }
148      }
149    }
150  }
151
152  /**
153   * Comparing the value associated with a given key across two TableDescriptor instances'
154   * properties.
155   * @return True if the table property <code>key</code> is the same in both.
156   */
157  private boolean isTablePropertyModified(TableDescriptor oldDescriptor,
158    TableDescriptor newDescriptor, String key) {
159    String oldV = oldDescriptor.getValue(key);
160    String newV = newDescriptor.getValue(key);
161    if (oldV == null && newV == null) {
162      return false;
163    } else if (oldV != null && newV != null && oldV.equals(newV)) {
164      return false;
165    }
166    return true;
167  }
168
169  private void initialize(final TableDescriptor unmodifiedTableDescriptor,
170    final boolean shouldCheckDescriptor) {
171    this.unmodifiedTableDescriptor = unmodifiedTableDescriptor;
172    this.shouldCheckDescriptor = shouldCheckDescriptor;
173    this.deleteColumnFamilyInModify = false;
174  }
175
176  @Override
177  protected Flow executeFromState(final MasterProcedureEnv env, final ModifyTableState state)
178    throws InterruptedException {
179    LOG.trace("{} execute state={}", this, state);
180    try {
181      switch (state) {
182        case MODIFY_TABLE_PREPARE:
183          prepareModify(env);
184          setNextState(ModifyTableState.MODIFY_TABLE_PRE_OPERATION);
185          break;
186        case MODIFY_TABLE_PRE_OPERATION:
187          preModify(env, state);
188          // We cannot allow changes to region replicas when 'reopenRegions==false',
189          // as this mode bypasses the state management required for modifying region replicas.
190          if (reopenRegions) {
191            setNextState(ModifyTableState.MODIFY_TABLE_CLOSE_EXCESS_REPLICAS);
192          } else {
193            setNextState(ModifyTableState.MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR);
194          }
195          break;
196        case MODIFY_TABLE_CLOSE_EXCESS_REPLICAS:
197          if (isTableEnabled(env)) {
198            closeExcessReplicasIfNeeded(env);
199          }
200          setNextState(ModifyTableState.MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR);
201          break;
202        case MODIFY_TABLE_UPDATE_TABLE_DESCRIPTOR:
203          updateTableDescriptor(env);
204          if (reopenRegions) {
205            setNextState(ModifyTableState.MODIFY_TABLE_REMOVE_REPLICA_COLUMN);
206          } else {
207            setNextState(ModifyTableState.MODIFY_TABLE_POST_OPERATION);
208          }
209          break;
210        case MODIFY_TABLE_REMOVE_REPLICA_COLUMN:
211          removeReplicaColumnsIfNeeded(env);
212          setNextState(ModifyTableState.MODIFY_TABLE_POST_OPERATION);
213          break;
214        case MODIFY_TABLE_POST_OPERATION:
215          postModify(env, state);
216          if (reopenRegions) {
217            setNextState(ModifyTableState.MODIFY_TABLE_REOPEN_ALL_REGIONS);
218          } else
219            if (ErasureCodingUtils.needsSync(unmodifiedTableDescriptor, modifiedTableDescriptor)) {
220              setNextState(ModifyTableState.MODIFY_TABLE_SYNC_ERASURE_CODING_POLICY);
221            } else {
222              return Flow.NO_MORE_STATE;
223            }
224          break;
225        case MODIFY_TABLE_REOPEN_ALL_REGIONS:
226          if (isTableEnabled(env)) {
227            addChildProcedure(ReopenTableRegionsProcedure.throttled(env.getMasterConfiguration(),
228              env.getMasterServices().getTableDescriptors().get(getTableName())));
229          }
230          setNextState(ModifyTableState.MODIFY_TABLE_ASSIGN_NEW_REPLICAS);
231          break;
232        case MODIFY_TABLE_ASSIGN_NEW_REPLICAS:
233          assignNewReplicasIfNeeded(env);
234          if (TableName.isMetaTableName(getTableName())) {
235            MetaLocationSyncer syncer = env.getMasterServices().getMetaLocationSyncer();
236            if (syncer != null) {
237              syncer.setMetaReplicaCount(modifiedTableDescriptor.getRegionReplication());
238            }
239          }
240          if (deleteColumnFamilyInModify) {
241            setNextState(ModifyTableState.MODIFY_TABLE_DELETE_FS_LAYOUT);
242          } else
243            if (ErasureCodingUtils.needsSync(unmodifiedTableDescriptor, modifiedTableDescriptor)) {
244              setNextState(ModifyTableState.MODIFY_TABLE_SYNC_ERASURE_CODING_POLICY);
245            } else {
246              return Flow.NO_MORE_STATE;
247            }
248          break;
249        case MODIFY_TABLE_DELETE_FS_LAYOUT:
250          deleteFromFs(env, unmodifiedTableDescriptor, modifiedTableDescriptor);
251          if (ErasureCodingUtils.needsSync(unmodifiedTableDescriptor, modifiedTableDescriptor)) {
252            setNextState(ModifyTableState.MODIFY_TABLE_SYNC_ERASURE_CODING_POLICY);
253            break;
254          } else {
255            return Flow.NO_MORE_STATE;
256          }
257        case MODIFY_TABLE_SYNC_ERASURE_CODING_POLICY:
258          ErasureCodingUtils.sync(env.getMasterFileSystem().getFileSystem(),
259            env.getMasterFileSystem().getRootDir(), modifiedTableDescriptor);
260          return Flow.NO_MORE_STATE;
261        default:
262          throw new UnsupportedOperationException("unhandled state=" + state);
263      }
264    } catch (IOException e) {
265      if (isRollbackSupported(state)) {
266        setFailure("master-modify-table", e);
267      } else {
268        LOG.warn("Retriable error trying to modify table={} (in state={})", getTableName(), state,
269          e);
270      }
271    }
272    return Flow.HAS_MORE_STATE;
273  }
274
275  @Override
276  protected void rollbackState(final MasterProcedureEnv env, final ModifyTableState state)
277    throws IOException {
278    if (
279      state == ModifyTableState.MODIFY_TABLE_PREPARE
280        || state == ModifyTableState.MODIFY_TABLE_PRE_OPERATION
281    ) {
282      // nothing to rollback, pre-modify is just checks.
283      // TODO: coprocessor rollback semantic is still undefined.
284      return;
285    }
286
287    // The delete doesn't have a rollback. The execution will succeed, at some point.
288    throw new UnsupportedOperationException("unhandled state=" + state);
289  }
290
291  @Override
292  protected boolean isRollbackSupported(final ModifyTableState state) {
293    switch (state) {
294      case MODIFY_TABLE_PRE_OPERATION:
295      case MODIFY_TABLE_PREPARE:
296        return true;
297      default:
298        return false;
299    }
300  }
301
302  @Override
303  protected void completionCleanup(final MasterProcedureEnv env) {
304    releaseSyncLatch();
305  }
306
307  @Override
308  protected ModifyTableState getState(final int stateId) {
309    return ModifyTableState.forNumber(stateId);
310  }
311
312  @Override
313  protected int getStateId(final ModifyTableState state) {
314    return state.getNumber();
315  }
316
317  @Override
318  protected ModifyTableState getInitialState() {
319    return ModifyTableState.MODIFY_TABLE_PREPARE;
320  }
321
322  @Override
323  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
324    super.serializeStateData(serializer);
325
326    MasterProcedureProtos.ModifyTableStateData.Builder modifyTableMsg =
327      MasterProcedureProtos.ModifyTableStateData.newBuilder()
328        .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
329        .setModifiedTableSchema(ProtobufUtil.toTableSchema(modifiedTableDescriptor))
330        .setDeleteColumnFamilyInModify(deleteColumnFamilyInModify)
331        .setShouldCheckDescriptor(shouldCheckDescriptor).setReopenRegions(reopenRegions);
332
333    if (unmodifiedTableDescriptor != null) {
334      modifyTableMsg
335        .setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedTableDescriptor));
336    }
337
338    serializer.serialize(modifyTableMsg.build());
339  }
340
341  @Override
342  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
343    super.deserializeStateData(serializer);
344
345    MasterProcedureProtos.ModifyTableStateData modifyTableMsg =
346      serializer.deserialize(MasterProcedureProtos.ModifyTableStateData.class);
347    setUser(MasterProcedureUtil.toUserInfo(modifyTableMsg.getUserInfo()));
348    modifiedTableDescriptor =
349      ProtobufUtil.toTableDescriptor(modifyTableMsg.getModifiedTableSchema());
350    deleteColumnFamilyInModify = modifyTableMsg.getDeleteColumnFamilyInModify();
351    shouldCheckDescriptor =
352      modifyTableMsg.hasShouldCheckDescriptor() ? modifyTableMsg.getShouldCheckDescriptor() : false;
353    reopenRegions = modifyTableMsg.hasReopenRegions() ? modifyTableMsg.getReopenRegions() : true;
354
355    if (modifyTableMsg.hasUnmodifiedTableSchema()) {
356      unmodifiedTableDescriptor =
357        ProtobufUtil.toTableDescriptor(modifyTableMsg.getUnmodifiedTableSchema());
358    }
359  }
360
361  @Override
362  public TableName getTableName() {
363    return modifiedTableDescriptor.getTableName();
364  }
365
366  @Override
367  public TableOperationType getTableOperationType() {
368    return TableOperationType.EDIT;
369  }
370
371  /**
372   * Check conditions before any real action of modifying a table.
373   */
374  private void prepareModify(final MasterProcedureEnv env) throws IOException {
375    // Checks whether the table exists
376    if (!env.getMasterServices().getTableDescriptors().exists(getTableName())) {
377      throw new TableNotFoundException(getTableName());
378    }
379
380    // check that we have at least 1 CF
381    if (modifiedTableDescriptor.getColumnFamilyCount() == 0) {
382      throw new DoNotRetryIOException(
383        "Table " + getTableName().toString() + " should have at least one column family.");
384    }
385
386    // If descriptor check is enabled, check whether the table descriptor when procedure was
387    // submitted matches with the current
388    // table descriptor of the table, else retrieve the old descriptor
389    // for comparison in order to update the descriptor.
390    if (shouldCheckDescriptor) {
391      if (
392        TableDescriptor.COMPARATOR.compare(unmodifiedTableDescriptor,
393          env.getMasterServices().getTableDescriptors().get(getTableName())) != 0
394      ) {
395        LOG.error("Error while modifying table '" + getTableName().toString()
396          + "' Skipping procedure : " + this);
397        throw new ConcurrentTableModificationException(
398          "Skipping modify table operation on table '" + getTableName().toString()
399            + "' as it has already been modified by some other concurrent operation, "
400            + "Please retry.");
401      }
402    } else {
403      this.unmodifiedTableDescriptor =
404        env.getMasterServices().getTableDescriptors().get(getTableName());
405    }
406
407    this.deleteColumnFamilyInModify =
408      isDeleteColumnFamily(unmodifiedTableDescriptor, modifiedTableDescriptor);
409    if (
410      !unmodifiedTableDescriptor.getRegionServerGroup()
411        .equals(modifiedTableDescriptor.getRegionServerGroup())
412    ) {
413      Supplier<String> forWhom = () -> "table " + getTableName();
414      RSGroupInfo rsGroupInfo = MasterProcedureUtil.checkGroupExists(
415        env.getMasterServices().getRSGroupInfoManager()::getRSGroup,
416        modifiedTableDescriptor.getRegionServerGroup(), forWhom);
417      MasterProcedureUtil.checkGroupNotEmpty(rsGroupInfo, forWhom);
418    }
419
420    // check for store file tracker configurations
421    StoreFileTrackerValidationUtils.checkForModifyTable(env.getMasterConfiguration(),
422      unmodifiedTableDescriptor, modifiedTableDescriptor, !isTableEnabled(env));
423  }
424
425  /**
426   * Find out whether all column families in unmodifiedTableDescriptor also exists in the
427   * modifiedTableDescriptor.
428   * @return True if we are deleting a column family.
429   */
430  private static boolean isDeleteColumnFamily(TableDescriptor originalDescriptor,
431    TableDescriptor newDescriptor) {
432    boolean result = false;
433    final Set<byte[]> originalFamilies = originalDescriptor.getColumnFamilyNames();
434    final Set<byte[]> newFamilies = newDescriptor.getColumnFamilyNames();
435    for (byte[] familyName : originalFamilies) {
436      if (!newFamilies.contains(familyName)) {
437        result = true;
438        break;
439      }
440    }
441    return result;
442  }
443
444  /**
445   * Action before modifying table.
446   * @param env   MasterProcedureEnv
447   * @param state the procedure state
448   */
449  private void preModify(final MasterProcedureEnv env, final ModifyTableState state)
450    throws IOException, InterruptedException {
451    runCoprocessorAction(env, state);
452  }
453
454  /**
455   * Update descriptor
456   * @param env MasterProcedureEnv
457   **/
458  private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException {
459    env.getMasterServices().getTableDescriptors().update(modifiedTableDescriptor);
460  }
461
462  /**
463   * Removes from hdfs the families that are not longer present in the new table descriptor.
464   * @param env MasterProcedureEnv
465   */
466  private void deleteFromFs(final MasterProcedureEnv env, final TableDescriptor oldTableDescriptor,
467    final TableDescriptor newTableDescriptor) throws IOException {
468    final Set<byte[]> oldFamilies = oldTableDescriptor.getColumnFamilyNames();
469    final Set<byte[]> newFamilies = newTableDescriptor.getColumnFamilyNames();
470    for (byte[] familyName : oldFamilies) {
471      if (!newFamilies.contains(familyName)) {
472        MasterDDLOperationHelper.deleteColumnFamilyFromFileSystem(env, getTableName(),
473          getRegionInfoList(env), familyName,
474          oldTableDescriptor.getColumnFamily(familyName).isMobEnabled());
475      }
476    }
477  }
478
479  /**
480   * remove replica columns if necessary.
481   */
482  private void removeReplicaColumnsIfNeeded(MasterProcedureEnv env) throws IOException {
483    final int oldReplicaCount = unmodifiedTableDescriptor.getRegionReplication();
484    final int newReplicaCount = modifiedTableDescriptor.getRegionReplication();
485    if (newReplicaCount >= oldReplicaCount) {
486      return;
487    }
488    env.getAssignmentManager().getRegionStateStore().removeRegionReplicas(getTableName(),
489      oldReplicaCount, newReplicaCount);
490    env.getAssignmentManager().getRegionStates().getRegionsOfTable(getTableName()).stream()
491      .filter(r -> r.getReplicaId() >= newReplicaCount)
492      .forEach(env.getAssignmentManager().getRegionStates()::deleteRegion);
493  }
494
495  private void assignNewReplicasIfNeeded(MasterProcedureEnv env) throws IOException {
496    final int oldReplicaCount = unmodifiedTableDescriptor.getRegionReplication();
497    final int newReplicaCount = modifiedTableDescriptor.getRegionReplication();
498    int existingReplicasCount = env.getAssignmentManager().getRegionStates()
499      .getRegionsOfTable(modifiedTableDescriptor.getTableName()).size();
500    if (newReplicaCount <= Math.min(oldReplicaCount, existingReplicasCount)) {
501      return;
502    }
503    if (isTableEnabled(env)) {
504      List<RegionInfo> newReplicas = env.getAssignmentManager().getRegionStates()
505        .getRegionsOfTable(getTableName()).stream().filter(RegionReplicaUtil::isDefaultReplica)
506        .flatMap(primaryRegion -> IntStream
507          .range(Math.min(oldReplicaCount, existingReplicasCount), newReplicaCount).mapToObj(
508            replicaId -> RegionReplicaUtil.getRegionInfoForReplica(primaryRegion, replicaId)))
509        .collect(Collectors.toList());
510      addChildProcedure(env.getAssignmentManager().createAssignProcedures(newReplicas));
511    }
512  }
513
514  private void closeExcessReplicasIfNeeded(MasterProcedureEnv env) {
515    final int oldReplicaCount = unmodifiedTableDescriptor.getRegionReplication();
516    final int newReplicaCount = modifiedTableDescriptor.getRegionReplication();
517    if (newReplicaCount >= oldReplicaCount) {
518      return;
519    }
520    addChildProcedure(new CloseExcessRegionReplicasProcedure(getTableName(), newReplicaCount));
521  }
522
523  /**
524   * Action after modifying table.
525   * @param env   MasterProcedureEnv
526   * @param state the procedure state
527   */
528  private void postModify(final MasterProcedureEnv env, final ModifyTableState state)
529    throws IOException, InterruptedException {
530    runCoprocessorAction(env, state);
531  }
532
533  /**
534   * Coprocessor Action.
535   * @param env   MasterProcedureEnv
536   * @param state the procedure state
537   */
538  private void runCoprocessorAction(final MasterProcedureEnv env, final ModifyTableState state)
539    throws IOException, InterruptedException {
540    final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
541    if (cpHost != null) {
542      switch (state) {
543        case MODIFY_TABLE_PRE_OPERATION:
544          cpHost.preModifyTableAction(getTableName(), unmodifiedTableDescriptor,
545            modifiedTableDescriptor, getUser());
546          break;
547        case MODIFY_TABLE_POST_OPERATION:
548          cpHost.postCompletedModifyTableAction(getTableName(), unmodifiedTableDescriptor,
549            modifiedTableDescriptor, getUser());
550          break;
551        default:
552          throw new UnsupportedOperationException(this + " unhandled state=" + state);
553      }
554    }
555  }
556
557  /**
558   * Fetches all Regions for a table. Cache the result of this method if you need to use it multiple
559   * times. Be aware that it may change over in between calls to this procedure.
560   */
561  private List<RegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
562    return env.getAssignmentManager().getRegionStates().getRegionsOfTable(getTableName());
563  }
564}