Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -32,4 +32,14 @@ public interface Event<TYPE extends Enum<TYPE>> {
TYPE getType();
long getTimestamp();
String toString();

/**
* In case of parallel execution of events in the same dispatcher,
* the result of this method will be used as semaphore.
* If method returns null, then a default semaphore will be used.
* @return the semaphore
*/
default String getLockKey() {
return null;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is returning null expected here?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @slfan1989 !

Thanks for the review.
Yes, that is expected.
If we dont specify lockKey for an event we should return with null, so these events will be executed in sequential not parallel. The method is used in the MultiDispatcherLocks.

};
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,154 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.hadoop.yarn.event.multidispatcher;

import java.util.List;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.commons.lang3.concurrent.BasicThreadFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.metrics.DispatcherEventMetrics;
import org.apache.hadoop.yarn.metrics.DispatcherEventMetricsImpl;
import org.apache.hadoop.yarn.metrics.DispatcherEventMetricsNoOps;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.MonotonicClock;

/**
* Dispatches {@link Event}s in a parallel thread.
* The {@link Dispatcher#getEventHandler()} method can be used to post an event to the dispatcher.
* The posted event will be separated based on the hashcode of the {@link Event#getLockKey()}.
* If the getLockKey() method returns with null,
* then the first executor thread will be used as default worker
* The {@link MultiDispatcherConfig} contains the information,
* how many thread will be used, for parallel execution.
* The {@link MultiDispatcherExecutor} contains the worker threads, which handle the events.
* The {@link MultiDispatcherLibrary} contains the information,
* how to pair event types with {@link EventHandler}s
* The Dispatcher provides metric data using the {@link DispatcherEventMetricsImpl}.
*/
public class MultiDispatcher extends AbstractService implements Dispatcher {

private final Logger log;
private final String dispatcherName;
private final MultiDispatcherLibrary library;
private final Clock clock = new MonotonicClock();

private MultiDispatcherExecutor workerExecutor;
private ScheduledThreadPoolExecutor monitorExecutor;
private DispatcherEventMetrics metrics;

public MultiDispatcher(String dispatcherName) {
super("Dispatcher");
this.dispatcherName = dispatcherName.replaceAll(" ", "-").toLowerCase();
this.log = LoggerFactory.getLogger(MultiDispatcher.class.getCanonicalName() + "." + this.dispatcherName);
this.library = new MultiDispatcherLibrary();
}

@Override
protected void serviceInit(Configuration conf) throws Exception{
super.serviceInit(conf);
MultiDispatcherConfig config = new MultiDispatcherConfig(getConfig(), dispatcherName);
workerExecutor = new MultiDispatcherExecutor(log, config, dispatcherName);
workerExecutor.start();
createMonitorThread(config);
if (config.getMetricsEnabled()) {
metrics = new DispatcherEventMetricsImpl(dispatcherName);
DefaultMetricsSystem.instance().register(
"Event metrics for " + dispatcherName,
"Event metrics for " + dispatcherName,
metrics
);
} else {
metrics = new DispatcherEventMetricsNoOps(log);
}
}

@Override
protected void serviceStop() throws Exception {
if (monitorExecutor != null) {
monitorExecutor.shutdown();
}
workerExecutor.stop();
}

@Override
public EventHandler getEventHandler() {
return event -> {
if (isInState(STATE.STOPPED)) {
log.warn("Discard event {} because stopped state", event);
} else {
EventHandler handler = library.getEventHandler(event);
Runnable runnable = createRunnable(event, handler);
workerExecutor.execute(event, runnable);
metrics.addEvent(event.getType());
}
};
}

@Override
public void register(Class<? extends Enum> eventType, EventHandler handler) {
library.register(eventType, handler);
metrics.init(eventType);
}

private Runnable createRunnable(Event event, EventHandler handler) {
return () -> {
long start = clock.getTime();
try {
handler.handle(event);
} finally {
metrics.updateRate(event.getType(), clock.getTime() - start);
metrics.removeEvent(event.getType());
}
};
}

private void createMonitorThread(MultiDispatcherConfig config) {
int interval = config.getMonitorSeconds();
if (interval < 1) {
return;
}
this.monitorExecutor = new ScheduledThreadPoolExecutor(
1,
new BasicThreadFactory.Builder()
.namingPattern(this.dispatcherName + "-monitor-%d")
.build());
monitorExecutor.scheduleAtFixedRate(() -> {
List<String> notEmptyQueues = workerExecutor.getQueuesSize().entrySet().stream()
.filter(e -> 0 < e.getValue())
.map(e -> String.format("%s has queue size %d", e.getKey(), e.getValue()))
.sorted()
.collect(Collectors.toList());
if (!notEmptyQueues.isEmpty()) {
log.info("Event queue sizes: {}", notEmptyQueues);
}
log.debug("Metrics: {}", metrics);
},10, interval, TimeUnit.SECONDS);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.hadoop.yarn.event.multidispatcher;

import org.apache.hadoop.conf.Configuration;

/**
* All the config what can be use in the {@link MultiDispatcher}
*/
class MultiDispatcherConfig extends Configuration {

private final String prefix;

public MultiDispatcherConfig(Configuration configuration, String dispatcherName) {
super(configuration);
this.prefix = String.format("yarn.dispatcher.multi-thread.%s.", dispatcherName);
}

/**
* How many executor thread should be created to handle the incoming events
* @return configured value, or default 4
*/
public int getDefaultPoolSize() {
return super.getInt(prefix + "default-pool-size", 4);
}

/**
* Maximus size of the event queue of the executor threads.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just a typo, if you touch the code again anyways :)

* If limit is reached then the queue#add method will block.
* @return configured value, or default 1_000_000
*/
public int getQueueSize() {
return super.getInt(prefix + "queue-size", 1_000_000);
}

/**
* How frequently the monitor thread should write the state of the dispatcher to the LOG.
* If less than 1 this monitoring will be turned off.
* @return configured value, or default 0
*/
public int getMonitorSeconds() {
return super.getInt(prefix + "monitor-seconds", 0);
}

/**
* How long should the dispatcher wait to drain all event queue of workers,
* after stop signal is received.
* @return configured value, or default 60
*/
public int getGracefulStopSeconds() {
return super.getInt(prefix + "graceful-stop-seconds", 60);
}

/**
* Dispatcher metrics should be published to the metric system.
* @return configured value, or default false
*/
public boolean getMetricsEnabled() {
return super.getBoolean(prefix + "metrics-enabled", false);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,122 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.hadoop.yarn.event.multidispatcher;

import java.util.Arrays;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.stream.Collectors;

import org.slf4j.Logger;

import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.MonotonicClock;

/**
* This class contains the thread which process the {@link MultiDispatcher}'s events.
*/
public class MultiDispatcherExecutor {

private final Logger log;
private final MultiDispatcherConfig config;
private final MultiDispatcherExecutorThread[] threads;
private final Clock clock = new MonotonicClock();

public MultiDispatcherExecutor(
Logger log,
MultiDispatcherConfig config,
String dispatcherName
) {
this.log = log;
this.config = config;
this.threads = new MultiDispatcherExecutorThread[config.getDefaultPoolSize()];
ThreadGroup group = new ThreadGroup(dispatcherName);
for (int i = 0; i < threads.length; ++i) {
threads[i] = new MultiDispatcherExecutorThread(group, i, config.getQueueSize());
}
}

public void start() {
for(Thread t : threads) {
t.start();
}
}

public void execute(Event event, Runnable runnable) {
String lockKey = event.getLockKey();
// abs of Integer.MIN_VALUE is Integer.MIN_VALUE
int threadIndex = lockKey == null || lockKey.hashCode() == Integer.MIN_VALUE ?
0 : Math.abs(lockKey.hashCode() % threads.length);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Based on our discussion, I think a comment or description probably would be useful to make the goal of this computation more clear

MultiDispatcherExecutorThread thread = threads[threadIndex];
thread.add(runnable);
log.trace("The {} with lock key {} will be handled by {}",
event.getType(), lockKey, thread.getName());
}

public void stop() throws InterruptedException {
long timeOut = clock.getTime() + config.getGracefulStopSeconds() * 1_000L;
// if not all queue is empty
if (Arrays.stream(threads).anyMatch(t -> 0 < t.queueSize())
// and not timeout yet
&& clock.getTime() < timeOut) {
log.debug("Not all event queue is empty, waiting to drain ...");
Thread.sleep(1_000);
}
for (MultiDispatcherExecutorThread thread : threads) {
thread.interrupt();
}
}

public Map<String, Long> getQueuesSize() {
return Arrays.stream(threads).collect(Collectors.toMap(
MultiDispatcherExecutorThread::getName,
MultiDispatcherExecutorThread::queueSize
));
}

private final class MultiDispatcherExecutorThread extends Thread {
private final BlockingQueue<Runnable> queue;

MultiDispatcherExecutorThread(ThreadGroup group, int index, int queueSize) {
super(group, String.format("%s-worker-%d", group.getName(), index));
this.queue = new LinkedBlockingQueue<>(queueSize);
}

void add(Runnable runnable) {
queue.add(runnable);
}

long queueSize() {
return queue.size();
}

@Override
public void run() {
try {
while (true) {
queue.take().run();
}
} catch (InterruptedException e) {
log.warn("{} get interrupted", getName());
}
}
}
}
Loading