mirror of https://github.com/apache/nifi.git
NIFI-1627: Do not call context.yield for funnels and local ports; use batch size of 100 instead of 10 for local ports and funnels
Signed-off-by: joewitt <joewitt@apache.org>
This commit is contained in:
parent
54670486bc
commit
a4a7d53769
|
@ -349,22 +349,16 @@ public class StandardFunnel implements Funnel {
|
||||||
readLock.lock();
|
readLock.lock();
|
||||||
try {
|
try {
|
||||||
Set<Relationship> available = context.getAvailableRelationships();
|
Set<Relationship> available = context.getAvailableRelationships();
|
||||||
int transferred = 0;
|
|
||||||
while (!available.isEmpty()) {
|
while (!available.isEmpty()) {
|
||||||
final List<FlowFile> flowFiles = session.get(10);
|
final List<FlowFile> flowFiles = session.get(100);
|
||||||
if (flowFiles.isEmpty()) {
|
if (flowFiles.isEmpty()) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
transferred += flowFiles.size();
|
|
||||||
session.transfer(flowFiles, Relationship.ANONYMOUS);
|
session.transfer(flowFiles, Relationship.ANONYMOUS);
|
||||||
session.commit();
|
session.commit();
|
||||||
available = context.getAvailableRelationships();
|
available = context.getAvailableRelationships();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (transferred == 0) {
|
|
||||||
context.yield();
|
|
||||||
}
|
|
||||||
} finally {
|
} finally {
|
||||||
readLock.unlock();
|
readLock.unlock();
|
||||||
}
|
}
|
||||||
|
|
|
@ -73,10 +73,8 @@ public class LocalPort extends AbstractPort {
|
||||||
public void onTrigger(final ProcessContext context, final ProcessSession session) {
|
public void onTrigger(final ProcessContext context, final ProcessSession session) {
|
||||||
readLock.lock();
|
readLock.lock();
|
||||||
try {
|
try {
|
||||||
final List<FlowFile> flowFiles = session.get(10);
|
final List<FlowFile> flowFiles = session.get(100);
|
||||||
if (flowFiles.isEmpty()) {
|
if (!flowFiles.isEmpty()) {
|
||||||
context.yield();
|
|
||||||
} else {
|
|
||||||
session.transfer(flowFiles, Relationship.ANONYMOUS);
|
session.transfer(flowFiles, Relationship.ANONYMOUS);
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
|
|
Loading…
Reference in New Issue