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.procedure2; 019 020import edu.umd.cs.findbugs.annotations.Nullable; 021import java.io.IOException; 022import java.io.UncheckedIOException; 023import java.util.ArrayDeque; 024import java.util.ArrayList; 025import java.util.Arrays; 026import java.util.Collection; 027import java.util.Deque; 028import java.util.HashSet; 029import java.util.List; 030import java.util.Set; 031import java.util.concurrent.ConcurrentHashMap; 032import java.util.concurrent.CopyOnWriteArrayList; 033import java.util.concurrent.Executor; 034import java.util.concurrent.Executors; 035import java.util.concurrent.TimeUnit; 036import java.util.concurrent.atomic.AtomicBoolean; 037import java.util.concurrent.atomic.AtomicInteger; 038import java.util.concurrent.atomic.AtomicLong; 039import java.util.stream.Collectors; 040import java.util.stream.Stream; 041import org.apache.hadoop.conf.Configuration; 042import org.apache.hadoop.hbase.HConstants; 043import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException; 044import org.apache.hadoop.hbase.log.HBaseMarkers; 045import org.apache.hadoop.hbase.procedure2.Procedure.LockState; 046import org.apache.hadoop.hbase.procedure2.store.ProcedureStore; 047import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator; 048import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureStoreListener; 049import org.apache.hadoop.hbase.procedure2.trace.ProcedureSpanBuilder; 050import org.apache.hadoop.hbase.procedure2.util.StringUtils; 051import org.apache.hadoop.hbase.security.User; 052import org.apache.hadoop.hbase.trace.TraceUtil; 053import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 054import org.apache.hadoop.hbase.util.IdLock; 055import org.apache.hadoop.hbase.util.NonceKey; 056import org.apache.hadoop.hbase.util.Threads; 057import org.apache.yetus.audience.InterfaceAudience; 058import org.slf4j.Logger; 059import org.slf4j.LoggerFactory; 060 061import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 062import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; 063 064import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState; 065 066/** 067 * Thread Pool that executes the submitted procedures. The executor has a ProcedureStore associated. 068 * Each operation is logged and on restart the pending procedures are resumed. Unless the Procedure 069 * code throws an error (e.g. invalid user input) the procedure will complete (at some point in 070 * time), On restart the pending procedures are resumed and the once failed will be rolledback. The 071 * user can add procedures to the executor via submitProcedure(proc) check for the finished state 072 * via isFinished(procId) and get the result via getResult(procId) 073 */ 074@InterfaceAudience.Private 075public class ProcedureExecutor<TEnvironment> { 076 private static final Logger LOG = LoggerFactory.getLogger(ProcedureExecutor.class); 077 078 public static final String CHECK_OWNER_SET_CONF_KEY = "hbase.procedure.check.owner.set"; 079 private static final boolean DEFAULT_CHECK_OWNER_SET = false; 080 081 public static final String WORKER_KEEP_ALIVE_TIME_CONF_KEY = 082 "hbase.procedure.worker.keep.alive.time.msec"; 083 private static final long DEFAULT_WORKER_KEEP_ALIVE_TIME = TimeUnit.MINUTES.toMillis(1); 084 085 public static final String EVICT_TTL_CONF_KEY = "hbase.procedure.cleaner.evict.ttl"; 086 static final int DEFAULT_EVICT_TTL = 15 * 60000; // 15min 087 088 public static final String EVICT_ACKED_TTL_CONF_KEY = "hbase.procedure.cleaner.acked.evict.ttl"; 089 static final int DEFAULT_ACKED_EVICT_TTL = 5 * 60000; // 5min 090 091 /** 092 * {@link #testing} is non-null when ProcedureExecutor is being tested. Tests will try to break PE 093 * having it fail at various junctures. When non-null, testing is set to an instance of the below 094 * internal {@link Testing} class with flags set for the particular test. 095 */ 096 volatile Testing testing = null; 097 098 /** 099 * Class with parameters describing how to fail/die when in testing-context. 100 */ 101 public static class Testing { 102 protected volatile boolean killIfHasParent = true; 103 protected volatile boolean killIfSuspended = false; 104 105 /** 106 * Kill the PE BEFORE we store state to the WAL. Good for figuring out if a Procedure is 107 * persisting all the state it needs to recover after a crash. 108 */ 109 protected volatile boolean killBeforeStoreUpdate = false; 110 protected volatile boolean toggleKillBeforeStoreUpdate = false; 111 112 /** 113 * Set when we want to fail AFTER state has been stored into the WAL. Rarely used. HBASE-20978 114 * is about a case where memory-state was being set after store to WAL where a crash could cause 115 * us to get stuck. This flag allows killing at what was a vulnerable time. 116 */ 117 protected volatile boolean killAfterStoreUpdate = false; 118 protected volatile boolean toggleKillAfterStoreUpdate = false; 119 120 protected boolean shouldKillBeforeStoreUpdate() { 121 final boolean kill = this.killBeforeStoreUpdate; 122 if (this.toggleKillBeforeStoreUpdate) { 123 this.killBeforeStoreUpdate = !kill; 124 LOG.warn("Toggle KILL before store update to: " + this.killBeforeStoreUpdate); 125 } 126 return kill; 127 } 128 129 protected boolean shouldKillBeforeStoreUpdate(boolean isSuspended, boolean hasParent) { 130 if (isSuspended && !killIfSuspended) { 131 return false; 132 } 133 if (hasParent && !killIfHasParent) { 134 return false; 135 } 136 return shouldKillBeforeStoreUpdate(); 137 } 138 139 protected boolean shouldKillAfterStoreUpdate() { 140 final boolean kill = this.killAfterStoreUpdate; 141 if (this.toggleKillAfterStoreUpdate) { 142 this.killAfterStoreUpdate = !kill; 143 LOG.warn("Toggle KILL after store update to: " + this.killAfterStoreUpdate); 144 } 145 return kill; 146 } 147 148 protected boolean shouldKillAfterStoreUpdate(final boolean isSuspended) { 149 return (isSuspended && !killIfSuspended) ? false : shouldKillAfterStoreUpdate(); 150 } 151 } 152 153 public interface ProcedureExecutorListener { 154 void procedureLoaded(long procId); 155 156 void procedureAdded(long procId); 157 158 void procedureFinished(long procId); 159 } 160 161 /** 162 * Map the the procId returned by submitProcedure(), the Root-ProcID, to the Procedure. Once a 163 * Root-Procedure completes (success or failure), the result will be added to this map. The user 164 * of ProcedureExecutor should call getResult(procId) to get the result. 165 */ 166 private final ConcurrentHashMap<Long, CompletedProcedureRetainer<TEnvironment>> completed = 167 new ConcurrentHashMap<>(); 168 169 /** 170 * Map the the procId returned by submitProcedure(), the Root-ProcID, to the RootProcedureState. 171 * The RootProcedureState contains the execution stack of the Root-Procedure, It is added to the 172 * map by submitProcedure() and removed on procedure completion. 173 */ 174 private final ConcurrentHashMap<Long, RootProcedureState<TEnvironment>> rollbackStack = 175 new ConcurrentHashMap<>(); 176 177 /** 178 * Helper map to lookup the live procedures by ID. This map contains every procedure. 179 * root-procedures and subprocedures. 180 */ 181 private final ConcurrentHashMap<Long, Procedure<TEnvironment>> procedures = 182 new ConcurrentHashMap<>(); 183 184 /** 185 * Helper map to lookup whether the procedure already issued from the same client. This map 186 * contains every root procedure. 187 */ 188 private final ConcurrentHashMap<NonceKey, Long> nonceKeysToProcIdsMap = new ConcurrentHashMap<>(); 189 190 private final CopyOnWriteArrayList<ProcedureExecutorListener> listeners = 191 new CopyOnWriteArrayList<>(); 192 193 private Configuration conf; 194 195 /** 196 * Created in the {@link #init(int, boolean)} method. Destroyed in {@link #join()} (FIX! Doing 197 * resource handling rather than observing in a #join is unexpected). Overridden when we do the 198 * ProcedureTestingUtility.testRecoveryAndDoubleExecution trickery (Should be ok). 199 */ 200 private ThreadGroup threadGroup; 201 202 /** 203 * Created in the {@link #init(int, boolean)} method. Terminated in {@link #join()} (FIX! Doing 204 * resource handling rather than observing in a #join is unexpected). Overridden when we do the 205 * ProcedureTestingUtility.testRecoveryAndDoubleExecution trickery (Should be ok). 206 */ 207 private CopyOnWriteArrayList<WorkerThread> workerThreads; 208 209 /** 210 * Created in the {@link #init(int, boolean)} method. Terminated in {@link #join()} (FIX! Doing 211 * resource handling rather than observing in a #join is unexpected). Overridden when we do the 212 * ProcedureTestingUtility.testRecoveryAndDoubleExecution trickery (Should be ok). 213 */ 214 private TimeoutExecutorThread<TEnvironment> timeoutExecutor; 215 216 /** 217 * WorkerMonitor check for stuck workers and new worker thread when necessary, for example if 218 * there is no worker to assign meta, it will new worker thread for it, so it is very important. 219 * TimeoutExecutor execute many tasks like DeadServerMetricRegionChore RegionInTransitionChore and 220 * so on, some tasks may execute for a long time so will block other tasks like WorkerMonitor, so 221 * use a dedicated thread for executing WorkerMonitor. 222 */ 223 private TimeoutExecutorThread<TEnvironment> workerMonitorExecutor; 224 225 private int corePoolSize; 226 private int maxPoolSize; 227 228 private volatile long keepAliveTime; 229 230 /** 231 * Scheduler/Queue that contains runnable procedures. 232 */ 233 private final ProcedureScheduler scheduler; 234 235 private final Executor forceUpdateExecutor = Executors.newSingleThreadExecutor( 236 new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Force-Update-PEWorker-%d").build()); 237 238 private final AtomicLong lastProcId = new AtomicLong(-1); 239 private final AtomicLong workerId = new AtomicLong(0); 240 private final AtomicInteger activeExecutorCount = new AtomicInteger(0); 241 private final AtomicBoolean running = new AtomicBoolean(false); 242 private final TEnvironment environment; 243 private final ProcedureStore store; 244 245 private final boolean checkOwnerSet; 246 247 // To prevent concurrent execution of the same procedure. 248 // For some rare cases, especially if the procedure uses ProcedureEvent, it is possible that the 249 // procedure is woken up before we finish the suspend which causes the same procedures to be 250 // executed in parallel. This does lead to some problems, see HBASE-20939&HBASE-20949, and is also 251 // a bit confusing to the developers. So here we introduce this lock to prevent the concurrent 252 // execution of the same procedure. 253 private final IdLock procExecutionLock = new IdLock(); 254 255 public ProcedureExecutor(final Configuration conf, final TEnvironment environment, 256 final ProcedureStore store) { 257 this(conf, environment, store, new SimpleProcedureScheduler()); 258 } 259 260 private boolean isRootFinished(Procedure<?> proc) { 261 Procedure<?> rootProc = procedures.get(proc.getRootProcId()); 262 return rootProc == null || rootProc.isFinished(); 263 } 264 265 private void forceUpdateProcedure(long procId) throws IOException { 266 IdLock.Entry lockEntry = procExecutionLock.getLockEntry(procId); 267 try { 268 Procedure<TEnvironment> proc = procedures.get(procId); 269 if (proc != null) { 270 if (proc.isFinished() && proc.hasParent() && isRootFinished(proc)) { 271 LOG.debug("Procedure {} has already been finished and parent is succeeded," 272 + " skip force updating", proc); 273 return; 274 } 275 } else { 276 CompletedProcedureRetainer<TEnvironment> retainer = completed.get(procId); 277 if (retainer == null || retainer.getProcedure() instanceof FailedProcedure) { 278 LOG.debug("No pending procedure with id = {}, skip force updating.", procId); 279 return; 280 } 281 long evictTtl = conf.getInt(EVICT_TTL_CONF_KEY, DEFAULT_EVICT_TTL); 282 long evictAckTtl = conf.getInt(EVICT_ACKED_TTL_CONF_KEY, DEFAULT_ACKED_EVICT_TTL); 283 if (retainer.isExpired(EnvironmentEdgeManager.currentTime(), evictTtl, evictAckTtl)) { 284 LOG.debug("Procedure {} has already been finished and expired, skip force updating", 285 procId); 286 return; 287 } 288 proc = retainer.getProcedure(); 289 } 290 LOG.debug("Force update procedure {}", proc); 291 store.update(proc); 292 } finally { 293 procExecutionLock.releaseLockEntry(lockEntry); 294 } 295 } 296 297 public ProcedureExecutor(final Configuration conf, final TEnvironment environment, 298 final ProcedureStore store, final ProcedureScheduler scheduler) { 299 this.environment = environment; 300 this.scheduler = scheduler; 301 this.store = store; 302 this.conf = conf; 303 this.checkOwnerSet = conf.getBoolean(CHECK_OWNER_SET_CONF_KEY, DEFAULT_CHECK_OWNER_SET); 304 refreshConfiguration(conf); 305 store.registerListener(new ProcedureStoreListener() { 306 307 @Override 308 public void forceUpdate(long[] procIds) { 309 Arrays.stream(procIds).forEach(procId -> forceUpdateExecutor.execute(() -> { 310 try { 311 forceUpdateProcedure(procId); 312 } catch (IOException e) { 313 LOG.warn("Failed to force update procedure with pid={}", procId); 314 } 315 })); 316 } 317 }); 318 } 319 320 private void load(final boolean abortOnCorruption) throws IOException { 321 Preconditions.checkArgument(completed.isEmpty(), "completed not empty"); 322 Preconditions.checkArgument(rollbackStack.isEmpty(), "rollback state not empty"); 323 Preconditions.checkArgument(procedures.isEmpty(), "procedure map not empty"); 324 Preconditions.checkArgument(scheduler.size() == 0, "run queue not empty"); 325 326 store.load(new ProcedureStore.ProcedureLoader() { 327 @Override 328 public void setMaxProcId(long maxProcId) { 329 assert lastProcId.get() < 0 : "expected only one call to setMaxProcId()"; 330 lastProcId.set(maxProcId); 331 } 332 333 @Override 334 public void load(ProcedureIterator procIter) throws IOException { 335 loadProcedures(procIter, abortOnCorruption); 336 } 337 338 @Override 339 public void handleCorrupted(ProcedureIterator procIter) throws IOException { 340 int corruptedCount = 0; 341 while (procIter.hasNext()) { 342 Procedure<?> proc = procIter.next(); 343 LOG.error("Corrupt " + proc); 344 corruptedCount++; 345 } 346 if (abortOnCorruption && corruptedCount > 0) { 347 throw new IOException("found " + corruptedCount + " corrupted procedure(s) on replay"); 348 } 349 } 350 }); 351 } 352 353 private void restoreLock(Procedure<TEnvironment> proc, Set<Long> restored) { 354 proc.restoreLock(getEnvironment()); 355 restored.add(proc.getProcId()); 356 } 357 358 private void restoreLocks(Deque<Procedure<TEnvironment>> stack, Set<Long> restored) { 359 while (!stack.isEmpty()) { 360 restoreLock(stack.pop(), restored); 361 } 362 } 363 364 // Restore the locks for all the procedures. 365 // Notice that we need to restore the locks starting from the root proc, otherwise there will be 366 // problem that a sub procedure may hold the exclusive lock first and then we are stuck when 367 // calling the acquireLock method for the parent procedure. 368 // The algorithm is straight-forward: 369 // 1. Use a set to record the procedures which locks have already been restored. 370 // 2. Use a stack to store the hierarchy of the procedures 371 // 3. For all the procedure, we will first try to find its parent and push it into the stack, 372 // unless 373 // a. We have no parent, i.e, we are the root procedure 374 // b. The lock has already been restored(by checking the set introduced in #1) 375 // then we start to pop the stack and call acquireLock for each procedure. 376 // Notice that this should be done for all procedures, not only the ones in runnableList. 377 private void restoreLocks() { 378 Set<Long> restored = new HashSet<>(); 379 Deque<Procedure<TEnvironment>> stack = new ArrayDeque<>(); 380 procedures.values().forEach(proc -> { 381 for (;;) { 382 if (restored.contains(proc.getProcId())) { 383 restoreLocks(stack, restored); 384 return; 385 } 386 if (!proc.hasParent()) { 387 restoreLock(proc, restored); 388 restoreLocks(stack, restored); 389 return; 390 } 391 stack.push(proc); 392 proc = procedures.get(proc.getParentProcId()); 393 } 394 }); 395 } 396 397 private void loadProcedures(ProcedureIterator procIter, boolean abortOnCorruption) 398 throws IOException { 399 // 1. Build the rollback stack 400 int runnableCount = 0; 401 int failedCount = 0; 402 int waitingCount = 0; 403 int waitingTimeoutCount = 0; 404 while (procIter.hasNext()) { 405 boolean finished = procIter.isNextFinished(); 406 @SuppressWarnings("unchecked") 407 Procedure<TEnvironment> proc = procIter.next(); 408 NonceKey nonceKey = proc.getNonceKey(); 409 long procId = proc.getProcId(); 410 411 if (finished) { 412 completed.put(proc.getProcId(), new CompletedProcedureRetainer<>(proc)); 413 LOG.debug("Completed {}", proc); 414 } else { 415 if (!proc.hasParent()) { 416 assert !proc.isFinished() : "unexpected finished procedure"; 417 rollbackStack.put(proc.getProcId(), new RootProcedureState<>()); 418 } 419 420 // add the procedure to the map 421 proc.beforeReplay(getEnvironment()); 422 procedures.put(proc.getProcId(), proc); 423 switch (proc.getState()) { 424 case RUNNABLE: 425 runnableCount++; 426 break; 427 case FAILED: 428 failedCount++; 429 break; 430 case WAITING: 431 waitingCount++; 432 break; 433 case WAITING_TIMEOUT: 434 waitingTimeoutCount++; 435 break; 436 default: 437 break; 438 } 439 } 440 441 if (nonceKey != null) { 442 nonceKeysToProcIdsMap.put(nonceKey, procId); // add the nonce to the map 443 } 444 } 445 446 // 2. Initialize the stacks: In the old implementation, for procedures in FAILED state, we will 447 // push it into the ProcedureScheduler directly to execute the rollback. But this does not work 448 // after we introduce the restore lock stage. For now, when we acquire a xlock, we will remove 449 // the queue from runQueue in scheduler, and then when a procedure which has lock access, for 450 // example, a sub procedure of the procedure which has the xlock, is pushed into the scheduler, 451 // we will add the queue back to let the workers poll from it. The assumption here is that, the 452 // procedure which has the xlock should have been polled out already, so when loading we can not 453 // add the procedure to scheduler first and then call acquireLock, since the procedure is still 454 // in the queue, and since we will remove the queue from runQueue, then no one can poll it out, 455 // then there is a dead lock 456 List<Procedure<TEnvironment>> runnableList = new ArrayList<>(runnableCount); 457 List<Procedure<TEnvironment>> failedList = new ArrayList<>(failedCount); 458 List<Procedure<TEnvironment>> waitingList = new ArrayList<>(waitingCount); 459 List<Procedure<TEnvironment>> waitingTimeoutList = new ArrayList<>(waitingTimeoutCount); 460 procIter.reset(); 461 while (procIter.hasNext()) { 462 if (procIter.isNextFinished()) { 463 procIter.skipNext(); 464 continue; 465 } 466 467 @SuppressWarnings("unchecked") 468 Procedure<TEnvironment> proc = procIter.next(); 469 assert !(proc.isFinished() && !proc.hasParent()) : "unexpected completed proc=" + proc; 470 LOG.debug("Loading {}", proc); 471 Long rootProcId = getRootProcedureId(proc); 472 // The orphan procedures will be passed to handleCorrupted, so add an assert here 473 assert rootProcId != null; 474 475 if (proc.hasParent()) { 476 Procedure<TEnvironment> parent = procedures.get(proc.getParentProcId()); 477 if (parent != null && !proc.isFinished()) { 478 parent.incChildrenLatch(); 479 } 480 } 481 482 RootProcedureState<TEnvironment> procStack = rollbackStack.get(rootProcId); 483 procStack.loadStack(proc); 484 485 proc.setRootProcId(rootProcId); 486 switch (proc.getState()) { 487 case RUNNABLE: 488 runnableList.add(proc); 489 break; 490 case WAITING: 491 waitingList.add(proc); 492 break; 493 case WAITING_TIMEOUT: 494 waitingTimeoutList.add(proc); 495 break; 496 case FAILED: 497 failedList.add(proc); 498 break; 499 case ROLLEDBACK: 500 case INITIALIZING: 501 String msg = "Unexpected " + proc.getState() + " state for " + proc; 502 LOG.error(msg); 503 throw new UnsupportedOperationException(msg); 504 default: 505 break; 506 } 507 } 508 509 // 3. Check the waiting procedures to see if some of them can be added to runnable. 510 waitingList.forEach(proc -> { 511 if (!proc.hasChildren()) { 512 // Normally, WAITING procedures should be waken by its children. But, there is a case that, 513 // all the children are successful and before they can wake up their parent procedure, the 514 // master was killed. So, during recovering the procedures from ProcedureWal, its children 515 // are not loaded because of their SUCCESS state. So we need to continue to run this WAITING 516 // procedure. But before executing, we need to set its state to RUNNABLE, otherwise, a 517 // exception will throw: 518 // Preconditions.checkArgument(procedure.getState() == ProcedureState.RUNNABLE, 519 // "NOT RUNNABLE! " + procedure.toString()); 520 proc.setState(ProcedureState.RUNNABLE); 521 runnableList.add(proc); 522 } else { 523 proc.afterReplay(getEnvironment()); 524 } 525 }); 526 // 4. restore locks 527 restoreLocks(); 528 529 // 5. Push the procedures to the timeout executor 530 waitingTimeoutList.forEach(proc -> { 531 proc.afterReplay(getEnvironment()); 532 timeoutExecutor.add(proc); 533 }); 534 535 // 6. Push the procedure to the scheduler 536 failedList.forEach(scheduler::addBack); 537 runnableList.forEach(p -> { 538 p.afterReplay(getEnvironment()); 539 if (!p.hasParent()) { 540 sendProcedureLoadedNotification(p.getProcId()); 541 } 542 scheduler.addBack(p); 543 }); 544 // After all procedures put into the queue, signal the worker threads. 545 // Otherwise, there is a race condition. See HBASE-21364. 546 scheduler.signalAll(); 547 } 548 549 /** 550 * Initialize the procedure executor, but do not start workers. We will start them later. 551 * <p/> 552 * It calls ProcedureStore.recoverLease() and ProcedureStore.load() to recover the lease, and 553 * ensure a single executor, and start the procedure replay to resume and recover the previous 554 * pending and in-progress procedures. 555 * @param numThreads number of threads available for procedure execution. 556 * @param abortOnCorruption true if you want to abort your service in case a corrupted procedure 557 * is found on replay. otherwise false. 558 */ 559 public void init(int numThreads, boolean abortOnCorruption) throws IOException { 560 // We have numThreads executor + one timer thread used for timing out 561 // procedures and triggering periodic procedures. 562 this.corePoolSize = numThreads; 563 this.maxPoolSize = 10 * numThreads; 564 LOG.info("Starting {} core workers (bigger of cpus/4 or 16) with max (burst) worker count={}", 565 corePoolSize, maxPoolSize); 566 567 this.threadGroup = new ThreadGroup("PEWorkerGroup"); 568 this.timeoutExecutor = new TimeoutExecutorThread<>(this, threadGroup, "ProcExecTimeout"); 569 this.workerMonitorExecutor = new TimeoutExecutorThread<>(this, threadGroup, "WorkerMonitor"); 570 571 // Create the workers 572 workerId.set(0); 573 workerThreads = new CopyOnWriteArrayList<>(); 574 for (int i = 0; i < corePoolSize; ++i) { 575 workerThreads.add(new WorkerThread(threadGroup)); 576 } 577 578 long st, et; 579 580 // Acquire the store lease. 581 st = System.nanoTime(); 582 store.recoverLease(); 583 et = System.nanoTime(); 584 LOG.info("Recovered {} lease in {}", store.getClass().getSimpleName(), 585 StringUtils.humanTimeDiff(TimeUnit.NANOSECONDS.toMillis(et - st))); 586 587 // start the procedure scheduler 588 scheduler.start(); 589 590 // TODO: Split in two steps. 591 // TODO: Handle corrupted procedures (currently just a warn) 592 // The first one will make sure that we have the latest id, 593 // so we can start the threads and accept new procedures. 594 // The second step will do the actual load of old procedures. 595 st = System.nanoTime(); 596 load(abortOnCorruption); 597 et = System.nanoTime(); 598 LOG.info("Loaded {} in {}", store.getClass().getSimpleName(), 599 StringUtils.humanTimeDiff(TimeUnit.NANOSECONDS.toMillis(et - st))); 600 } 601 602 /** 603 * Start the workers. 604 */ 605 public void startWorkers() throws IOException { 606 if (!running.compareAndSet(false, true)) { 607 LOG.warn("Already running"); 608 return; 609 } 610 // Start the executors. Here we must have the lastProcId set. 611 LOG.trace("Start workers {}", workerThreads.size()); 612 timeoutExecutor.start(); 613 workerMonitorExecutor.start(); 614 for (WorkerThread worker : workerThreads) { 615 worker.start(); 616 } 617 618 // Internal chores 619 workerMonitorExecutor.add(new WorkerMonitor()); 620 621 // Add completed cleaner chore 622 addChore(new CompletedProcedureCleaner<>(conf, store, procExecutionLock, completed, 623 nonceKeysToProcIdsMap)); 624 } 625 626 public void stop() { 627 if (!running.getAndSet(false)) { 628 return; 629 } 630 631 LOG.info("Stopping"); 632 scheduler.stop(); 633 timeoutExecutor.sendStopSignal(); 634 workerMonitorExecutor.sendStopSignal(); 635 } 636 637 public void join() { 638 assert !isRunning() : "expected not running"; 639 640 // stop the timeout executor 641 timeoutExecutor.awaitTermination(); 642 // stop the work monitor executor 643 workerMonitorExecutor.awaitTermination(); 644 645 // stop the worker threads 646 for (WorkerThread worker : workerThreads) { 647 worker.awaitTermination(); 648 } 649 650 // Destroy the Thread Group for the executors 651 // TODO: Fix. #join is not place to destroy resources. 652 try { 653 threadGroup.destroy(); 654 } catch (IllegalThreadStateException e) { 655 LOG.error("ThreadGroup {} contains running threads; {}: See STDOUT", this.threadGroup, 656 e.getMessage()); 657 // This dumps list of threads on STDOUT. 658 this.threadGroup.list(); 659 } 660 661 // reset the in-memory state for testing 662 completed.clear(); 663 rollbackStack.clear(); 664 procedures.clear(); 665 nonceKeysToProcIdsMap.clear(); 666 scheduler.clear(); 667 lastProcId.set(-1); 668 } 669 670 public void refreshConfiguration(final Configuration conf) { 671 this.conf = conf; 672 setKeepAliveTime(conf.getLong(WORKER_KEEP_ALIVE_TIME_CONF_KEY, DEFAULT_WORKER_KEEP_ALIVE_TIME), 673 TimeUnit.MILLISECONDS); 674 } 675 676 // ========================================================================== 677 // Accessors 678 // ========================================================================== 679 public boolean isRunning() { 680 return running.get(); 681 } 682 683 /** Returns the current number of worker threads. */ 684 public int getWorkerThreadCount() { 685 return workerThreads.size(); 686 } 687 688 /** Returns the core pool size settings. */ 689 public int getCorePoolSize() { 690 return corePoolSize; 691 } 692 693 public int getActiveExecutorCount() { 694 return activeExecutorCount.get(); 695 } 696 697 public TEnvironment getEnvironment() { 698 return this.environment; 699 } 700 701 public ProcedureStore getStore() { 702 return this.store; 703 } 704 705 ProcedureScheduler getScheduler() { 706 return scheduler; 707 } 708 709 public void setKeepAliveTime(final long keepAliveTime, final TimeUnit timeUnit) { 710 this.keepAliveTime = timeUnit.toMillis(keepAliveTime); 711 this.scheduler.signalAll(); 712 } 713 714 public long getKeepAliveTime(final TimeUnit timeUnit) { 715 return timeUnit.convert(keepAliveTime, TimeUnit.MILLISECONDS); 716 } 717 718 // ========================================================================== 719 // Submit/Remove Chores 720 // ========================================================================== 721 722 /** 723 * Add a chore procedure to the executor 724 * @param chore the chore to add 725 */ 726 public void addChore(@Nullable ProcedureInMemoryChore<TEnvironment> chore) { 727 if (chore == null) { 728 return; 729 } 730 chore.setState(ProcedureState.WAITING_TIMEOUT); 731 timeoutExecutor.add(chore); 732 } 733 734 /** 735 * Remove a chore procedure from the executor 736 * @param chore the chore to remove 737 * @return whether the chore is removed, or it will be removed later 738 */ 739 public boolean removeChore(@Nullable ProcedureInMemoryChore<TEnvironment> chore) { 740 if (chore == null) { 741 return true; 742 } 743 chore.setState(ProcedureState.SUCCESS); 744 return timeoutExecutor.remove(chore); 745 } 746 747 // ========================================================================== 748 // Nonce Procedure helpers 749 // ========================================================================== 750 /** 751 * Create a NonceKey from the specified nonceGroup and nonce. 752 * @param nonceGroup the group to use for the {@link NonceKey} 753 * @param nonce the nonce to use in the {@link NonceKey} 754 * @return the generated NonceKey 755 */ 756 public NonceKey createNonceKey(final long nonceGroup, final long nonce) { 757 return (nonce == HConstants.NO_NONCE) ? null : new NonceKey(nonceGroup, nonce); 758 } 759 760 /** 761 * Register a nonce for a procedure that is going to be submitted. A procId will be reserved and 762 * on submitProcedure(), the procedure with the specified nonce will take the reserved ProcId. If 763 * someone already reserved the nonce, this method will return the procId reserved, otherwise an 764 * invalid procId will be returned. and the caller should procede and submit the procedure. 765 * @param nonceKey A unique identifier for this operation from the client or process. 766 * @return the procId associated with the nonce, if any otherwise an invalid procId. 767 */ 768 public long registerNonce(final NonceKey nonceKey) { 769 if (nonceKey == null) { 770 return -1; 771 } 772 773 // check if we have already a Reserved ID for the nonce 774 Long oldProcId = nonceKeysToProcIdsMap.get(nonceKey); 775 if (oldProcId == null) { 776 // reserve a new Procedure ID, this will be associated with the nonce 777 // and the procedure submitted with the specified nonce will use this ID. 778 final long newProcId = nextProcId(); 779 oldProcId = nonceKeysToProcIdsMap.putIfAbsent(nonceKey, newProcId); 780 if (oldProcId == null) { 781 return -1; 782 } 783 } 784 785 // we found a registered nonce, but the procedure may not have been submitted yet. 786 // since the client expect the procedure to be submitted, spin here until it is. 787 final boolean traceEnabled = LOG.isTraceEnabled(); 788 while ( 789 isRunning() && !(procedures.containsKey(oldProcId) || completed.containsKey(oldProcId)) 790 && nonceKeysToProcIdsMap.containsKey(nonceKey) 791 ) { 792 if (traceEnabled) { 793 LOG.trace("Waiting for pid=" + oldProcId.longValue() + " to be submitted"); 794 } 795 Threads.sleep(100); 796 } 797 return oldProcId.longValue(); 798 } 799 800 /** 801 * Remove the NonceKey if the procedure was not submitted to the executor. 802 * @param nonceKey A unique identifier for this operation from the client or process. 803 */ 804 public void unregisterNonceIfProcedureWasNotSubmitted(final NonceKey nonceKey) { 805 if (nonceKey == null) { 806 return; 807 } 808 809 final Long procId = nonceKeysToProcIdsMap.get(nonceKey); 810 if (procId == null) { 811 return; 812 } 813 814 // if the procedure was not submitted, remove the nonce 815 if (!(procedures.containsKey(procId) || completed.containsKey(procId))) { 816 nonceKeysToProcIdsMap.remove(nonceKey); 817 } 818 } 819 820 /** 821 * If the failure failed before submitting it, we may want to give back the same error to the 822 * requests with the same nonceKey. 823 * @param nonceKey A unique identifier for this operation from the client or process 824 * @param procName name of the procedure, used to inform the user 825 * @param procOwner name of the owner of the procedure, used to inform the user 826 * @param exception the failure to report to the user 827 */ 828 public void setFailureResultForNonce(NonceKey nonceKey, String procName, User procOwner, 829 IOException exception) { 830 if (nonceKey == null) { 831 return; 832 } 833 834 Long procId = nonceKeysToProcIdsMap.get(nonceKey); 835 if (procId == null || completed.containsKey(procId)) { 836 return; 837 } 838 839 completed.computeIfAbsent(procId, (key) -> { 840 Procedure<TEnvironment> proc = 841 new FailedProcedure<>(procId.longValue(), procName, procOwner, nonceKey, exception); 842 843 return new CompletedProcedureRetainer<>(proc); 844 }); 845 } 846 847 // ========================================================================== 848 // Submit/Abort Procedure 849 // ========================================================================== 850 /** 851 * Add a new root-procedure to the executor. 852 * @param proc the new procedure to execute. 853 * @return the procedure id, that can be used to monitor the operation 854 */ 855 public long submitProcedure(Procedure<TEnvironment> proc) { 856 return submitProcedure(proc, null); 857 } 858 859 /** 860 * Bypass a procedure. If the procedure is set to bypass, all the logic in execute/rollback will 861 * be ignored and it will return success, whatever. It is used to recover buggy stuck procedures, 862 * releasing the lock resources and letting other procedures run. Bypassing one procedure (and its 863 * ancestors will be bypassed automatically) may leave the cluster in a middle state, e.g. region 864 * not assigned, or some hdfs files left behind. After getting rid of those stuck procedures, the 865 * operators may have to do some clean up on hdfs or schedule some assign procedures to let region 866 * online. DO AT YOUR OWN RISK. 867 * <p> 868 * A procedure can be bypassed only if 1. The procedure is in state of RUNNABLE, WAITING, 869 * WAITING_TIMEOUT or it is a root procedure without any child. 2. No other worker thread is 870 * executing it 3. No child procedure has been submitted 871 * <p> 872 * If all the requirements are meet, the procedure and its ancestors will be bypassed and 873 * persisted to WAL. 874 * <p> 875 * If the procedure is in WAITING state, will set it to RUNNABLE add it to run queue. TODO: What 876 * about WAITING_TIMEOUT? 877 * @param pids the procedure id 878 * @param lockWait time to wait lock 879 * @param force if force set to true, we will bypass the procedure even if it is executing. 880 * This is for procedures which can't break out during executing(due to bug, 881 * mostly) In this case, bypassing the procedure is not enough, since it is 882 * already stuck there. We need to restart the master after bypassing, and 883 * letting the problematic procedure to execute wth bypass=true, so in that 884 * condition, the procedure can be successfully bypassed. 885 * @param recursive We will do an expensive search for children of each pid. EXPENSIVE! 886 * @return true if bypass success 887 * @throws IOException IOException 888 */ 889 public List<Boolean> bypassProcedure(List<Long> pids, long lockWait, boolean force, 890 boolean recursive) throws IOException { 891 List<Boolean> result = new ArrayList<Boolean>(pids.size()); 892 for (long pid : pids) { 893 result.add(bypassProcedure(pid, lockWait, force, recursive)); 894 } 895 return result; 896 } 897 898 boolean bypassProcedure(long pid, long lockWait, boolean override, boolean recursive) 899 throws IOException { 900 Preconditions.checkArgument(lockWait > 0, "lockWait should be positive"); 901 final Procedure<TEnvironment> procedure = getProcedure(pid); 902 if (procedure == null) { 903 LOG.debug("Procedure pid={} does not exist, skipping bypass", pid); 904 return false; 905 } 906 907 LOG.debug("Begin bypass {} with lockWait={}, override={}, recursive={}", procedure, lockWait, 908 override, recursive); 909 IdLock.Entry lockEntry = procExecutionLock.tryLockEntry(procedure.getProcId(), lockWait); 910 if (lockEntry == null && !override) { 911 LOG.debug("Waited {} ms, but {} is still running, skipping bypass with force={}", lockWait, 912 procedure, override); 913 return false; 914 } else if (lockEntry == null) { 915 LOG.debug("Waited {} ms, but {} is still running, begin bypass with force={}", lockWait, 916 procedure, override); 917 } 918 try { 919 // check whether the procedure is already finished 920 if (procedure.isFinished()) { 921 LOG.debug("{} is already finished, skipping bypass", procedure); 922 return false; 923 } 924 925 if (procedure.hasChildren()) { 926 if (recursive) { 927 // EXPENSIVE. Checks each live procedure of which there could be many!!! 928 // Is there another way to get children of a procedure? 929 LOG.info("Recursive bypass on children of pid={}", procedure.getProcId()); 930 this.procedures.forEachValue(1 /* Single-threaded */, 931 // Transformer 932 v -> v.getParentProcId() == procedure.getProcId() ? v : null, 933 // Consumer 934 v -> { 935 try { 936 bypassProcedure(v.getProcId(), lockWait, override, recursive); 937 } catch (IOException e) { 938 LOG.warn("Recursive bypass of pid={}", v.getProcId(), e); 939 } 940 }); 941 } else { 942 LOG.debug("{} has children, skipping bypass", procedure); 943 return false; 944 } 945 } 946 947 // If the procedure has no parent or no child, we are safe to bypass it in whatever state 948 if ( 949 procedure.hasParent() && procedure.getState() != ProcedureState.RUNNABLE 950 && procedure.getState() != ProcedureState.WAITING 951 && procedure.getState() != ProcedureState.WAITING_TIMEOUT 952 ) { 953 LOG.debug("Bypassing procedures in RUNNABLE, WAITING and WAITING_TIMEOUT states " 954 + "(with no parent), {}", procedure); 955 // Question: how is the bypass done here? 956 return false; 957 } 958 959 // Now, the procedure is not finished, and no one can execute it since we take the lock now 960 // And we can be sure that its ancestor is not running too, since their child has not 961 // finished yet 962 Procedure<TEnvironment> current = procedure; 963 while (current != null) { 964 LOG.debug("Bypassing {}", current); 965 current.bypass(getEnvironment()); 966 store.update(current); 967 long parentID = current.getParentProcId(); 968 current = getProcedure(parentID); 969 } 970 971 // wake up waiting procedure, already checked there is no child 972 if (procedure.getState() == ProcedureState.WAITING) { 973 procedure.setState(ProcedureState.RUNNABLE); 974 store.update(procedure); 975 } 976 977 // If state of procedure is WAITING_TIMEOUT, we can directly submit it to the scheduler. 978 // Instead we should remove it from timeout Executor queue and tranfer its state to RUNNABLE 979 if (procedure.getState() == ProcedureState.WAITING_TIMEOUT) { 980 LOG.debug("transform procedure {} from WAITING_TIMEOUT to RUNNABLE", procedure); 981 if (timeoutExecutor.remove(procedure)) { 982 LOG.debug("removed procedure {} from timeoutExecutor", procedure); 983 timeoutExecutor.executeTimedoutProcedure(procedure); 984 } 985 } else if (lockEntry != null) { 986 scheduler.addFront(procedure); 987 LOG.debug("Bypassing {} and its ancestors successfully, adding to queue", procedure); 988 } else { 989 // If we don't have the lock, we can't re-submit the queue, 990 // since it is already executing. To get rid of the stuck situation, we 991 // need to restart the master. With the procedure set to bypass, the procedureExecutor 992 // will bypass it and won't get stuck again. 993 LOG.debug("Bypassing {} and its ancestors successfully, but since it is already running, " 994 + "skipping add to queue", procedure); 995 } 996 return true; 997 998 } finally { 999 if (lockEntry != null) { 1000 procExecutionLock.releaseLockEntry(lockEntry); 1001 } 1002 } 1003 } 1004 1005 /** 1006 * Add a new root-procedure to the executor. 1007 * @param proc the new procedure to execute. 1008 * @param nonceKey the registered unique identifier for this operation from the client or process. 1009 * @return the procedure id, that can be used to monitor the operation 1010 */ 1011 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NP_NULL_ON_SOME_PATH", 1012 justification = "FindBugs is blind to the check-for-null") 1013 public long submitProcedure(Procedure<TEnvironment> proc, NonceKey nonceKey) { 1014 Preconditions.checkArgument(lastProcId.get() >= 0); 1015 1016 prepareProcedure(proc); 1017 1018 final Long currentProcId; 1019 if (nonceKey != null) { 1020 currentProcId = nonceKeysToProcIdsMap.get(nonceKey); 1021 Preconditions.checkArgument(currentProcId != null, 1022 "Expected nonceKey=" + nonceKey + " to be reserved, use registerNonce(); proc=" + proc); 1023 } else { 1024 currentProcId = nextProcId(); 1025 } 1026 1027 // Initialize the procedure 1028 proc.setNonceKey(nonceKey); 1029 proc.setProcId(currentProcId.longValue()); 1030 1031 // Commit the transaction 1032 store.insert(proc, null); 1033 LOG.debug("Stored {}", proc); 1034 1035 // Add the procedure to the executor 1036 return pushProcedure(proc); 1037 } 1038 1039 /** 1040 * Add a set of new root-procedure to the executor. 1041 * @param procs the new procedures to execute. 1042 */ 1043 // TODO: Do we need to take nonces here? 1044 public void submitProcedures(Procedure<TEnvironment>[] procs) { 1045 Preconditions.checkArgument(lastProcId.get() >= 0); 1046 if (procs == null || procs.length <= 0) { 1047 return; 1048 } 1049 1050 // Prepare procedure 1051 for (int i = 0; i < procs.length; ++i) { 1052 prepareProcedure(procs[i]).setProcId(nextProcId()); 1053 } 1054 1055 // Commit the transaction 1056 store.insert(procs); 1057 if (LOG.isDebugEnabled()) { 1058 LOG.debug("Stored " + Arrays.toString(procs)); 1059 } 1060 1061 // Add the procedure to the executor 1062 for (int i = 0; i < procs.length; ++i) { 1063 pushProcedure(procs[i]); 1064 } 1065 } 1066 1067 private Procedure<TEnvironment> prepareProcedure(Procedure<TEnvironment> proc) { 1068 Preconditions.checkArgument(proc.getState() == ProcedureState.INITIALIZING); 1069 Preconditions.checkArgument(!proc.hasParent(), "unexpected parent", proc); 1070 if (this.checkOwnerSet) { 1071 Preconditions.checkArgument(proc.hasOwner(), "missing owner"); 1072 } 1073 return proc; 1074 } 1075 1076 private long pushProcedure(Procedure<TEnvironment> proc) { 1077 final long currentProcId = proc.getProcId(); 1078 1079 // Update metrics on start of a procedure 1080 proc.updateMetricsOnSubmit(getEnvironment()); 1081 1082 // Create the rollback stack for the procedure 1083 RootProcedureState<TEnvironment> stack = new RootProcedureState<>(); 1084 rollbackStack.put(currentProcId, stack); 1085 1086 // Submit the new subprocedures 1087 assert !procedures.containsKey(currentProcId); 1088 procedures.put(currentProcId, proc); 1089 sendProcedureAddedNotification(currentProcId); 1090 scheduler.addBack(proc); 1091 return proc.getProcId(); 1092 } 1093 1094 /** 1095 * Send an abort notification the specified procedure. Depending on the procedure implementation 1096 * the abort can be considered or ignored. 1097 * @param procId the procedure to abort 1098 * @return true if the procedure exists and has received the abort, otherwise false. 1099 */ 1100 public boolean abort(long procId) { 1101 return abort(procId, true); 1102 } 1103 1104 /** 1105 * Send an abort notification to the specified procedure. Depending on the procedure 1106 * implementation, the abort can be considered or ignored. 1107 * @param procId the procedure to abort 1108 * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted? 1109 * @return true if the procedure exists and has received the abort, otherwise false. 1110 */ 1111 public boolean abort(long procId, boolean mayInterruptIfRunning) { 1112 Procedure<TEnvironment> proc = procedures.get(procId); 1113 if (proc != null) { 1114 if (!mayInterruptIfRunning && proc.wasExecuted()) { 1115 return false; 1116 } 1117 return proc.abort(getEnvironment()); 1118 } 1119 return false; 1120 } 1121 1122 // ========================================================================== 1123 // Executor query helpers 1124 // ========================================================================== 1125 public Procedure<TEnvironment> getProcedure(final long procId) { 1126 return procedures.get(procId); 1127 } 1128 1129 public <T extends Procedure<TEnvironment>> T getProcedure(Class<T> clazz, long procId) { 1130 Procedure<TEnvironment> proc = getProcedure(procId); 1131 if (clazz.isInstance(proc)) { 1132 return clazz.cast(proc); 1133 } 1134 return null; 1135 } 1136 1137 public Procedure<TEnvironment> getResult(long procId) { 1138 CompletedProcedureRetainer<TEnvironment> retainer = completed.get(procId); 1139 if (retainer == null) { 1140 return null; 1141 } else { 1142 return retainer.getProcedure(); 1143 } 1144 } 1145 1146 /** 1147 * Return true if the procedure is finished. The state may be "completed successfully" or "failed 1148 * and rolledback". Use getResult() to check the state or get the result data. 1149 * @param procId the ID of the procedure to check 1150 * @return true if the procedure execution is finished, otherwise false. 1151 */ 1152 public boolean isFinished(final long procId) { 1153 return !procedures.containsKey(procId); 1154 } 1155 1156 /** 1157 * Return true if the procedure is started. 1158 * @param procId the ID of the procedure to check 1159 * @return true if the procedure execution is started, otherwise false. 1160 */ 1161 public boolean isStarted(long procId) { 1162 Procedure<?> proc = procedures.get(procId); 1163 if (proc == null) { 1164 return completed.get(procId) != null; 1165 } 1166 return proc.wasExecuted(); 1167 } 1168 1169 /** 1170 * Mark the specified completed procedure, as ready to remove. 1171 * @param procId the ID of the procedure to remove 1172 */ 1173 public void removeResult(long procId) { 1174 CompletedProcedureRetainer<TEnvironment> retainer = completed.get(procId); 1175 if (retainer == null) { 1176 assert !procedures.containsKey(procId) : "pid=" + procId + " is still running"; 1177 LOG.debug("pid={} already removed by the cleaner.", procId); 1178 return; 1179 } 1180 1181 // The CompletedProcedureCleaner will take care of deletion, once the TTL is expired. 1182 retainer.setClientAckTime(EnvironmentEdgeManager.currentTime()); 1183 } 1184 1185 public Procedure<TEnvironment> getResultOrProcedure(long procId) { 1186 CompletedProcedureRetainer<TEnvironment> retainer = completed.get(procId); 1187 if (retainer == null) { 1188 return procedures.get(procId); 1189 } else { 1190 return retainer.getProcedure(); 1191 } 1192 } 1193 1194 /** 1195 * Check if the user is this procedure's owner 1196 * @param procId the target procedure 1197 * @param user the user 1198 * @return true if the user is the owner of the procedure, false otherwise or the owner is 1199 * unknown. 1200 */ 1201 public boolean isProcedureOwner(long procId, User user) { 1202 if (user == null) { 1203 return false; 1204 } 1205 final Procedure<TEnvironment> runningProc = procedures.get(procId); 1206 if (runningProc != null) { 1207 return runningProc.getOwner().equals(user.getShortName()); 1208 } 1209 1210 final CompletedProcedureRetainer<TEnvironment> retainer = completed.get(procId); 1211 if (retainer != null) { 1212 return retainer.getProcedure().getOwner().equals(user.getShortName()); 1213 } 1214 1215 // Procedure either does not exist or has already completed and got cleaned up. 1216 // At this time, we cannot check the owner of the procedure 1217 return false; 1218 } 1219 1220 /** 1221 * Should only be used when starting up, where the procedure workers have not been started. 1222 * <p/> 1223 * If the procedure works has been started, the return values maybe changed when you are 1224 * processing it so usually this is not safe. Use {@link #getProcedures()} below for most cases as 1225 * it will do a copy, and also include the finished procedures. 1226 */ 1227 public Collection<Procedure<TEnvironment>> getActiveProceduresNoCopy() { 1228 return procedures.values(); 1229 } 1230 1231 /** 1232 * Get procedures. 1233 * @return the procedures in a list 1234 */ 1235 public List<Procedure<TEnvironment>> getProcedures() { 1236 List<Procedure<TEnvironment>> procedureList = 1237 new ArrayList<>(procedures.size() + completed.size()); 1238 procedureList.addAll(procedures.values()); 1239 // Note: The procedure could show up twice in the list with different state, as 1240 // it could complete after we walk through procedures list and insert into 1241 // procedureList - it is ok, as we will use the information in the Procedure 1242 // to figure it out; to prevent this would increase the complexity of the logic. 1243 completed.values().stream().map(CompletedProcedureRetainer::getProcedure) 1244 .forEach(procedureList::add); 1245 return procedureList; 1246 } 1247 1248 // ========================================================================== 1249 // Listeners helpers 1250 // ========================================================================== 1251 public void registerListener(ProcedureExecutorListener listener) { 1252 this.listeners.add(listener); 1253 } 1254 1255 public boolean unregisterListener(ProcedureExecutorListener listener) { 1256 return this.listeners.remove(listener); 1257 } 1258 1259 private void sendProcedureLoadedNotification(final long procId) { 1260 if (!this.listeners.isEmpty()) { 1261 for (ProcedureExecutorListener listener : this.listeners) { 1262 try { 1263 listener.procedureLoaded(procId); 1264 } catch (Throwable e) { 1265 LOG.error("Listener " + listener + " had an error: " + e.getMessage(), e); 1266 } 1267 } 1268 } 1269 } 1270 1271 private void sendProcedureAddedNotification(final long procId) { 1272 if (!this.listeners.isEmpty()) { 1273 for (ProcedureExecutorListener listener : this.listeners) { 1274 try { 1275 listener.procedureAdded(procId); 1276 } catch (Throwable e) { 1277 LOG.error("Listener " + listener + " had an error: " + e.getMessage(), e); 1278 } 1279 } 1280 } 1281 } 1282 1283 private void sendProcedureFinishedNotification(final long procId) { 1284 if (!this.listeners.isEmpty()) { 1285 for (ProcedureExecutorListener listener : this.listeners) { 1286 try { 1287 listener.procedureFinished(procId); 1288 } catch (Throwable e) { 1289 LOG.error("Listener " + listener + " had an error: " + e.getMessage(), e); 1290 } 1291 } 1292 } 1293 } 1294 1295 // ========================================================================== 1296 // Procedure IDs helpers 1297 // ========================================================================== 1298 private long nextProcId() { 1299 long procId = lastProcId.incrementAndGet(); 1300 if (procId < 0) { 1301 while (!lastProcId.compareAndSet(procId, 0)) { 1302 procId = lastProcId.get(); 1303 if (procId >= 0) { 1304 break; 1305 } 1306 } 1307 while (procedures.containsKey(procId)) { 1308 procId = lastProcId.incrementAndGet(); 1309 } 1310 } 1311 assert procId >= 0 : "Invalid procId " + procId; 1312 return procId; 1313 } 1314 1315 protected long getLastProcId() { 1316 return lastProcId.get(); 1317 } 1318 1319 public Set<Long> getActiveProcIds() { 1320 return procedures.keySet(); 1321 } 1322 1323 Long getRootProcedureId(Procedure<TEnvironment> proc) { 1324 return Procedure.getRootProcedureId(procedures, proc); 1325 } 1326 1327 // ========================================================================== 1328 // Executions 1329 // ========================================================================== 1330 private void executeProcedure(Procedure<TEnvironment> proc) { 1331 if (proc.isFinished()) { 1332 LOG.debug("{} is already finished, skipping execution", proc); 1333 return; 1334 } 1335 final Long rootProcId = getRootProcedureId(proc); 1336 if (rootProcId == null) { 1337 // The 'proc' was ready to run but the root procedure was rolledback 1338 LOG.warn("Rollback because parent is done/rolledback proc=" + proc); 1339 executeRollback(proc); 1340 return; 1341 } 1342 1343 RootProcedureState<TEnvironment> procStack = rollbackStack.get(rootProcId); 1344 if (procStack == null) { 1345 LOG.warn("RootProcedureState is null for " + proc.getProcId()); 1346 return; 1347 } 1348 do { 1349 // Try to acquire the execution 1350 if (!procStack.acquire(proc)) { 1351 if (procStack.setRollback()) { 1352 // we have the 'rollback-lock' we can start rollingback 1353 switch (executeRollback(rootProcId, procStack)) { 1354 case LOCK_ACQUIRED: 1355 break; 1356 case LOCK_YIELD_WAIT: 1357 procStack.unsetRollback(); 1358 scheduler.yield(proc); 1359 break; 1360 case LOCK_EVENT_WAIT: 1361 LOG.info("LOCK_EVENT_WAIT rollback..." + proc); 1362 procStack.unsetRollback(); 1363 break; 1364 default: 1365 throw new UnsupportedOperationException(); 1366 } 1367 } else { 1368 // if we can't rollback means that some child is still running. 1369 // the rollback will be executed after all the children are done. 1370 // If the procedure was never executed, remove and mark it as rolledback. 1371 if (!proc.wasExecuted()) { 1372 switch (executeRollback(proc)) { 1373 case LOCK_ACQUIRED: 1374 break; 1375 case LOCK_YIELD_WAIT: 1376 scheduler.yield(proc); 1377 break; 1378 case LOCK_EVENT_WAIT: 1379 LOG.info("LOCK_EVENT_WAIT can't rollback child running?..." + proc); 1380 break; 1381 default: 1382 throw new UnsupportedOperationException(); 1383 } 1384 } 1385 } 1386 break; 1387 } 1388 1389 // Execute the procedure 1390 assert proc.getState() == ProcedureState.RUNNABLE : proc; 1391 // Note that lock is NOT about concurrency but rather about ensuring 1392 // ownership of a procedure of an entity such as a region or table 1393 LockState lockState = acquireLock(proc); 1394 switch (lockState) { 1395 case LOCK_ACQUIRED: 1396 execProcedure(procStack, proc); 1397 break; 1398 case LOCK_YIELD_WAIT: 1399 LOG.info(lockState + " " + proc); 1400 scheduler.yield(proc); 1401 break; 1402 case LOCK_EVENT_WAIT: 1403 // Someone will wake us up when the lock is available 1404 LOG.debug(lockState + " " + proc); 1405 break; 1406 default: 1407 throw new UnsupportedOperationException(); 1408 } 1409 procStack.release(proc); 1410 1411 if (proc.isSuccess()) { 1412 // update metrics on finishing the procedure 1413 proc.updateMetricsOnFinish(getEnvironment(), proc.elapsedTime(), true); 1414 LOG.info("Finished " + proc + " in " + StringUtils.humanTimeDiff(proc.elapsedTime())); 1415 // Finalize the procedure state 1416 if (proc.getProcId() == rootProcId) { 1417 procedureFinished(proc); 1418 } else { 1419 execCompletionCleanup(proc); 1420 } 1421 break; 1422 } 1423 } while (procStack.isFailed()); 1424 } 1425 1426 private LockState acquireLock(Procedure<TEnvironment> proc) { 1427 TEnvironment env = getEnvironment(); 1428 // if holdLock is true, then maybe we already have the lock, so just return LOCK_ACQUIRED if 1429 // hasLock is true. 1430 if (proc.hasLock()) { 1431 return LockState.LOCK_ACQUIRED; 1432 } 1433 return proc.doAcquireLock(env, store); 1434 } 1435 1436 private void releaseLock(Procedure<TEnvironment> proc, boolean force) { 1437 TEnvironment env = getEnvironment(); 1438 // For how the framework works, we know that we will always have the lock 1439 // when we call releaseLock(), so we can avoid calling proc.hasLock() 1440 if (force || !proc.holdLock(env) || proc.isFinished()) { 1441 proc.doReleaseLock(env, store); 1442 } 1443 } 1444 1445 /** 1446 * Execute the rollback of the full procedure stack. Once the procedure is rolledback, the 1447 * root-procedure will be visible as finished to user, and the result will be the fatal exception. 1448 */ 1449 private LockState executeRollback(long rootProcId, RootProcedureState<TEnvironment> procStack) { 1450 Procedure<TEnvironment> rootProc = procedures.get(rootProcId); 1451 RemoteProcedureException exception = rootProc.getException(); 1452 // TODO: This needs doc. The root proc doesn't have an exception. Maybe we are 1453 // rolling back because the subprocedure does. Clarify. 1454 if (exception == null) { 1455 exception = procStack.getException(); 1456 rootProc.setFailure(exception); 1457 store.update(rootProc); 1458 } 1459 1460 List<Procedure<TEnvironment>> subprocStack = procStack.getSubproceduresStack(); 1461 assert subprocStack != null : "Called rollback with no steps executed rootProc=" + rootProc; 1462 1463 int stackTail = subprocStack.size(); 1464 while (stackTail-- > 0) { 1465 Procedure<TEnvironment> proc = subprocStack.get(stackTail); 1466 IdLock.Entry lockEntry = null; 1467 // Hold the execution lock if it is not held by us. The IdLock is not reentrant so we need 1468 // this check, as the worker will hold the lock before executing a procedure. This is the only 1469 // place where we may hold two procedure execution locks, and there is a fence in the 1470 // RootProcedureState where we can make sure that only one worker can execute the rollback of 1471 // a RootProcedureState, so there is no dead lock problem. And the lock here is necessary to 1472 // prevent race between us and the force update thread. 1473 if (!procExecutionLock.isHeldByCurrentThread(proc.getProcId())) { 1474 try { 1475 lockEntry = procExecutionLock.getLockEntry(proc.getProcId()); 1476 } catch (IOException e) { 1477 // can only happen if interrupted, so not a big deal to propagate it 1478 throw new UncheckedIOException(e); 1479 } 1480 } 1481 try { 1482 // For the sub procedures which are successfully finished, we do not rollback them. 1483 // Typically, if we want to rollback a procedure, we first need to rollback it, and then 1484 // recursively rollback its ancestors. The state changes which are done by sub procedures 1485 // should be handled by parent procedures when rolling back. For example, when rolling back 1486 // a MergeTableProcedure, we will schedule new procedures to bring the offline regions 1487 // online, instead of rolling back the original procedures which offlined the regions(in 1488 // fact these procedures can not be rolled back...). 1489 if (proc.isSuccess()) { 1490 // Just do the cleanup work, without actually executing the rollback 1491 subprocStack.remove(stackTail); 1492 cleanupAfterRollbackOneStep(proc); 1493 continue; 1494 } 1495 LockState lockState = acquireLock(proc); 1496 if (lockState != LockState.LOCK_ACQUIRED) { 1497 // can't take a lock on the procedure, add the root-proc back on the 1498 // queue waiting for the lock availability 1499 return lockState; 1500 } 1501 1502 lockState = executeRollback(proc); 1503 releaseLock(proc, false); 1504 boolean abortRollback = lockState != LockState.LOCK_ACQUIRED; 1505 abortRollback |= !isRunning() || !store.isRunning(); 1506 1507 // allows to kill the executor before something is stored to the wal. 1508 // useful to test the procedure recovery. 1509 if (abortRollback) { 1510 return lockState; 1511 } 1512 1513 subprocStack.remove(stackTail); 1514 1515 // if the procedure is kind enough to pass the slot to someone else, yield 1516 // if the proc is already finished, do not yield 1517 if (!proc.isFinished() && proc.isYieldAfterExecutionStep(getEnvironment())) { 1518 return LockState.LOCK_YIELD_WAIT; 1519 } 1520 1521 if (proc != rootProc) { 1522 execCompletionCleanup(proc); 1523 } 1524 } finally { 1525 if (lockEntry != null) { 1526 procExecutionLock.releaseLockEntry(lockEntry); 1527 } 1528 } 1529 } 1530 1531 // Finalize the procedure state 1532 LOG.info("Rolled back {} exec-time={}", rootProc, 1533 StringUtils.humanTimeDiff(rootProc.elapsedTime())); 1534 procedureFinished(rootProc); 1535 return LockState.LOCK_ACQUIRED; 1536 } 1537 1538 private void cleanupAfterRollbackOneStep(Procedure<TEnvironment> proc) { 1539 if (proc.removeStackIndex()) { 1540 if (!proc.isSuccess()) { 1541 proc.setState(ProcedureState.ROLLEDBACK); 1542 } 1543 1544 // update metrics on finishing the procedure (fail) 1545 proc.updateMetricsOnFinish(getEnvironment(), proc.elapsedTime(), false); 1546 1547 if (proc.hasParent()) { 1548 store.delete(proc.getProcId()); 1549 procedures.remove(proc.getProcId()); 1550 } else { 1551 final long[] childProcIds = rollbackStack.get(proc.getProcId()).getSubprocedureIds(); 1552 if (childProcIds != null) { 1553 store.delete(proc, childProcIds); 1554 } else { 1555 store.update(proc); 1556 } 1557 } 1558 } else { 1559 store.update(proc); 1560 } 1561 } 1562 1563 /** 1564 * Execute the rollback of the procedure step. It updates the store with the new state (stack 1565 * index) or will remove completly the procedure in case it is a child. 1566 */ 1567 private LockState executeRollback(Procedure<TEnvironment> proc) { 1568 try { 1569 proc.doRollback(getEnvironment()); 1570 } catch (IOException e) { 1571 LOG.debug("Roll back attempt failed for {}", proc, e); 1572 return LockState.LOCK_YIELD_WAIT; 1573 } catch (InterruptedException e) { 1574 handleInterruptedException(proc, e); 1575 return LockState.LOCK_YIELD_WAIT; 1576 } catch (Throwable e) { 1577 // Catch NullPointerExceptions or similar errors... 1578 LOG.error(HBaseMarkers.FATAL, "CODE-BUG: Uncaught runtime exception for " + proc, e); 1579 } 1580 1581 // allows to kill the executor before something is stored to the wal. 1582 // useful to test the procedure recovery. 1583 if (testing != null && testing.shouldKillBeforeStoreUpdate()) { 1584 String msg = "TESTING: Kill before store update"; 1585 LOG.debug(msg); 1586 stop(); 1587 throw new RuntimeException(msg); 1588 } 1589 1590 cleanupAfterRollbackOneStep(proc); 1591 1592 return LockState.LOCK_ACQUIRED; 1593 } 1594 1595 private void yieldProcedure(Procedure<TEnvironment> proc) { 1596 releaseLock(proc, false); 1597 scheduler.yield(proc); 1598 } 1599 1600 /** 1601 * Executes <code>procedure</code> 1602 * <ul> 1603 * <li>Calls the doExecute() of the procedure 1604 * <li>If the procedure execution didn't fail (i.e. valid user input) 1605 * <ul> 1606 * <li>...and returned subprocedures 1607 * <ul> 1608 * <li>The subprocedures are initialized. 1609 * <li>The subprocedures are added to the store 1610 * <li>The subprocedures are added to the runnable queue 1611 * <li>The procedure is now in a WAITING state, waiting for the subprocedures to complete 1612 * </ul> 1613 * </li> 1614 * <li>...if there are no subprocedure 1615 * <ul> 1616 * <li>the procedure completed successfully 1617 * <li>if there is a parent (WAITING) 1618 * <li>the parent state will be set to RUNNABLE 1619 * </ul> 1620 * </li> 1621 * </ul> 1622 * </li> 1623 * <li>In case of failure 1624 * <ul> 1625 * <li>The store is updated with the new state</li> 1626 * <li>The executor (caller of this method) will start the rollback of the procedure</li> 1627 * </ul> 1628 * </li> 1629 * </ul> 1630 */ 1631 private void execProcedure(RootProcedureState<TEnvironment> procStack, 1632 Procedure<TEnvironment> procedure) { 1633 Preconditions.checkArgument(procedure.getState() == ProcedureState.RUNNABLE, 1634 "NOT RUNNABLE! " + procedure.toString()); 1635 1636 // Procedures can suspend themselves. They skip out by throwing a ProcedureSuspendedException. 1637 // The exception is caught below and then we hurry to the exit without disturbing state. The 1638 // idea is that the processing of this procedure will be unsuspended later by an external event 1639 // such the report of a region open. 1640 boolean suspended = false; 1641 1642 // Whether to 're-' -execute; run through the loop again. 1643 boolean reExecute = false; 1644 1645 Procedure<TEnvironment>[] subprocs = null; 1646 do { 1647 reExecute = false; 1648 procedure.resetPersistence(); 1649 try { 1650 subprocs = procedure.doExecute(getEnvironment()); 1651 if (subprocs != null && subprocs.length == 0) { 1652 subprocs = null; 1653 } 1654 } catch (ProcedureSuspendedException e) { 1655 LOG.trace("Suspend {}", procedure); 1656 suspended = true; 1657 } catch (ProcedureYieldException e) { 1658 LOG.trace("Yield {}", procedure, e); 1659 yieldProcedure(procedure); 1660 return; 1661 } catch (InterruptedException e) { 1662 LOG.trace("Yield interrupt {}", procedure, e); 1663 handleInterruptedException(procedure, e); 1664 yieldProcedure(procedure); 1665 return; 1666 } catch (Throwable e) { 1667 // Catch NullPointerExceptions or similar errors... 1668 String msg = "CODE-BUG: Uncaught runtime exception: " + procedure; 1669 LOG.error(msg, e); 1670 procedure.setFailure(new RemoteProcedureException(msg, e)); 1671 } 1672 1673 if (!procedure.isFailed()) { 1674 if (subprocs != null) { 1675 if (subprocs.length == 1 && subprocs[0] == procedure) { 1676 // Procedure returned itself. Quick-shortcut for a state machine-like procedure; 1677 // i.e. we go around this loop again rather than go back out on the scheduler queue. 1678 subprocs = null; 1679 reExecute = true; 1680 LOG.trace("Short-circuit to next step on pid={}", procedure.getProcId()); 1681 } else { 1682 // Yield the current procedure, and make the subprocedure runnable 1683 // subprocs may come back 'null'. 1684 subprocs = initializeChildren(procStack, procedure, subprocs); 1685 LOG.info("Initialized subprocedures=" + (subprocs == null 1686 ? null 1687 : Stream.of(subprocs).map(e -> "{" + e.toString() + "}").collect(Collectors.toList()) 1688 .toString())); 1689 } 1690 } else if (procedure.getState() == ProcedureState.WAITING_TIMEOUT) { 1691 LOG.trace("Added to timeoutExecutor {}", procedure); 1692 timeoutExecutor.add(procedure); 1693 } else if (!suspended) { 1694 // No subtask, so we are done 1695 procedure.setState(ProcedureState.SUCCESS); 1696 } 1697 } 1698 1699 // Add the procedure to the stack 1700 procStack.addRollbackStep(procedure); 1701 1702 // allows to kill the executor before something is stored to the wal. 1703 // useful to test the procedure recovery. 1704 if ( 1705 testing != null && testing.shouldKillBeforeStoreUpdate(suspended, procedure.hasParent()) 1706 ) { 1707 kill("TESTING: Kill BEFORE store update: " + procedure); 1708 } 1709 1710 // TODO: The code here doesn't check if store is running before persisting to the store as 1711 // it relies on the method call below to throw RuntimeException to wind up the stack and 1712 // executor thread to stop. The statement following the method call below seems to check if 1713 // store is not running, to prevent scheduling children procedures, re-execution or yield 1714 // of this procedure. This may need more scrutiny and subsequent cleanup in future 1715 // 1716 // Commit the transaction even if a suspend (state may have changed). Note this append 1717 // can take a bunch of time to complete. 1718 if (procedure.needPersistence()) { 1719 updateStoreOnExec(procStack, procedure, subprocs); 1720 } 1721 1722 // if the store is not running we are aborting 1723 if (!store.isRunning()) { 1724 return; 1725 } 1726 // if the procedure is kind enough to pass the slot to someone else, yield 1727 if ( 1728 procedure.isRunnable() && !suspended 1729 && procedure.isYieldAfterExecutionStep(getEnvironment()) 1730 ) { 1731 yieldProcedure(procedure); 1732 return; 1733 } 1734 1735 assert (reExecute && subprocs == null) || !reExecute; 1736 } while (reExecute); 1737 1738 // Allows to kill the executor after something is stored to the WAL but before the below 1739 // state settings are done -- in particular the one on the end where we make parent 1740 // RUNNABLE again when its children are done; see countDownChildren. 1741 if (testing != null && testing.shouldKillAfterStoreUpdate(suspended)) { 1742 kill("TESTING: Kill AFTER store update: " + procedure); 1743 } 1744 1745 // Submit the new subprocedures 1746 if (subprocs != null && !procedure.isFailed()) { 1747 submitChildrenProcedures(subprocs); 1748 } 1749 1750 // we need to log the release lock operation before waking up the parent procedure, as there 1751 // could be race that the parent procedure may call updateStoreOnExec ahead of us and remove all 1752 // the sub procedures from store and cause problems... 1753 releaseLock(procedure, false); 1754 1755 // if the procedure is complete and has a parent, count down the children latch. 1756 // If 'suspended', do nothing to change state -- let other threads handle unsuspend event. 1757 if (!suspended && procedure.isFinished() && procedure.hasParent()) { 1758 countDownChildren(procStack, procedure); 1759 } 1760 } 1761 1762 private void kill(String msg) { 1763 LOG.debug(msg); 1764 stop(); 1765 throw new RuntimeException(msg); 1766 } 1767 1768 private Procedure<TEnvironment>[] initializeChildren(RootProcedureState<TEnvironment> procStack, 1769 Procedure<TEnvironment> procedure, Procedure<TEnvironment>[] subprocs) { 1770 assert subprocs != null : "expected subprocedures"; 1771 final long rootProcId = getRootProcedureId(procedure); 1772 for (int i = 0; i < subprocs.length; ++i) { 1773 Procedure<TEnvironment> subproc = subprocs[i]; 1774 if (subproc == null) { 1775 String msg = "subproc[" + i + "] is null, aborting the procedure"; 1776 procedure 1777 .setFailure(new RemoteProcedureException(msg, new IllegalArgumentIOException(msg))); 1778 return null; 1779 } 1780 1781 assert subproc.getState() == ProcedureState.INITIALIZING : subproc; 1782 subproc.setParentProcId(procedure.getProcId()); 1783 subproc.setRootProcId(rootProcId); 1784 subproc.setProcId(nextProcId()); 1785 procStack.addSubProcedure(subproc); 1786 } 1787 1788 if (!procedure.isFailed()) { 1789 procedure.setChildrenLatch(subprocs.length); 1790 switch (procedure.getState()) { 1791 case RUNNABLE: 1792 procedure.setState(ProcedureState.WAITING); 1793 break; 1794 case WAITING_TIMEOUT: 1795 timeoutExecutor.add(procedure); 1796 break; 1797 default: 1798 break; 1799 } 1800 } 1801 return subprocs; 1802 } 1803 1804 private void submitChildrenProcedures(Procedure<TEnvironment>[] subprocs) { 1805 for (int i = 0; i < subprocs.length; ++i) { 1806 Procedure<TEnvironment> subproc = subprocs[i]; 1807 subproc.updateMetricsOnSubmit(getEnvironment()); 1808 assert !procedures.containsKey(subproc.getProcId()); 1809 procedures.put(subproc.getProcId(), subproc); 1810 scheduler.addFront(subproc); 1811 } 1812 } 1813 1814 private void countDownChildren(RootProcedureState<TEnvironment> procStack, 1815 Procedure<TEnvironment> procedure) { 1816 Procedure<TEnvironment> parent = procedures.get(procedure.getParentProcId()); 1817 if (parent == null) { 1818 assert procStack.isRollingback(); 1819 return; 1820 } 1821 1822 // If this procedure is the last child awake the parent procedure 1823 if (parent.tryRunnable()) { 1824 // If we succeeded in making the parent runnable -- i.e. all of its 1825 // children have completed, move parent to front of the queue. 1826 store.update(parent); 1827 scheduler.addFront(parent); 1828 LOG.info("Finished subprocedure pid={}, resume processing ppid={}", procedure.getProcId(), 1829 parent.getProcId()); 1830 return; 1831 } 1832 } 1833 1834 private void updateStoreOnExec(RootProcedureState<TEnvironment> procStack, 1835 Procedure<TEnvironment> procedure, Procedure<TEnvironment>[] subprocs) { 1836 if (subprocs != null && !procedure.isFailed()) { 1837 if (LOG.isTraceEnabled()) { 1838 LOG.trace("Stored " + procedure + ", children " + Arrays.toString(subprocs)); 1839 } 1840 store.insert(procedure, subprocs); 1841 } else { 1842 LOG.trace("Store update {}", procedure); 1843 if (procedure.isFinished() && !procedure.hasParent()) { 1844 // remove child procedures 1845 final long[] childProcIds = procStack.getSubprocedureIds(); 1846 if (childProcIds != null) { 1847 store.delete(procedure, childProcIds); 1848 for (int i = 0; i < childProcIds.length; ++i) { 1849 procedures.remove(childProcIds[i]); 1850 } 1851 } else { 1852 store.update(procedure); 1853 } 1854 } else { 1855 store.update(procedure); 1856 } 1857 } 1858 } 1859 1860 private void handleInterruptedException(Procedure<TEnvironment> proc, InterruptedException e) { 1861 LOG.trace("Interrupt during {}. suspend and retry it later.", proc, e); 1862 // NOTE: We don't call Thread.currentThread().interrupt() 1863 // because otherwise all the subsequent calls e.g. Thread.sleep() will throw 1864 // the InterruptedException. If the master is going down, we will be notified 1865 // and the executor/store will be stopped. 1866 // (The interrupted procedure will be retried on the next run) 1867 } 1868 1869 private void execCompletionCleanup(Procedure<TEnvironment> proc) { 1870 final TEnvironment env = getEnvironment(); 1871 if (proc.hasLock()) { 1872 LOG.warn("Usually this should not happen, we will release the lock before if the procedure" 1873 + " is finished, even if the holdLock is true, arrive here means we have some holes where" 1874 + " we do not release the lock. And the releaseLock below may fail since the procedure may" 1875 + " have already been deleted from the procedure store."); 1876 releaseLock(proc, true); 1877 } 1878 try { 1879 proc.completionCleanup(env); 1880 } catch (Throwable e) { 1881 // Catch NullPointerExceptions or similar errors... 1882 LOG.error("CODE-BUG: uncatched runtime exception for procedure: " + proc, e); 1883 } 1884 } 1885 1886 private void procedureFinished(Procedure<TEnvironment> proc) { 1887 // call the procedure completion cleanup handler 1888 execCompletionCleanup(proc); 1889 1890 CompletedProcedureRetainer<TEnvironment> retainer = new CompletedProcedureRetainer<>(proc); 1891 1892 // update the executor internal state maps 1893 if (!proc.shouldWaitClientAck(getEnvironment())) { 1894 retainer.setClientAckTime(0); 1895 } 1896 1897 completed.put(proc.getProcId(), retainer); 1898 rollbackStack.remove(proc.getProcId()); 1899 procedures.remove(proc.getProcId()); 1900 1901 // call the runnableSet completion cleanup handler 1902 try { 1903 scheduler.completionCleanup(proc); 1904 } catch (Throwable e) { 1905 // Catch NullPointerExceptions or similar errors... 1906 LOG.error("CODE-BUG: uncatched runtime exception for completion cleanup: {}", proc, e); 1907 } 1908 1909 // Notify the listeners 1910 sendProcedureFinishedNotification(proc.getProcId()); 1911 } 1912 1913 RootProcedureState<TEnvironment> getProcStack(long rootProcId) { 1914 return rollbackStack.get(rootProcId); 1915 } 1916 1917 ProcedureScheduler getProcedureScheduler() { 1918 return scheduler; 1919 } 1920 1921 int getCompletedSize() { 1922 return completed.size(); 1923 } 1924 1925 public IdLock getProcExecutionLock() { 1926 return procExecutionLock; 1927 } 1928 1929 // ========================================================================== 1930 // Worker Thread 1931 // ========================================================================== 1932 private class WorkerThread extends StoppableThread { 1933 private final AtomicLong executionStartTime = new AtomicLong(Long.MAX_VALUE); 1934 private volatile Procedure<TEnvironment> activeProcedure; 1935 1936 public WorkerThread(ThreadGroup group) { 1937 this(group, "PEWorker-"); 1938 } 1939 1940 protected WorkerThread(ThreadGroup group, String prefix) { 1941 super(group, prefix + workerId.incrementAndGet()); 1942 setDaemon(true); 1943 } 1944 1945 @Override 1946 public void sendStopSignal() { 1947 scheduler.signalAll(); 1948 } 1949 1950 /** 1951 * Encapsulates execution of the current {@link #activeProcedure} for easy tracing. 1952 */ 1953 private long runProcedure() throws IOException { 1954 final Procedure<TEnvironment> proc = this.activeProcedure; 1955 int activeCount = activeExecutorCount.incrementAndGet(); 1956 int runningCount = store.setRunningProcedureCount(activeCount); 1957 LOG.trace("Execute pid={} runningCount={}, activeCount={}", proc.getProcId(), runningCount, 1958 activeCount); 1959 executionStartTime.set(EnvironmentEdgeManager.currentTime()); 1960 IdLock.Entry lockEntry = procExecutionLock.getLockEntry(proc.getProcId()); 1961 try { 1962 executeProcedure(proc); 1963 } catch (AssertionError e) { 1964 LOG.info("ASSERT pid=" + proc.getProcId(), e); 1965 throw e; 1966 } finally { 1967 procExecutionLock.releaseLockEntry(lockEntry); 1968 activeCount = activeExecutorCount.decrementAndGet(); 1969 runningCount = store.setRunningProcedureCount(activeCount); 1970 LOG.trace("Halt pid={} runningCount={}, activeCount={}", proc.getProcId(), runningCount, 1971 activeCount); 1972 this.activeProcedure = null; 1973 executionStartTime.set(Long.MAX_VALUE); 1974 } 1975 return EnvironmentEdgeManager.currentTime(); 1976 } 1977 1978 @Override 1979 public void run() { 1980 long lastUpdate = EnvironmentEdgeManager.currentTime(); 1981 try { 1982 while (isRunning() && keepAlive(lastUpdate)) { 1983 @SuppressWarnings("unchecked") 1984 Procedure<TEnvironment> proc = scheduler.poll(keepAliveTime, TimeUnit.MILLISECONDS); 1985 if (proc == null) { 1986 continue; 1987 } 1988 this.activeProcedure = proc; 1989 lastUpdate = TraceUtil.trace(this::runProcedure, new ProcedureSpanBuilder(proc)); 1990 } 1991 } catch (Throwable t) { 1992 LOG.warn("Worker terminating UNNATURALLY {}", this.activeProcedure, t); 1993 } finally { 1994 LOG.trace("Worker terminated."); 1995 } 1996 workerThreads.remove(this); 1997 } 1998 1999 @Override 2000 public String toString() { 2001 Procedure<?> p = this.activeProcedure; 2002 return getName() + "(pid=" + (p == null ? Procedure.NO_PROC_ID : p.getProcId() + ")"); 2003 } 2004 2005 /** Returns the time since the current procedure is running */ 2006 public long getCurrentRunTime() { 2007 return EnvironmentEdgeManager.currentTime() - executionStartTime.get(); 2008 } 2009 2010 // core worker never timeout 2011 protected boolean keepAlive(long lastUpdate) { 2012 return true; 2013 } 2014 } 2015 2016 // A worker thread which can be added when core workers are stuck. Will timeout after 2017 // keepAliveTime if there is no procedure to run. 2018 private final class KeepAliveWorkerThread extends WorkerThread { 2019 public KeepAliveWorkerThread(ThreadGroup group) { 2020 super(group, "KeepAlivePEWorker-"); 2021 } 2022 2023 @Override 2024 protected boolean keepAlive(long lastUpdate) { 2025 return EnvironmentEdgeManager.currentTime() - lastUpdate < keepAliveTime; 2026 } 2027 } 2028 2029 // ---------------------------------------------------------------------------- 2030 // TODO-MAYBE: Should we provide a InlineChore to notify the store with the 2031 // full set of procedures pending and completed to write a compacted 2032 // version of the log (in case is a log)? 2033 // In theory no, procedures are have a short life, so at some point the store 2034 // will have the tracker saying everything is in the last log. 2035 // ---------------------------------------------------------------------------- 2036 2037 private final class WorkerMonitor extends InlineChore { 2038 public static final String WORKER_MONITOR_INTERVAL_CONF_KEY = 2039 "hbase.procedure.worker.monitor.interval.msec"; 2040 private static final int DEFAULT_WORKER_MONITOR_INTERVAL = 5000; // 5sec 2041 2042 public static final String WORKER_STUCK_THRESHOLD_CONF_KEY = 2043 "hbase.procedure.worker.stuck.threshold.msec"; 2044 private static final int DEFAULT_WORKER_STUCK_THRESHOLD = 10000; // 10sec 2045 2046 public static final String WORKER_ADD_STUCK_PERCENTAGE_CONF_KEY = 2047 "hbase.procedure.worker.add.stuck.percentage"; 2048 private static final float DEFAULT_WORKER_ADD_STUCK_PERCENTAGE = 0.5f; // 50% stuck 2049 2050 private float addWorkerStuckPercentage = DEFAULT_WORKER_ADD_STUCK_PERCENTAGE; 2051 private int timeoutInterval = DEFAULT_WORKER_MONITOR_INTERVAL; 2052 private int stuckThreshold = DEFAULT_WORKER_STUCK_THRESHOLD; 2053 2054 public WorkerMonitor() { 2055 refreshConfig(); 2056 } 2057 2058 @Override 2059 public void run() { 2060 final int stuckCount = checkForStuckWorkers(); 2061 checkThreadCount(stuckCount); 2062 2063 // refresh interval (poor man dynamic conf update) 2064 refreshConfig(); 2065 } 2066 2067 private int checkForStuckWorkers() { 2068 // check if any of the worker is stuck 2069 int stuckCount = 0; 2070 for (WorkerThread worker : workerThreads) { 2071 if (worker.getCurrentRunTime() < stuckThreshold) { 2072 continue; 2073 } 2074 2075 // WARN the worker is stuck 2076 stuckCount++; 2077 LOG.warn("Worker stuck {}, run time {}", worker, 2078 StringUtils.humanTimeDiff(worker.getCurrentRunTime())); 2079 } 2080 return stuckCount; 2081 } 2082 2083 private void checkThreadCount(final int stuckCount) { 2084 // nothing to do if there are no runnable tasks 2085 if (stuckCount < 1 || !scheduler.hasRunnables()) { 2086 return; 2087 } 2088 2089 // add a new thread if the worker stuck percentage exceed the threshold limit 2090 // and every handler is active. 2091 final float stuckPerc = ((float) stuckCount) / workerThreads.size(); 2092 // let's add new worker thread more aggressively, as they will timeout finally if there is no 2093 // work to do. 2094 if (stuckPerc >= addWorkerStuckPercentage && workerThreads.size() < maxPoolSize) { 2095 final KeepAliveWorkerThread worker = new KeepAliveWorkerThread(threadGroup); 2096 workerThreads.add(worker); 2097 worker.start(); 2098 LOG.debug("Added new worker thread {}", worker); 2099 } 2100 } 2101 2102 private void refreshConfig() { 2103 addWorkerStuckPercentage = 2104 conf.getFloat(WORKER_ADD_STUCK_PERCENTAGE_CONF_KEY, DEFAULT_WORKER_ADD_STUCK_PERCENTAGE); 2105 timeoutInterval = 2106 conf.getInt(WORKER_MONITOR_INTERVAL_CONF_KEY, DEFAULT_WORKER_MONITOR_INTERVAL); 2107 stuckThreshold = conf.getInt(WORKER_STUCK_THRESHOLD_CONF_KEY, DEFAULT_WORKER_STUCK_THRESHOLD); 2108 } 2109 2110 @Override 2111 public int getTimeoutInterval() { 2112 return timeoutInterval; 2113 } 2114 } 2115}