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.executor; 019 020import java.io.IOException; 021import java.io.Writer; 022import java.lang.management.ThreadInfo; 023import java.util.List; 024import java.util.Map; 025import java.util.Map.Entry; 026import java.util.concurrent.BlockingQueue; 027import java.util.concurrent.ConcurrentHashMap; 028import java.util.concurrent.ConcurrentMap; 029import java.util.concurrent.Executors; 030import java.util.concurrent.LinkedBlockingQueue; 031import java.util.concurrent.ThreadPoolExecutor; 032import java.util.concurrent.TimeUnit; 033import java.util.concurrent.atomic.AtomicLong; 034import org.apache.hadoop.hbase.monitoring.ThreadMonitoring; 035import org.apache.yetus.audience.InterfaceAudience; 036import org.slf4j.Logger; 037import org.slf4j.LoggerFactory; 038 039import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 040import org.apache.hbase.thirdparty.com.google.common.collect.Lists; 041import org.apache.hbase.thirdparty.com.google.common.collect.Maps; 042import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ListenableFuture; 043import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ListeningScheduledExecutorService; 044import org.apache.hbase.thirdparty.com.google.common.util.concurrent.MoreExecutors; 045import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; 046 047/** 048 * This is a generic executor service. This component abstracts a threadpool, a queue to which 049 * {@link EventType}s can be submitted, and a <code>Runnable</code> that handles the object that is 050 * added to the queue. 051 * <p> 052 * In order to create a new service, create an instance of this class and then do: 053 * <code>instance.startExecutorService(executorConfig);</code>. {@link ExecutorConfig} wraps the 054 * configuration needed by this service. When done call {@link #shutdown()}. 055 * <p> 056 * In order to use the service created above, call {@link #submit(EventHandler)}. 057 */ 058@InterfaceAudience.Private 059public class ExecutorService { 060 private static final Logger LOG = LoggerFactory.getLogger(ExecutorService.class); 061 062 // hold the all the executors created in a map addressable by their names 063 private final ConcurrentHashMap<String, Executor> executorMap = new ConcurrentHashMap<>(); 064 065 // Name of the server hosting this executor service. 066 private final String servername; 067 068 private final ListeningScheduledExecutorService delayedSubmitTimer = 069 MoreExecutors.listeningDecorator(Executors.newScheduledThreadPool(1, new ThreadFactoryBuilder() 070 .setDaemon(true).setNameFormat("Event-Executor-Delay-Submit-Timer").build())); 071 072 /** 073 * Default constructor. 074 * @param servername Name of the hosting server. 075 */ 076 public ExecutorService(final String servername) { 077 this.servername = servername; 078 } 079 080 /** 081 * Start an executor service with a given name. If there was a service already started with the 082 * same name, this throws a RuntimeException. 083 * @param config Configuration to use for the executor. 084 */ 085 public void startExecutorService(final ExecutorConfig config) { 086 final String name = config.getName(); 087 if (this.executorMap.get(name) != null) { 088 throw new RuntimeException( 089 "An executor service with the name " + name + " is already running!"); 090 } 091 Executor hbes = new Executor(config); 092 if (this.executorMap.putIfAbsent(name, hbes) != null) { 093 throw new RuntimeException( 094 "An executor service with the name " + name + " is already running (2)!"); 095 } 096 LOG.debug("Starting executor service name=" + name + ", corePoolSize=" 097 + hbes.threadPoolExecutor.getCorePoolSize() + ", maxPoolSize=" 098 + hbes.threadPoolExecutor.getMaximumPoolSize()); 099 } 100 101 boolean isExecutorServiceRunning(String name) { 102 return this.executorMap.containsKey(name); 103 } 104 105 public void shutdown() { 106 this.delayedSubmitTimer.shutdownNow(); 107 for (Entry<String, Executor> entry : this.executorMap.entrySet()) { 108 List<Runnable> wasRunning = entry.getValue().threadPoolExecutor.shutdownNow(); 109 if (!wasRunning.isEmpty()) { 110 LOG.info(entry.getValue() + " had " + wasRunning + " on shutdown"); 111 } 112 } 113 this.executorMap.clear(); 114 } 115 116 Executor getExecutor(final ExecutorType type) { 117 return getExecutor(type.getExecutorName(this.servername)); 118 } 119 120 Executor getExecutor(String name) { 121 return this.executorMap.get(name); 122 } 123 124 public ThreadPoolExecutor getExecutorThreadPool(final ExecutorType type) { 125 return getExecutor(type).getThreadPoolExecutor(); 126 } 127 128 /** 129 * Initialize the executor lazily, Note if an executor need to be initialized lazily, then all 130 * paths should use this method to get the executor, should not start executor by using 131 * {@link ExecutorService#startExecutorService(ExecutorConfig)} 132 */ 133 public ThreadPoolExecutor getExecutorLazily(ExecutorConfig config) { 134 return executorMap.computeIfAbsent(config.getName(), (executorName) -> new Executor(config)) 135 .getThreadPoolExecutor(); 136 } 137 138 public void submit(final EventHandler eh) { 139 Executor executor = getExecutor(eh.getEventType().getExecutorServiceType()); 140 if (executor == null) { 141 // This happens only when events are submitted after shutdown() was 142 // called, so dropping them should be "ok" since it means we're 143 // shutting down. 144 LOG.error("Cannot submit [" + eh + "] because the executor is missing." 145 + " Is this process shutting down?"); 146 } else { 147 executor.submit(eh); 148 } 149 } 150 151 // Submit the handler after the given delay. Used for retrying. 152 public void delayedSubmit(EventHandler eh, long delay, TimeUnit unit) { 153 ListenableFuture<?> future = delayedSubmitTimer.schedule(() -> submit(eh), delay, unit); 154 future.addListener(() -> { 155 try { 156 future.get(); 157 } catch (Exception e) { 158 LOG.error("Failed to submit the event handler {} to executor", eh, e); 159 } 160 }, MoreExecutors.directExecutor()); 161 } 162 163 public Map<String, ExecutorStatus> getAllExecutorStatuses() { 164 Map<String, ExecutorStatus> ret = Maps.newHashMap(); 165 for (Map.Entry<String, Executor> e : executorMap.entrySet()) { 166 ret.put(e.getKey(), e.getValue().getStatus()); 167 } 168 return ret; 169 } 170 171 /** 172 * Configuration wrapper for {@link Executor}. 173 */ 174 public class ExecutorConfig { 175 // Refer to ThreadPoolExecutor javadoc for details of these configuration. 176 // Argument validation and bound checks delegated to the underlying ThreadPoolExecutor 177 // implementation. 178 public static final long KEEP_ALIVE_TIME_MILLIS_DEFAULT = 1000; 179 private int corePoolSize = -1; 180 private boolean allowCoreThreadTimeout = false; 181 private long keepAliveTimeMillis = KEEP_ALIVE_TIME_MILLIS_DEFAULT; 182 private ExecutorType executorType; 183 184 public ExecutorConfig setExecutorType(ExecutorType type) { 185 this.executorType = type; 186 return this; 187 } 188 189 private ExecutorType getExecutorType() { 190 return Preconditions.checkNotNull(executorType, "ExecutorType not set."); 191 } 192 193 public int getCorePoolSize() { 194 return corePoolSize; 195 } 196 197 public ExecutorConfig setCorePoolSize(int corePoolSize) { 198 this.corePoolSize = corePoolSize; 199 return this; 200 } 201 202 public boolean allowCoreThreadTimeout() { 203 return allowCoreThreadTimeout; 204 } 205 206 /** 207 * Allows timing out of core threads. Good to set this for non-critical thread pools for release 208 * of unused resources. Refer to {@link ThreadPoolExecutor#allowCoreThreadTimeOut} for 209 * additional details. 210 */ 211 public ExecutorConfig setAllowCoreThreadTimeout(boolean allowCoreThreadTimeout) { 212 this.allowCoreThreadTimeout = allowCoreThreadTimeout; 213 return this; 214 } 215 216 /** 217 * Returns the executor name inferred from the type and the servername on which this is running. 218 */ 219 public String getName() { 220 return getExecutorType().getExecutorName(servername); 221 } 222 223 public long getKeepAliveTimeMillis() { 224 return keepAliveTimeMillis; 225 } 226 227 public ExecutorConfig setKeepAliveTimeMillis(long keepAliveTimeMillis) { 228 this.keepAliveTimeMillis = keepAliveTimeMillis; 229 return this; 230 } 231 } 232 233 /** 234 * Executor instance. 235 */ 236 static class Executor { 237 // the thread pool executor that services the requests 238 final TrackingThreadPoolExecutor threadPoolExecutor; 239 // work queue to use - unbounded queue 240 final BlockingQueue<Runnable> q = new LinkedBlockingQueue<>(); 241 private final String name; 242 private static final AtomicLong seqids = new AtomicLong(0); 243 private final long id; 244 245 protected Executor(ExecutorConfig config) { 246 this.id = seqids.incrementAndGet(); 247 this.name = config.getName(); 248 // create the thread pool executor 249 this.threadPoolExecutor = new TrackingThreadPoolExecutor( 250 // setting maxPoolSize > corePoolSize has no effect since we use an unbounded task queue. 251 config.getCorePoolSize(), config.getCorePoolSize(), config.getKeepAliveTimeMillis(), 252 TimeUnit.MILLISECONDS, q); 253 this.threadPoolExecutor.allowCoreThreadTimeOut(config.allowCoreThreadTimeout()); 254 // name the threads for this threadpool 255 ThreadFactoryBuilder tfb = new ThreadFactoryBuilder(); 256 tfb.setNameFormat(this.name + "-%d"); 257 tfb.setDaemon(true); 258 this.threadPoolExecutor.setThreadFactory(tfb.build()); 259 } 260 261 /** 262 * Submit the event to the queue for handling. n 263 */ 264 void submit(final EventHandler event) { 265 // If there is a listener for this type, make sure we call the before 266 // and after process methods. 267 this.threadPoolExecutor.execute(event); 268 } 269 270 TrackingThreadPoolExecutor getThreadPoolExecutor() { 271 return threadPoolExecutor; 272 } 273 274 @Override 275 public String toString() { 276 return getClass().getSimpleName() + "-" + id + "-" + name; 277 } 278 279 public ExecutorStatus getStatus() { 280 List<EventHandler> queuedEvents = Lists.newArrayList(); 281 for (Runnable r : q) { 282 if (!(r instanceof EventHandler)) { 283 LOG.warn("Non-EventHandler " + r + " queued in " + name); 284 continue; 285 } 286 queuedEvents.add((EventHandler) r); 287 } 288 289 List<RunningEventStatus> running = Lists.newArrayList(); 290 for (Map.Entry<Thread, Runnable> e : threadPoolExecutor.getRunningTasks().entrySet()) { 291 Runnable r = e.getValue(); 292 if (!(r instanceof EventHandler)) { 293 LOG.warn("Non-EventHandler " + r + " running in " + name); 294 continue; 295 } 296 running.add(new RunningEventStatus(e.getKey(), (EventHandler) r)); 297 } 298 299 return new ExecutorStatus(this, queuedEvents, running); 300 } 301 } 302 303 /** 304 * A subclass of ThreadPoolExecutor that keeps track of the Runnables that are executing at any 305 * given point in time. 306 */ 307 static class TrackingThreadPoolExecutor extends ThreadPoolExecutor { 308 private ConcurrentMap<Thread, Runnable> running = Maps.newConcurrentMap(); 309 310 public TrackingThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, 311 TimeUnit unit, BlockingQueue<Runnable> workQueue) { 312 super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue); 313 } 314 315 @Override 316 protected void afterExecute(Runnable r, Throwable t) { 317 super.afterExecute(r, t); 318 running.remove(Thread.currentThread()); 319 } 320 321 @Override 322 protected void beforeExecute(Thread t, Runnable r) { 323 Runnable oldPut = running.put(t, r); 324 assert oldPut == null : "inconsistency for thread " + t; 325 super.beforeExecute(t, r); 326 } 327 328 /** 329 * @return a map of the threads currently running tasks inside this executor. Each key is an 330 * active thread, and the value is the task that is currently running. Note that this is 331 * not a stable snapshot of the map. 332 */ 333 public ConcurrentMap<Thread, Runnable> getRunningTasks() { 334 return running; 335 } 336 } 337 338 /** 339 * A snapshot of the status of a particular executor. This includes the contents of the executor's 340 * pending queue, as well as the threads and events currently being processed. This is a 341 * consistent snapshot that is immutable once constructed. 342 */ 343 public static class ExecutorStatus { 344 final Executor executor; 345 final List<EventHandler> queuedEvents; 346 final List<RunningEventStatus> running; 347 348 ExecutorStatus(Executor executor, List<EventHandler> queuedEvents, 349 List<RunningEventStatus> running) { 350 this.executor = executor; 351 this.queuedEvents = queuedEvents; 352 this.running = running; 353 } 354 355 /** 356 * Dump a textual representation of the executor's status to the given writer. 357 * @param out the stream to write to 358 * @param indent a string prefix for each line, used for indentation 359 */ 360 public void dumpTo(Writer out, String indent) throws IOException { 361 out.write(indent + "Status for executor: " + executor + "\n"); 362 out.write(indent + "=======================================\n"); 363 out.write(indent + queuedEvents.size() + " events queued, " + running.size() + " running\n"); 364 if (!queuedEvents.isEmpty()) { 365 out.write(indent + "Queued:\n"); 366 for (EventHandler e : queuedEvents) { 367 out.write(indent + " " + e + "\n"); 368 } 369 out.write("\n"); 370 } 371 if (!running.isEmpty()) { 372 out.write(indent + "Running:\n"); 373 for (RunningEventStatus stat : running) { 374 out.write(indent + " Running on thread '" + stat.threadInfo.getThreadName() + "': " 375 + stat.event + "\n"); 376 out.write(ThreadMonitoring.formatThreadInfo(stat.threadInfo, indent + " ")); 377 out.write("\n"); 378 } 379 } 380 out.flush(); 381 } 382 } 383 384 /** 385 * The status of a particular event that is in the middle of being handled by an executor. 386 */ 387 public static class RunningEventStatus { 388 final ThreadInfo threadInfo; 389 final EventHandler event; 390 391 public RunningEventStatus(Thread t, EventHandler event) { 392 this.threadInfo = ThreadMonitoring.getThreadInfo(t); 393 this.event = event; 394 } 395 } 396}