mirror of https://github.com/apache/nifi.git
NIFI-10703 - Updated VersionedDataflow to support MaxEventDrivenThreadCount
NIFI-10703 - Also setting the maxEventDrivenThreadCount on the controller when using a versioned flow. NIFI-10703 - Corrected to use getMaxEventDrivenThreadCount(), set a default value for event driven thread count in VersionedDataflow NIFI-10703 - Updated log message for setMaxThreadCount in FlowController.java NIFI-10703 - Updated default value for maxEventDrivenThreadCount NIFI-10703 - Set private for DEFAULT_MAX_EVENT_DRIVEN_THREAD_COUNT Signed-off-by: Matthew Burgess <mattyb149@apache.org> This closes #6638
This commit is contained in:
parent
709110da35
commit
1b8cd8349b
|
@ -30,6 +30,7 @@ import java.util.Set;
|
|||
public class VersionedDataflow {
|
||||
private VersionedFlowEncodingVersion encodingVersion;
|
||||
private int maxTimerDrivenThreadCount;
|
||||
private int maxEventDrivenThreadCount;
|
||||
private List<VersionedFlowRegistryClient> registries;
|
||||
private List<VersionedParameterContext> parameterContexts;
|
||||
private List<VersionedParameterProvider> parameterProviders;
|
||||
|
@ -38,6 +39,8 @@ public class VersionedDataflow {
|
|||
private Set<VersionedTemplate> templates;
|
||||
private VersionedProcessGroup rootGroup;
|
||||
|
||||
private final static int DEFAULT_MAX_EVENT_DRIVEN_THREAD_COUNT = 1;
|
||||
|
||||
public VersionedFlowEncodingVersion getEncodingVersion() {
|
||||
return encodingVersion;
|
||||
}
|
||||
|
@ -54,6 +57,14 @@ public class VersionedDataflow {
|
|||
this.maxTimerDrivenThreadCount = maxTimerDrivenThreadCount;
|
||||
}
|
||||
|
||||
public int getMaxEventDrivenThreadCount() {
|
||||
return maxEventDrivenThreadCount < 1 ? DEFAULT_MAX_EVENT_DRIVEN_THREAD_COUNT : maxEventDrivenThreadCount;
|
||||
}
|
||||
|
||||
public void setMaxEventDrivenThreadCount(final int maxEventDrivenThreadCount) {
|
||||
this.maxEventDrivenThreadCount = maxEventDrivenThreadCount;
|
||||
}
|
||||
|
||||
public List<VersionedFlowRegistryClient> getRegistries() {
|
||||
return registries;
|
||||
}
|
||||
|
|
|
@ -1591,7 +1591,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node
|
|||
*/
|
||||
private void setMaxThreadCount(final int maxThreadCount, final FlowEngine engine, final AtomicInteger maxThreads) {
|
||||
if (maxThreadCount < 1) {
|
||||
throw new IllegalArgumentException("Cannot set max number of threads to less than 2");
|
||||
throw new IllegalArgumentException("Cannot set max number of threads to less than 1");
|
||||
}
|
||||
|
||||
maxThreads.getAndSet(maxThreadCount);
|
||||
|
|
|
@ -82,6 +82,7 @@ public class VersionedDataflowMapper {
|
|||
final VersionedDataflow dataflow = new VersionedDataflow();
|
||||
dataflow.setEncodingVersion(ENCODING_VERSION);
|
||||
dataflow.setMaxTimerDrivenThreadCount(flowController.getMaxTimerDrivenThreadCount());
|
||||
dataflow.setMaxEventDrivenThreadCount(flowController.getMaxEventDrivenThreadCount());
|
||||
dataflow.setControllerServices(mapControllerServices());
|
||||
dataflow.setParameterContexts(mapParameterContexts());
|
||||
dataflow.setRegistries(mapRegistries());
|
||||
|
|
|
@ -370,6 +370,7 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
|
|||
|
||||
if (versionedFlow != null) {
|
||||
controller.setMaxTimerDrivenThreadCount(versionedFlow.getMaxTimerDrivenThreadCount());
|
||||
controller.setMaxEventDrivenThreadCount(versionedFlow.getMaxEventDrivenThreadCount());
|
||||
ProcessGroup rootGroup = controller.getFlowManager().getRootGroup();
|
||||
|
||||
final Map<String, VersionedParameterContext> versionedParameterContextMap = new HashMap<>();
|
||||
|
|
Loading…
Reference in New Issue