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.assignment; 019 020import java.io.IOException; 021import java.io.InterruptedIOException; 022import java.util.ArrayList; 023import java.util.Arrays; 024import java.util.Collection; 025import java.util.Collections; 026import java.util.HashMap; 027import java.util.List; 028import java.util.Map; 029import java.util.concurrent.Callable; 030import java.util.concurrent.ExecutionException; 031import java.util.concurrent.ExecutorService; 032import java.util.concurrent.Executors; 033import java.util.concurrent.Future; 034import java.util.concurrent.TimeUnit; 035import java.util.stream.Stream; 036import org.apache.hadoop.conf.Configuration; 037import org.apache.hadoop.fs.FileSystem; 038import org.apache.hadoop.fs.Path; 039import org.apache.hadoop.hbase.DoNotRetryIOException; 040import org.apache.hadoop.hbase.HConstants; 041import org.apache.hadoop.hbase.ServerName; 042import org.apache.hadoop.hbase.TableName; 043import org.apache.hadoop.hbase.UnknownRegionException; 044import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; 045import org.apache.hadoop.hbase.client.MasterSwitchType; 046import org.apache.hadoop.hbase.client.Mutation; 047import org.apache.hadoop.hbase.client.RegionInfo; 048import org.apache.hadoop.hbase.client.RegionInfoBuilder; 049import org.apache.hadoop.hbase.client.TableDescriptor; 050import org.apache.hadoop.hbase.io.hfile.CacheConfig; 051import org.apache.hadoop.hbase.master.MasterCoprocessorHost; 052import org.apache.hadoop.hbase.master.MasterFileSystem; 053import org.apache.hadoop.hbase.master.RegionState.State; 054import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan; 055import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure; 056import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; 057import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil; 058import org.apache.hadoop.hbase.procedure2.ProcedureMetrics; 059import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; 060import org.apache.hadoop.hbase.quotas.QuotaExceededException; 061import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; 062import org.apache.hadoop.hbase.regionserver.HStore; 063import org.apache.hadoop.hbase.regionserver.HStoreFile; 064import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy; 065import org.apache.hadoop.hbase.regionserver.RegionSplitRestriction; 066import org.apache.hadoop.hbase.regionserver.StoreFileInfo; 067import org.apache.hadoop.hbase.regionserver.StoreUtils; 068import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; 069import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; 070import org.apache.hadoop.hbase.util.Bytes; 071import org.apache.hadoop.hbase.util.CommonFSUtils; 072import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 073import org.apache.hadoop.hbase.util.FSUtils; 074import org.apache.hadoop.hbase.util.Pair; 075import org.apache.hadoop.hbase.util.Threads; 076import org.apache.hadoop.hbase.wal.WALSplitUtil; 077import org.apache.hadoop.util.ReflectionUtils; 078import org.apache.yetus.audience.InterfaceAudience; 079import org.slf4j.Logger; 080import org.slf4j.LoggerFactory; 081 082import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; 083 084import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 085import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse; 086import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; 087import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState; 088 089/** 090 * The procedure to split a region in a table. Takes lock on the parent region. It holds the lock 091 * for the life of the procedure. 092 * <p> 093 * Throws exception on construction if determines context hostile to spllt (cluster going down or 094 * master is shutting down or table is disabled). 095 * </p> 096 */ 097@InterfaceAudience.Private 098public class SplitTableRegionProcedure 099 extends AbstractStateMachineRegionProcedure<SplitTableRegionState> { 100 private static final Logger LOG = LoggerFactory.getLogger(SplitTableRegionProcedure.class); 101 private RegionInfo daughterOneRI; 102 private RegionInfo daughterTwoRI; 103 private byte[] bestSplitRow; 104 private RegionSplitPolicy splitPolicy; 105 106 public SplitTableRegionProcedure() { 107 // Required by the Procedure framework to create the procedure on replay 108 } 109 110 public SplitTableRegionProcedure(final MasterProcedureEnv env, final RegionInfo regionToSplit, 111 final byte[] splitRow) throws IOException { 112 super(env, regionToSplit); 113 preflightChecks(env, true); 114 // When procedure goes to run in its prepare step, it also does these checkOnline checks. Here 115 // we fail-fast on construction. There it skips the split with just a warning. 116 checkOnline(env, regionToSplit); 117 this.bestSplitRow = splitRow; 118 TableDescriptor tableDescriptor = 119 env.getMasterServices().getTableDescriptors().get(getTableName()); 120 Configuration conf = env.getMasterConfiguration(); 121 if (hasBestSplitRow()) { 122 // Apply the split restriction for the table to the user-specified split point 123 RegionSplitRestriction splitRestriction = 124 RegionSplitRestriction.create(tableDescriptor, conf); 125 byte[] restrictedSplitRow = splitRestriction.getRestrictedSplitPoint(bestSplitRow); 126 if (!Bytes.equals(bestSplitRow, restrictedSplitRow)) { 127 LOG.warn( 128 "The specified split point {} violates the split restriction of the table. " 129 + "Using {} as a split point.", 130 Bytes.toStringBinary(bestSplitRow), Bytes.toStringBinary(restrictedSplitRow)); 131 bestSplitRow = restrictedSplitRow; 132 } 133 } 134 checkSplittable(env, regionToSplit); 135 final TableName table = regionToSplit.getTable(); 136 final long rid = getDaughterRegionIdTimestamp(regionToSplit); 137 this.daughterOneRI = 138 RegionInfoBuilder.newBuilder(table).setStartKey(regionToSplit.getStartKey()) 139 .setEndKey(bestSplitRow).setSplit(false).setRegionId(rid).build(); 140 this.daughterTwoRI = RegionInfoBuilder.newBuilder(table).setStartKey(bestSplitRow) 141 .setEndKey(regionToSplit.getEndKey()).setSplit(false).setRegionId(rid).build(); 142 143 if (tableDescriptor.getRegionSplitPolicyClassName() != null) { 144 // Since we don't have region reference here, creating the split policy instance without it. 145 // This can be used to invoke methods which don't require Region reference. This instantiation 146 // of a class on Master-side though it only makes sense on the RegionServer-side is 147 // for Phoenix Local Indexing. Refer HBASE-12583 for more information. 148 Class<? extends RegionSplitPolicy> clazz = 149 RegionSplitPolicy.getSplitPolicyClass(tableDescriptor, conf); 150 this.splitPolicy = ReflectionUtils.newInstance(clazz, conf); 151 } 152 } 153 154 @Override 155 protected LockState acquireLock(final MasterProcedureEnv env) { 156 if ( 157 env.getProcedureScheduler().waitRegions(this, getTableName(), getParentRegion(), 158 daughterOneRI, daughterTwoRI) 159 ) { 160 try { 161 LOG.debug(LockState.LOCK_EVENT_WAIT + " " + env.getProcedureScheduler().dumpLocks()); 162 } catch (IOException e) { 163 // Ignore, just for logging 164 } 165 return LockState.LOCK_EVENT_WAIT; 166 } 167 return LockState.LOCK_ACQUIRED; 168 } 169 170 @Override 171 protected void releaseLock(final MasterProcedureEnv env) { 172 env.getProcedureScheduler().wakeRegions(this, getTableName(), getParentRegion(), daughterOneRI, 173 daughterTwoRI); 174 } 175 176 public RegionInfo getDaughterOneRI() { 177 return daughterOneRI; 178 } 179 180 public RegionInfo getDaughterTwoRI() { 181 return daughterTwoRI; 182 } 183 184 private boolean hasBestSplitRow() { 185 return bestSplitRow != null && bestSplitRow.length > 0; 186 } 187 188 /** 189 * Check whether the region is splittable 190 * @param env MasterProcedureEnv 191 * @param regionToSplit parent Region to be split 192 */ 193 private void checkSplittable(final MasterProcedureEnv env, final RegionInfo regionToSplit) 194 throws IOException { 195 // Ask the remote RS if this region is splittable. 196 // If we get an IOE, report it along w/ the failure so can see why we are not splittable at 197 // this time. 198 if (regionToSplit.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) { 199 throw new IllegalArgumentException("Can't invoke split on non-default regions directly"); 200 } 201 RegionStateNode node = 202 env.getAssignmentManager().getRegionStates().getRegionStateNode(getParentRegion()); 203 IOException splittableCheckIOE = null; 204 boolean splittable = false; 205 if (node != null) { 206 try { 207 GetRegionInfoResponse response; 208 if (!hasBestSplitRow()) { 209 LOG.info( 210 "{} splitKey isn't explicitly specified, will try to find a best split key from RS {}", 211 node.getRegionInfo().getRegionNameAsString(), node.getRegionLocation()); 212 response = AssignmentManagerUtil.getRegionInfoResponse(env, node.getRegionLocation(), 213 node.getRegionInfo(), true); 214 bestSplitRow = 215 response.hasBestSplitRow() ? response.getBestSplitRow().toByteArray() : null; 216 } else { 217 response = AssignmentManagerUtil.getRegionInfoResponse(env, node.getRegionLocation(), 218 node.getRegionInfo(), false); 219 } 220 splittable = response.hasSplittable() && response.getSplittable(); 221 if (LOG.isDebugEnabled()) { 222 LOG.debug("Splittable=" + splittable + " " + node.toShortString()); 223 } 224 } catch (IOException e) { 225 splittableCheckIOE = e; 226 } 227 } 228 229 if (!splittable) { 230 IOException e = 231 new DoNotRetryIOException(regionToSplit.getShortNameToLog() + " NOT splittable"); 232 if (splittableCheckIOE != null) { 233 e.initCause(splittableCheckIOE); 234 } 235 throw e; 236 } 237 238 if (!hasBestSplitRow()) { 239 throw new DoNotRetryIOException("Region not splittable because bestSplitPoint = null, " 240 + "maybe table is too small for auto split. For force split, try specifying split row"); 241 } 242 243 if (Bytes.equals(regionToSplit.getStartKey(), bestSplitRow)) { 244 throw new DoNotRetryIOException( 245 "Split row is equal to startkey: " + Bytes.toStringBinary(bestSplitRow)); 246 } 247 248 if (!regionToSplit.containsRow(bestSplitRow)) { 249 throw new DoNotRetryIOException("Split row is not inside region key range splitKey:" 250 + Bytes.toStringBinary(bestSplitRow) + " region: " + regionToSplit); 251 } 252 } 253 254 /** 255 * Calculate daughter regionid to use. 256 * @param hri Parent {@link RegionInfo} 257 * @return Daughter region id (timestamp) to use. 258 */ 259 private static long getDaughterRegionIdTimestamp(final RegionInfo hri) { 260 long rid = EnvironmentEdgeManager.currentTime(); 261 // Regionid is timestamp. Can't be less than that of parent else will insert 262 // at wrong location in hbase:meta (See HBASE-710). 263 if (rid < hri.getRegionId()) { 264 LOG.warn("Clock skew; parent regions id is " + hri.getRegionId() 265 + " but current time here is " + rid); 266 rid = hri.getRegionId() + 1; 267 } 268 return rid; 269 } 270 271 private void removeNonDefaultReplicas(MasterProcedureEnv env) throws IOException { 272 AssignmentManagerUtil.removeNonDefaultReplicas(env, Stream.of(getParentRegion()), 273 getRegionReplication(env)); 274 } 275 276 private void checkClosedRegions(MasterProcedureEnv env) throws IOException { 277 // theoretically this should not happen any more after we use TRSP, but anyway let's add a check 278 // here 279 AssignmentManagerUtil.checkClosedRegion(env, getParentRegion()); 280 } 281 282 @Override 283 protected Flow executeFromState(MasterProcedureEnv env, SplitTableRegionState state) 284 throws InterruptedException { 285 LOG.trace("{} execute state={}", this, state); 286 287 try { 288 switch (state) { 289 case SPLIT_TABLE_REGION_PREPARE: 290 if (prepareSplitRegion(env)) { 291 setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION); 292 break; 293 } else { 294 return Flow.NO_MORE_STATE; 295 } 296 case SPLIT_TABLE_REGION_PRE_OPERATION: 297 preSplitRegion(env); 298 setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CLOSE_PARENT_REGION); 299 break; 300 case SPLIT_TABLE_REGION_CLOSE_PARENT_REGION: 301 addChildProcedure(createUnassignProcedures(env)); 302 // createUnassignProcedures() can throw out IOException. If this happens, 303 // it wont reach state SPLIT_TABLE_REGIONS_CHECK_CLOSED_REGION and no parent regions 304 // is closed as all created UnassignProcedures are rolled back. If it rolls back with 305 // state SPLIT_TABLE_REGION_CLOSE_PARENT_REGION, no need to call openParentRegion(), 306 // otherwise, it will result in OpenRegionProcedure for an already open region. 307 setNextState(SplitTableRegionState.SPLIT_TABLE_REGIONS_CHECK_CLOSED_REGIONS); 308 break; 309 case SPLIT_TABLE_REGIONS_CHECK_CLOSED_REGIONS: 310 checkClosedRegions(env); 311 setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS); 312 break; 313 case SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS: 314 removeNonDefaultReplicas(env); 315 createDaughterRegions(env); 316 setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_WRITE_MAX_SEQUENCE_ID_FILE); 317 break; 318 case SPLIT_TABLE_REGION_WRITE_MAX_SEQUENCE_ID_FILE: 319 writeMaxSequenceIdFile(env); 320 setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_META); 321 break; 322 case SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_META: 323 preSplitRegionBeforeMETA(env); 324 setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_UPDATE_META); 325 break; 326 case SPLIT_TABLE_REGION_UPDATE_META: 327 updateMeta(env); 328 setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_META); 329 break; 330 case SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_META: 331 preSplitRegionAfterMETA(env); 332 setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS); 333 break; 334 case SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS: 335 addChildProcedure(createAssignProcedures(env)); 336 setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_POST_OPERATION); 337 break; 338 case SPLIT_TABLE_REGION_POST_OPERATION: 339 postSplitRegion(env); 340 return Flow.NO_MORE_STATE; 341 default: 342 throw new UnsupportedOperationException(this + " unhandled state=" + state); 343 } 344 } catch (IOException e) { 345 String msg = "Splitting " + getParentRegion().getEncodedName() + ", " + this; 346 if (!isRollbackSupported(state)) { 347 // We reach a state that cannot be rolled back. We just need to keep retrying. 348 LOG.warn(msg, e); 349 } else { 350 LOG.error(msg, e); 351 setFailure("master-split-regions", e); 352 } 353 } 354 // if split fails, need to call ((HRegion)parent).clearSplit() when it is a force split 355 return Flow.HAS_MORE_STATE; 356 } 357 358 /** 359 * To rollback {@link SplitTableRegionProcedure}, an AssignProcedure is asynchronously submitted 360 * for parent region to be split (rollback doesn't wait on the completion of the AssignProcedure) 361 * . This can be improved by changing rollback() to support sub-procedures. See HBASE-19851 for 362 * details. 363 */ 364 @Override 365 protected void rollbackState(final MasterProcedureEnv env, final SplitTableRegionState state) 366 throws IOException, InterruptedException { 367 LOG.trace("{} rollback state={}", this, state); 368 369 try { 370 switch (state) { 371 case SPLIT_TABLE_REGION_POST_OPERATION: 372 case SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS: 373 case SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_META: 374 case SPLIT_TABLE_REGION_UPDATE_META: 375 // PONR 376 throw new UnsupportedOperationException(this + " unhandled state=" + state); 377 case SPLIT_TABLE_REGION_PRE_OPERATION_BEFORE_META: 378 break; 379 case SPLIT_TABLE_REGION_CREATE_DAUGHTER_REGIONS: 380 case SPLIT_TABLE_REGION_WRITE_MAX_SEQUENCE_ID_FILE: 381 deleteDaughterRegions(env); 382 break; 383 case SPLIT_TABLE_REGIONS_CHECK_CLOSED_REGIONS: 384 openParentRegion(env); 385 break; 386 case SPLIT_TABLE_REGION_CLOSE_PARENT_REGION: 387 // If it rolls back with state SPLIT_TABLE_REGION_CLOSE_PARENT_REGION, no need to call 388 // openParentRegion(), otherwise, it will result in OpenRegionProcedure for an 389 // already open region. 390 break; 391 case SPLIT_TABLE_REGION_PRE_OPERATION: 392 postRollBackSplitRegion(env); 393 break; 394 case SPLIT_TABLE_REGION_PREPARE: 395 break; // nothing to do 396 default: 397 throw new UnsupportedOperationException(this + " unhandled state=" + state); 398 } 399 } catch (IOException e) { 400 // This will be retried. Unless there is a bug in the code, 401 // this should be just a "temporary error" (e.g. network down) 402 LOG.warn("pid=" + getProcId() + " failed rollback attempt step " + state 403 + " for splitting the region " + getParentRegion().getEncodedName() + " in table " 404 + getTableName(), e); 405 throw e; 406 } 407 } 408 409 /* 410 * Check whether we are in the state that can be rollback 411 */ 412 @Override 413 protected boolean isRollbackSupported(final SplitTableRegionState state) { 414 switch (state) { 415 case SPLIT_TABLE_REGION_POST_OPERATION: 416 case SPLIT_TABLE_REGION_OPEN_CHILD_REGIONS: 417 case SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_META: 418 case SPLIT_TABLE_REGION_UPDATE_META: 419 // It is not safe to rollback if we reach to these states. 420 return false; 421 default: 422 break; 423 } 424 return true; 425 } 426 427 @Override 428 protected SplitTableRegionState getState(final int stateId) { 429 return SplitTableRegionState.forNumber(stateId); 430 } 431 432 @Override 433 protected int getStateId(final SplitTableRegionState state) { 434 return state.getNumber(); 435 } 436 437 @Override 438 protected SplitTableRegionState getInitialState() { 439 return SplitTableRegionState.SPLIT_TABLE_REGION_PREPARE; 440 } 441 442 @Override 443 protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { 444 super.serializeStateData(serializer); 445 446 final MasterProcedureProtos.SplitTableRegionStateData.Builder splitTableRegionMsg = 447 MasterProcedureProtos.SplitTableRegionStateData.newBuilder() 448 .setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser())) 449 .setParentRegionInfo(ProtobufUtil.toRegionInfo(getRegion())) 450 .addChildRegionInfo(ProtobufUtil.toRegionInfo(daughterOneRI)) 451 .addChildRegionInfo(ProtobufUtil.toRegionInfo(daughterTwoRI)); 452 serializer.serialize(splitTableRegionMsg.build()); 453 } 454 455 @Override 456 protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { 457 super.deserializeStateData(serializer); 458 459 final MasterProcedureProtos.SplitTableRegionStateData splitTableRegionsMsg = 460 serializer.deserialize(MasterProcedureProtos.SplitTableRegionStateData.class); 461 setUser(MasterProcedureUtil.toUserInfo(splitTableRegionsMsg.getUserInfo())); 462 setRegion(ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getParentRegionInfo())); 463 assert (splitTableRegionsMsg.getChildRegionInfoCount() == 2); 464 daughterOneRI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(0)); 465 daughterTwoRI = ProtobufUtil.toRegionInfo(splitTableRegionsMsg.getChildRegionInfo(1)); 466 } 467 468 @Override 469 public void toStringClassDetails(StringBuilder sb) { 470 sb.append(getClass().getSimpleName()); 471 sb.append(" table="); 472 sb.append(getTableName()); 473 sb.append(", parent="); 474 sb.append(getParentRegion().getShortNameToLog()); 475 sb.append(", daughterA="); 476 sb.append(daughterOneRI.getShortNameToLog()); 477 sb.append(", daughterB="); 478 sb.append(daughterTwoRI.getShortNameToLog()); 479 } 480 481 private RegionInfo getParentRegion() { 482 return getRegion(); 483 } 484 485 @Override 486 public TableOperationType getTableOperationType() { 487 return TableOperationType.REGION_SPLIT; 488 } 489 490 @Override 491 protected ProcedureMetrics getProcedureMetrics(MasterProcedureEnv env) { 492 return env.getAssignmentManager().getAssignmentManagerMetrics().getSplitProcMetrics(); 493 } 494 495 private byte[] getSplitRow() { 496 return daughterTwoRI.getStartKey(); 497 } 498 499 private static final State[] EXPECTED_SPLIT_STATES = new State[] { State.OPEN, State.CLOSED }; 500 501 /** 502 * Prepare to Split region. 503 * @param env MasterProcedureEnv 504 */ 505 public boolean prepareSplitRegion(final MasterProcedureEnv env) throws IOException { 506 // Fail if we are taking snapshot for the given table 507 if ( 508 env.getMasterServices().getSnapshotManager() 509 .isTableTakingAnySnapshot(getParentRegion().getTable()) 510 ) { 511 setFailure(new IOException("Skip splitting region " + getParentRegion().getShortNameToLog() 512 + ", because we are taking snapshot for the table " + getParentRegion().getTable())); 513 return false; 514 } 515 // Check whether the region is splittable 516 RegionStateNode node = 517 env.getAssignmentManager().getRegionStates().getRegionStateNode(getParentRegion()); 518 519 if (node == null) { 520 throw new UnknownRegionException(getParentRegion().getRegionNameAsString()); 521 } 522 523 RegionInfo parentHRI = node.getRegionInfo(); 524 if (parentHRI == null) { 525 LOG.info("Unsplittable; parent region is null; node={}", node); 526 return false; 527 } 528 // Lookup the parent HRI state from the AM, which has the latest updated info. 529 // Protect against the case where concurrent SPLIT requests came in and succeeded 530 // just before us. 531 if (node.isInState(State.SPLIT)) { 532 LOG.info("Split of " + parentHRI + " skipped; state is already SPLIT"); 533 return false; 534 } 535 if (parentHRI.isSplit() || parentHRI.isOffline()) { 536 LOG.info("Split of " + parentHRI + " skipped because offline/split."); 537 return false; 538 } 539 540 // expected parent to be online or closed 541 if (!node.isInState(EXPECTED_SPLIT_STATES)) { 542 // We may have SPLIT already? 543 setFailure( 544 new IOException("Split " + parentHRI.getRegionNameAsString() + " FAILED because state=" 545 + node.getState() + "; expected " + Arrays.toString(EXPECTED_SPLIT_STATES))); 546 return false; 547 } 548 549 // Mostly this check is not used because we already check the switch before submit a split 550 // procedure. Just for safe, check the switch again. This procedure can be rollbacked if 551 // the switch was set to false after submit. 552 if (!env.getMasterServices().isSplitOrMergeEnabled(MasterSwitchType.SPLIT)) { 553 LOG.warn("pid=" + getProcId() + " split switch is off! skip split of " + parentHRI); 554 setFailure(new IOException( 555 "Split region " + parentHRI.getRegionNameAsString() + " failed due to split switch off")); 556 return false; 557 } 558 559 if (!env.getMasterServices().getTableDescriptors().get(getTableName()).isSplitEnabled()) { 560 LOG.warn("pid={}, split is disabled for the table! Skipping split of {}", getProcId(), 561 parentHRI); 562 setFailure(new IOException("Split region " + parentHRI.getRegionNameAsString() 563 + " failed as region split is disabled for the table")); 564 return false; 565 } 566 567 // set node state as SPLITTING 568 node.setState(State.SPLITTING); 569 570 // Since we have the lock and the master is coordinating the operation 571 // we are always able to split the region 572 return true; 573 } 574 575 /** 576 * Action before splitting region in a table. 577 * @param env MasterProcedureEnv 578 */ 579 private void preSplitRegion(final MasterProcedureEnv env) 580 throws IOException, InterruptedException { 581 final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); 582 if (cpHost != null) { 583 cpHost.preSplitRegionAction(getTableName(), getSplitRow(), getUser()); 584 } 585 586 // TODO: Clean up split and merge. Currently all over the place. 587 // Notify QuotaManager and RegionNormalizer 588 try { 589 env.getMasterServices().getMasterQuotaManager().onRegionSplit(this.getParentRegion()); 590 } catch (QuotaExceededException e) { 591 // TODO: why is this here? split requests can be submitted by actors other than the normalizer 592 env.getMasterServices().getRegionNormalizerManager() 593 .planSkipped(NormalizationPlan.PlanType.SPLIT); 594 throw e; 595 } 596 } 597 598 /** 599 * Action after rollback a split table region action. 600 * @param env MasterProcedureEnv 601 */ 602 private void postRollBackSplitRegion(final MasterProcedureEnv env) throws IOException { 603 final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); 604 if (cpHost != null) { 605 cpHost.postRollBackSplitRegionAction(getUser()); 606 } 607 } 608 609 /** 610 * Rollback close parent region 611 */ 612 private void openParentRegion(MasterProcedureEnv env) throws IOException { 613 AssignmentManagerUtil.reopenRegionsForRollback(env, 614 Collections.singletonList((getParentRegion())), getRegionReplication(env), 615 getParentRegionServerName(env)); 616 } 617 618 /** 619 * Create daughter regions 620 */ 621 public void createDaughterRegions(final MasterProcedureEnv env) throws IOException { 622 final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem(); 623 final Path tabledir = CommonFSUtils.getTableDir(mfs.getRootDir(), getTableName()); 624 final FileSystem fs = mfs.getFileSystem(); 625 HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem( 626 env.getMasterConfiguration(), fs, tabledir, getParentRegion(), false); 627 regionFs.createSplitsDir(daughterOneRI, daughterTwoRI); 628 629 Pair<List<Path>, List<Path>> expectedReferences = splitStoreFiles(env, regionFs); 630 631 assertSplitResultFilesCount(fs, expectedReferences.getFirst().size(), 632 regionFs.getSplitsDir(daughterOneRI)); 633 regionFs.commitDaughterRegion(daughterOneRI, expectedReferences.getFirst(), env); 634 assertSplitResultFilesCount(fs, expectedReferences.getFirst().size(), 635 new Path(tabledir, daughterOneRI.getEncodedName())); 636 637 assertSplitResultFilesCount(fs, expectedReferences.getSecond().size(), 638 regionFs.getSplitsDir(daughterTwoRI)); 639 regionFs.commitDaughterRegion(daughterTwoRI, expectedReferences.getSecond(), env); 640 assertSplitResultFilesCount(fs, expectedReferences.getSecond().size(), 641 new Path(tabledir, daughterTwoRI.getEncodedName())); 642 } 643 644 private void deleteDaughterRegions(final MasterProcedureEnv env) throws IOException { 645 final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem(); 646 final Path tabledir = CommonFSUtils.getTableDir(mfs.getRootDir(), getTableName()); 647 HRegionFileSystem.deleteRegionFromFileSystem(env.getMasterConfiguration(), mfs.getFileSystem(), 648 tabledir, daughterOneRI); 649 HRegionFileSystem.deleteRegionFromFileSystem(env.getMasterConfiguration(), mfs.getFileSystem(), 650 tabledir, daughterTwoRI); 651 } 652 653 /** 654 * Create Split directory 655 * @param env MasterProcedureEnv 656 */ 657 private Pair<List<Path>, List<Path>> splitStoreFiles(final MasterProcedureEnv env, 658 final HRegionFileSystem regionFs) throws IOException { 659 final Configuration conf = env.getMasterConfiguration(); 660 TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName()); 661 // The following code sets up a thread pool executor with as many slots as 662 // there's files to split. It then fires up everything, waits for 663 // completion and finally checks for any exception 664 // 665 // Note: From HBASE-26187, splitStoreFiles now creates daughter region dirs straight under the 666 // table dir. In case of failure, the proc would go through this again, already existing 667 // region dirs and split files would just be ignored, new split files should get created. 668 int nbFiles = 0; 669 final Map<String, Collection<StoreFileInfo>> files = 670 new HashMap<String, Collection<StoreFileInfo>>(htd.getColumnFamilyCount()); 671 for (ColumnFamilyDescriptor cfd : htd.getColumnFamilies()) { 672 String family = cfd.getNameAsString(); 673 StoreFileTracker tracker = 674 StoreFileTrackerFactory.create(env.getMasterConfiguration(), htd, cfd, regionFs); 675 Collection<StoreFileInfo> sfis = tracker.load(); 676 if (sfis == null) { 677 continue; 678 } 679 Collection<StoreFileInfo> filteredSfis = null; 680 for (StoreFileInfo sfi : sfis) { 681 // Filter. There is a lag cleaning up compacted reference files. They get cleared 682 // after a delay in case outstanding Scanners still have references. Because of this, 683 // the listing of the Store content may have straggler reference files. Skip these. 684 // It should be safe to skip references at this point because we checked above with 685 // the region if it thinks it is splittable and if we are here, it thinks it is 686 // splitable. 687 if (sfi.isReference()) { 688 LOG.info("Skipping split of " + sfi + "; presuming ready for archiving."); 689 continue; 690 } 691 if (filteredSfis == null) { 692 filteredSfis = new ArrayList<StoreFileInfo>(sfis.size()); 693 files.put(family, filteredSfis); 694 } 695 filteredSfis.add(sfi); 696 nbFiles++; 697 } 698 } 699 if (nbFiles == 0) { 700 // no file needs to be splitted. 701 return new Pair<>(Collections.emptyList(), Collections.emptyList()); 702 } 703 // Max #threads is the smaller of the number of storefiles or the default max determined above. 704 int maxThreads = Math.min( 705 conf.getInt(HConstants.REGION_SPLIT_THREADS_MAX, 706 conf.getInt(HStore.BLOCKING_STOREFILES_KEY, HStore.DEFAULT_BLOCKING_STOREFILE_COUNT)), 707 nbFiles); 708 LOG.info("pid=" + getProcId() + " splitting " + nbFiles + " storefiles, region=" 709 + getParentRegion().getShortNameToLog() + ", threads=" + maxThreads); 710 final ExecutorService threadPool = Executors.newFixedThreadPool(maxThreads, 711 new ThreadFactoryBuilder().setNameFormat("StoreFileSplitter-pool-%d").setDaemon(true) 712 .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build()); 713 final List<Future<Pair<Path, Path>>> futures = new ArrayList<Future<Pair<Path, Path>>>(nbFiles); 714 715 // Split each store file. 716 for (Map.Entry<String, Collection<StoreFileInfo>> e : files.entrySet()) { 717 byte[] familyName = Bytes.toBytes(e.getKey()); 718 final ColumnFamilyDescriptor hcd = htd.getColumnFamily(familyName); 719 final Collection<StoreFileInfo> storeFiles = e.getValue(); 720 if (storeFiles != null && storeFiles.size() > 0) { 721 final Configuration storeConfiguration = 722 StoreUtils.createStoreConfiguration(env.getMasterConfiguration(), htd, hcd); 723 for (StoreFileInfo storeFileInfo : storeFiles) { 724 // As this procedure is running on master, use CacheConfig.DISABLED means 725 // don't cache any block. 726 // We also need to pass through a suitable CompoundConfiguration as if this 727 // is running in a regionserver's Store context, or we might not be able 728 // to read the hfiles. 729 storeFileInfo.setConf(storeConfiguration); 730 StoreFileSplitter sfs = new StoreFileSplitter(regionFs, familyName, 731 new HStoreFile(storeFileInfo, hcd.getBloomFilterType(), CacheConfig.DISABLED)); 732 futures.add(threadPool.submit(sfs)); 733 } 734 } 735 } 736 // Shutdown the pool 737 threadPool.shutdown(); 738 739 // Wait for all the tasks to finish. 740 // When splits ran on the RegionServer, how-long-to-wait-configuration was named 741 // hbase.regionserver.fileSplitTimeout. If set, use its value. 742 long fileSplitTimeout = conf.getLong("hbase.master.fileSplitTimeout", 743 conf.getLong("hbase.regionserver.fileSplitTimeout", 600000)); 744 try { 745 boolean stillRunning = !threadPool.awaitTermination(fileSplitTimeout, TimeUnit.MILLISECONDS); 746 if (stillRunning) { 747 threadPool.shutdownNow(); 748 // wait for the thread to shutdown completely. 749 while (!threadPool.isTerminated()) { 750 Thread.sleep(50); 751 } 752 throw new IOException( 753 "Took too long to split the" + " files and create the references, aborting split"); 754 } 755 } catch (InterruptedException e) { 756 throw (InterruptedIOException) new InterruptedIOException().initCause(e); 757 } 758 759 List<Path> daughterA = new ArrayList<>(); 760 List<Path> daughterB = new ArrayList<>(); 761 // Look for any exception 762 for (Future<Pair<Path, Path>> future : futures) { 763 try { 764 Pair<Path, Path> p = future.get(); 765 if (p.getFirst() != null) { 766 daughterA.add(p.getFirst()); 767 } 768 if (p.getSecond() != null) { 769 daughterB.add(p.getSecond()); 770 } 771 } catch (InterruptedException e) { 772 throw (InterruptedIOException) new InterruptedIOException().initCause(e); 773 } catch (ExecutionException e) { 774 throw new IOException(e); 775 } 776 } 777 778 if (LOG.isDebugEnabled()) { 779 LOG.debug("pid=" + getProcId() + " split storefiles for region " 780 + getParentRegion().getShortNameToLog() + " Daughter A: " + daughterA 781 + " storefiles, Daughter B: " + daughterB + " storefiles."); 782 } 783 return new Pair<>(daughterA, daughterB); 784 } 785 786 private void assertSplitResultFilesCount(final FileSystem fs, 787 final int expectedSplitResultFileCount, Path dir) throws IOException { 788 if (expectedSplitResultFileCount != 0) { 789 int resultFileCount = FSUtils.getRegionReferenceAndLinkFileCount(fs, dir); 790 if (expectedSplitResultFileCount != resultFileCount) { 791 throw new IOException("Failing split. Didn't have expected reference and HFileLink files" 792 + ", expected=" + expectedSplitResultFileCount + ", actual=" + resultFileCount); 793 } 794 } 795 } 796 797 private Pair<Path, Path> splitStoreFile(HRegionFileSystem regionFs, byte[] family, HStoreFile sf) 798 throws IOException { 799 if (LOG.isDebugEnabled()) { 800 LOG.debug("pid=" + getProcId() + " splitting started for store file: " + sf.getPath() 801 + " for region: " + getParentRegion().getShortNameToLog()); 802 } 803 804 final byte[] splitRow = getSplitRow(); 805 final String familyName = Bytes.toString(family); 806 final Path path_first = 807 regionFs.splitStoreFile(this.daughterOneRI, familyName, sf, splitRow, false, splitPolicy); 808 final Path path_second = 809 regionFs.splitStoreFile(this.daughterTwoRI, familyName, sf, splitRow, true, splitPolicy); 810 if (LOG.isDebugEnabled()) { 811 LOG.debug("pid=" + getProcId() + " splitting complete for store file: " + sf.getPath() 812 + " for region: " + getParentRegion().getShortNameToLog()); 813 } 814 return new Pair<Path, Path>(path_first, path_second); 815 } 816 817 /** 818 * Utility class used to do the file splitting / reference writing in parallel instead of 819 * sequentially. 820 */ 821 private class StoreFileSplitter implements Callable<Pair<Path, Path>> { 822 private final HRegionFileSystem regionFs; 823 private final byte[] family; 824 private final HStoreFile sf; 825 826 /** 827 * Constructor that takes what it needs to split 828 * @param regionFs the file system 829 * @param family Family that contains the store file 830 * @param sf which file 831 */ 832 public StoreFileSplitter(HRegionFileSystem regionFs, byte[] family, HStoreFile sf) { 833 this.regionFs = regionFs; 834 this.sf = sf; 835 this.family = family; 836 } 837 838 @Override 839 public Pair<Path, Path> call() throws IOException { 840 return splitStoreFile(regionFs, family, sf); 841 } 842 } 843 844 /** 845 * Post split region actions before the Point-of-No-Return step 846 * @param env MasterProcedureEnv 847 **/ 848 private void preSplitRegionBeforeMETA(final MasterProcedureEnv env) 849 throws IOException, InterruptedException { 850 final List<Mutation> metaEntries = new ArrayList<Mutation>(); 851 final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); 852 if (cpHost != null) { 853 cpHost.preSplitBeforeMETAAction(getSplitRow(), metaEntries, getUser()); 854 try { 855 for (Mutation p : metaEntries) { 856 RegionInfo.parseRegionName(p.getRow()); 857 } 858 } catch (IOException e) { 859 LOG.error("pid=" + getProcId() + " row key of mutation from coprocessor not parsable as " 860 + "region name." + "Mutations from coprocessor should only for hbase:meta table."); 861 throw e; 862 } 863 } 864 } 865 866 /** 867 * Add daughter regions to META 868 * @param env MasterProcedureEnv 869 */ 870 private void updateMeta(final MasterProcedureEnv env) throws IOException { 871 env.getAssignmentManager().markRegionAsSplit(getParentRegion(), getParentRegionServerName(env), 872 daughterOneRI, daughterTwoRI); 873 } 874 875 /** 876 * Pre split region actions after the Point-of-No-Return step 877 * @param env MasterProcedureEnv 878 **/ 879 private void preSplitRegionAfterMETA(final MasterProcedureEnv env) 880 throws IOException, InterruptedException { 881 final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); 882 if (cpHost != null) { 883 cpHost.preSplitAfterMETAAction(getUser()); 884 } 885 } 886 887 /** 888 * Post split region actions 889 * @param env MasterProcedureEnv 890 **/ 891 private void postSplitRegion(final MasterProcedureEnv env) throws IOException { 892 final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost(); 893 if (cpHost != null) { 894 cpHost.postCompletedSplitRegionAction(daughterOneRI, daughterTwoRI, getUser()); 895 } 896 } 897 898 private ServerName getParentRegionServerName(final MasterProcedureEnv env) { 899 return env.getMasterServices().getAssignmentManager().getRegionStates() 900 .getRegionServerOfRegion(getParentRegion()); 901 } 902 903 private TransitRegionStateProcedure[] createUnassignProcedures(MasterProcedureEnv env) 904 throws IOException { 905 return AssignmentManagerUtil.createUnassignProceduresForSplitOrMerge(env, 906 Stream.of(getParentRegion()), getRegionReplication(env)); 907 } 908 909 private TransitRegionStateProcedure[] createAssignProcedures(MasterProcedureEnv env) 910 throws IOException { 911 List<RegionInfo> hris = new ArrayList<RegionInfo>(2); 912 hris.add(daughterOneRI); 913 hris.add(daughterTwoRI); 914 return AssignmentManagerUtil.createAssignProceduresForSplitDaughters(env, hris, 915 getRegionReplication(env), getParentRegionServerName(env)); 916 } 917 918 private int getRegionReplication(final MasterProcedureEnv env) throws IOException { 919 final TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName()); 920 return htd.getRegionReplication(); 921 } 922 923 private void writeMaxSequenceIdFile(MasterProcedureEnv env) throws IOException { 924 MasterFileSystem fs = env.getMasterFileSystem(); 925 long maxSequenceId = WALSplitUtil.getMaxRegionSequenceId(env.getMasterConfiguration(), 926 getParentRegion(), fs::getFileSystem, fs::getWALFileSystem); 927 if (maxSequenceId > 0) { 928 WALSplitUtil.writeRegionSequenceIdFile(fs.getWALFileSystem(), 929 getWALRegionDir(env, daughterOneRI), maxSequenceId); 930 WALSplitUtil.writeRegionSequenceIdFile(fs.getWALFileSystem(), 931 getWALRegionDir(env, daughterTwoRI), maxSequenceId); 932 } 933 } 934 935 @Override 936 protected boolean abort(MasterProcedureEnv env) { 937 // Abort means rollback. We can't rollback all steps. HBASE-18018 added abort to all 938 // Procedures. Here is a Procedure that has a PONR and cannot be aborted wants it enters this 939 // range of steps; what do we do for these should an operator want to cancel them? HBASE-20022. 940 return isRollbackSupported(getCurrentState()) ? super.abort(env) : false; 941 } 942}