Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -345,6 +345,7 @@ public class FlowController implements ReportingTaskProvider, FlowAnalysisRulePr
private final Set<Connectable> startConnectablesAfterInitialization;
private final Set<ProcessGroup> startGroupsAfterInitialization;
private final Set<RemoteGroupPort> startRemoteGroupPortsAfterInitialization;
private final Set<ConnectorNode> startConnectorsAfterInitialization;
private final LeaderElectionManager leaderElectionManager;
private final ClusterCoordinator clusterCoordinator;
private final FlowEngine validationThreadPool;
Expand Down Expand Up @@ -679,6 +680,7 @@ private FlowController(
startConnectablesAfterInitialization = new HashSet<>();
startRemoteGroupPortsAfterInitialization = new HashSet<>();
startGroupsAfterInitialization = new HashSet<>();
startConnectorsAfterInitialization = new HashSet<>();

final String gracefulShutdownSecondsVal = nifiProperties.getProperty(GRACEFUL_SHUTDOWN_PERIOD);
long shutdownSecs;
Expand Down Expand Up @@ -1221,6 +1223,7 @@ public void purge() {
startConnectablesAfterInitialization.clear();
startRemoteGroupPortsAfterInitialization.clear();
startGroupsAfterInitialization.clear();
startConnectorsAfterInitialization.clear();
} finally {
writeLock.unlock("purge");
}
Expand Down Expand Up @@ -1490,6 +1493,23 @@ public void trigger(final ComponentNode component) {

LOG.info("Started {} Remote Group Ports transmitting", startedTransmitting);
startRemoteGroupPortsAfterInitialization.clear();

LOG.info("Starting {} Connectors", startConnectorsAfterInitialization.size());
for (final ConnectorNode connectorNode : startConnectorsAfterInitialization) {
try {
final ConnectorNode existingConnector = connectorRepository.getConnector(connectorNode.getIdentifier());
if (existingConnector == null) {
LOG.debug("Will not start {} because it no longer exists", connectorNode);
continue;
}

connectorRepository.startConnector(connectorNode);
} catch (final Throwable t) {
LOG.error("Unable to start {}", connectorNode, t);
}
}

startConnectorsAfterInitialization.clear();
} else {
// We don't want to start all of the delayed components. However, funnels need to be started anyway
// because we don't provide users the ability to start or stop them - they are just notional.
Expand All @@ -1505,6 +1525,19 @@ public void trigger(final ComponentNode component) {

startConnectablesAfterInitialization.clear();
startRemoteGroupPortsAfterInitialization.clear();

// Explicitly stop Connectors so that their state is properly transitioned from UPDATED to STOPPED.
for (final ConnectorNode connectorNode : startConnectorsAfterInitialization) {
try {
final ConnectorNode existingConnector = connectorRepository.getConnector(connectorNode.getIdentifier());
if (existingConnector != null) {
connectorRepository.stopConnector(connectorNode);
}
} catch (final Throwable t) {
LOG.error("Unable to stop {}", connectorNode, t);
}
}
startConnectorsAfterInitialization.clear();
}

flowManager.getRootGroup().findAllRemoteProcessGroups().forEach(RemoteProcessGroup::initialize);
Expand Down Expand Up @@ -1574,6 +1607,10 @@ public boolean isStartAfterInitialization(final ProcessGroup group) {
return startGroupsAfterInitialization.contains(group);
}

public boolean isStartAfterInitialization(final ConnectorNode connectorNode) {
return startConnectorsAfterInitialization.contains(connectorNode);
}

private ContentRepository createContentRepository(final NiFiProperties properties) {
final String implementationClassName = properties.getProperty(NiFiProperties.CONTENT_REPOSITORY_IMPLEMENTATION, DEFAULT_CONTENT_REPO_IMPLEMENTATION);
if (implementationClassName == null) {
Expand Down Expand Up @@ -2363,6 +2400,19 @@ public void startProcessGroup(final ProcessGroup processGroup) {
}
}

public void startConnector(final ConnectorNode connectorNode) {
writeLock.lock();
try {
if (initialized.get()) {
connectorRepository.startConnector(connectorNode);
} else {
startConnectorsAfterInitialization.add(connectorNode);
}
} finally {
writeLock.unlock("startConnector");
}
}

public boolean isInitialized() {
return initialized.get();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,10 @@ private List<VersionedConnector> mapConnectors() {

for (final ConnectorNode connectorNode : flowController.getConnectorRepository().getConnectors()) {
final VersionedConnector versionedConnector = flowMapper.mapConnector(connectorNode);
if (flowController.isStartAfterInitialization(connectorNode)) {
versionedConnector.setScheduledState(ScheduledState.RUNNING);
}

connectors.add(versionedConnector);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1057,10 +1057,10 @@ private void inheritConnectors(final FlowController flowController, final Versio
logger.info("Connector {} of type {} with name {} is not in the current flow. Will add Connector.",
versionedConnector.getInstanceIdentifier(), versionedConnector.getType(), versionedConnector.getName());

addConnector(versionedConnector, connectorRepository, flowController.getFlowManager());
addConnector(versionedConnector, flowController);
} else if (isConnectorConfigurationUpdated(existingConnector, versionedConnector)) {
logger.info("{} configuration has changed, updating configuration", existingConnector);
updateConnector(versionedConnector, connectorRepository);
updateConnector(versionedConnector, flowController);
} else {
logger.debug("{} configuration is up to date, no update necessary", existingConnector);
}
Expand Down Expand Up @@ -1162,15 +1162,18 @@ private boolean equals(final VersionedConnectorValueReference versionedReference
};
}

private void addConnector(final VersionedConnector versionedConnector, final ConnectorRepository connectorRepository, final FlowManager flowManager) {
private void addConnector(final VersionedConnector versionedConnector, final FlowController flowController) {
final ConnectorRepository connectorRepository = flowController.getConnectorRepository();
final FlowManager flowManager = flowController.getFlowManager();
final BundleCoordinate coordinate = createBundleCoordinate(extensionManager, versionedConnector.getBundle(), versionedConnector.getType());
final ConnectorNode connectorNode = flowManager.createConnector(versionedConnector.getType(), versionedConnector.getInstanceIdentifier(), coordinate, false, true);
connectorRepository.restoreConnector(connectorNode);
updateConnector(versionedConnector, connectorRepository);
updateConnector(versionedConnector, flowController);
}


private void updateConnector(final VersionedConnector versionedConnector, final ConnectorRepository connectorRepository) {
private void updateConnector(final VersionedConnector versionedConnector, final FlowController flowController) {
final ConnectorRepository connectorRepository = flowController.getConnectorRepository();
final ConnectorNode connectorNode = connectorRepository.getConnector(versionedConnector.getInstanceIdentifier());

connectorRepository.updateConnector(connectorNode, versionedConnector.getName());
Expand All @@ -1182,7 +1185,7 @@ private void updateConnector(final VersionedConnector versionedConnector, final

final ScheduledState desiredState = versionedConnector.getScheduledState();
if (desiredState == ScheduledState.RUNNING) {
connectorRepository.startConnector(connectorNode);
flowController.startConnector(connectorNode);
} else if (desiredState == ScheduledState.ENABLED) {
connectorRepository.stopConnector(connectorNode);
}
Expand Down Expand Up @@ -1514,6 +1517,9 @@ private static boolean isFlowEmpty(final VersionedDataflow dataflow) {
if (!CollectionUtils.isEmpty(dataflow.getParameterContexts())) {
return false;
}
if (!CollectionUtils.isEmpty(dataflow.getConnectors())) {
return false;
}

final VersionedProcessGroup rootGroup = dataflow.getRootGroup();
return isFlowEmpty(rootGroup);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

import org.apache.nifi.cluster.protocol.DataFlow;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.connector.ConnectorNode;
import org.apache.nifi.components.connector.ConnectorRepository;
import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.ReportingTaskNode;
Expand All @@ -30,6 +31,7 @@
import org.apache.nifi.encrypt.PropertyEncryptor;
import org.apache.nifi.flow.Bundle;
import org.apache.nifi.flow.ScheduledState;
import org.apache.nifi.flow.VersionedConnector;
import org.apache.nifi.flow.VersionedControllerService;
import org.apache.nifi.flow.VersionedProcessGroup;
import org.apache.nifi.flow.VersionedReportingTask;
Expand Down Expand Up @@ -63,6 +65,7 @@
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.inOrder;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;

Expand Down Expand Up @@ -271,6 +274,10 @@ private void setRootGroup() {
}

private void setFlowController() {
setFlowController(mock(ConnectorRepository.class));
}

private void setFlowController(final ConnectorRepository connectorRepository) {
when(rootGroup.isEmpty()).thenReturn(false);
when(flowController.getSnippetManager()).thenReturn(snippetManager);
when(snippetManager.export()).thenReturn(new byte[]{});
Expand All @@ -281,8 +288,62 @@ private void setFlowController() {
when(flowController.createVersionedComponentStateLookup(any())).thenReturn(stateLookup);
when(flowController.getControllerServiceProvider()).thenReturn(controllerServiceProvider);

final ConnectorRepository connectorRepository = mock(ConnectorRepository.class);
when(connectorRepository.getConnectors()).thenReturn(Collections.emptyList());
when(flowController.getConnectorRepository()).thenReturn(connectorRepository);
}

@Test
void testSyncInheritConnectorStartRoutedThroughFlowController() {
setRootGroup();

final String connectorId = UUID.randomUUID().toString();
final String connectorType = "org.apache.nifi.connectors.TestConnector";

final VersionedConnector versionedConnector = new VersionedConnector();
versionedConnector.setInstanceIdentifier(connectorId);
versionedConnector.setName("Test Connector");
versionedConnector.setType(connectorType);
versionedConnector.setBundle(CORE_BUNDLE);
versionedConnector.setScheduledState(ScheduledState.RUNNING);

final ConnectorRepository connectorRepository = mock(ConnectorRepository.class);
final ConnectorNode connectorNode = mock(ConnectorNode.class);
when(connectorRepository.getConnector(eq(connectorId))).thenReturn(null).thenReturn(connectorNode);
when(flowManager.createConnector(eq(connectorType), eq(connectorId), any(), eq(false), eq(true))).thenReturn(connectorNode);

setFlowController(connectorRepository);
when(versionedDataflow.getConnectors()).thenReturn(List.of(versionedConnector));

versionedFlowSynchronizer.sync(flowController, dataFlow, flowService, BundleUpdateStrategy.USE_SPECIFIED_OR_GHOST);

verify(flowController).startConnector(connectorNode);
}

@Test
void testSyncInheritConnectorNotStartedWhenEnabled() {
setRootGroup();

final String connectorId = UUID.randomUUID().toString();
final String connectorType = "org.apache.nifi.connectors.TestConnector";

final VersionedConnector versionedConnector = new VersionedConnector();
versionedConnector.setInstanceIdentifier(connectorId);
versionedConnector.setName("Test Connector");
versionedConnector.setType(connectorType);
versionedConnector.setBundle(CORE_BUNDLE);
versionedConnector.setScheduledState(ScheduledState.ENABLED);

final ConnectorRepository connectorRepository = mock(ConnectorRepository.class);
final ConnectorNode connectorNode = mock(ConnectorNode.class);
when(connectorRepository.getConnector(eq(connectorId))).thenReturn(null).thenReturn(connectorNode);
when(flowManager.createConnector(eq(connectorType), eq(connectorId), any(), eq(false), eq(true))).thenReturn(connectorNode);

setFlowController(connectorRepository);
when(versionedDataflow.getConnectors()).thenReturn(List.of(versionedConnector));

versionedFlowSynchronizer.sync(flowController, dataFlow, flowService, BundleUpdateStrategy.USE_SPECIFIED_OR_GHOST);

verify(flowController, never()).startConnector(any());
verify(connectorRepository).stopConnector(connectorNode);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.nifi.tests.system.connectors;

import org.apache.nifi.components.connector.ConnectorState;
import org.apache.nifi.tests.system.NiFiSystemIT;
import org.apache.nifi.toolkit.client.NiFiClientException;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.api.entity.ConnectorEntity;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;

import static org.junit.jupiter.api.Assertions.assertEquals;

/**
* System test verifying that a running Connector is not auto-resumed after a restart
* when the {@code nifi.flowcontroller.autoResumeState} property is set to {@code false}.
*/
public class ConnectorAutoResumeIT extends NiFiSystemIT {

private static final Logger logger = LoggerFactory.getLogger(ConnectorAutoResumeIT.class);

@Override
protected boolean isDestroyEnvironmentAfterEachTest() {
return true;
}

@Override
protected boolean isAllowFactoryReuse() {
return false;
}

@Test
public void testConnectorNotResumedWhenAutoResumeDisabled() throws NiFiClientException, IOException, InterruptedException {
final ConnectorEntity connector = getClientUtil().createConnector("NopConnector");
final String connectorId = connector.getId();

getClientUtil().applyConnectorUpdate(connector);
getClientUtil().waitForValidConnector(connectorId);

getClientUtil().startConnector(connectorId);
getClientUtil().waitForConnectorState(connectorId, ConnectorState.RUNNING);
logger.info("Connector {} is RUNNING", connectorId);

getNiFiInstance().stop();
logger.info("NiFi stopped");

getNiFiInstance().setProperty(NiFiProperties.AUTO_RESUME_STATE, "false");
getNiFiInstance().start();
setupClient();
logger.info("NiFi restarted with autoResumeState=false");

final ConnectorEntity connectorAfterRestart = getNifiClient().getConnectorClient().getConnector(connectorId);
final String stateAfterRestart = connectorAfterRestart.getComponent().getState();

assertEquals(ConnectorState.STOPPED.name(), stateAfterRestart);
}
}
Loading