diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java index c19fa94cb9..38b7845ef4 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java @@ -14,82 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.nifi.controller.repository; +import org.apache.nifi.controller.queue.FlowFileQueue; +import org.wali.SerDeFactory; + import java.util.Map; -import org.apache.nifi.controller.queue.FlowFileQueue; -import org.apache.nifi.controller.repository.claim.ResourceClaimManager; -import org.wali.SerDe; -import org.wali.SerDeFactory; -import org.wali.UpdateType; - -public class RepositoryRecordSerdeFactory implements SerDeFactory { - private final String LEGACY_SERDE_ENCODING_NAME = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository$WriteAheadRecordSerde"; - private final ResourceClaimManager resourceClaimManager; - private Map flowFileQueueMap = null; - - public RepositoryRecordSerdeFactory(final ResourceClaimManager claimManager) { - this.resourceClaimManager = claimManager; - } - - protected void setQueueMap(final Map queueMap) { - this.flowFileQueueMap = queueMap; - } - - protected Map getQueueMap() { - return flowFileQueueMap; - } - - @Override - public SerDe createSerDe(final String encodingName) { - if (encodingName == null || SchemaRepositoryRecordSerde.class.getName().equals(encodingName)) { - final SchemaRepositoryRecordSerde serde = new SchemaRepositoryRecordSerde(resourceClaimManager); - serde.setQueueMap(flowFileQueueMap); - return serde; - } - - if (WriteAheadRepositoryRecordSerde.class.getName().equals(encodingName) - || LEGACY_SERDE_ENCODING_NAME.equals(encodingName)) { - final WriteAheadRepositoryRecordSerde serde = new WriteAheadRepositoryRecordSerde(resourceClaimManager); - serde.setQueueMap(flowFileQueueMap); - return serde; - } - - throw new IllegalArgumentException("Cannot create Deserializer for Repository Records because the encoding '" + encodingName + "' is not known"); - } - - protected FlowFileQueue getFlowFileQueue(final String queueId) { - return flowFileQueueMap.get(queueId); - } - - @Override - public Long getRecordIdentifier(final RepositoryRecord record) { - return record.getCurrent().getId(); - } - - @Override - public UpdateType getUpdateType(final RepositoryRecord record) { - switch (record.getType()) { - case CONTENTMISSING: - case DELETE: - return UpdateType.DELETE; - case CREATE: - return UpdateType.CREATE; - case UPDATE: - return UpdateType.UPDATE; - case SWAP_OUT: - return UpdateType.SWAP_OUT; - case SWAP_IN: - return UpdateType.SWAP_IN; - } - return null; - } - - @Override - public String getLocation(final RepositoryRecord record) { - return record.getSwapLocation(); - } +public interface RepositoryRecordSerdeFactory extends SerDeFactory { + void setQueueMap(Map queueMap); + Long getRecordIdentifier(RepositoryRecord record); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecordSerdeFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecordSerdeFactory.java new file mode 100644 index 0000000000..f24ac08cd5 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecordSerdeFactory.java @@ -0,0 +1,95 @@ +/* + * 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.controller.repository; + +import org.apache.nifi.controller.queue.FlowFileQueue; +import org.apache.nifi.controller.repository.claim.ResourceClaimManager; +import org.wali.SerDe; +import org.wali.UpdateType; + +import java.util.Map; + +public class StandardRepositoryRecordSerdeFactory implements RepositoryRecordSerdeFactory { + private final String LEGACY_SERDE_ENCODING_NAME = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository$WriteAheadRecordSerde"; + private final ResourceClaimManager resourceClaimManager; + private Map flowFileQueueMap = null; + + public StandardRepositoryRecordSerdeFactory(final ResourceClaimManager claimManager) { + this.resourceClaimManager = claimManager; + } + + @Override + public void setQueueMap(final Map queueMap) { + this.flowFileQueueMap = queueMap; + } + + protected Map getQueueMap() { + return flowFileQueueMap; + } + + @Override + public SerDe createSerDe(final String encodingName) { + if (encodingName == null || SchemaRepositoryRecordSerde.class.getName().equals(encodingName)) { + final SchemaRepositoryRecordSerde serde = new SchemaRepositoryRecordSerde(resourceClaimManager); + serde.setQueueMap(flowFileQueueMap); + return serde; + } + + if (WriteAheadRepositoryRecordSerde.class.getName().equals(encodingName) + || LEGACY_SERDE_ENCODING_NAME.equals(encodingName)) { + final WriteAheadRepositoryRecordSerde serde = new WriteAheadRepositoryRecordSerde(resourceClaimManager); + serde.setQueueMap(flowFileQueueMap); + return serde; + } + + throw new IllegalArgumentException("Cannot create Deserializer for Repository Records because the encoding '" + encodingName + "' is not known"); + } + + protected FlowFileQueue getFlowFileQueue(final String queueId) { + return flowFileQueueMap.get(queueId); + } + + @Override + public Long getRecordIdentifier(final RepositoryRecord record) { + return record.getCurrent().getId(); + } + + @Override + public UpdateType getUpdateType(final RepositoryRecord record) { + switch (record.getType()) { + case CONTENTMISSING: + case DELETE: + return UpdateType.DELETE; + case CREATE: + return UpdateType.CREATE; + case UPDATE: + return UpdateType.UPDATE; + case SWAP_OUT: + return UpdateType.SWAP_OUT; + case SWAP_IN: + return UpdateType.SWAP_IN; + } + return null; + } + + @Override + public String getLocation(final RepositoryRecord record) { + return record.getSwapLocation(); + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java index f78c9d20d9..d2032a2d82 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java @@ -67,7 +67,6 @@ public abstract class AbstractComponentNode implements ComponentNode { private final ControllerServiceProvider serviceProvider; private final AtomicReference name; private final AtomicReference annotationData = new AtomicReference<>(); - private final AtomicReference validationContext = new AtomicReference<>(); private final String componentType; private final String componentCanonicalClass; private final ComponentVariableRegistry variableRegistry; @@ -79,10 +78,13 @@ public abstract class AbstractComponentNode implements ComponentNode { private final Lock lock = new ReentrantLock(); private final ConcurrentMap properties = new ConcurrentHashMap<>(); private volatile String additionalResourcesFingerprint; - private AtomicReference validationState = new AtomicReference<>(new ValidationState(ValidationStatus.VALIDATING, Collections.emptyList())); + private final AtomicReference validationState = new AtomicReference<>(new ValidationState(ValidationStatus.VALIDATING, Collections.emptyList())); private final ValidationTrigger validationTrigger; private volatile boolean triggerValidation = true; + // guaraded by lock + private ValidationContext validationContext = null; + public AbstractComponentNode(final String id, final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider, final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry, @@ -575,7 +577,7 @@ public abstract class AbstractComponentNode implements ComponentNode { } - private final void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { + private void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(extensionManager, getComponent().getClass(), getComponent().getIdentifier())) { getComponent().onPropertyModified(descriptor, oldValue, newValue); } @@ -627,13 +629,18 @@ public abstract class AbstractComponentNode implements ComponentNode { } protected void resetValidationState() { - validationContext.set(null); - validationState.set(new ValidationState(ValidationStatus.VALIDATING, Collections.emptyList())); + lock.lock(); + try { + validationContext = null; + validationState.set(new ValidationState(ValidationStatus.VALIDATING, Collections.emptyList())); - if (isTriggerValidation()) { - validationTrigger.triggerAsync(this); - } else { - logger.debug("Reset validation state of {} but will not trigger async validation because trigger has been paused", this); + if (isTriggerValidation()) { + validationTrigger.triggerAsync(this); + } else { + logger.debug("Reset validation state of {} but will not trigger async validation because trigger has been paused", this); + } + } finally { + lock.unlock(); } } @@ -714,27 +721,21 @@ public abstract class AbstractComponentNode implements ComponentNode { } protected ValidationContext getValidationContext() { - while (true) { - ValidationContext context = this.validationContext.get(); + lock.lock(); + try { + ValidationContext context = this.validationContext; if (context != null) { return context; } - // Use a lock here because we want to prevent calls to getProperties() from happening while setProperties() is also happening. - final Map properties; - lock.lock(); - try { - properties = getProperties(); - } finally { - lock.unlock(); - } + final Map properties = getProperties(); context = getValidationContextFactory().newValidationContext(properties, getAnnotationData(), getProcessGroupIdentifier(), getIdentifier()); - final boolean updated = validationContext.compareAndSet(null, context); - if (updated) { - logger.debug("Updating validation context to {}", context); - return context; - } + this.validationContext = context; + logger.debug("Updating validation context to {}", context); + return context; + } finally { + lock.unlock(); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java index eb00402377..76728bab36 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java @@ -16,13 +16,13 @@ */ package org.apache.nifi.events; -import java.util.concurrent.atomic.AtomicLong; - import org.apache.nifi.connectable.Connectable; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.reporting.Bulletin; import org.apache.nifi.reporting.ComponentType; +import java.util.concurrent.atomic.AtomicLong; + public final class BulletinFactory { private static final AtomicLong currentId = new AtomicLong(0); @@ -47,7 +47,7 @@ public final class BulletinFactory { } final ProcessGroup group = connectable.getProcessGroup(); - final String groupId = group == null ? null : group.getIdentifier(); + final String groupId = connectable.getProcessGroupIdentifier(); final String groupName = group == null ? null : group.getName(); return BulletinFactory.createBulletin(groupId, groupName, connectable.getIdentifier(), type, connectable.getName(), category, severity, message); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml index 85c5281ef4..ca5ab87a3c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml @@ -225,6 +225,12 @@ 1.10.0-SNAPSHOT test + + org.apache.nifi + nifi-persistent-provenance-repository + 1.10.0-SNAPSHOT + test + diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index b731610561..4141832b57 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -86,6 +86,7 @@ import org.apache.nifi.controller.repository.FlowFileEventRepository; import org.apache.nifi.controller.repository.FlowFileRecord; import org.apache.nifi.controller.repository.FlowFileRepository; import org.apache.nifi.controller.repository.FlowFileSwapManager; +import org.apache.nifi.controller.repository.QueueProvider; import org.apache.nifi.controller.repository.StandardCounterRepository; import org.apache.nifi.controller.repository.StandardFlowFileRecord; import org.apache.nifi.controller.repository.StandardQueueProvider; @@ -294,6 +295,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node private final ProvenanceAuthorizableFactory provenanceAuthorizableFactory; private final UserAwareEventAccess eventAccess; private final StandardFlowManager flowManager; + private final RepositoryContextFactory repositoryContextFactory; /** * true if controller is configured to operate in a clustered environment @@ -433,7 +435,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node final ExtensionManager extensionManager) { maxTimerDrivenThreads = new AtomicInteger(10); - maxEventDrivenThreads = new AtomicInteger(5); + maxEventDrivenThreads = new AtomicInteger(1); this.encryptor = encryptor; this.nifiProperties = nifiProperties; @@ -484,18 +486,17 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node processScheduler = new StandardProcessScheduler(timerDrivenEngineRef.get(), this, encryptor, stateManagerProvider, this.nifiProperties); eventDrivenWorkerQueue = new EventDrivenWorkerQueue(false, false, processScheduler); - final RepositoryContextFactory contextFactory = new RepositoryContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository); - + repositoryContextFactory = new RepositoryContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository); this.flowManager = new StandardFlowManager(nifiProperties, sslContext, this, flowFileEventRepository); controllerServiceProvider = new StandardControllerServiceProvider(this, processScheduler, bulletinRepository); eventDrivenSchedulingAgent = new EventDrivenSchedulingAgent( - eventDrivenEngineRef.get(), controllerServiceProvider, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor, extensionManager); + eventDrivenEngineRef.get(), controllerServiceProvider, stateManagerProvider, eventDrivenWorkerQueue, repositoryContextFactory, maxEventDrivenThreads.get(), encryptor, extensionManager); processScheduler.setSchedulingAgent(SchedulingStrategy.EVENT_DRIVEN, eventDrivenSchedulingAgent); - final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor); - final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.nifiProperties); + final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), repositoryContextFactory, encryptor); + final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), repositoryContextFactory, encryptor, this.nifiProperties); processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, timerDrivenAgent); // PRIMARY_NODE_ONLY is deprecated, but still exists to handle processors that are still defined with it (they haven't been re-configured with executeNode = PRIMARY). processScheduler.setSchedulingAgent(SchedulingStrategy.PRIMARY_NODE_ONLY, timerDrivenAgent); @@ -743,12 +744,16 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node } public void initializeFlow() throws IOException { + initializeFlow(new StandardQueueProvider(getFlowManager())); + } + + public void initializeFlow(final QueueProvider queueProvider) throws IOException { writeLock.lock(); try { // get all connections/queues and recover from swap files. final List connections = flowManager.getRootGroup().findAllConnections(); - flowFileRepository.loadFlowFiles(new StandardQueueProvider(this)); + flowFileRepository.loadFlowFiles(queueProvider); long maxIdFromSwapFiles = -1L; if (flowFileRepository.isVolatile()) { @@ -1190,6 +1195,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node for (final RemoteSiteListener listener : externalSiteListeners) { listener.stop(); + listener.destroy(); } if (loadBalanceServer != null) { @@ -1719,6 +1725,10 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node return flowManager; } + public RepositoryContextFactory getRepositoryContextFactory() { + return repositoryContextFactory; + } + /** * Creates a connection between two Connectable objects. * diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/ConnectionEventListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/ConnectionEventListener.java index a3ae6ee79a..a5e2c5f20a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/ConnectionEventListener.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/ConnectionEventListener.java @@ -21,4 +21,14 @@ public interface ConnectionEventListener { void triggerSourceEvent(); void triggerDestinationEvent(); + + ConnectionEventListener NOP_EVENT_LISTENER = new ConnectionEventListener() { + @Override + public void triggerSourceEvent() { + } + + @Override + public void triggerDestinationEvent() { + } + }; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java index 125cd500e8..be7f17d77d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java @@ -525,7 +525,7 @@ public class FileSystemRepository implements ContentRepository { return containerPath.resolve(resourceClaim.getSection()).resolve(resourceClaim.getId()); } - private Path getPath(final ContentClaim claim, final boolean verifyExists) throws ContentNotFoundException { + public Path getPath(final ContentClaim claim, final boolean verifyExists) throws ContentNotFoundException { final ResourceClaim resourceClaim = claim.getResourceClaim(); final Path containerPath = containers.get(resourceClaim.getContainer()); if (containerPath == null) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryContext.java index ecf5046779..44675ea299 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryContext.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryContext.java @@ -16,14 +16,6 @@ */ package org.apache.nifi.controller.repository; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; - import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; @@ -32,6 +24,14 @@ import org.apache.nifi.processor.Relationship; import org.apache.nifi.provenance.ProvenanceEventRepository; import org.apache.nifi.util.Connectables; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; + public class RepositoryContext { private final Connectable connectable; @@ -128,11 +128,11 @@ public class RepositoryContext { counterRepo.adjustCounter(globalContext, name, delta); } - ContentRepository getContentRepository() { + public ContentRepository getContentRepository() { return contentRepo; } - FlowFileRepository getFlowFileRepository() { + public FlowFileRepository getFlowFileRepository() { return flowFileRepo; } @@ -140,7 +140,7 @@ public class RepositoryContext { return flowFileEventRepo; } - ProvenanceEventRepository getProvenanceRepository() { + public ProvenanceEventRepository getProvenanceRepository() { return provenanceRepo; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java index 78e93a1747..14cb70a994 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java @@ -614,6 +614,17 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE } } + // Next, process any JOIN events because we need to ensure that the JOINed FlowFile is created before any processor-emitted events occur. + for (final Map.Entry> entry : checkpoint.generatedProvenanceEvents.entrySet()) { + for (final ProvenanceEventRecord event : entry.getValue()) { + final ProvenanceEventType eventType = event.getEventType(); + if (eventType == ProvenanceEventType.JOIN) { + recordsToSubmit.add(event); + addEventType(eventTypesPerFlowFileId, event.getFlowFileUuid(), event.getEventType()); + } + } + } + // Now add any Processor-reported events. for (final ProvenanceEventRecord event : processorGenerated) { if (isSpuriousForkEvent(event, checkpoint.removedFlowFiles)) { @@ -633,6 +644,10 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE // Finally, add any other events that we may have generated. for (final List eventList : checkpoint.generatedProvenanceEvents.values()) { for (final ProvenanceEventRecord event : eventList) { + if (event.getEventType() == ProvenanceEventType.JOIN) { + continue; // JOIN events are handled above. + } + if (isSpuriousForkEvent(event, checkpoint.removedFlowFiles)) { continue; } @@ -647,17 +662,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE final ContentClaim original = repoRecord.getOriginalClaim(); final ContentClaim current = repoRecord.getCurrentClaim(); - boolean contentChanged = false; - if (original == null && current != null) { - contentChanged = true; - } - if (original != null && current == null) { - contentChanged = true; - } - if (original != null && current != null && !original.equals(current)) { - contentChanged = true; - } - + final boolean contentChanged = !Objects.equals(original, current); final FlowFileRecord curFlowFile = repoRecord.getCurrent(); final String flowFileId = curFlowFile.getAttribute(CoreAttributes.UUID.key()); boolean eventAdded = false; @@ -1742,11 +1747,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE private void registerJoinEvent(final FlowFile child, final Collection parents) { final ProvenanceEventRecord eventRecord = provenanceReporter.generateJoinEvent(parents, child); - List existingRecords = generatedProvenanceEvents.get(child); - if (existingRecords == null) { - existingRecords = new ArrayList<>(); - generatedProvenanceEvents.put(child, existingRecords); - } + final List existingRecords = generatedProvenanceEvents.computeIfAbsent(child, k -> new ArrayList<>()); existingRecords.add(eventRecord); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardQueueProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardQueueProvider.java index 55e36c64f5..cd891e9ba2 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardQueueProvider.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardQueueProvider.java @@ -17,7 +17,7 @@ package org.apache.nifi.controller.repository; import org.apache.nifi.connectable.Connection; -import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.flow.FlowManager; import org.apache.nifi.controller.queue.FlowFileQueue; import java.util.ArrayList; @@ -25,16 +25,16 @@ import java.util.Collection; import java.util.List; public class StandardQueueProvider implements QueueProvider { - private final FlowController flowController; + private final FlowManager flowManager; - public StandardQueueProvider(final FlowController flowController) { - this.flowController = flowController; + public StandardQueueProvider(final FlowManager flowManager) { + this.flowManager = flowManager; } @Override public Collection getAllQueues() { - final Collection connections = flowController.getFlowManager().findAllConnections(); + final Collection connections = flowManager.findAllConnections(); final List queues = new ArrayList<>(connections.size()); for (final Connection connection : connections) { queues.add(connection.getFlowFileQueue()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java index 4215633cf2..fdde18c8fe 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java @@ -181,6 +181,10 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis @Override public void initialize(final ResourceClaimManager claimManager) throws IOException { + initialize(claimManager, new StandardRepositoryRecordSerdeFactory(claimManager)); + } + + protected void initialize(final ResourceClaimManager claimManager, final RepositoryRecordSerdeFactory serdeFactory) throws IOException { this.claimManager = claimManager; for (final File file : flowFileRepositoryPaths) { @@ -191,7 +195,7 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis // TODO: Allow for backup path that can be used if disk out of space?? Would allow a snapshot to be stored on // backup and then the data deleted from the normal location; then can move backup to normal location and // delete backup. On restore, if no files exist in partition's directory, would have to check backup directory - serdeFactory = new RepositoryRecordSerdeFactory(claimManager); + this.serdeFactory = serdeFactory; if (walImplementation.equals(SEQUENTIAL_ACCESS_WAL)) { wal = new SequentialAccessWriteAheadLog<>(flowFileRepositoryPaths.get(0), serdeFactory, this); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/RingBufferEventRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/RingBufferEventRepository.java index bcd03448f3..629ec5e2ab 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/RingBufferEventRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/RingBufferEventRepository.java @@ -25,7 +25,6 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; public class RingBufferEventRepository implements FlowFileEventRepository { - private final int numMinutes; private final ConcurrentMap componentEventMap = new ConcurrentHashMap<>(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/SecondPrecisionEventContainer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/SecondPrecisionEventContainer.java index 2482177d3d..b901f3f9c1 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/SecondPrecisionEventContainer.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/metrics/SecondPrecisionEventContainer.java @@ -28,8 +28,8 @@ public class SecondPrecisionEventContainer implements EventContainer { private final int numBins; private final EventSum[] sums; - private final EventSumValue aggregateValue = new EventSumValue(0); - private final AtomicLong lastUpdateSecond = new AtomicLong(0); + private final EventSumValue aggregateValue = new EventSumValue(0L); + private final AtomicLong lastUpdateSecond = new AtomicLong(System.currentTimeMillis() / 1000L); public SecondPrecisionEventContainer(final int numMinutes) { // number of bins is number of seconds in 'numMinutes' plus 1. We add one because diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java index db937e9cdb..f7e496327d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java @@ -16,12 +16,6 @@ */ package org.apache.nifi.controller.scheduling; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; - import org.apache.nifi.connectable.Connectable; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.ReportingTaskNode; @@ -35,6 +29,12 @@ import org.apache.nifi.util.NiFiProperties; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + public class TimerDrivenSchedulingAgent extends AbstractSchedulingAgent { private static final Logger logger = LoggerFactory.getLogger(TimerDrivenSchedulingAgent.class); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java index 180e2a2ba8..45d0c8f412 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java @@ -149,19 +149,20 @@ public class ConnectableTask { } public InvocationResult invoke() { - logger.trace("Triggering {}", connectable); - if (scheduleState.isTerminated()) { + logger.debug("Will not trigger {} because task is terminated", connectable); return InvocationResult.DO_NOT_YIELD; } // make sure processor is not yielded if (isYielded()) { + logger.debug("Will not trigger {} because component is yielded", connectable); return InvocationResult.DO_NOT_YIELD; } // make sure that either we're not clustered or this processor runs on all nodes or that this is the primary node if (!isRunOnCluster(flowController)) { + logger.debug("Will not trigger {} because this is not the primary node", connectable); return InvocationResult.DO_NOT_YIELD; } @@ -179,6 +180,8 @@ public class ConnectableTask { } } + logger.debug("Triggering {}", connectable); + final long batchNanos = connectable.getRunDuration(TimeUnit.NANOSECONDS); final ProcessSessionFactory sessionFactory; final StandardProcessSession rawSession; @@ -206,8 +209,8 @@ public class ConnectableTask { try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), connectable.getRunnableComponent().getClass(), connectable.getIdentifier())) { boolean shouldRun = connectable.getScheduledState() == ScheduledState.RUNNING; while (shouldRun) { - connectable.onTrigger(processContext, activeSessionFactory); invocationCount++; + connectable.onTrigger(processContext, activeSessionFactory); if (!batch) { return InvocationResult.DO_NOT_YIELD; @@ -258,14 +261,14 @@ public class ConnectableTask { if (batch) { try { rawSession.commit(); - } catch (final Exception e) { + } catch (final Throwable t) { final ComponentLog procLog = new SimpleProcessLogger(connectable.getIdentifier(), connectable.getRunnableComponent()); - procLog.error("Failed to commit session {} due to {}; rolling back", new Object[] { rawSession, e.toString() }, e); + procLog.error("Failed to commit session {} due to {}; rolling back", new Object[] { rawSession, t.toString() }, t); try { rawSession.rollback(true); } catch (final Exception e1) { - procLog.error("Failed to roll back session {} due to {}", new Object[] { rawSession, e.toString() }, e); + procLog.error("Failed to roll back session {} due to {}", new Object[] { rawSession, t.toString() }, t); } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java index 1bc06a196f..ed123b8e0f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java @@ -3577,7 +3577,7 @@ public final class StandardProcessGroup implements ProcessGroup { if (childGroup == null) { final ProcessGroup added = addProcessGroup(group, proposedChildGroup, componentIdSeed, variablesToSkip); flowManager.onProcessGroupAdded(added); - added.findAllRemoteProcessGroups().stream().forEach(RemoteProcessGroup::initialize); + added.findAllRemoteProcessGroups().forEach(RemoteProcessGroup::initialize); LOG.info("Added {} to {}", added, this); } else if (childCoordinates == null || updateDescendantVersionedGroups) { updateProcessGroup(childGroup, proposedChildGroup, componentIdSeed, updatedVersionedComponentIds, true, true, updateDescendantVersionedGroups, variablesToSkip); @@ -3695,7 +3695,7 @@ public final class StandardProcessGroup implements ProcessGroup { final Set proposedAutoTerminated = proposedProcessor.getAutoTerminatedRelationships() == null ? Collections.emptySet() : proposedProcessor.getAutoTerminatedRelationships().stream() - .map(relName -> added.getRelationship(relName)) + .map(added::getRelationship) .collect(Collectors.toSet()); autoTerminatedRelationships.put(added, proposedAutoTerminated); LOG.info("Added {} to {}", added, this); @@ -3704,7 +3704,7 @@ public final class StandardProcessGroup implements ProcessGroup { final Set proposedAutoTerminated = proposedProcessor.getAutoTerminatedRelationships() == null ? Collections.emptySet() : proposedProcessor.getAutoTerminatedRelationships().stream() - .map(relName -> processor.getRelationship(relName)) + .map(processor::getRelationship) .collect(Collectors.toSet()); if (!processor.getAutoTerminatedRelationships().equals(proposedAutoTerminated)) { @@ -3778,7 +3778,7 @@ public final class StandardProcessGroup implements ProcessGroup { // We cannot do this above, in the 'updateProcessor' call because if a connection is removed and changed to auto-terminated, // then updating this in the updateProcessor call above would attempt to set the Relationship to being auto-terminated while a // Connection for that relationship exists. This will throw an Exception. - autoTerminatedRelationships.forEach((proc, rels) -> proc.setAutoTerminatedRelationships(rels)); + autoTerminatedRelationships.forEach(ProcessorNode::setAutoTerminatedRelationships); // Remove all controller services no longer in use for (final String removedVersionedId : controllerServicesRemoved) { @@ -4317,7 +4317,7 @@ public final class StandardProcessGroup implements ProcessGroup { } private String getServiceInstanceId(final String serviceVersionedComponentId, final ProcessGroup group) { - for (final ControllerServiceNode serviceNode : group.getControllerServices(true)) { + for (final ControllerServiceNode serviceNode : group.getControllerServices(false)) { final Optional optionalVersionedId = serviceNode.getVersionedComponentId(); final String versionedId = optionalVersionedId.orElseGet(() -> UUID.nameUUIDFromBytes(serviceNode.getIdentifier().getBytes(StandardCharsets.UTF_8)).toString()); if (versionedId.equals(serviceVersionedComponentId)) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java index a1fa975000..dbb84be5e6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestStandardProcessSession.java @@ -2259,11 +2259,9 @@ public class TestStandardProcessSession { public Set getExistingClaims() { final Set claims = new HashSet<>(); - for (long i = 0; i < idGenerator.get(); i++) { - final ResourceClaim resourceClaim = resourceClaimManager.newResourceClaim("container", "section", String.valueOf(i), false, false); - final ContentClaim contentClaim = new StandardContentClaim(resourceClaim, 0L); - if (getClaimantCount(contentClaim) > 0) { - claims.add(contentClaim); + for (final Map.Entry entry : claimantCounts.entrySet()) { + if (entry.getValue().get() > 0) { + claims.add(entry.getKey()); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java index cd3ee1cfda..4faba26b6e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java @@ -323,7 +323,7 @@ public class TestWriteAheadFlowFileRepository { assertTrue(path.toFile().mkdirs()); final ResourceClaimManager claimManager = new StandardResourceClaimManager(); - final RepositoryRecordSerdeFactory serdeFactory = new RepositoryRecordSerdeFactory(claimManager); + final StandardRepositoryRecordSerdeFactory serdeFactory = new StandardRepositoryRecordSerdeFactory(claimManager); final WriteAheadRepository repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serdeFactory, null); final Collection initialRecs = repo.recoverRecords(); assertTrue(initialRecs.isEmpty()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/AlwaysAuthorizedAuthorizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/AlwaysAuthorizedAuthorizer.java new file mode 100644 index 0000000000..872ac153da --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/AlwaysAuthorizedAuthorizer.java @@ -0,0 +1,56 @@ +/* + * 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.integration; + +import org.apache.nifi.authorization.AuthorizationRequest; +import org.apache.nifi.authorization.AuthorizationResult; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.AuthorizerConfigurationContext; +import org.apache.nifi.authorization.AuthorizerInitializationContext; +import org.apache.nifi.authorization.exception.AuthorizationAccessException; +import org.apache.nifi.authorization.exception.AuthorizerCreationException; +import org.apache.nifi.authorization.exception.AuthorizerDestructionException; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class AlwaysAuthorizedAuthorizer implements Authorizer { + private List requests = new ArrayList<>(); + + @Override + public AuthorizationResult authorize(final AuthorizationRequest request) throws AuthorizationAccessException { + requests.add(request); + return AuthorizationResult.approved(); + } + + @Override + public void initialize(final AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException { + } + + @Override + public void onConfigured(final AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException { + } + + @Override + public void preDestruction() throws AuthorizerDestructionException { + } + + public List getAuthorizationRequests() { + return Collections.unmodifiableList(requests); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/DirectInjectionExtensionManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/DirectInjectionExtensionManager.java new file mode 100644 index 0000000000..56cc212a4e --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/DirectInjectionExtensionManager.java @@ -0,0 +1,64 @@ +/* + * 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.integration; + +import org.apache.nifi.bundle.Bundle; +import org.apache.nifi.bundle.BundleDetails; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.nar.StandardExtensionDiscoveringManager; +import org.apache.nifi.nar.SystemBundle; +import org.apache.nifi.processor.Processor; +import org.apache.nifi.reporting.ReportingTask; + +import java.io.File; + +public class DirectInjectionExtensionManager extends StandardExtensionDiscoveringManager { + private static final Bundle INTEGRATION_TEST_BUNDLE = new Bundle(new BundleDetails.Builder() + .workingDir(new File("target")) + .coordinate(SystemBundle.SYSTEM_BUNDLE_COORDINATE) + .build(), ClassLoader.getSystemClassLoader()); + + public void injectExtension(final Object extension) { + final Class extensionType; + if (extension instanceof Processor) { + extensionType = Processor.class; + } else if (extension instanceof ControllerService) { + extensionType = ControllerService.class; + } else if (extension instanceof ReportingTask) { + extensionType = ReportingTask.class; + } else { + throw new IllegalArgumentException("Given extension is not a Processor, Controller Service, or Reporting Task"); + } + + super.loadExtension(extension, extensionType, INTEGRATION_TEST_BUNDLE); + } + + public void injectExtensionType(final Class extensionType, final String implementationClassName) { + final Class implementationClass; + try { + implementationClass = Class.forName(implementationClassName, false, INTEGRATION_TEST_BUNDLE.getClassLoader()); + } catch (ClassNotFoundException e) { + throw new IllegalArgumentException("Could not find implementation class: " + implementationClassName); + } + + injectExtensionType(extensionType, implementationClass); + } + + public void injectExtensionType(final Class extensionType, final Class implementationClass) { + super.registerExtensionClass(extensionType, implementationClass, INTEGRATION_TEST_BUNDLE); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/FrameworkIntegrationTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/FrameworkIntegrationTest.java new file mode 100644 index 0000000000..157f9d1123 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/FrameworkIntegrationTest.java @@ -0,0 +1,461 @@ +/* + * 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.integration; + +import org.apache.nifi.admin.service.AuditService; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.bundle.Bundle; +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.components.state.StateProvider; +import org.apache.nifi.components.validation.ValidationStatus; +import org.apache.nifi.connectable.Connection; +import org.apache.nifi.connectable.StandardConnection; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.FileSystemSwapManager; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.StandardSnippet; +import org.apache.nifi.controller.flow.StandardFlowManager; +import org.apache.nifi.controller.queue.ConnectionEventListener; +import org.apache.nifi.controller.queue.FlowFileQueue; +import org.apache.nifi.controller.queue.FlowFileQueueFactory; +import org.apache.nifi.controller.queue.LoadBalanceStrategy; +import org.apache.nifi.controller.queue.StandardFlowFileQueue; +import org.apache.nifi.controller.repository.ContentRepository; +import org.apache.nifi.controller.repository.FileSystemRepository; +import org.apache.nifi.controller.repository.FlowFileEvent; +import org.apache.nifi.controller.repository.FlowFileEventRepository; +import org.apache.nifi.controller.repository.FlowFileRecord; +import org.apache.nifi.controller.repository.FlowFileRepository; +import org.apache.nifi.controller.repository.FlowFileSwapManager; +import org.apache.nifi.controller.repository.QueueProvider; +import org.apache.nifi.controller.repository.RepositoryContext; +import org.apache.nifi.controller.repository.RepositoryStatusReport; +import org.apache.nifi.controller.repository.WriteAheadFlowFileRepository; +import org.apache.nifi.controller.repository.claim.ResourceClaimManager; +import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager; +import org.apache.nifi.controller.repository.metrics.RingBufferEventRepository; +import org.apache.nifi.controller.scheduling.RepositoryContextFactory; +import org.apache.nifi.controller.scheduling.SchedulingAgent; +import org.apache.nifi.controller.scheduling.StandardProcessScheduler; +import org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.controller.service.StandardControllerServiceProvider; +import org.apache.nifi.controller.state.providers.local.WriteAheadLocalStateProvider; +import org.apache.nifi.controller.status.history.ComponentStatusRepository; +import org.apache.nifi.controller.status.history.VolatileComponentStatusRepository; +import org.apache.nifi.encrypt.StringEncryptor; +import org.apache.nifi.engine.FlowEngine; +import org.apache.nifi.events.VolatileBulletinRepository; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.integration.processor.BiConsumerProcessor; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.nar.SystemBundle; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Processor; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.ProvenanceEventRepository; +import org.apache.nifi.provenance.ProvenanceEventType; +import org.apache.nifi.provenance.ProvenanceRepository; +import org.apache.nifi.provenance.WriteAheadProvenanceRepository; +import org.apache.nifi.registry.VariableRegistry; +import org.apache.nifi.registry.flow.FlowRegistryClient; +import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.scheduling.SchedulingStrategy; +import org.apache.nifi.util.FileUtils; +import org.apache.nifi.util.NiFiProperties; +import org.junit.After; +import org.junit.Before; +import org.junit.rules.Timeout; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; + +import static org.junit.Assert.assertEquals; + +public class FrameworkIntegrationTest { + //@Rule + public Timeout globalTimeout = Timeout.seconds(20); + + private ResourceClaimManager resourceClaimManager; + private StandardProcessScheduler processScheduler; + + private FlowEngine flowEngine; + private FlowController flowController; + private FlowRegistryClient flowRegistryClient = null; + private ProcessorNode nopProcessor; + private ProcessorNode terminateProcessor; + private ProcessorNode terminateAllProcessor; + private FlowFileQueueFactory flowFileQueueFactory; + private FlowFileSwapManager flowFileSwapManager; + private DirectInjectionExtensionManager extensionManager; + private ProcessGroup rootProcessGroup; + private Bundle systemBundle; + + public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success").build(); + + @Before + public void setup() throws IOException { + cleanup(); + initialize(null); + } + + protected String getNiFiPropertiesFilename() { + return "src/test/resources/int-tests/default-nifi.properties"; + } + + protected Map getNiFiPropertiesOverrides() { + return Collections.emptyMap(); + } + + protected void injectExtensionTypes(final DirectInjectionExtensionManager extensionManager) { + // Placeholder for subclasses. + } + + protected final void initialize(final QueueProvider queueProvider) throws IOException { + final NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties(getNiFiPropertiesFilename(), getNiFiPropertiesOverrides()); + initialize(nifiProperties, queueProvider); + } + + protected final void initialize(final NiFiProperties nifiProperties, final QueueProvider queueProvider) throws IOException { + final FlowFileEventRepository flowFileEventRepository = new RingBufferEventRepository(5); + resourceClaimManager = new StandardResourceClaimManager(); + + final BulletinRepository bulletinRepo = new VolatileBulletinRepository(); + flowEngine = new FlowEngine(4, "unit test flow engine"); + extensionManager = new DirectInjectionExtensionManager(); + + extensionManager.injectExtensionType(FlowFileRepository.class, WriteAheadFlowFileRepository.class); + extensionManager.injectExtensionType(ContentRepository.class, FileSystemRepository.class); + extensionManager.injectExtensionType(ProvenanceRepository.class, WriteAheadProvenanceRepository.class); + extensionManager.injectExtensionType(StateProvider.class, WriteAheadLocalStateProvider.class); + extensionManager.injectExtensionType(ComponentStatusRepository.class, VolatileComponentStatusRepository.class); + extensionManager.injectExtensionType(FlowFileSwapManager.class, FileSystemSwapManager.class); + extensionManager.injectExtensionType(Processor.class, BiConsumerProcessor.class); + + injectExtensionTypes(extensionManager); + systemBundle = SystemBundle.create(nifiProperties); + extensionManager.discoverExtensions(systemBundle, Collections.emptySet()); + + final StringEncryptor encryptor = StringEncryptor.createEncryptor("PBEWITHMD5AND256BITAES-CBC-OPENSSL", "BC", "unit-test"); + final Authorizer authorizer = new AlwaysAuthorizedAuthorizer(); + final AuditService auditService = new NopAuditService(); + + flowController = FlowController.createStandaloneInstance(flowFileEventRepository, nifiProperties, authorizer, auditService, encryptor, bulletinRepo, + VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY, flowRegistryClient, extensionManager); + + processScheduler = new StandardProcessScheduler(flowEngine, flowController, encryptor, flowController.getStateManagerProvider(), nifiProperties); + + final RepositoryContextFactory repositoryContextFactory = flowController.getRepositoryContextFactory(); + final SchedulingAgent timerDrivenSchedulingAgent = new TimerDrivenSchedulingAgent(flowController, flowEngine, repositoryContextFactory, encryptor, nifiProperties); + processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, timerDrivenSchedulingAgent); + + final ControllerServiceProvider controllerServiceProvider = new StandardControllerServiceProvider(flowController, processScheduler, bulletinRepo); + + rootProcessGroup = flowController.getFlowManager().createProcessGroup(UUID.randomUUID().toString()); + ((StandardFlowManager) flowController.getFlowManager()).setRootGroup(rootProcessGroup); + + nopProcessor = createProcessorNode((context, session) -> {}); + + terminateProcessor = createProcessorNode((context, session) -> { + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; + } + + session.remove(flowFile); + }); + + terminateAllProcessor = createProcessorNode((context, session) -> { + FlowFile flowFile; + while ((flowFile = session.get()) != null) { + session.remove(flowFile); + } + }); + + flowFileSwapManager = flowController.createSwapManager(); + flowFileQueueFactory = new FlowFileQueueFactory() { + @Override + public FlowFileQueue createFlowFileQueue(final LoadBalanceStrategy loadBalanceStrategy, final String partitioningAttribute, final ConnectionEventListener connectionEventListener) { + return FrameworkIntegrationTest.this.createFlowFileQueue(UUID.randomUUID().toString()); + } + }; + + if (queueProvider == null) { + flowController.initializeFlow(); + } else { + flowController.initializeFlow(queueProvider); + } + } + + @After + public final void shutdown() { + flowController.shutdown(true); + flowEngine.shutdownNow(); + processScheduler.shutdown(); + } + + @After + public final void cleanup() throws IOException { + deleteDirectory(new File("target/int-tests")); + } + + private void deleteDirectory(final File dir) throws IOException { + if (!dir.exists()) { + return; + } + + FileUtils.deleteFile(dir, true); + } + + protected FlowFileQueue createFlowFileQueue(final String uuid) { + final RepositoryContext repoContext = getRepositoryContext(); + return new StandardFlowFileQueue(uuid, ConnectionEventListener.NOP_EVENT_LISTENER, repoContext.getFlowFileRepository(), repoContext.getProvenanceRepository(), + resourceClaimManager, processScheduler, flowFileSwapManager, flowController.createEventReporter(), 20000, 10000L, "1 GB"); + } + + protected final ProcessorNode createProcessorNode(final Class processorType) { + return createProcessorNode(processorType.getName()); + } + + protected final ProcessorNode createProcessorNode(final String processorType) { + final String uuid = getSimpleTypeName(processorType) + "-" + UUID.randomUUID().toString(); + final BundleCoordinate bundleCoordinate = SystemBundle.SYSTEM_BUNDLE_COORDINATE; + final ProcessorNode procNode = flowController.getFlowManager().createProcessor(processorType, uuid, bundleCoordinate, Collections.emptySet(), true, true); + rootProcessGroup.addProcessor(procNode); + + return procNode; + } + + protected final ControllerServiceNode createControllerServiceNode(final Class controllerServiceType) { + return createControllerServiceNode(controllerServiceType.getName()); + } + + protected final ControllerServiceNode createControllerServiceNode(final String controllerServiceType) { + final String uuid = getSimpleTypeName(controllerServiceType) + "-" + UUID.randomUUID().toString(); + final BundleCoordinate bundleCoordinate = SystemBundle.SYSTEM_BUNDLE_COORDINATE; + final ControllerServiceNode serviceNode = flowController.getFlowManager().createControllerService(controllerServiceType, uuid, bundleCoordinate, Collections.emptySet(), true, true); + rootProcessGroup.addControllerService(serviceNode); + return serviceNode; + } + + private String getSimpleTypeName(final String className) { + final int index = className.lastIndexOf("."); + if (index >= 0 && index < className.length()) { + return className.substring(index + 1); + } else { + return ""; + } + } + + protected ProcessGroup getRootGroup() { + return rootProcessGroup; + } + + /** + * Creates a Processor that is responsible for generating a FlowFile of the given size and routing to "success". + * + * @param contentSize the number of bytes for the content + * + * @return the ProcessorNode + */ + protected final ProcessorNode createGenerateProcessor(final int contentSize) { + return createGenerateProcessor(contentSize, null); + } + + /** + * Creates a Processor that is responsible for generating a FlowFile of the given size and routing to "success". The generated FlowFile is set in the given AtomicReference + * + * @param contentSize the number of bytes for the content + * @param flowFileReference an AtomicReference to hold the flowfile + * + * @return the ProcessorNode + */ + protected final ProcessorNode createGenerateProcessor(final int contentSize, final AtomicReference flowFileReference) { + return createProcessorNode((context, session) -> { + FlowFile flowFile = session.create(); + flowFile = session.write(flowFile, out -> out.write(new byte[contentSize])); + + if (flowFileReference != null) { + flowFileReference.set((FlowFileRecord) flowFile); + } + + session.transfer(flowFile, REL_SUCCESS); + }, REL_SUCCESS); + } + + protected final ProcessorNode createProcessorNode(final BiConsumer trigger, final Relationship... relationships) { + final Set relationshipSet = new HashSet<>(Arrays.asList(relationships)); + + final ProcessorNode processorNode = createProcessorNode(BiConsumerProcessor.class.getName()); + final BiConsumerProcessor biConsumerProcessor = (BiConsumerProcessor) processorNode.getProcessor(); + biConsumerProcessor.setRelationships(relationshipSet); + biConsumerProcessor.setTrigger(trigger); + + return processorNode; + } + + protected final void connect(final ProcessorNode source, final ProcessorNode destination, final Relationship relationship) { + connect(source, destination, Collections.singleton(relationship)); + } + + protected final void connect(final ProcessorNode source, final ProcessorNode destination, final Collection relationships) { + final Connection connection = new StandardConnection.Builder(processScheduler) + .source(source) + .destination(destination) + .relationships(relationships) + .id(UUID.randomUUID().toString()) + .clustered(false) + .flowFileQueueFactory(flowFileQueueFactory) + .build(); + + source.addConnection(connection); + destination.addConnection(connection); + rootProcessGroup.addConnection(connection); + } + + protected final Future start(final ProcessorNode procNode) { + final ValidationStatus validationStatus = procNode.performValidation(); + if (validationStatus != ValidationStatus.VALID) { + throw new IllegalStateException("Processor is invalid: " + procNode + ": " + procNode.getValidationErrors()); + } + + return rootProcessGroup.startProcessor(procNode, true); + } + + protected final Future stop(final ProcessorNode procNode) { + return rootProcessGroup.stopProcessor(procNode); + } + + protected final FlowFileQueue getDestinationQueue(final ProcessorNode procNode, final Relationship relationship) { + return procNode.getConnections(relationship).stream() + .map(Connection::getFlowFileQueue) + .findAny() + .orElseThrow(() -> new IllegalArgumentException("Could not find queue for relationship with name <" + relationship + ">")); + } + + protected final FlowFileRepository getFlowFileRepository() { + return getRepositoryContext().getFlowFileRepository(); + } + + protected Bundle getSystemBundle() { + return systemBundle; + } + + protected final ContentRepository getContentRepository() { + return getRepositoryContext().getContentRepository(); + } + + protected final ProvenanceEventRepository getProvenanceRepository() { + return getRepositoryContext().getProvenanceRepository(); + } + + private RepositoryContext getRepositoryContext() { + return flowController.getRepositoryContextFactory().newProcessContext(nopProcessor, new AtomicLong(0L)); + } + + protected final ProcessorNode getNopProcessor() { + return nopProcessor; + } + + protected final ProcessorNode getTerminateProcessor() { + return terminateProcessor; + } + + protected final ProcessorNode getTerminateAllProcessor() { + return terminateAllProcessor; + } + + protected final FlowController getFlowController() { + return flowController; + } + + protected void assertProvenanceEventCount(final ProvenanceEventType eventType, final int count) throws IOException { + int encountered = 0; + + for (final ProvenanceEventRecord event : getProvenanceRepository().getEvents(0L, 100_000_000)) { + if (event.getEventType() == eventType) { + encountered++; + } + } + + assertEquals("Expected to encounter " + count + " Provenance Events of type " + eventType + " but encountered " + encountered, count, encountered); + } + + protected void triggerOnce(final ProcessorNode processor) throws ExecutionException, InterruptedException { + final String schedulingPeriod = processor.getSchedulingPeriod(); + try { + final FlowFileEvent initialReport = getStatusReport(processor); + final int initialInvocations = (initialReport == null) ? 0 : initialReport.getInvocations(); + + processor.setScheduldingPeriod("1 hour"); + + // We will only trigger the Processor to run once per hour. So we need to ensure that + // we don't trigger the Processor while it's yielded. So if its yield expiration is in the future, + // wait until the yield expires. + while (processor.getYieldExpiration() > System.currentTimeMillis()) { + Thread.sleep(1L); + } + + start(processor).get(); + + int totalInvocations = initialInvocations; + while (totalInvocations < initialInvocations + 1) { + final FlowFileEvent currentReport = getStatusReport(processor); + totalInvocations = currentReport == null ? 0 : currentReport.getInvocations(); + } + + stop(processor).get(); + } finally { + processor.setScheduldingPeriod(schedulingPeriod); + } + } + + protected FlowFileEvent getStatusReport(final ProcessorNode processor) { + final FlowFileEventRepository repository = getRepositoryContext().getFlowFileEventRepository(); + RepositoryStatusReport statusReport = repository.reportTransferEvents(0L); + return statusReport.getReportEntry(processor.getIdentifier()); + } + + protected void moveProcessor(final ProcessorNode processor, final ProcessGroup destination) { + final StandardSnippet snippet = new StandardSnippet(); + snippet.setParentGroupId(processor.getProcessGroupIdentifier()); + snippet.addProcessors(Collections.singletonMap(processor.getIdentifier(), null)); + + processor.getProcessGroup().move(snippet, destination); + } + + protected ExtensionManager getExtensionManager() { + return extensionManager; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/NopAuditService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/NopAuditService.java new file mode 100644 index 0000000000..878dc387cb --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/NopAuditService.java @@ -0,0 +1,60 @@ +/* + * 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.integration; + +import org.apache.nifi.action.Action; +import org.apache.nifi.admin.service.AuditService; +import org.apache.nifi.history.History; +import org.apache.nifi.history.HistoryQuery; +import org.apache.nifi.history.PreviousValue; + +import java.util.Collection; +import java.util.Date; +import java.util.List; +import java.util.Map; + +public class NopAuditService implements AuditService { + @Override + public void addActions(final Collection actions) { + + } + + @Override + public Map> getPreviousValues(final String componentId) { + return null; + } + + @Override + public History getActions(final HistoryQuery actionQuery) { + return null; + } + + @Override + public History getActions(final int firstActionId, final int maxActions) { + return null; + } + + @Override + public Action getAction(final Integer actionId) { + return null; + } + + @Override + public void purgeActions(final Date end, final Action purgeAction) { + + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/ControllerServiceReferenceIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/ControllerServiceReferenceIT.java new file mode 100644 index 0000000000..b586444d12 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/ControllerServiceReferenceIT.java @@ -0,0 +1,70 @@ +/* + * 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.integration.cs; + +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.validation.ValidationStatus; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.integration.DirectInjectionExtensionManager; +import org.apache.nifi.integration.FrameworkIntegrationTest; +import org.apache.nifi.processor.Processor; +import org.junit.Test; + +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.ExecutionException; + +import static org.junit.Assert.assertEquals; +import static org.testng.Assert.assertSame; + +public class ControllerServiceReferenceIT extends FrameworkIntegrationTest { + @Override + protected void injectExtensionTypes(final DirectInjectionExtensionManager extensionManager) { + extensionManager.injectExtensionType(ControllerService.class, CounterControllerService.class); + extensionManager.injectExtensionType(ControllerService.class, LongValidatingControllerService.class); + extensionManager.injectExtensionType(Processor.class, ControllerServiceReferencingProcessor.class); + } + + + @Test + public void testCallingControllerService() throws ExecutionException, InterruptedException { + final ProcessorNode counter = createProcessorNode(ControllerServiceReferencingProcessor.class.getName()); + + final ControllerServiceNode serviceNode = createControllerServiceNode(CounterControllerService.class.getName()); + assertSame(ValidationStatus.VALID, serviceNode.performValidation()); + getFlowController().getControllerServiceProvider().enableControllerService(serviceNode).get(); + + counter.setAutoTerminatedRelationships(Collections.singleton(REL_SUCCESS)); + counter.setProperties(Collections.singletonMap("Counter Service", serviceNode.getIdentifier())); + + triggerOnce(counter); + + assertEquals(1, ((Counter) serviceNode.getControllerServiceImplementation()).getValue()); + } + + @Test + public void testLongValidatingControllerService() { + final ControllerServiceNode serviceNode = createControllerServiceNode(LongValidatingControllerService.class.getName()); + serviceNode.setProperties(Collections.singletonMap(LongValidatingControllerService.DELAY.getName(), "250 millis")); + final ValidationStatus validationStatus = serviceNode.performValidation(); + final Collection validationErrors = serviceNode.getValidationErrors(); + assertSame(validationStatus, ValidationStatus.VALID); + assertEquals(0, validationErrors.size()); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/ControllerServiceReferencingProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/ControllerServiceReferencingProcessor.java new file mode 100644 index 0000000000..82438e8f01 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/ControllerServiceReferencingProcessor.java @@ -0,0 +1,52 @@ +/* + * 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.integration.cs; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.integration.FrameworkIntegrationTest; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; + +import java.util.Collections; +import java.util.List; +import java.util.Set; + +public class ControllerServiceReferencingProcessor extends AbstractProcessor { + private final PropertyDescriptor SERVICE = new PropertyDescriptor.Builder() + .name("Counter Service") + .identifiesControllerService(Counter.class) + .required(true) + .build(); + + @Override + public Set getRelationships() { + return Collections.singleton(FrameworkIntegrationTest.REL_SUCCESS); + } + + @Override + protected List getSupportedPropertyDescriptors() { + return Collections.singletonList(SERVICE); + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + context.getProperty(SERVICE).asControllerService(Counter.class).increment(1L); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/Counter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/Counter.java new file mode 100644 index 0000000000..6a46400602 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/Counter.java @@ -0,0 +1,25 @@ +/* + * 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.integration.cs; + +import org.apache.nifi.controller.ControllerService; + +public interface Counter extends ControllerService { + long increment(long delta); + + long getValue(); +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/CounterControllerService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/CounterControllerService.java new file mode 100644 index 0000000000..5d49466ab6 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/CounterControllerService.java @@ -0,0 +1,35 @@ +/* + * 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.integration.cs; + +import org.apache.nifi.controller.AbstractControllerService; + +import java.util.concurrent.atomic.AtomicLong; + +public class CounterControllerService extends AbstractControllerService implements Counter { + private final AtomicLong count = new AtomicLong(0L); + + @Override + public long increment(final long delta) { + return count.addAndGet(delta); + } + + @Override + public long getValue() { + return count.get(); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/LongValidatingControllerService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/LongValidatingControllerService.java new file mode 100644 index 0000000000..4bad93a61a --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/LongValidatingControllerService.java @@ -0,0 +1,59 @@ +/* + * 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.integration.cs; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyDescriptor.Builder; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.processor.util.StandardValidators; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.apache.nifi.expression.ExpressionLanguageScope.NONE; + +public class LongValidatingControllerService extends AbstractControllerService { + public static final PropertyDescriptor DELAY = new Builder() + .name("Delay") + .displayName("Delay") + .description("Amount of time to sleep during validation") + .required(true) + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(NONE) + .defaultValue("2 sec") + .build(); + + @Override + protected List getSupportedPropertyDescriptors() { + return Collections.singletonList(DELAY); + } + + @Override + protected Collection customValidate(final ValidationContext validationContext) { + final long millis = validationContext.getProperty(DELAY).asTimePeriod(TimeUnit.MILLISECONDS); + try { + Thread.sleep(millis); + } catch (final InterruptedException ie) { + } + + return Collections.emptyList(); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/NopControllerService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/NopControllerService.java new file mode 100644 index 0000000000..2dcc5aa5ca --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/NopControllerService.java @@ -0,0 +1,22 @@ +/* + * 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.integration.cs; + +import org.apache.nifi.controller.ControllerService; + +public interface NopControllerService extends ControllerService { +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/NopServiceReferencingProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/NopServiceReferencingProcessor.java new file mode 100644 index 0000000000..8c55326ec5 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/cs/NopServiceReferencingProcessor.java @@ -0,0 +1,46 @@ +/* + * 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.integration.cs; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyDescriptor.Builder; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; + +import java.util.Collections; +import java.util.List; + +public class NopServiceReferencingProcessor extends AbstractProcessor { + public static final PropertyDescriptor SERVICE = new Builder() + .name("Controller Service") + .displayName("Controller Service") + .description("The Controller Service") + .required(true) + .identifiesControllerService(NopControllerService.class) + .build(); + + @Override + protected List getSupportedPropertyDescriptors() { + return Collections.singletonList(SERVICE); + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/flowfilerepo/OOMEFlowFileRepoUpdateIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/flowfilerepo/OOMEFlowFileRepoUpdateIT.java new file mode 100644 index 0000000000..64a7657e82 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/flowfilerepo/OOMEFlowFileRepoUpdateIT.java @@ -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.integration.flowfilerepo; + +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.repository.FlowFileRepository; +import org.apache.nifi.controller.repository.WriteAheadFlowFileRepository; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.integration.DirectInjectionExtensionManager; +import org.apache.nifi.integration.FrameworkIntegrationTest; +import org.apache.nifi.reporting.Bulletin; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; + +import static org.junit.Assert.assertEquals; + +public class OOMEFlowFileRepoUpdateIT extends FrameworkIntegrationTest { + @Override + protected Map getNiFiPropertiesOverrides() { + return Collections.singletonMap("nifi.flowfile.repository.implementation", OOMEWriteAheadFlowFileRepository.class.getName()); + } + + @Override + protected void injectExtensionTypes(final DirectInjectionExtensionManager extensionManager) { + extensionManager.injectExtensionType(FlowFileRepository.class, OOMEWriteAheadFlowFileRepository.class); + } + + @Test + public void testOOMEOnUpdatePreventsSubsequentUpdates() throws ExecutionException, InterruptedException, IOException { + final ProcessorNode generate = createProcessorNode((context, session) -> { + FlowFile flowFile = session.create(); + session.transfer(flowFile, REL_SUCCESS); + }, REL_SUCCESS); + connect(generate, getTerminateAllProcessor(), REL_SUCCESS); + + for (int i=0; i < 4; i++) { + triggerOnce(generate); + } + + List processorBulletins = getFlowController().getBulletinRepository().findBulletinsForSource(generate.getIdentifier()); + assertEquals(1, processorBulletins.size()); + + triggerOnce(generate); + + // FlowFile Repository should not allow us to udpate until it has been checkpointed. + processorBulletins = getFlowController().getBulletinRepository().findBulletinsForSource(generate.getIdentifier()); + assertEquals(2, processorBulletins.size()); + + // Checkpoint the repository. + ((WriteAheadFlowFileRepository) getFlowFileRepository()).checkpoint(); + + // Should now succeed. + triggerOnce(generate); + processorBulletins = getFlowController().getBulletinRepository().findBulletinsForSource(generate.getIdentifier()); + assertEquals(2, processorBulletins.size()); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/flowfilerepo/OOMEWriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/flowfilerepo/OOMEWriteAheadFlowFileRepository.java new file mode 100644 index 0000000000..1509e4136f --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/flowfilerepo/OOMEWriteAheadFlowFileRepository.java @@ -0,0 +1,142 @@ +/* + * 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.integration.flowfilerepo; + +import org.apache.nifi.controller.queue.FlowFileQueue; +import org.apache.nifi.controller.repository.RepositoryRecord; +import org.apache.nifi.controller.repository.RepositoryRecordSerdeFactory; +import org.apache.nifi.controller.repository.StandardRepositoryRecordSerdeFactory; +import org.apache.nifi.controller.repository.WriteAheadFlowFileRepository; +import org.apache.nifi.controller.repository.claim.ResourceClaimManager; +import org.apache.nifi.util.NiFiProperties; +import org.wali.SerDe; +import org.wali.UpdateType; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.Map; + +public class OOMEWriteAheadFlowFileRepository extends WriteAheadFlowFileRepository { + + public OOMEWriteAheadFlowFileRepository() { + } + + public OOMEWriteAheadFlowFileRepository(final NiFiProperties nifiProperties) { + super(nifiProperties); + } + + @Override + public void initialize(final ResourceClaimManager claimManager) throws IOException { + super.initialize(claimManager, new ThrowOOMERepositoryRecordSerdeFactory(new StandardRepositoryRecordSerdeFactory(claimManager))); + } + + + private static class ThrowOOMERepositoryRecordSerdeFactory implements RepositoryRecordSerdeFactory { + private final RepositoryRecordSerdeFactory factory; + + public ThrowOOMERepositoryRecordSerdeFactory(final RepositoryRecordSerdeFactory factory) { + this.factory = factory; + } + + @Override + public void setQueueMap(final Map queueMap) { + factory.setQueueMap(queueMap); + } + + @Override + public SerDe createSerDe(final String encodingName) { + final SerDe serde = factory.createSerDe(encodingName); + return new ThrowOOMESerde(serde, 3); + } + + @Override + public Long getRecordIdentifier(final RepositoryRecord record) { + return factory.getRecordIdentifier(record); + } + + @Override + public UpdateType getUpdateType(final RepositoryRecord record) { + return factory.getUpdateType(record); + } + + @Override + public String getLocation(final RepositoryRecord record) { + return factory.getLocation(record); + } + } + + + private static class ThrowOOMESerde implements SerDe { + private final SerDe serde; + private final int afterSuccessfulAttempts; + private int successfulUpdates = 0; + + public ThrowOOMESerde(final SerDe serde, final int afterSuccessfulAttempts) { + this.serde = serde; + this.afterSuccessfulAttempts = afterSuccessfulAttempts; + } + + @Override + public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord newRecordState, final DataOutputStream out) throws IOException { + if (successfulUpdates++ == afterSuccessfulAttempts) { + throw new OutOfMemoryError("Intentional OOME for unit test"); + } + + serde.serializeEdit(previousRecordState, newRecordState, out); + } + + @Override + public void serializeRecord(final RepositoryRecord record, final DataOutputStream out) throws IOException { + if (successfulUpdates++ == afterSuccessfulAttempts) { + throw new OutOfMemoryError("Intentional OOME for unit test"); + } + + serde.serializeRecord(record, out); + } + + @Override + public RepositoryRecord deserializeEdit(final DataInputStream in, final Map currentRecordStates, final int version) throws IOException { + return serde.deserializeEdit(in, currentRecordStates, version); + } + + @Override + public RepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException { + return serde.deserializeRecord(in, version); + } + + @Override + public Object getRecordIdentifier(final RepositoryRecord record) { + return serde.getRecordIdentifier(record); + } + + @Override + public UpdateType getUpdateType(final RepositoryRecord record) { + return serde.getUpdateType(record); + } + + @Override + public String getLocation(final RepositoryRecord record) { + return serde.getLocation(record); + } + + @Override + public int getVersion() { + return serde.getVersion(); + } + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/lifecycle/ContentCleanupIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/lifecycle/ContentCleanupIT.java new file mode 100644 index 0000000000..90f1f3df3a --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/lifecycle/ContentCleanupIT.java @@ -0,0 +1,184 @@ +/* + * 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.integration.lifecycle; + +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.repository.FileSystemRepository; +import org.apache.nifi.controller.repository.FlowFileRecord; +import org.apache.nifi.controller.repository.WriteAheadFlowFileRepository; +import org.apache.nifi.controller.repository.claim.ContentClaim; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.integration.FrameworkIntegrationTest; +import org.apache.nifi.provenance.ProvenanceEventType; +import org.junit.Test; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicReference; + +import static junit.framework.TestCase.assertTrue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; + +public class ContentCleanupIT extends FrameworkIntegrationTest { + + @Test + public void testCompletedContentClaimCleanedUpOnCheckpoint() throws InterruptedException, IOException, ExecutionException { + final AtomicReference largeFlowFileReference = new AtomicReference<>(); + final AtomicReference smallFlowFileReference = new AtomicReference<>(); + + // Processor to write 1 MB of content to a FlowFile + final ProcessorNode createLargeProcessor = createGenerateProcessor(1024 * 1024, largeFlowFileReference); + final ProcessorNode createSmallProcessor = createGenerateProcessor(5, smallFlowFileReference); + + connect(createLargeProcessor, getTerminateProcessor(), REL_SUCCESS); + connect(createSmallProcessor, getTerminateProcessor(), REL_SUCCESS); + + // Trigger the create processor. + triggerOnce(createLargeProcessor); + triggerOnce(createSmallProcessor); + + // Ensure content available and has a claim count of 1. + final ContentClaim largeContentClaim = largeFlowFileReference.get().getContentClaim(); + final ContentClaim smallContentClaim = smallFlowFileReference.get().getContentClaim(); + assertNotEquals(largeContentClaim.getResourceClaim(), smallContentClaim.getResourceClaim()); + assertEquals(1, getContentRepository().getClaimantCount(largeContentClaim)); + assertEquals(1, getContentRepository().getClaimantCount(smallContentClaim)); + + // Ensure that content is still available and considered 'in use' + final FileSystemRepository fileSystemRepository = (FileSystemRepository) getContentRepository(); + final Path largeClaimPath = fileSystemRepository.getPath(largeContentClaim, false); + final Path smallClaimPath = fileSystemRepository.getPath(smallContentClaim, false); + assertTrue(Files.exists(largeClaimPath)); + assertTrue(largeContentClaim.getResourceClaim().isInUse()); + assertTrue(Files.exists(smallClaimPath)); + assertTrue(smallContentClaim.getResourceClaim().isInUse()); + + int recordCount = ((WriteAheadFlowFileRepository) getFlowFileRepository()).checkpoint(); + assertEquals(2, recordCount); + + // Trigger the delete Processor. + triggerOnce(getTerminateProcessor()); + triggerOnce(getTerminateProcessor()); + + // Claim count should now be 0 and resource claim should not be in use. + assertEquals(0, getContentRepository().getClaimantCount(largeContentClaim)); + assertEquals(0, getContentRepository().getClaimantCount(largeContentClaim)); + + assertFalse(largeContentClaim.getResourceClaim().isInUse()); + assertTrue(smallContentClaim.getResourceClaim().isInUse()); + + // Checkpoint the FlowFile Repo + recordCount = ((WriteAheadFlowFileRepository) getFlowFileRepository()).checkpoint(); + assertEquals(0, recordCount); + + // Wait for the data to be deleted/archived. + waitForClaimDestruction(largeContentClaim); + + assertTrue(Files.exists(smallClaimPath)); + + assertProvenanceEventCount(ProvenanceEventType.CREATE, 2); + assertProvenanceEventCount(ProvenanceEventType.DROP, 2); + } + + + @Test + public void testTransientClaimsNotHeld() throws ExecutionException, InterruptedException, IOException { + final AtomicReference claimReference = new AtomicReference<>(); + + final ProcessorNode processor = createProcessorNode((context, session) -> { + FlowFile flowFile = session.create(); + + for (int i=0; i < 1000; i++) { + final byte[] bytes = String.valueOf(i).getBytes(); + flowFile = session.write(flowFile, out -> out.write(bytes)); + } + + // Write 1 MB to the content claim in order to ensure that the claim is no longer usable. + final byte[] oneMB = new byte[1024 * 1024]; + flowFile = session.write(flowFile, out -> out.write(oneMB)); + + claimReference.set(((FlowFileRecord) flowFile).getContentClaim()); + session.transfer(flowFile, REL_SUCCESS); + + }, REL_SUCCESS); + + connect(processor, getTerminateProcessor(), REL_SUCCESS); + triggerOnce(processor); + + final int claimCount = getContentRepository().getClaimantCount(claimReference.get()); + assertEquals(1, claimCount); + + int recordCount = ((WriteAheadFlowFileRepository) getFlowFileRepository()).checkpoint(); + assertEquals(1, recordCount); + assertTrue(claimReference.get().getResourceClaim().isInUse()); + + triggerOnce(getTerminateProcessor()); + assertFalse(claimReference.get().getResourceClaim().isInUse()); + + recordCount = ((WriteAheadFlowFileRepository) getFlowFileRepository()).checkpoint(); + assertEquals(0, recordCount); + + waitForClaimDestruction(claimReference.get()); + + assertProvenanceEventCount(ProvenanceEventType.CREATE, 1); + assertProvenanceEventCount(ProvenanceEventType.DROP, 1); + } + + + @Test + public void testCloneIncrementsContentClaim() throws ExecutionException, InterruptedException, IOException { + final AtomicReference flowFileReference = new AtomicReference<>(); + final ProcessorNode createProcessor = createGenerateProcessor(1024 * 1024, flowFileReference); + + connect(createProcessor, getTerminateProcessor(), REL_SUCCESS); + connect(createProcessor, getTerminateAllProcessor(), REL_SUCCESS); + + triggerOnce(createProcessor); + + final ContentClaim contentClaim = flowFileReference.get().getContentClaim(); + assertEquals(2, getContentRepository().getClaimantCount(contentClaim)); + assertTrue(contentClaim.getResourceClaim().isInUse()); + + triggerOnce(getTerminateProcessor()); + assertEquals(1, getContentRepository().getClaimantCount(contentClaim)); + assertTrue(contentClaim.getResourceClaim().isInUse()); + + triggerOnce(getTerminateAllProcessor()); + assertEquals(0, getContentRepository().getClaimantCount(contentClaim)); + assertFalse(contentClaim.getResourceClaim().isInUse()); + + assertEquals(0, ((WriteAheadFlowFileRepository) getFlowFileRepository()).checkpoint()); + + waitForClaimDestruction(contentClaim); + } + + + private void waitForClaimDestruction(final ContentClaim contentClaim) { + final Path path = ((FileSystemRepository) getContentRepository()).getPath(contentClaim, false); + + while (Files.exists(path)) { + try { + Thread.sleep(10L); + } catch (final Exception e) { + } + } + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/lifecycle/FlowFileRepositoryLifecycleIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/lifecycle/FlowFileRepositoryLifecycleIT.java new file mode 100644 index 0000000000..257111b123 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/lifecycle/FlowFileRepositoryLifecycleIT.java @@ -0,0 +1,147 @@ +/* + * 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.integration.lifecycle; + +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.queue.DropFlowFileState; +import org.apache.nifi.controller.queue.DropFlowFileStatus; +import org.apache.nifi.controller.queue.FlowFileQueue; +import org.apache.nifi.controller.repository.FlowFileRecord; +import org.apache.nifi.controller.repository.FlowFileSwapManager; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.integration.FrameworkIntegrationTest; +import org.apache.nifi.provenance.ProvenanceEventType; +import org.apache.nifi.reporting.Bulletin; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ExecutionException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.testng.Assert.assertNull; +import static org.testng.AssertJUnit.assertNotNull; + +public class FlowFileRepositoryLifecycleIT extends FrameworkIntegrationTest { + + @Test + public void testFlowFilesReloadedIntoQueuesUponRestart() throws ExecutionException, InterruptedException, IOException { + final ProcessorNode procNode = createProcessorNode((context, session) -> { + FlowFile flowFile = session.get(); + if (flowFile == null) { + flowFile = session.create(); + } + + flowFile = session.putAttribute(flowFile,"creator", "unit-test"); + session.transfer(flowFile, REL_SUCCESS); + }, REL_SUCCESS); + + connect(procNode, getNopProcessor(), REL_SUCCESS); + start(procNode); + final FlowFileQueue queue = getDestinationQueue(procNode, REL_SUCCESS); + + while (queue.isEmpty()) { + try { + Thread.sleep(10L); + } catch (InterruptedException e) { + } + } + + stop(procNode).get(); + + final int queueSize = queue.size().getObjectCount(); + assertTrue(queueSize > 0); + assertEquals(0L, queue.size().getByteCount()); + + assertProvenanceEventCount(ProvenanceEventType.CREATE, queueSize); + + shutdown(); + + final FlowFileQueue restoredQueue = createFlowFileQueue(queue.getIdentifier()); + initialize(() -> Collections.singleton(restoredQueue)); + + for (int i=0; i < queueSize; i++) { + final FlowFileRecord flowFileRecord = restoredQueue.poll(Collections.emptySet()); + assertNotNull(flowFileRecord); + assertEquals("unit-test", flowFileRecord.getAttribute("creator")); + } + + assertFalse(restoredQueue.isEmpty()); + assertTrue(restoredQueue.isActiveQueueEmpty()); + assertNull(restoredQueue.poll(Collections.emptySet())); + } + + + @Test + public void testMissingSwapFileOnSwapIn() throws ExecutionException, InterruptedException, IOException { + final ProcessorNode createProcessor = createProcessorNode((context, session) -> { + for (int i=0; i < 30_000; i++) { + session.transfer(session.create(), REL_SUCCESS); + } + }, REL_SUCCESS); + + connect(createProcessor, getTerminateAllProcessor(), REL_SUCCESS); + triggerOnce(createProcessor); + + // Verify queue sizes + final FlowFileQueue flowFileQueue = getDestinationQueue(createProcessor, REL_SUCCESS); + assertEquals(30_000, flowFileQueue.size().getObjectCount()); + assertEquals(10_000, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapQueueSize().getObjectCount()); + assertEquals(1, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapFileCount()); + assertEquals(20_000, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getActiveQueueSize().getObjectCount()); + + // Delete all swap files + final FlowFileSwapManager swapManager = getFlowController().createSwapManager(); + swapManager.purge(); + + // Verify queue sizes haven't changed + assertEquals(30_000, flowFileQueue.size().getObjectCount()); + assertEquals(10_000, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapQueueSize().getObjectCount()); + assertEquals(1, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapFileCount()); + assertEquals(20_000, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getActiveQueueSize().getObjectCount()); + + for (int i=0; i < 5; i++) { + triggerOnce(getTerminateAllProcessor()); + + // Verify new queue sizes + assertEquals(10_000, flowFileQueue.size().getObjectCount()); + assertEquals(10_000, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapQueueSize().getObjectCount()); + assertEquals(1, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapFileCount()); + assertEquals(0, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getActiveQueueSize().getObjectCount()); + + final List bulletins = getFlowController().getBulletinRepository().findBulletinsForController(); + assertEquals(1, bulletins.size()); + } + + assertProvenanceEventCount(ProvenanceEventType.CREATE, 30_000); + assertProvenanceEventCount(ProvenanceEventType.DROP, 20_000); + + final DropFlowFileStatus status = flowFileQueue.dropFlowFiles("unit-test-id-1", "unit-test"); + while (status.getState() != DropFlowFileState.COMPLETE) { + Thread.sleep(10L); + } + + assertEquals(0, status.getDroppedSize().getObjectCount()); + assertEquals(10_000, flowFileQueue.size().getObjectCount()); + assertEquals(10_000, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapQueueSize().getObjectCount()); + assertEquals(1, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getSwapFileCount()); + assertEquals(0, flowFileQueue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getActiveQueueSize().getObjectCount()); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processgroup/StandardProcessGroupIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processgroup/StandardProcessGroupIT.java new file mode 100644 index 0000000000..15dc5a8a8e --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processgroup/StandardProcessGroupIT.java @@ -0,0 +1,100 @@ +/* + * 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.integration.processgroup; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyDescriptor.Builder; +import org.apache.nifi.controller.ComponentNode; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.expression.ExpressionLanguageScope; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.integration.FrameworkIntegrationTest; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; +import java.util.Set; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class StandardProcessGroupIT extends FrameworkIntegrationTest { + + @Test + public void testComponentsAffectedByVariableOverridden() { + final ProcessGroup child = getFlowController().getFlowManager().createProcessGroup("child"); + child.setName("Child"); + child.setVariables(Collections.singletonMap("number", "5")); + + getRootGroup().setVariables(Collections.singletonMap("number", "1")); + getRootGroup().addProcessGroup(child); + + final ProcessorNode processor = createProcessorNode(NumberRefProcessor.class); + processor.setProperties(Collections.singletonMap(NumberRefProcessor.NUMBER.getName(), "${number}")); + moveProcessor(processor, child); + + final Set componentsAffected = child.getComponentsAffectedByVariable("number"); + assertEquals(1, componentsAffected.size()); + assertTrue(componentsAffected.contains(processor)); + + final Set rootAffected = getRootGroup().getComponentsAffectedByVariable("number"); + assertTrue(rootAffected.isEmpty()); + + processor.setScheduldingPeriod("1 hour"); + child.startProcessor(processor, false); + + getRootGroup().setVariables(Collections.singletonMap("number", "2")); + + try { + child.setVariables(Collections.singletonMap("number", "10")); + Assert.fail("Updated variable that is referenced by a running processor"); + } catch (final IllegalStateException ise) { + // Expected + } + + child.stopProcessor(processor); + } + + + public static class NumberRefProcessor extends AbstractProcessor { + static final PropertyDescriptor NUMBER = new Builder() + .name("Number") + .displayName("Number") + .description("A Number") + .required(true) + .addValidator(StandardValidators.NUMBER_VALIDATOR) + .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) + .defaultValue("1") + .build(); + + @Override + protected List getSupportedPropertyDescriptors() { + return Collections.singletonList(NUMBER); + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + + } + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processor/BiConsumerProcessor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processor/BiConsumerProcessor.java new file mode 100644 index 0000000000..2fe6029dd5 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/processor/BiConsumerProcessor.java @@ -0,0 +1,57 @@ +/* + * 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.integration.processor; + +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; + +import java.util.Set; +import java.util.function.BiConsumer; + +public class BiConsumerProcessor extends AbstractProcessor { + private BiConsumer trigger; + private Set relationships; + + public void setTrigger(final BiConsumer trigger) { + this.trigger = trigger; + } + + public void setRelationships(final Set relationships) { + this.relationships = relationships; + } + + @Override + public Set getRelationships() { + if (relationships == null) { + throw new IllegalStateException("Relationships have not been initialized"); + } + + return relationships; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + if (trigger == null) { + throw new IllegalStateException("Trigger has not been initialized"); + } + + trigger.accept(context, session); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/provenance/ProvenanceEventsIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/provenance/ProvenanceEventsIT.java new file mode 100644 index 0000000000..eb710fbd84 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/provenance/ProvenanceEventsIT.java @@ -0,0 +1,423 @@ +/* + * 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.integration.provenance; + +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.integration.FrameworkIntegrationTest; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.ProvenanceEventRepository; +import org.apache.nifi.provenance.ProvenanceEventType; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; + +import static junit.framework.TestCase.assertTrue; +import static org.junit.Assert.assertEquals; +import static org.testng.Assert.assertNull; + +public class ProvenanceEventsIT extends FrameworkIntegrationTest { + + @Test + public void testCreateEventIfNewFlowFileWithoutReceive() throws ExecutionException, InterruptedException, IOException { + final ProcessorNode createProcessor = createProcessorNode((context, session) -> { + FlowFile flowFile = session.create(); + + final Map attrs = new HashMap<>(); + attrs.put("test", "integration"); + attrs.put("integration", "true"); + + flowFile = session.putAllAttributes(flowFile, attrs); + session.transfer(flowFile, REL_SUCCESS); + }, REL_SUCCESS); + + connect(createProcessor, getTerminateAllProcessor(), REL_SUCCESS); + triggerOnce(createProcessor); + + // There should be exactly 1 event. + final ProvenanceEventRepository provRepo = getProvenanceRepository(); + assertEquals(0L, provRepo.getMaxEventId().longValue()); + + final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L); + assertEquals(ProvenanceEventType.CREATE, firstEvent.getEventType()); + assertEquals("integration", firstEvent.getAttribute("test")); + assertEquals("true", firstEvent.getAttribute("integration")); + } + + @Test + public void testNoCreateEventIfReceiveExplicitlyCreated() throws ExecutionException, InterruptedException, IOException { + final ProcessorNode createProcessor = createProcessorNode((context, session) -> { + FlowFile flowFile = session.create(); + + final Map attrs = new HashMap<>(); + attrs.put("test", "integration"); + attrs.put("integration", "true"); + + flowFile = session.putAllAttributes(flowFile, attrs); + session.getProvenanceReporter().receive(flowFile, "nifi://unit.test"); + session.transfer(flowFile, REL_SUCCESS); + }, REL_SUCCESS); + + connect(createProcessor, getTerminateAllProcessor(), REL_SUCCESS); + triggerOnce(createProcessor); + + // There should be exactly 1 event. + final ProvenanceEventRepository provRepo = getProvenanceRepository(); + assertEquals(0L, provRepo.getMaxEventId().longValue()); + + final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L); + assertEquals(ProvenanceEventType.RECEIVE, firstEvent.getEventType()); + assertEquals("integration", firstEvent.getAttribute("test")); + assertEquals("true", firstEvent.getAttribute("integration")); + assertEquals("nifi://unit.test", firstEvent.getTransitUri()); + } + + @Test + public void testDropEventIfRoutedToAutoTerminatedRelationship() throws ExecutionException, InterruptedException, IOException { + final ProcessorNode createProcessor = createProcessorNode((context, session) -> { + FlowFile flowFile = session.create(); + + final Map attrs = new HashMap<>(); + attrs.put("test", "integration"); + attrs.put("integration", "true"); + + flowFile = session.putAllAttributes(flowFile, attrs); + session.transfer(flowFile, REL_SUCCESS); + }, REL_SUCCESS); + + createProcessor.setAutoTerminatedRelationships(Collections.singleton(REL_SUCCESS)); + + triggerOnce(createProcessor); + + // There should be exactly 1 event. + final ProvenanceEventRepository provRepo = getProvenanceRepository(); + assertEquals(1L, provRepo.getMaxEventId().longValue()); + + final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L); + assertEquals(ProvenanceEventType.CREATE, firstEvent.getEventType()); + assertEquals("integration", firstEvent.getAttribute("test")); + assertEquals("true", firstEvent.getAttribute("integration")); + + final ProvenanceEventRecord secondEvent = provRepo.getEvent(1L); + assertEquals(ProvenanceEventType.DROP, secondEvent.getEventType()); + assertEquals("integration", secondEvent.getAttribute("test")); + assertEquals("true", secondEvent.getAttribute("integration")); + } + + @Test + public void testNoEventsIfExplicitlyRemoved() throws ExecutionException, InterruptedException, IOException { + final ProcessorNode createProcessor = createProcessorNode((context, session) -> { + FlowFile flowFile = session.create(); + + final Map attrs = new HashMap<>(); + attrs.put("test", "integration"); + attrs.put("integration", "true"); + + flowFile = session.putAllAttributes(flowFile, attrs); + session.remove(flowFile); + }, REL_SUCCESS); + + createProcessor.setAutoTerminatedRelationships(Collections.singleton(REL_SUCCESS)); + + triggerOnce(createProcessor); + + // There should be exactly 1 event. + final ProvenanceEventRepository provRepo = getProvenanceRepository(); + assertEquals(-1L, provRepo.getMaxEventId().longValue()); + assertTrue(provRepo.getEvents(0L, 1000).isEmpty()); + } + + @Test + public void testAttributesModifiedIfNothingElse() throws ExecutionException, InterruptedException, IOException { + final ProcessorNode createProcessor = createGenerateProcessor(0); + + final ProcessorNode updateProcessor = createProcessorNode((context, session) -> { + FlowFile flowFile = session.get(); + + final Map attrs = new HashMap<>(); + attrs.put("test", "integration"); + attrs.put("integration", "true"); + + flowFile = session.putAllAttributes(flowFile, attrs); + session.transfer(flowFile, REL_SUCCESS); + }, REL_SUCCESS); + + connect(createProcessor, updateProcessor, REL_SUCCESS); + connect(updateProcessor, getTerminateAllProcessor(), REL_SUCCESS); + + triggerOnce(createProcessor); + triggerOnce(updateProcessor); + + // There should be exactly 1 event. + final ProvenanceEventRepository provRepo = getProvenanceRepository(); + assertEquals(1L, provRepo.getMaxEventId().longValue()); + + final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L); + assertEquals(ProvenanceEventType.CREATE, firstEvent.getEventType()); + assertNull(firstEvent.getAttribute("test")); + assertNull(firstEvent.getAttribute("integration")); + + final ProvenanceEventRecord secondEvent = provRepo.getEvent(1L); + assertEquals(ProvenanceEventType.ATTRIBUTES_MODIFIED, secondEvent.getEventType()); + assertEquals("integration", secondEvent.getAttribute("test")); + assertEquals("true", secondEvent.getAttribute("integration")); + } + + @Test + public void testAttributesModifiedNotCreatedIfProcessorEmitsIt() throws ExecutionException, InterruptedException, IOException { + final ProcessorNode createProcessor = createGenerateProcessor(0); + + final ProcessorNode updateProcessor = createProcessorNode((context, session) -> { + FlowFile flowFile = session.get(); + + final Map attrs = new HashMap<>(); + attrs.put("test", "integration"); + attrs.put("integration", "true"); + + flowFile = session.putAllAttributes(flowFile, attrs); + session.getProvenanceReporter().modifyAttributes(flowFile, "Unit Test Details"); + session.transfer(flowFile, REL_SUCCESS); + }, REL_SUCCESS); + + connect(createProcessor, updateProcessor, REL_SUCCESS); + connect(updateProcessor, getTerminateAllProcessor(), REL_SUCCESS); + + triggerOnce(createProcessor); + triggerOnce(updateProcessor); + + // There should be exactly 1 event. + final ProvenanceEventRepository provRepo = getProvenanceRepository(); + assertEquals(1L, provRepo.getMaxEventId().longValue()); + + final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L); + assertEquals(ProvenanceEventType.CREATE, firstEvent.getEventType()); + assertNull(firstEvent.getAttribute("test")); + assertNull(firstEvent.getAttribute("integration")); + + final ProvenanceEventRecord secondEvent = provRepo.getEvent(1L); + assertEquals(ProvenanceEventType.ATTRIBUTES_MODIFIED, secondEvent.getEventType()); + assertEquals("integration", secondEvent.getAttribute("test")); + assertEquals("true", secondEvent.getAttribute("integration")); + assertEquals("Unit Test Details", secondEvent.getDetails()); + } + + @Test + public void testAttributesModifiedNotCreatedIfProcessorEmitsOtherEvent() throws ExecutionException, InterruptedException, IOException { + final ProcessorNode createProcessor = createGenerateProcessor(0); + + final ProcessorNode updateProcessor = createProcessorNode((context, session) -> { + FlowFile flowFile = session.get(); + + final Map attrs = new HashMap<>(); + attrs.put("test", "integration"); + attrs.put("integration", "true"); + + flowFile = session.putAllAttributes(flowFile, attrs); + session.getProvenanceReporter().fetch(flowFile, "nifi://unit.test"); + session.transfer(flowFile, REL_SUCCESS); + }, REL_SUCCESS); + + connect(createProcessor, updateProcessor, REL_SUCCESS); + connect(updateProcessor, getTerminateAllProcessor(), REL_SUCCESS); + + triggerOnce(createProcessor); + triggerOnce(updateProcessor); + + // There should be exactly 1 event. + final ProvenanceEventRepository provRepo = getProvenanceRepository(); + assertEquals(1L, provRepo.getMaxEventId().longValue()); + + final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L); + assertEquals(ProvenanceEventType.CREATE, firstEvent.getEventType()); + assertNull(firstEvent.getAttribute("test")); + assertNull(firstEvent.getAttribute("integration")); + + final ProvenanceEventRecord secondEvent = provRepo.getEvent(1L); + assertEquals(ProvenanceEventType.FETCH, secondEvent.getEventType()); + assertEquals("integration", secondEvent.getAttribute("test")); + assertEquals("true", secondEvent.getAttribute("integration")); + assertEquals("nifi://unit.test", secondEvent.getTransitUri()); + } + + + @Test + public void testAttributesModifiedNotCreatedIfContentModified() throws ExecutionException, InterruptedException, IOException { + final ProcessorNode createProcessor = createGenerateProcessor(0); + + final ProcessorNode updateProcessor = createProcessorNode((context, session) -> { + FlowFile flowFile = session.get(); + + final Map attrs = new HashMap<>(); + attrs.put("test", "integration"); + attrs.put("integration", "true"); + + flowFile = session.putAllAttributes(flowFile, attrs); + flowFile = session.write(flowFile, out -> out.write('A')); + session.transfer(flowFile, REL_SUCCESS); + }, REL_SUCCESS); + + connect(createProcessor, updateProcessor, REL_SUCCESS); + connect(updateProcessor, getTerminateAllProcessor(), REL_SUCCESS); + + triggerOnce(createProcessor); + triggerOnce(updateProcessor); + + // There should be exactly 1 event. + final ProvenanceEventRepository provRepo = getProvenanceRepository(); + assertEquals(1L, provRepo.getMaxEventId().longValue()); + + final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L); + assertEquals(ProvenanceEventType.CREATE, firstEvent.getEventType()); + assertNull(firstEvent.getAttribute("test")); + assertNull(firstEvent.getAttribute("integration")); + + final ProvenanceEventRecord secondEvent = provRepo.getEvent(1L); + assertEquals(ProvenanceEventType.CONTENT_MODIFIED, secondEvent.getEventType()); + assertEquals("integration", secondEvent.getAttribute("test")); + assertEquals("true", secondEvent.getAttribute("integration")); + } + + + @Test + public void testNoAttributesModifiedOnJoin() throws ExecutionException, InterruptedException, IOException { + testJoin(false); + } + + @Test + public void testNoAttributesModifiedOnJoinWithExplicitJoinEvent() throws ExecutionException, InterruptedException, IOException { + testJoin(true); + } + + private void testJoin(final boolean emitJoinEventExplicitly) throws ExecutionException, InterruptedException, IOException { + final ProcessorNode createProcessor = createGenerateProcessor(0); + + final ProcessorNode joinProcessor = createProcessorNode((context, session) -> { + final List originals = new ArrayList<>(); + FlowFile flowFile; + while ((flowFile = session.get()) != null) { + originals.add(flowFile); + } + + FlowFile merged = session.create(originals); + + final Map attrs = new HashMap<>(); + attrs.put("test", "integration"); + attrs.put("integration", "true"); + + merged = session.putAllAttributes(merged, attrs); + merged = session.write(merged, out -> out.write('A')); + + if (emitJoinEventExplicitly) { + session.getProvenanceReporter().join(originals, merged); + } + + session.remove(originals); + session.transfer(merged, REL_SUCCESS); + session.getProvenanceReporter().route(merged, REL_SUCCESS); + }, REL_SUCCESS); + + connect(createProcessor, joinProcessor, REL_SUCCESS); + joinProcessor.setAutoTerminatedRelationships(Collections.singleton(REL_SUCCESS)); + + for (int i=0; i < 3; i++) { + triggerOnce(createProcessor); + } + + triggerOnce(joinProcessor); + + final ProvenanceEventRepository provRepo = getProvenanceRepository(); + assertEquals(8L, provRepo.getMaxEventId().longValue()); + + // Crete events are from the first 'generate' processor. + for (int i=0; i < 3; i++) { + assertEquals(ProvenanceEventType.CREATE, provRepo.getEvent(i).getEventType()); + } + + // Any FORK/JOIN events will occur first in the Process Session to ensure that any other events that reference the FlowFile + // that is created as a result have a FlowFile to actually reference. + final ProvenanceEventRecord joinEvent = provRepo.getEvent(3); + assertEquals(ProvenanceEventType.JOIN, joinEvent.getEventType()); + assertEquals("integration", joinEvent.getAttribute("test")); + assertEquals("true", joinEvent.getAttribute("integration")); + assertEquals(3, joinEvent.getParentUuids().size()); + assertEquals(1, joinEvent.getChildUuids().size()); + assertEquals(joinEvent.getFlowFileUuid(), joinEvent.getChildUuids().get(0)); + + // Next event to occur in the Processor is the DORP event + for (int i=4; i < 7; i++) { + assertEquals(ProvenanceEventType.DROP, provRepo.getEvent(i).getEventType()); + } + + // Finally Processor will ROUTE the FlowFile + final ProvenanceEventRecord routeEvent = provRepo.getEvent(7); + assertEquals(ProvenanceEventType.ROUTE, routeEvent.getEventType()); + assertEquals("success", routeEvent.getRelationship()); + assertEquals("integration", routeEvent.getAttribute("test")); + assertEquals("true", routeEvent.getAttribute("integration")); + + // Merged FlowFile is then auto-terminated. + final ProvenanceEventRecord dropJoinedEvent = provRepo.getEvent(8); + assertEquals(ProvenanceEventType.DROP, dropJoinedEvent.getEventType()); + assertEquals("integration", dropJoinedEvent.getAttribute("test")); + assertEquals("true", dropJoinedEvent.getAttribute("integration")); + } + + @Test + public void testForkAutoGenerated() throws ExecutionException, InterruptedException, IOException { + final ProcessorNode generateProcessor = createGenerateProcessor(0); + final ProcessorNode forkProcessor = createProcessorNode((context, session) -> { + FlowFile original = session.get(); + + for (int i=0; i < 3; i++) { + FlowFile child = session.create(original); + child = session.putAttribute(child, "i", String.valueOf(i)); + session.transfer(child, REL_SUCCESS); + } + + session.remove(original); + }, REL_SUCCESS); + + connect(generateProcessor, forkProcessor, REL_SUCCESS); + connect(forkProcessor, getTerminateAllProcessor(), REL_SUCCESS); + + triggerOnce(generateProcessor); + triggerOnce(forkProcessor); + + final ProvenanceEventRepository provRepo = getProvenanceRepository(); + assertEquals(2L, provRepo.getMaxEventId().longValue()); + + final ProvenanceEventRecord firstEvent = provRepo.getEvent(0L); + assertEquals(ProvenanceEventType.CREATE, firstEvent.getEventType()); + assertNull(firstEvent.getAttribute("test")); + assertNull(firstEvent.getAttribute("integration")); + + final ProvenanceEventRecord secondEvent = provRepo.getEvent(1L); + assertEquals(ProvenanceEventType.FORK, secondEvent.getEventType()); + assertEquals(1, secondEvent.getParentUuids().size()); + assertEquals(3, secondEvent.getChildUuids().size()); + assertEquals(secondEvent.getFlowFileUuid(), secondEvent.getParentUuids().get(0)); + + final ProvenanceEventRecord thirdEvent = provRepo.getEvent(2L); + assertEquals(ProvenanceEventType.DROP, thirdEvent.getEventType()); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java new file mode 100644 index 0000000000..9320ce563d --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/integration/versioned/ImportFlowIT.java @@ -0,0 +1,163 @@ +/* + * 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.integration.versioned; + +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.validation.ValidationStatus; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.integration.DirectInjectionExtensionManager; +import org.apache.nifi.integration.FrameworkIntegrationTest; +import org.apache.nifi.integration.cs.LongValidatingControllerService; +import org.apache.nifi.integration.cs.NopServiceReferencingProcessor; +import org.apache.nifi.processor.Processor; +import org.apache.nifi.registry.bucket.Bucket; +import org.apache.nifi.registry.flow.Bundle; +import org.apache.nifi.registry.flow.VersionedControllerService; +import org.apache.nifi.registry.flow.VersionedFlow; +import org.apache.nifi.registry.flow.VersionedFlowSnapshot; +import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata; +import org.apache.nifi.registry.flow.VersionedProcessGroup; +import org.apache.nifi.registry.flow.VersionedProcessor; +import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.junit.Test; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ExecutionException; + +import static junit.framework.TestCase.assertTrue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +public class ImportFlowIT extends FrameworkIntegrationTest { + + @Override + protected void injectExtensionTypes(final DirectInjectionExtensionManager extensionManager) { + extensionManager.injectExtensionType(Processor.class, NopServiceReferencingProcessor.class); + extensionManager.injectExtensionType(ControllerService.class, LongValidatingControllerService.class); + } + + @Test + public void testImportFlowWithProcessorAndControllerService() throws ExecutionException, InterruptedException { + // Build a Versioned Flow that consists of a Controller Service and a Processor + // that references that Controller Service. + final ControllerServiceNode controllerService = createControllerServiceNode(LongValidatingControllerService.class); + controllerService.setProperties(Collections.singletonMap(LongValidatingControllerService.DELAY.getName(), "250 millis")); + + final ProcessorNode processor = createProcessorNode(NopServiceReferencingProcessor.class); + processor.setAutoTerminatedRelationships(Collections.singleton(REL_SUCCESS)); + processor.setProperties(Collections.singletonMap(NopServiceReferencingProcessor.SERVICE.getName(), controllerService.getIdentifier())); + + final VersionedFlowSnapshot proposedFlow = createFlowSnapshot(Collections.singletonList(controllerService), Collections.singletonList(processor)); + + // Create an Inner Process Group and update it to match the Versioned Flow. + final ProcessGroup innerGroup = getFlowController().getFlowManager().createProcessGroup("inner-group-id"); + innerGroup.setName("Inner Group"); + getRootGroup().addProcessGroup(innerGroup); + + innerGroup.updateFlow(proposedFlow, null, false, true, false); + + // Ensure that the controller service is valid and enable it. + final Set serviceNodes = innerGroup.findAllControllerServices(); + assertEquals(1, serviceNodes.size()); + + final ControllerServiceNode serviceNode = serviceNodes.iterator().next(); + final ValidationStatus validationStatus = serviceNode.performValidation(); + assertEquals(ValidationStatus.VALID, validationStatus); + getFlowController().getControllerServiceProvider().enableControllerService(serviceNode).get(); + assertTrue(serviceNode.isActive()); + + // Ensure that the processor is valid. + final List processorNodes = innerGroup.findAllProcessors(); + assertEquals(1, processorNodes.size()); + + final ProcessorNode procNode = processorNodes.get(0); + final ValidationStatus procValidationStatus = procNode.performValidation(); + final Collection validationErrors = procNode.getValidationErrors(); + System.out.println(validationErrors); + assertEquals(Collections.emptyList(), validationErrors); + assertEquals(ValidationStatus.VALID, procValidationStatus); + + // Ensure that the reference to the controller service was properly updated + final String referencedServiceId = procNode.getProperty(NopServiceReferencingProcessor.SERVICE); + assertEquals(serviceNode.getIdentifier(), referencedServiceId); + assertNotEquals("service-id", referencedServiceId); + } + + + private VersionedFlowSnapshot createFlowSnapshot(final List controllerServices, final List processors) { + final VersionedFlowSnapshotMetadata snapshotMetadata = new VersionedFlowSnapshotMetadata(); + snapshotMetadata.setAuthor("unit-test"); + snapshotMetadata.setBucketIdentifier("unit-test-bucket"); + snapshotMetadata.setFlowIdentifier("unit-test-flow"); + snapshotMetadata.setTimestamp(System.currentTimeMillis()); + snapshotMetadata.setVersion(1); + + final Bucket bucket = new Bucket(); + bucket.setCreatedTimestamp(System.currentTimeMillis()); + bucket.setIdentifier("unit-test-bucket"); + bucket.setName("Unit Test Bucket"); + + final VersionedFlow flow = new VersionedFlow(); + flow.setBucketIdentifier("unit-test-bucket"); + flow.setBucketName("Unit Test Bucket"); + flow.setCreatedTimestamp(System.currentTimeMillis()); + flow.setIdentifier("unit-test-flow"); + flow.setName("Unit Test Flow"); + + final BundleCoordinate coordinate = getSystemBundle().getBundleDetails().getCoordinate(); + final Bundle bundle = new Bundle(); + bundle.setArtifact(coordinate.getId()); + bundle.setGroup(coordinate.getGroup()); + bundle.setVersion(coordinate.getVersion()); + + final NiFiRegistryFlowMapper flowMapper = new NiFiRegistryFlowMapper(getExtensionManager()); + + final Set versionedProcessors = new HashSet<>(); + for (final ProcessorNode processor : processors) { + final VersionedProcessor versionedProcessor = flowMapper.mapProcessor(processor, getFlowController().getControllerServiceProvider()); + versionedProcessors.add(versionedProcessor); + } + + final Set services = new HashSet<>(); + for (final ControllerServiceNode serviceNode : controllerServices) { + final VersionedControllerService service = flowMapper.mapControllerService(serviceNode, getFlowController().getControllerServiceProvider()); + services.add(service); + } + + final VersionedProcessGroup flowContents = new VersionedProcessGroup(); + flowContents.setIdentifier("unit-test-flow-contents"); + flowContents.setName("Unit Test"); + flowContents.setProcessors(versionedProcessors); + flowContents.setControllerServices(services); + + final VersionedFlowSnapshot versionedFlowSnapshot = new VersionedFlowSnapshot(); + versionedFlowSnapshot.setSnapshotMetadata(snapshotMetadata); + versionedFlowSnapshot.setBucket(bucket); + versionedFlowSnapshot.setFlow(flow); + versionedFlowSnapshot.setFlowContents(flowContents); + + return versionedFlowSnapshot; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/int-tests/default-nifi.properties b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/int-tests/default-nifi.properties new file mode 100644 index 0000000000..1bbb49a02d --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/int-tests/default-nifi.properties @@ -0,0 +1,256 @@ +# 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. + +# Core Properties # +nifi.flow.configuration.file=./target/conf/flow.xml.gz +nifi.flow.configuration.archive.enabled=true +nifi.flow.configuration.archive.dir=./target/conf/archive/ +nifi.flow.configuration.archive.max.time=30 days +nifi.flow.configuration.archive.max.storage=500 MB +nifi.flow.configuration.archive.max.count= +nifi.flowcontroller.autoResumeState=true +nifi.flowcontroller.graceful.shutdown.period=10 sec +nifi.flowservice.writedelay.interval=500 ms +nifi.administrative.yield.duration=100 millis +# If a component has no work to do (is "bored"), how long should we wait before checking again for work? +nifi.bored.yield.duration=10 millis +nifi.queue.backpressure.count=10000 +nifi.queue.backpressure.size=1 GB + +nifi.authorizer.configuration.file=./target/conf/authorizers.xml +nifi.login.identity.provider.configuration.file=./target/conf/login-identity-providers.xml +nifi.templates.directory=./target/conf/templates +nifi.ui.banner.text= +nifi.ui.autorefresh.interval=30 sec +nifi.nar.library.directory=./target/lib +nifi.nar.library.autoload.directory=./target/extensions +nifi.nar.working.directory=./target/work/nar/ +nifi.documentation.working.directory=./target/work/docs/components + +#################### +# State Management # +#################### +nifi.state.management.configuration.file=src/test/resources/int-tests/state-management.xml +# The ID of the local state provider +nifi.state.management.provider.local=local-provider +# The ID of the cluster-wide state provider. This will be ignored if NiFi is not clustered but must be populated if running in a cluster. +nifi.state.management.provider.cluster=zk-provider +# Specifies whether or not this instance of NiFi should run an embedded ZooKeeper server +nifi.state.management.embedded.zookeeper.start=false +# Properties file that provides the ZooKeeper properties to use if is set to true +nifi.state.management.embedded.zookeeper.properties=./target/conf/zookeeper.properties + + +# H2 Settings +nifi.database.directory=./target/database_repository +nifi.h2.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE + +# FlowFile Repository +nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository +nifi.flowfile.repository.wal.implementation=org.apache.nifi.wali.SequentialAccessWriteAheadLog +nifi.flowfile.repository.directory=./target/int-tests/flowfile_repository +nifi.flowfile.repository.partitions=256 +nifi.flowfile.repository.checkpoint.interval=5 mins +nifi.flowfile.repository.always.sync=false + +nifi.swap.manager.implementation=org.apache.nifi.controller.FileSystemSwapManager +nifi.queue.swap.threshold=20000 +nifi.swap.in.period=5 sec +nifi.swap.in.threads=1 +nifi.swap.out.period=5 sec +nifi.swap.out.threads=4 + +# Content Repository +nifi.content.repository.implementation=org.apache.nifi.controller.repository.FileSystemRepository +nifi.content.claim.max.appendable.size=1 MB +nifi.content.claim.max.flow.files=100 +nifi.content.repository.directory.default=./target/int-tests/content_repository +nifi.content.repository.archive.max.retention.period=12 hours +nifi.content.repository.archive.max.usage.percentage=50% +nifi.content.repository.archive.enabled=true +nifi.content.repository.always.sync=false +nifi.content.viewer.url=../nifi-content-viewer/ + +# Provenance Repository Properties +nifi.provenance.repository.implementation=org.apache.nifi.provenance.WriteAheadProvenanceRepository +nifi.provenance.repository.debug.frequency=1_000_000 +nifi.provenance.repository.encryption.key.provider.implementation= +nifi.provenance.repository.encryption.key.provider.location= +nifi.provenance.repository.encryption.key.id= +nifi.provenance.repository.encryption.key= + +# Persistent Provenance Repository Properties +nifi.provenance.repository.directory.default=./target/int-tests/provenance_repository +nifi.provenance.repository.max.storage.time=24 hours +nifi.provenance.repository.max.storage.size=1 GB +nifi.provenance.repository.rollover.time=30 secs +nifi.provenance.repository.rollover.size=100 MB +nifi.provenance.repository.query.threads=2 +nifi.provenance.repository.index.threads=2 +nifi.provenance.repository.compress.on.rollover=true +nifi.provenance.repository.always.sync=false +# Comma-separated list of fields. Fields that are not indexed will not be searchable. Valid fields are: +# EventType, FlowFileUUID, Filename, TransitURI, ProcessorID, AlternateIdentifierURI, Relationship, Details +nifi.provenance.repository.indexed.fields=EventType, FlowFileUUID, Filename, ProcessorID, Relationship +# FlowFile Attributes that should be indexed and made searchable. Some examples to consider are filename, uuid, mime.type +nifi.provenance.repository.indexed.attributes= +# Large values for the shard size will result in more Java heap usage when searching the Provenance Repository +# but should provide better performance +nifi.provenance.repository.index.shard.size=500 MB +# Indicates the maximum length that a FlowFile attribute can be when retrieving a Provenance Event from +# the repository. If the length of any attribute exceeds this value, it will be truncated when the event is retrieved. +nifi.provenance.repository.max.attribute.length=65536 +nifi.provenance.repository.concurrent.merge.threads=2 + + +# Volatile Provenance Respository Properties +nifi.provenance.repository.buffer.size=100000 + +# Component Status Repository +nifi.components.status.repository.implementation=org.apache.nifi.controller.status.history.VolatileComponentStatusRepository +nifi.components.status.repository.buffer.size=1440 +nifi.components.status.snapshot.frequency=1 min + +# Site to Site properties +nifi.remote.input.host= +nifi.remote.input.secure=false +nifi.remote.input.socket.port= +nifi.remote.input.http.enabled=true +nifi.remote.input.http.transaction.ttl=30 sec +nifi.remote.contents.cache.expiration=30 secs + +# web properties # +nifi.web.war.directory=./target/lib +nifi.web.http.host= +nifi.web.http.port=8080 +nifi.web.http.network.interface.default= +nifi.web.https.host= +nifi.web.https.port= +nifi.web.https.network.interface.default= +nifi.web.jetty.working.directory=./target/work/jetty +nifi.web.jetty.threads=200 +nifi.web.max.header.size=16 KB +nifi.web.proxy.context.path= +nifi.web.proxy.host= + +# security properties # +nifi.sensitive.props.key= +nifi.sensitive.props.key.protected= +nifi.sensitive.props.algorithm=PBEWITHMD5AND256BITAES-CBC-OPENSSL +nifi.sensitive.props.provider=BC +nifi.sensitive.props.additional.keys= + +nifi.security.keystore= +nifi.security.keystoreType= +nifi.security.keystorePasswd= +nifi.security.keyPasswd= +nifi.security.truststore= +nifi.security.truststoreType= +nifi.security.truststorePasswd= +nifi.security.user.authorizer=managed-authorizer +nifi.security.user.login.identity.provider= +nifi.security.ocsp.responder.url= +nifi.security.ocsp.responder.certificate= + +# OpenId Connect SSO Properties # +nifi.security.user.oidc.discovery.url= +nifi.security.user.oidc.connect.timeout=5 secs +nifi.security.user.oidc.read.timeout=5 secs +nifi.security.user.oidc.client.id= +nifi.security.user.oidc.client.secret= +nifi.security.user.oidc.preferred.jwsalgorithm= + +# Apache Knox SSO Properties # +nifi.security.user.knox.url= +nifi.security.user.knox.publicKey= +nifi.security.user.knox.cookieName=hadoop-jwt +nifi.security.user.knox.audiences= + +# Identity Mapping Properties # +# These properties allow normalizing user identities such that identities coming from different identity providers +# (certificates, LDAP, Kerberos) can be treated the same internally in NiFi. The following example demonstrates normalizing +# DNs from certificates and principals from Kerberos into a common identity string: +# +# nifi.security.identity.mapping.pattern.dn=^CN=(.*?), OU=(.*?), O=(.*?), L=(.*?), ST=(.*?), C=(.*?)$ +# nifi.security.identity.mapping.value.dn=$1@$2 +# nifi.security.identity.mapping.transform.dn=NONE +# nifi.security.identity.mapping.pattern.kerb=^(.*?)/instance@(.*?)$ +# nifi.security.identity.mapping.value.kerb=$1@$2 +# nifi.security.identity.mapping.transform.kerb=UPPER + +# Group Mapping Properties # +# These properties allow normalizing group names coming from external sources like LDAP. The following example +# lowercases any group name. +# +# nifi.security.group.mapping.pattern.anygroup=^(.*)$ +# nifi.security.group.mapping.value.anygroup=$1 +# nifi.security.group.mapping.transform.anygroup=LOWER + +# cluster common properties (all nodes must have same values) # +nifi.cluster.protocol.heartbeat.interval=5 sec +nifi.cluster.protocol.is.secure=false + +# cluster node properties (only configure for cluster nodes) # +nifi.cluster.is.node=false +nifi.cluster.node.address= +nifi.cluster.node.protocol.port= +nifi.cluster.node.protocol.threads=10 +nifi.cluster.node.protocol.max.threads=50 +nifi.cluster.node.event.history.size=25 +nifi.cluster.node.connection.timeout=5 sec +nifi.cluster.node.read.timeout=5 sec +nifi.cluster.node.max.concurrent.requests=100 +nifi.cluster.firewall.file= +nifi.cluster.flow.election.max.wait.time=5 mins +nifi.cluster.flow.election.max.candidates= + +# cluster load balancing properties # +nifi.cluster.load.balance.host= +nifi.cluster.load.balance.port=6342 +nifi.cluster.load.balance.connections.per.node=4 +nifi.cluster.load.balance.max.thread.count=8 +nifi.cluster.load.balance.comms.timeout=30 sec + +# zookeeper properties, used for cluster management # +nifi.zookeeper.connect.string= +nifi.zookeeper.connect.timeout=3 secs +nifi.zookeeper.session.timeout=3 secs +nifi.zookeeper.root.node=/nifi + +# Zookeeper properties for the authentication scheme used when creating acls on znodes used for cluster management +# Values supported for nifi.zookeeper.auth.type are "default", which will apply world/anyone rights on znodes +# and "sasl" which will give rights to the sasl/kerberos identity used to authenticate the nifi node +# The identity is determined using the value in nifi.kerberos.service.principal and the removeHostFromPrincipal +# and removeRealmFromPrincipal values (which should align with the kerberos.removeHostFromPrincipal and kerberos.removeRealmFromPrincipal +# values configured on the zookeeper server). +nifi.zookeeper.auth.type= +nifi.zookeeper.kerberos.removeHostFromPrincipal= +nifi.zookeeper.kerberos.removeRealmFromPrincipal= + +# kerberos # +nifi.kerberos.krb5.file= + +# kerberos service principal # +nifi.kerberos.service.principal= +nifi.kerberos.service.keytab.location= + +# kerberos spnego principal # +nifi.kerberos.spnego.principal= +nifi.kerberos.spnego.keytab.location= +nifi.kerberos.spnego.authentication.expiration=12 hours + +# external properties files for variable registry +# supports a comma delimited list of file locations +nifi.variable.registry.properties= \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/int-tests/state-management.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/int-tests/state-management.xml new file mode 100644 index 0000000000..f6b9768b16 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/int-tests/state-management.xml @@ -0,0 +1,24 @@ + + + + local-provider + org.apache.nifi.controller.state.providers.local.WriteAheadLocalStateProvider + ./target/int-tests/state/local + false + 16 + 2 mins + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml index f6cb8c5465..d8801cefd9 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/logback-test.xml @@ -30,6 +30,7 @@ + " diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java index 8567f32b58..adadea116a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java @@ -151,37 +151,7 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering final ServiceLoader serviceLoader = ServiceLoader.load(entry.getKey(), bundle.getClassLoader()); for (final Object o : serviceLoader) { try { - // create a cache of temp ConfigurableComponent instances, the initialize here has to happen before the checks below - if ((isControllerService || isProcessor || isReportingTask) && o instanceof ConfigurableComponent) { - final ConfigurableComponent configurableComponent = (ConfigurableComponent) o; - initializeTempComponent(configurableComponent); - - final String cacheKey = getClassBundleKey(o.getClass().getCanonicalName(), bundle.getBundleDetails().getCoordinate()); - tempComponentLookup.put(cacheKey, (ConfigurableComponent) o); - } - - // only consider extensions discovered directly in this bundle - boolean registerExtension = bundle.getClassLoader().equals(o.getClass().getClassLoader()); - - if (registerExtension) { - final Class extensionType = o.getClass(); - if (isControllerService && !checkControllerServiceEligibility(extensionType)) { - registerExtension = false; - logger.error(String.format( - "Skipping Controller Service %s because it is bundled with its supporting APIs and requires instance class loading.", extensionType.getName())); - } - - final boolean canReferenceControllerService = (isControllerService || isProcessor || isReportingTask) && o instanceof ConfigurableComponent; - if (canReferenceControllerService && !checkControllerServiceReferenceEligibility((ConfigurableComponent) o, bundle.getClassLoader())) { - registerExtension = false; - logger.error(String.format( - "Skipping component %s because it is bundled with its referenced Controller Service APIs and requires instance class loading.", extensionType.getName())); - } - - if (registerExtension) { - registerServiceClass(o.getClass(), classNameBundleLookup, bundleCoordinateClassesLookup, bundle, entry.getValue()); - } - } + loadExtension(o, entry.getKey(), bundle); } catch (Exception e) { logger.warn("Failed to register extension {} due to: {}" , new Object[]{o.getClass().getCanonicalName(), e.getMessage()}); if (logger.isDebugEnabled()) { @@ -194,6 +164,50 @@ public class StandardExtensionDiscoveringManager implements ExtensionDiscovering } } + protected void loadExtension(final Object extension, final Class extensionType, final Bundle bundle) { + final boolean isControllerService = ControllerService.class.equals(extensionType); + final boolean isProcessor = Processor.class.equals(extensionType); + final boolean isReportingTask = ReportingTask.class.equals(extensionType); + + // create a cache of temp ConfigurableComponent instances, the initialize here has to happen before the checks below + if ((isControllerService || isProcessor || isReportingTask) && extension instanceof ConfigurableComponent) { + final ConfigurableComponent configurableComponent = (ConfigurableComponent) extension; + initializeTempComponent(configurableComponent); + + final String cacheKey = getClassBundleKey(extension.getClass().getCanonicalName(), bundle.getBundleDetails().getCoordinate()); + tempComponentLookup.put(cacheKey, configurableComponent); + } + + // only consider extensions discovered directly in this bundle + boolean registerExtension = bundle.getClassLoader().equals(extension.getClass().getClassLoader()); + + if (registerExtension) { + final Class extensionClass = extension.getClass(); + if (isControllerService && !checkControllerServiceEligibility(extensionClass)) { + registerExtension = false; + logger.error(String.format( + "Skipping Controller Service %s because it is bundled with its supporting APIs and requires instance class loading.", extensionClass.getName())); + } + + final boolean canReferenceControllerService = (isControllerService || isProcessor || isReportingTask) && extension instanceof ConfigurableComponent; + if (canReferenceControllerService && !checkControllerServiceReferenceEligibility((ConfigurableComponent) extension, bundle.getClassLoader())) { + registerExtension = false; + logger.error(String.format( + "Skipping component %s because it is bundled with its referenced Controller Service APIs and requires instance class loading.", extensionClass.getName())); + } + + if (registerExtension) { + registerExtensionClass(extensionType, extension.getClass(), bundle); + } + } + } + + protected void registerExtensionClass(final Class extensionType, final Class implementationClass, final Bundle bundle) { + final Set registeredClasses = definitionMap.get(extensionType); + registerServiceClass(implementationClass, classNameBundleLookup, bundleCoordinateClassesLookup, bundle, registeredClasses); + } + + private void initializeTempComponent(final ConfigurableComponent configurableComponent) { ConfigurableComponentInitializer initializer = null; try { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java index 39a25912a3..7a3e24f1ab 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java @@ -70,6 +70,10 @@ public final class StandardContentClaim implements ContentClaim, Comparable { int originalSize = transactions.size(); logger.trace("Transaction maintenance task started."); try { - Set transactionIds = transactions.keySet().stream().collect(Collectors.toSet()); - transactionIds.stream().filter(tid -> !isTransactionActive(tid)) - .forEach(tid -> cancelTransaction(tid)); + for (final String transactionId : transactions.keySet()) { + if (!isTransactionActive(transactionId)) { + cancelTransaction(transactionId); + } + } } catch (Exception e) { // Swallow exception so that this thread can keep working. logger.error("An exception occurred while maintaining transactions", e); @@ -168,6 +167,12 @@ public class HttpRemoteSiteListener implements RemoteSiteListener { } } + @Override + public void destroy() { + stop(); + instance = null; + } + public String createTransaction() { final String transactionId = UUID.randomUUID().toString(); transactions.put(transactionId, new TransactionWrapper(null, null)); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteSiteListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteSiteListener.java index 1183fc5527..888f9ead13 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteSiteListener.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/RemoteSiteListener.java @@ -16,10 +16,10 @@ */ package org.apache.nifi.remote; -import java.io.IOException; - import org.apache.nifi.groups.ProcessGroup; +import java.io.IOException; + public interface RemoteSiteListener { void setRootGroup(ProcessGroup rootGroup); @@ -27,4 +27,6 @@ public interface RemoteSiteListener { void start() throws IOException; void stop(); + + void destroy(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java index 07c5920edc..f864ab79ed 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/SocketRemoteSiteListener.java @@ -57,8 +57,6 @@ import java.util.concurrent.atomic.AtomicReference; public class SocketRemoteSiteListener implements RemoteSiteListener { - public static final String DEFAULT_FLOWFILE_PATH = "./"; - private final int socketPort; private final SSLContext sslContext; private final NodeInformant nodeInformant; @@ -375,6 +373,10 @@ public class SocketRemoteSiteListener implements RemoteSiteListener { stopped.set(true); } + @Override + public void destroy() { + } + private void verifyMagicBytes(final InputStream in, final String peerDescription) throws IOException, HandshakeException { final byte[] receivedMagicBytes = new byte[CommunicationsSession.MAGIC_BYTES.length]; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java index c2a180a902..20669f6277 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java @@ -361,7 +361,7 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou final ProcessGroup group = locateProcessGroup(flowController, groupId); group.updateFlow(proposedSnapshot, componentIdSeed, verifyNotModified, updateSettings, updateDescendantVersionedFlows); - group.findAllRemoteProcessGroups().stream().forEach(RemoteProcessGroup::initialize); + group.findAllRemoteProcessGroups().forEach(RemoteProcessGroup::initialize); final StandardVersionControlInformation svci = StandardVersionControlInformation.Builder.fromDto(versionControlInformation) .flowSnapshot(proposedSnapshot.getFlowContents()) diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/EventIndexTask.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/EventIndexTask.java index f8bbd3be22..df752b8b30 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/EventIndexTask.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/EventIndexTask.java @@ -17,16 +17,6 @@ package org.apache.nifi.provenance.index.lucene; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; - import org.apache.lucene.document.Document; import org.apache.lucene.search.NumericRangeQuery; import org.apache.nifi.events.EventReporter; @@ -38,6 +28,18 @@ import org.apache.nifi.reporting.Severity; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + public class EventIndexTask implements Runnable { private static final Logger logger = LoggerFactory.getLogger(EventIndexTask.class); private static final String EVENT_CATEGORY = "Provenance Repository"; @@ -52,6 +54,8 @@ public class EventIndexTask implements Runnable { private final EventReporter eventReporter; private final int commitThreshold; + private volatile CompletableFuture shutdownComplete; + public EventIndexTask(final BlockingQueue documentQueue, final RepositoryConfiguration repoConfig, final IndexManager indexManager, final IndexDirectoryManager directoryManager, final int maxEventsPerCommit, final EventReporter eventReporter) { this.documentQueue = documentQueue; @@ -61,8 +65,13 @@ public class EventIndexTask implements Runnable { this.eventReporter = eventReporter; } - public void shutdown() { + public synchronized Future shutdown() { + if (shutdownComplete == null) { + shutdownComplete = new CompletableFuture<>(); + } + this.shutdown = true; + return shutdownComplete; } private void fetchDocuments(final List destination) throws InterruptedException { @@ -108,6 +117,11 @@ public class EventIndexTask implements Runnable { eventReporter.reportEvent(Severity.ERROR, EVENT_CATEGORY, "Failed to index Provenance Events. See logs for more information."); } } + + final CompletableFuture future = this.shutdownComplete; + if (future != null) { + future.complete(null); + } } @@ -119,7 +133,7 @@ public class EventIndexTask implements Runnable { return; } - final Map> docsByIndexDir = toIndex.stream().collect(Collectors.groupingBy(doc -> doc.getIndexDirectory())); + final Map> docsByIndexDir = toIndex.stream().collect(Collectors.groupingBy(IndexableDocument::getIndexDirectory)); for (final Map.Entry> entry : docsByIndexDir.entrySet()) { final File indexDirectory = entry.getKey(); final List documentsForIndex = entry.getValue(); @@ -145,7 +159,7 @@ public class EventIndexTask implements Runnable { indexWriter.getIndexWriter().deleteDocuments(query); final List documents = documentsForIndex.stream() - .map(doc -> doc.getDocument()) + .map(IndexableDocument::getDocument) .collect(Collectors.toList()); indexWriter.index(documents, commitThreshold); diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java index d1a9714831..87b57c5995 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/index/lucene/LuceneEventIndex.java @@ -17,25 +17,6 @@ package org.apache.nifi.provenance.index.lucene; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Date; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; import org.apache.lucene.document.Document; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.Term; @@ -75,6 +56,27 @@ import org.apache.nifi.util.timebuffer.TimestampedLong; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Date; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + public class LuceneEventIndex implements EventIndex { private static final Logger logger = LoggerFactory.getLogger(LuceneEventIndex.class); @@ -188,9 +190,21 @@ public class LuceneEventIndex implements EventIndex { maintenanceExecutor.shutdown(); } + final List> futures = new ArrayList<>(); for (final EventIndexTask task : indexTasks) { - task.shutdown(); + futures.add(task.shutdown()); } + + // Wait for all tasks to complete before returning + for (final Future future : futures) { + try { + future.get(); + } catch (final Exception e) { + logger.error("Failed to shutdown Index Task", e); + } + } + + indexManager.close(); } long getMaxEventId(final String partitionName) { diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java index d59a81d818..7dd626bdfb 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/lucene/SimpleIndexManager.java @@ -17,17 +17,6 @@ package org.apache.nifi.provenance.lucene; -import java.io.Closeable; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; - import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.standard.StandardAnalyzer; import org.apache.lucene.index.ConcurrentMergeScheduler; @@ -44,6 +33,19 @@ import org.apache.nifi.provenance.util.NamedThreadFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + public class SimpleIndexManager implements IndexManager { private static final Logger logger = LoggerFactory.getLogger(SimpleIndexManager.class); @@ -69,6 +71,20 @@ public class SimpleIndexManager implements IndexManager { Thread.currentThread().interrupt(); searchExecutor.shutdownNow(); } + + synchronized (writerCounts) { + final Set closed = new HashSet<>(); + + for (final Map.Entry entry : writerCounts.entrySet()) { + final IndexWriterCount count = entry.getValue(); + if (count.getCount() < 1) { + count.close(); + closed.add(entry.getKey()); + } + } + + closed.forEach(writerCounts::remove); + } } @Override