NIFI-4379: Event-Driven threads are constantly active, polling a queue to see if there is any work to do. Instead of getting number of active threads from the ScheduledExecutor for these threads, updated code to use an AtomicInteger that is incremented when there's work to do and decremented once complete

Signed-off-by: Pierre Villard <pierre.villard.fr@gmail.com>

This closes #2153.
This commit is contained in:
Mark Payne 2017-09-13 11:01:07 -04:00 committed by Pierre Villard
parent 1e70e24267
commit e01d59a462
2 changed files with 108 additions and 93 deletions

View File

@ -280,6 +280,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
private final AtomicInteger maxEventDrivenThreads; private final AtomicInteger maxEventDrivenThreads;
private final AtomicReference<FlowEngine> timerDrivenEngineRef; private final AtomicReference<FlowEngine> timerDrivenEngineRef;
private final AtomicReference<FlowEngine> eventDrivenEngineRef; private final AtomicReference<FlowEngine> eventDrivenEngineRef;
private final EventDrivenSchedulingAgent eventDrivenSchedulingAgent;
private final ContentRepository contentRepository; private final ContentRepository contentRepository;
private final FlowFileRepository flowFileRepository; private final FlowFileRepository flowFileRepository;
@ -502,8 +503,10 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
eventDrivenWorkerQueue = new EventDrivenWorkerQueue(false, false, processScheduler); eventDrivenWorkerQueue = new EventDrivenWorkerQueue(false, false, processScheduler);
final ProcessContextFactory contextFactory = new ProcessContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository); final ProcessContextFactory contextFactory = new ProcessContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository);
processScheduler.setSchedulingAgent(SchedulingStrategy.EVENT_DRIVEN, new EventDrivenSchedulingAgent(
eventDrivenEngineRef.get(), this, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor)); eventDrivenSchedulingAgent = new EventDrivenSchedulingAgent(
eventDrivenEngineRef.get(), this, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor);
processScheduler.setSchedulingAgent(SchedulingStrategy.EVENT_DRIVEN, eventDrivenSchedulingAgent);
final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor); final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor);
final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.nifiProperties); final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.nifiProperties);
@ -3582,7 +3585,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
public int getActiveThreadCount() { public int getActiveThreadCount() {
final int timerDrivenCount = timerDrivenEngineRef.get().getActiveCount(); final int timerDrivenCount = timerDrivenEngineRef.get().getActiveCount();
final int eventDrivenCount = eventDrivenEngineRef.get().getActiveCount(); final int eventDrivenCount = eventDrivenSchedulingAgent.getActiveThreadCount();
return timerDrivenCount + eventDrivenCount; return timerDrivenCount + eventDrivenCount;
} }

View File

@ -60,6 +60,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
private final EventDrivenWorkerQueue workerQueue; private final EventDrivenWorkerQueue workerQueue;
private final ProcessContextFactory contextFactory; private final ProcessContextFactory contextFactory;
private final AtomicInteger maxThreadCount; private final AtomicInteger maxThreadCount;
private final AtomicInteger activeThreadCount = new AtomicInteger(0);
private final StringEncryptor encryptor; private final StringEncryptor encryptor;
private volatile String adminYieldDuration = "1 sec"; private volatile String adminYieldDuration = "1 sec";
@ -78,11 +79,15 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
this.encryptor = encryptor; this.encryptor = encryptor;
for (int i = 0; i < maxThreadCount; i++) { for (int i = 0; i < maxThreadCount; i++) {
final Runnable eventDrivenTask = new EventDrivenTask(workerQueue); final Runnable eventDrivenTask = new EventDrivenTask(workerQueue, activeThreadCount);
flowEngine.scheduleWithFixedDelay(eventDrivenTask, 0L, 1L, TimeUnit.NANOSECONDS); flowEngine.scheduleWithFixedDelay(eventDrivenTask, 0L, 1L, TimeUnit.NANOSECONDS);
} }
} }
public int getActiveThreadCount() {
return activeThreadCount.get();
}
private StateManager getStateManager(final String componentId) { private StateManager getStateManager(final String componentId) {
return stateManagerProvider.getStateManager(componentId); return stateManagerProvider.getStateManager(componentId);
} }
@ -127,7 +132,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
// if more threads have been allocated, add more tasks to the work queue // if more threads have been allocated, add more tasks to the work queue
final int tasksToAdd = maxThreadCount - oldMax; final int tasksToAdd = maxThreadCount - oldMax;
for (int i = 0; i < tasksToAdd; i++) { for (int i = 0; i < tasksToAdd; i++) {
final Runnable eventDrivenTask = new EventDrivenTask(workerQueue); final Runnable eventDrivenTask = new EventDrivenTask(workerQueue, activeThreadCount);
flowEngine.scheduleWithFixedDelay(eventDrivenTask, 0L, 1L, TimeUnit.NANOSECONDS); flowEngine.scheduleWithFixedDelay(eventDrivenTask, 0L, 1L, TimeUnit.NANOSECONDS);
} }
} }
@ -151,9 +156,11 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
private class EventDrivenTask implements Runnable { private class EventDrivenTask implements Runnable {
private final EventDrivenWorkerQueue workerQueue; private final EventDrivenWorkerQueue workerQueue;
private final AtomicInteger activeThreadCount;
public EventDrivenTask(final EventDrivenWorkerQueue workerQueue) { public EventDrivenTask(final EventDrivenWorkerQueue workerQueue, final AtomicInteger activeThreadCount) {
this.workerQueue = workerQueue; this.workerQueue = workerQueue;
this.activeThreadCount = activeThreadCount;
} }
@Override @Override
@ -170,6 +177,8 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
continue; continue;
} }
activeThreadCount.incrementAndGet();
try {
// get the connection index for this worker // get the connection index for this worker
AtomicLong connectionIndex = connectionIndexMap.get(connectable); AtomicLong connectionIndex = connectionIndexMap.get(connectable);
if (connectionIndex == null) { if (connectionIndex == null) {
@ -266,6 +275,9 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
onEvent(connectable); onEvent(connectable);
} }
} }
} finally {
activeThreadCount.decrementAndGet();
}
} }
} }