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