mirror of https://github.com/apache/nifi.git
NIFI-4476 Improving logic for determining when to yield in PutTCP/UDP/Syslog/Splunk
Signed-off-by: Pierre Villard <pierre.villard.fr@gmail.com> This closes #2204.
This commit is contained in:
parent
883c223ced
commit
9324a2a742
|
@ -252,21 +252,28 @@ public abstract class AbstractPutEventProcessor extends AbstractSessionFactoryPr
|
||||||
* Close any senders that haven't been active with in the given threshold
|
* Close any senders that haven't been active with in the given threshold
|
||||||
*
|
*
|
||||||
* @param idleThreshold the threshold to consider a sender as idle
|
* @param idleThreshold the threshold to consider a sender as idle
|
||||||
|
* @return the number of connections that were closed as a result of being idle
|
||||||
*/
|
*/
|
||||||
protected void pruneIdleSenders(final long idleThreshold) {
|
protected PruneResult pruneIdleSenders(final long idleThreshold) {
|
||||||
|
int numClosed = 0;
|
||||||
|
int numConsidered = 0;
|
||||||
|
|
||||||
long currentTime = System.currentTimeMillis();
|
long currentTime = System.currentTimeMillis();
|
||||||
final List<ChannelSender> putBack = new ArrayList<>();
|
final List<ChannelSender> putBack = new ArrayList<>();
|
||||||
|
|
||||||
// if a connection hasn't been used with in the threshold then it gets closed
|
// if a connection hasn't been used with in the threshold then it gets closed
|
||||||
ChannelSender sender;
|
ChannelSender sender;
|
||||||
while ((sender = senderPool.poll()) != null) {
|
while ((sender = senderPool.poll()) != null) {
|
||||||
|
numConsidered++;
|
||||||
if (currentTime > (sender.getLastUsed() + idleThreshold)) {
|
if (currentTime > (sender.getLastUsed() + idleThreshold)) {
|
||||||
getLogger().debug("Closing idle connection...");
|
getLogger().debug("Closing idle connection...");
|
||||||
sender.close();
|
sender.close();
|
||||||
|
numClosed++;
|
||||||
} else {
|
} else {
|
||||||
putBack.add(sender);
|
putBack.add(sender);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// re-queue senders that weren't idle, but if the queue is full then close the sender
|
// re-queue senders that weren't idle, but if the queue is full then close the sender
|
||||||
for (ChannelSender putBackSender : putBack) {
|
for (ChannelSender putBackSender : putBack) {
|
||||||
boolean returned = senderPool.offer(putBackSender);
|
boolean returned = senderPool.offer(putBackSender);
|
||||||
|
@ -274,6 +281,8 @@ public abstract class AbstractPutEventProcessor extends AbstractSessionFactoryPr
|
||||||
putBackSender.close();
|
putBackSender.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
return new PruneResult(numClosed, numConsidered);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -371,6 +380,31 @@ public abstract class AbstractPutEventProcessor extends AbstractSessionFactoryPr
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The results from pruning connections.
|
||||||
|
*/
|
||||||
|
protected static class PruneResult {
|
||||||
|
|
||||||
|
private final int numClosed;
|
||||||
|
|
||||||
|
private final int numConsidered;
|
||||||
|
|
||||||
|
public PruneResult(final int numClosed, final int numConsidered) {
|
||||||
|
this.numClosed = numClosed;
|
||||||
|
this.numConsidered = numConsidered;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getNumClosed() {
|
||||||
|
return numClosed;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getNumConsidered() {
|
||||||
|
return numConsidered;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Represents a range of messages from a FlowFile.
|
* Represents a range of messages from a FlowFile.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -138,8 +138,11 @@ public class PutSplunk extends AbstractPutEventProcessor {
|
||||||
final ProcessSession session = sessionFactory.createSession();
|
final ProcessSession session = sessionFactory.createSession();
|
||||||
final FlowFile flowFile = session.get();
|
final FlowFile flowFile = session.get();
|
||||||
if (flowFile == null) {
|
if (flowFile == null) {
|
||||||
pruneIdleSenders(context.getProperty(IDLE_EXPIRATION).asTimePeriod(TimeUnit.MILLISECONDS).longValue());
|
final PruneResult result = pruneIdleSenders(context.getProperty(IDLE_EXPIRATION).asTimePeriod(TimeUnit.MILLISECONDS).longValue());
|
||||||
|
// yield if we closed an idle connection, or if there were no connections in the first place
|
||||||
|
if (result.getNumClosed() > 0 || (result.getNumClosed() == 0 && result.getNumConsidered() == 0)) {
|
||||||
context.yield();
|
context.yield();
|
||||||
|
}
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -265,20 +265,26 @@ public class PutSyslog extends AbstractSyslogProcessor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void pruneIdleSenders(final long idleThreshold){
|
private PruneResult pruneIdleSenders(final long idleThreshold){
|
||||||
|
int numClosed = 0;
|
||||||
|
int numConsidered = 0;
|
||||||
|
|
||||||
long currentTime = System.currentTimeMillis();
|
long currentTime = System.currentTimeMillis();
|
||||||
final List<ChannelSender> putBack = new ArrayList<>();
|
final List<ChannelSender> putBack = new ArrayList<>();
|
||||||
|
|
||||||
// if a connection hasn't been used with in the threshold then it gets closed
|
// if a connection hasn't been used with in the threshold then it gets closed
|
||||||
ChannelSender sender;
|
ChannelSender sender;
|
||||||
while ((sender = senderPool.poll()) != null) {
|
while ((sender = senderPool.poll()) != null) {
|
||||||
|
numConsidered++;
|
||||||
if (currentTime > (sender.getLastUsed() + idleThreshold)) {
|
if (currentTime > (sender.getLastUsed() + idleThreshold)) {
|
||||||
getLogger().debug("Closing idle connection...");
|
getLogger().debug("Closing idle connection...");
|
||||||
sender.close();
|
sender.close();
|
||||||
|
numClosed++;
|
||||||
} else {
|
} else {
|
||||||
putBack.add(sender);
|
putBack.add(sender);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// re-queue senders that weren't idle, but if the queue is full then close the sender
|
// re-queue senders that weren't idle, but if the queue is full then close the sender
|
||||||
for (ChannelSender putBackSender : putBack) {
|
for (ChannelSender putBackSender : putBack) {
|
||||||
boolean returned = senderPool.offer(putBackSender);
|
boolean returned = senderPool.offer(putBackSender);
|
||||||
|
@ -286,6 +292,8 @@ public class PutSyslog extends AbstractSyslogProcessor {
|
||||||
putBackSender.close();
|
putBackSender.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
return new PruneResult(numClosed, numConsidered);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -295,8 +303,11 @@ public class PutSyslog extends AbstractSyslogProcessor {
|
||||||
|
|
||||||
final List<FlowFile> flowFiles = session.get(batchSize);
|
final List<FlowFile> flowFiles = session.get(batchSize);
|
||||||
if (flowFiles == null || flowFiles.isEmpty()) {
|
if (flowFiles == null || flowFiles.isEmpty()) {
|
||||||
pruneIdleSenders(context.getProperty(IDLE_EXPIRATION).asTimePeriod(TimeUnit.MILLISECONDS).longValue());
|
final PruneResult result = pruneIdleSenders(context.getProperty(IDLE_EXPIRATION).asTimePeriod(TimeUnit.MILLISECONDS).longValue());
|
||||||
|
// yield if we closed an idle connection, or if there were no connections in the first place
|
||||||
|
if (result.getNumClosed() > 0 || (result.getNumClosed() == 0 && result.getNumConsidered() == 0)) {
|
||||||
context.yield();
|
context.yield();
|
||||||
|
}
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -394,4 +405,25 @@ public class PutSyslog extends AbstractSyslogProcessor {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static class PruneResult {
|
||||||
|
|
||||||
|
private final int numClosed;
|
||||||
|
|
||||||
|
private final int numConsidered;
|
||||||
|
|
||||||
|
public PruneResult(final int numClosed, final int numConsidered) {
|
||||||
|
this.numClosed = numClosed;
|
||||||
|
this.numConsidered = numConsidered;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getNumClosed() {
|
||||||
|
return numClosed;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int getNumConsidered() {
|
||||||
|
return numConsidered;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -168,8 +168,11 @@ public class PutTCP extends AbstractPutEventProcessor {
|
||||||
final ProcessSession session = sessionFactory.createSession();
|
final ProcessSession session = sessionFactory.createSession();
|
||||||
final FlowFile flowFile = session.get();
|
final FlowFile flowFile = session.get();
|
||||||
if (flowFile == null) {
|
if (flowFile == null) {
|
||||||
pruneIdleSenders(context.getProperty(IDLE_EXPIRATION).asTimePeriod(TimeUnit.MILLISECONDS).longValue());
|
final PruneResult result = pruneIdleSenders(context.getProperty(IDLE_EXPIRATION).asTimePeriod(TimeUnit.MILLISECONDS).longValue());
|
||||||
|
// yield if we closed an idle connection, or if there were no connections in the first place
|
||||||
|
if (result.getNumClosed() > 0 || (result.getNumClosed() == 0 && result.getNumConsidered() == 0)) {
|
||||||
context.yield();
|
context.yield();
|
||||||
|
}
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -131,8 +131,11 @@ public class PutUDP extends AbstractPutEventProcessor {
|
||||||
final ProcessSession session = sessionFactory.createSession();
|
final ProcessSession session = sessionFactory.createSession();
|
||||||
final FlowFile flowFile = session.get();
|
final FlowFile flowFile = session.get();
|
||||||
if (flowFile == null) {
|
if (flowFile == null) {
|
||||||
pruneIdleSenders(context.getProperty(IDLE_EXPIRATION).asTimePeriod(TimeUnit.MILLISECONDS).longValue());
|
final PruneResult result = pruneIdleSenders(context.getProperty(IDLE_EXPIRATION).asTimePeriod(TimeUnit.MILLISECONDS).longValue());
|
||||||
|
// yield if we closed an idle connection, or if there were no connections in the first place
|
||||||
|
if (result.getNumClosed() > 0 || (result.getNumClosed() == 0 && result.getNumConsidered() == 0)) {
|
||||||
context.yield();
|
context.yield();
|
||||||
|
}
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue