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