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}