NIFI-5822: Ensure that we don't call FlowController.getControllerServiceProvider() before the ControllerServiceProvider has been initialized

This closes #3171.

Signed-off-by: Bryan Bende <bbende@apache.org>
This commit is contained in:
Mark Payne 2018-11-15 12:33:03 -05:00 committed by Bryan Bende
parent d319a3ef2f
commit 76b0065a67
No known key found for this signature in database
GPG Key ID: A0DDA9ED50711C39
1 changed files with 2 additions and 5 deletions

View File

@ -35,7 +35,6 @@ import org.apache.nifi.controller.SchedulingAgentCallback;
import org.apache.nifi.controller.StandardProcessorNode;
import org.apache.nifi.controller.exception.ProcessorInstantiationException;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.engine.FlowEngine;
import org.apache.nifi.logging.ComponentLog;
@ -72,7 +71,6 @@ public final class StandardProcessScheduler implements ProcessScheduler {
private static final Logger LOG = LoggerFactory.getLogger(StandardProcessScheduler.class);
private final ControllerServiceProvider controllerServiceProvider;
private final FlowController flowController;
private final long administrativeYieldMillis;
private final String administrativeYieldDuration;
@ -92,7 +90,6 @@ public final class StandardProcessScheduler implements ProcessScheduler {
public StandardProcessScheduler(final FlowEngine componentLifecycleThreadPool, final FlowController flowController, final StringEncryptor encryptor,
final StateManagerProvider stateManagerProvider, final NiFiProperties nifiProperties) {
this.componentLifeCycleThreadPool = componentLifecycleThreadPool;
this.controllerServiceProvider = flowController.getControllerServiceProvider();
this.flowController = flowController;
this.encryptor = encryptor;
this.stateManagerProvider = stateManagerProvider;
@ -296,7 +293,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
public synchronized CompletableFuture<Void> startProcessor(final ProcessorNode procNode, final boolean failIfStopping) {
final LifecycleState lifecycleState = getLifecycleState(requireNonNull(procNode), true);
final StandardProcessContext processContext = new StandardProcessContext(procNode, this.controllerServiceProvider,
final StandardProcessContext processContext = new StandardProcessContext(procNode, flowController.getControllerServiceProvider(),
this.encryptor, getStateManager(procNode.getIdentifier()), lifecycleState::isTerminated);
final CompletableFuture<Void> future = new CompletableFuture<>();
@ -336,7 +333,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
public synchronized CompletableFuture<Void> stopProcessor(final ProcessorNode procNode) {
final LifecycleState lifecycleState = getLifecycleState(procNode, false);
StandardProcessContext processContext = new StandardProcessContext(procNode, this.controllerServiceProvider,
StandardProcessContext processContext = new StandardProcessContext(procNode, flowController.getControllerServiceProvider(),
this.encryptor, getStateManager(procNode.getIdentifier()), lifecycleState::isTerminated);
LOG.info("Stopping {}", procNode);