|
| 1 | +/** |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.hadoop.yarn.event.multidispatcher; |
| 20 | + |
| 21 | +import java.util.concurrent.BlockingQueue; |
| 22 | +import java.util.concurrent.LinkedBlockingQueue; |
| 23 | +import java.util.concurrent.ScheduledThreadPoolExecutor; |
| 24 | +import java.util.concurrent.ThreadPoolExecutor; |
| 25 | +import java.util.concurrent.TimeUnit; |
| 26 | + |
| 27 | +import org.slf4j.Logger; |
| 28 | +import org.slf4j.LoggerFactory; |
| 29 | + |
| 30 | +import org.apache.commons.lang3.concurrent.BasicThreadFactory; |
| 31 | +import org.apache.hadoop.conf.Configuration; |
| 32 | +import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; |
| 33 | +import org.apache.hadoop.service.AbstractService; |
| 34 | +import org.apache.hadoop.yarn.event.Dispatcher; |
| 35 | +import org.apache.hadoop.yarn.event.Event; |
| 36 | +import org.apache.hadoop.yarn.event.EventHandler; |
| 37 | +import org.apache.hadoop.yarn.metrics.DispatcherEventMetrics; |
| 38 | +import org.apache.hadoop.yarn.util.Clock; |
| 39 | +import org.apache.hadoop.yarn.util.MonotonicClock; |
| 40 | + |
| 41 | +/** |
| 42 | + * Dispatches {@link Event}s in a parallel thread. |
| 43 | + * The {@link this#getEventHandler()} method can be used to post an event to the dispatcher. |
| 44 | + * The posted event will be added to the event queue what is polled by many thread, |
| 45 | + * based on the config values in the {@link MultiDispatcherConfig}. |
| 46 | + * The posted events can be parallel executed, |
| 47 | + * if the result of the {@link Event#getLockKey()} is different between 2 event. |
| 48 | + * If the method return with null, then a global semaphore will be used for these events. |
| 49 | + * The method usually returns with the applicationId based on the concept |
| 50 | + * parallel apps should not affect each others. |
| 51 | + * The locking logic is implemented in {@link MultiDispatcherLocks}. |
| 52 | + * The Dispatcher provides metric data using the {@link DispatcherEventMetrics} |
| 53 | + */ |
| 54 | +public class MultiDispatcher extends AbstractService implements Dispatcher { |
| 55 | + |
| 56 | + private final Logger LOG; |
| 57 | + private final String dispatcherName; |
| 58 | + private final DispatcherEventMetrics metrics; |
| 59 | + private final MultiDispatcherLocks locks; |
| 60 | + private final MultiDispatcherLibrary library; |
| 61 | + private final Clock clock = new MonotonicClock(); |
| 62 | + |
| 63 | + private MultiDispatcherConfig config; |
| 64 | + private BlockingQueue<Runnable> eventQueue; |
| 65 | + private ThreadPoolExecutor threadPoolExecutor; |
| 66 | + private ScheduledThreadPoolExecutor monitorExecutor; |
| 67 | + |
| 68 | + public MultiDispatcher(String dispatcherName) { |
| 69 | + super("Dispatcher"); |
| 70 | + this.dispatcherName = dispatcherName.replaceAll(" ", "-").toLowerCase(); |
| 71 | + this.LOG = LoggerFactory.getLogger(MultiDispatcher.class.getCanonicalName() + "." + this.dispatcherName); |
| 72 | + this.metrics = new DispatcherEventMetrics(this.dispatcherName); |
| 73 | + this.locks = new MultiDispatcherLocks(this.LOG); |
| 74 | + this.library = new MultiDispatcherLibrary(); |
| 75 | + } |
| 76 | + |
| 77 | + @Override |
| 78 | + protected void serviceInit(Configuration conf) throws Exception{ |
| 79 | + super.serviceInit(conf); |
| 80 | + this.config = new MultiDispatcherConfig(getConfig(), this.dispatcherName); |
| 81 | + this.eventQueue = new LinkedBlockingQueue<>(this.config.getQueueSize()); |
| 82 | + } |
| 83 | + |
| 84 | + @Override |
| 85 | + protected void serviceStart() throws Exception { |
| 86 | + super.serviceStart(); |
| 87 | + createWorkerPool(); |
| 88 | + createMonitorThread(); |
| 89 | + DefaultMetricsSystem.instance().register( |
| 90 | + "Event metrics for " + dispatcherName, |
| 91 | + "Event metrics for " + dispatcherName, |
| 92 | + metrics |
| 93 | + ); |
| 94 | + } |
| 95 | + |
| 96 | + @Override |
| 97 | + protected void serviceStop() throws Exception { |
| 98 | + if (monitorExecutor != null) { |
| 99 | + monitorExecutor.shutdownNow(); |
| 100 | + } |
| 101 | + threadPoolExecutor.shutdown(); |
| 102 | + threadPoolExecutor.awaitTermination(config.getGracefulStopSeconds(), TimeUnit.SECONDS); |
| 103 | + int terminatedSize = threadPoolExecutor.shutdownNow().size(); |
| 104 | + if (0 < terminatedSize) { |
| 105 | + LOG.error("{} tasks not finished in time, so they were terminated", terminatedSize); |
| 106 | + } |
| 107 | + } |
| 108 | + |
| 109 | + @Override |
| 110 | + public EventHandler getEventHandler() { |
| 111 | + return event -> { |
| 112 | + if (isInState(STATE.STOPPED)) { |
| 113 | + LOG.warn("Discard event {} because stopped state", event); |
| 114 | + } else { |
| 115 | + EventHandler handler = library.getEventHandler(event); |
| 116 | + threadPoolExecutor.execute(createRunnable(event, handler)); |
| 117 | + metrics.addEvent(event.getType()); |
| 118 | + } |
| 119 | + }; |
| 120 | + } |
| 121 | + |
| 122 | + @Override |
| 123 | + public void register(Class<? extends Enum> eventType, EventHandler handler) { |
| 124 | + library.register(eventType, handler); |
| 125 | + metrics.init(eventType); |
| 126 | + } |
| 127 | + |
| 128 | + private Runnable createRunnable(Event event, EventHandler handler) { |
| 129 | + return () -> { |
| 130 | + LOG.debug("{} handle {}, queue size: {}", |
| 131 | + Thread.currentThread().getName(), event.getClass().getSimpleName(), eventQueue.size()); |
| 132 | + locks.lock(event); |
| 133 | + long start = clock.getTime(); |
| 134 | + try { |
| 135 | + handler.handle(event); |
| 136 | + } finally { |
| 137 | + metrics.updateRate(event.getType(), clock.getTime() - start); |
| 138 | + locks.unLock(event); |
| 139 | + metrics.removeEvent(event.getType()); |
| 140 | + } |
| 141 | + }; |
| 142 | + } |
| 143 | + |
| 144 | + private void createWorkerPool() { |
| 145 | + this.threadPoolExecutor = new ThreadPoolExecutor( |
| 146 | + config.getDefaultPoolSize(), |
| 147 | + config.getMaxPoolSize(), |
| 148 | + config.getKeepAliveSeconds(), |
| 149 | + TimeUnit.SECONDS, |
| 150 | + eventQueue, |
| 151 | + new BasicThreadFactory.Builder() |
| 152 | + .namingPattern(this.dispatcherName + "-worker-%d") |
| 153 | + .build() |
| 154 | + ); |
| 155 | + } |
| 156 | + |
| 157 | + private void createMonitorThread() { |
| 158 | + int interval = config.getMonitorSeconds(); |
| 159 | + if (interval < 1) { |
| 160 | + return; |
| 161 | + } |
| 162 | + this.monitorExecutor = new ScheduledThreadPoolExecutor( |
| 163 | + 1, |
| 164 | + new BasicThreadFactory.Builder() |
| 165 | + .namingPattern(this.dispatcherName + "-monitor-%d") |
| 166 | + .build()); |
| 167 | + monitorExecutor.scheduleAtFixedRate(() -> { |
| 168 | + int size = eventQueue.size(); |
| 169 | + if (0 < size) { |
| 170 | + LOG.info("Event queue size is {}", size); |
| 171 | + LOG.debug("Metrics: {}", metrics); |
| 172 | + } |
| 173 | + },10, interval, TimeUnit.SECONDS); |
| 174 | + } |
| 175 | +} |
0 commit comments