From 5ffc024311c6cc635a189840de5f038cba24b536 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Fri, 2 Feb 2018 12:16:36 -0500 Subject: [PATCH 1/2] NIFI-4849: Implemented REST Endpoint and associated backend code to generate a Diagnostics Report for a Processor --- .../nifi/controller/queue/FlowFileQueue.java | 10 + .../repository/ContentRepository.java | 9 + .../repository/FlowFileRepository.java | 8 + .../history/ComponentStatusRepository.java | 13 +- .../history/GarbageCollectionHistory.java | 27 ++ .../history/GarbageCollectionStatus.java | 42 ++ .../nifi/provenance/ProvenanceRepository.java | 9 + .../provenance/MockProvenanceRepository.java | 7 +- .../ClassLoaderDiagnosticsDTO.java | 60 +++ .../diagnostics/ConnectionDiagnosticsDTO.java | 148 +++++++ .../ControllerServiceDiagnosticsDTO.java | 48 +++ .../diagnostics/GCDiagnosticsSnapshotDTO.java | 67 +++ .../GarbageCollectionDiagnosticsDTO.java | 61 +++ .../dto/diagnostics/JVMDiagnosticsDTO.java | 68 ++++ .../JVMDiagnosticsSnapshotDTO.java | 382 ++++++++++++++++++ .../NodeGCDiagnosticsSnapshotDTO.java | 67 +++ .../NodeJVMDiagnosticsSnapshotDTO.java | 68 ++++ .../diagnostics/ProcessorDiagnosticsDTO.java | 113 ++++++ .../dto/diagnostics/RepositoryUsageDTO.java | 111 +++++ .../api/dto/diagnostics/ThreadDumpDTO.java | 86 ++++ .../entity/ProcessorDiagnosticsEntity.java | 40 ++ .../http/StandardHttpResponseMapper.java | 2 + .../ProcessorDiagnosticsEndpointMerger.java | 65 +++ .../ProcessorDiagnosticsEntityMerger.java | 119 ++++++ .../nifi/cluster/manager/StatusMerger.java | 107 ++++- .../nifi/controller/ActiveThreadInfo.java | 42 ++ .../apache/nifi/controller/ProcessorNode.java | 3 + .../nifi/controller/FlowController.java | 51 ++- .../controller/StandardFlowFileQueue.java | 47 ++- .../controller/StandardProcessorNode.java | 68 +++- .../repository/FileSystemRepository.java | 10 + .../repository/VolatileContentRepository.java | 5 + .../VolatileFlowFileRepository.java | 5 + .../WriteAheadFlowFileRepository.java | 11 + .../StandardGarbageCollectionHistory.java | 50 +++ .../StandardGarbageCollectionStatus.java | 54 +++ .../VolatileComponentStatusRepository.java | 52 ++- .../nifi/diagnostics/SystemDiagnostics.java | 31 ++ .../diagnostics/SystemDiagnosticsFactory.java | 25 ++ .../org/apache/nifi/util/ThreadUtils.java | 90 +++++ .../TestStandardProcessSession.java | 10 + .../TestWriteAheadFlowFileRepository.java | 25 ++ .../org/apache/nifi/nar/ExtensionManager.java | 6 + .../apache/nifi/web/NiFiServiceFacade.java | 9 + .../nifi/web/StandardNiFiServiceFacade.java | 126 ++++++ .../nifi/web/api/ProcessorResource.java | 54 +++ .../apache/nifi/web/api/dto/DtoFactory.java | 327 ++++++++++++++- .../nifi/web/api/dto/EntityFactory.java | 23 ++ .../nifi/web/controller/ControllerFacade.java | 16 +- .../PersistentProvenanceRepository.java | 17 +- .../WriteAheadProvenanceRepository.java | 16 + .../VolatileProvenanceRepository.java | 5 + 52 files changed, 2883 insertions(+), 32 deletions(-) create mode 100644 nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/GarbageCollectionHistory.java create mode 100644 nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/GarbageCollectionStatus.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ClassLoaderDiagnosticsDTO.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ConnectionDiagnosticsDTO.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ControllerServiceDiagnosticsDTO.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/GCDiagnosticsSnapshotDTO.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/GarbageCollectionDiagnosticsDTO.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/JVMDiagnosticsDTO.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/JVMDiagnosticsSnapshotDTO.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/NodeGCDiagnosticsSnapshotDTO.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/NodeJVMDiagnosticsSnapshotDTO.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ProcessorDiagnosticsDTO.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/RepositoryUsageDTO.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ThreadDumpDTO.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorDiagnosticsEntity.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProcessorDiagnosticsEndpointMerger.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ProcessorDiagnosticsEntityMerger.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ActiveThreadInfo.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StandardGarbageCollectionHistory.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StandardGarbageCollectionStatus.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/ThreadUtils.java diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/FlowFileQueue.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/FlowFileQueue.java index 7948ecbc3357..9e637b01da42 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/FlowFileQueue.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/FlowFileQueue.java @@ -59,6 +59,8 @@ public interface FlowFileQueue { */ void purgeSwapFiles(); + int getSwapFileCount(); + /** * Resets the comparator used by this queue to maintain order. * @@ -115,6 +117,10 @@ public interface FlowFileQueue { */ QueueSize getUnacknowledgedQueueSize(); + QueueSize getActiveQueueSize(); + + QueueSize getSwapQueueSize(); + void acknowledge(FlowFileRecord flowFile); void acknowledge(Collection flowFiles); @@ -125,6 +131,10 @@ public interface FlowFileQueue { */ boolean isFull(); + boolean isAnyActiveFlowFilePenalized(); + + boolean isAllActiveFlowFilesPenalized(); + /** * places the given file into the queue * diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/ContentRepository.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/ContentRepository.java index b1ea87cf4240..d74948279d33 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/ContentRepository.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/ContentRepository.java @@ -76,6 +76,15 @@ public interface ContentRepository { */ long getContainerUsableSpace(String containerName) throws IOException; + /** + * Returns the name of the FileStore that the given container is stored on, or null + * if not applicable or unable to determine the file store name + * + * @param containerName the name of the container + * @return the name of the FileStore + */ + String getContainerFileStoreName(String containerName); + /** * Creates a new content claim * diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/FlowFileRepository.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/FlowFileRepository.java index 276d3b7c2393..6560c0aeb83c 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/FlowFileRepository.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/FlowFileRepository.java @@ -56,6 +56,14 @@ public interface FlowFileRepository extends Closeable { */ long getUsableStorageSpace() throws IOException; + /** + * Returns the name of the FileStore that the repository is stored on, or null + * if not applicable or unable to determine the file store name + * + * @return the name of the FileStore + */ + String getFileStoreName(); + /** * Updates the repository with the given RepositoryRecords. * diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java index 553903f9869b..98d4863d16c1 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/ComponentStatusRepository.java @@ -16,9 +16,10 @@ */ package org.apache.nifi.controller.status.history; -import org.apache.nifi.controller.status.ProcessGroupStatus; - import java.util.Date; +import java.util.List; + +import org.apache.nifi.controller.status.ProcessGroupStatus; /** * A repository for storing and retrieving components' historical status @@ -38,8 +39,9 @@ public interface ComponentStatusRepository { * Captures the status information provided in the given report * * @param rootGroupStatus status of root group + * @param garbageCollectionStatus status of garbage collection */ - void capture(ProcessGroupStatus rootGroupStatus); + void capture(ProcessGroupStatus rootGroupStatus, List garbageCollectionStatus); /** * Captures the status information provided in the given report, providing a @@ -48,8 +50,9 @@ public interface ComponentStatusRepository { * * @param rootGroupStatus status * @param timestamp timestamp of capture + * @param garbageCollectionStatus status of garbage collection */ - void capture(ProcessGroupStatus rootGroupStatus, Date timestamp); + void capture(ProcessGroupStatus rootGroupStatus, List garbageCollectionStatus, Date timestamp); /** * @return the Date at which the latest capture was performed @@ -127,4 +130,6 @@ public interface ComponentStatusRepository { * period */ StatusHistory getRemoteProcessGroupStatusHistory(String remoteGroupId, Date start, Date end, int preferredDataPoints); + + GarbageCollectionHistory getGarbageCollectionHistory(Date start, Date end); } diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/GarbageCollectionHistory.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/GarbageCollectionHistory.java new file mode 100644 index 000000000000..3f64b09156ce --- /dev/null +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/GarbageCollectionHistory.java @@ -0,0 +1,27 @@ +/* + * 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.status.history; + +import java.util.List; +import java.util.Set; + +public interface GarbageCollectionHistory { + Set getMemoryManagerNames(); + + List getGarbageCollectionStatuses(String memoryManagerName); +} diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/GarbageCollectionStatus.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/GarbageCollectionStatus.java new file mode 100644 index 000000000000..8ee65066840e --- /dev/null +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/status/history/GarbageCollectionStatus.java @@ -0,0 +1,42 @@ +/* + * 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.status.history; + +import java.util.Date; + +public interface GarbageCollectionStatus { + /** + * @return the name of the JVM memory manager that this status pertains to + */ + String getMemoryManagerName(); + + /** + * @return timestamp of when the status was created + */ + Date getTimestamp(); + + /** + * @return the number of times that garbage collection has occurred + */ + long getCollectionCount(); + + /** + * @return the number of milliseconds spent performing garbage collection + */ + long getCollectionMillis(); +} diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceRepository.java b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceRepository.java index 6d15d85bf4a3..73d207e82f39 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceRepository.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/provenance/ProvenanceRepository.java @@ -199,6 +199,15 @@ public interface ProvenanceRepository extends ProvenanceEventRepository { */ long getContainerCapacity(String containerName) throws IOException; + /** + * Returns the name of the FileStore that the given container is stored on, or null + * if not applicable or unable to determine the file store name + * + * @param containerName the name of the container + * @return the name of the FileStore + */ + String getContainerFileStoreName(String containerName); + /** * @param containerName to check space on * @return the number of bytes available to be used used by the storage diff --git a/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceRepository.java b/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceRepository.java index 0b4afcd02302..30e6bd17a34f 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceRepository.java +++ b/nifi-mock/src/main/java/org/apache/nifi/provenance/MockProvenanceRepository.java @@ -159,9 +159,14 @@ public long getContainerCapacity(String containerName) throws IOException { return 0; } + @Override + public String getContainerFileStoreName(String containerName) { + return null; + } + @Override public Set getContainerNames() { - return new HashSet(); + return new HashSet<>(); } @Override diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ClassLoaderDiagnosticsDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ClassLoaderDiagnosticsDTO.java new file mode 100644 index 000000000000..bf808b670fa1 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ClassLoaderDiagnosticsDTO.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.web.api.dto.diagnostics; + +import java.util.Set; + +import javax.xml.bind.annotation.XmlType; + +import org.apache.nifi.web.api.dto.BundleDTO; + +import io.swagger.annotations.ApiModelProperty; + +@XmlType(name = "classLoaderDiagnostics") +public class ClassLoaderDiagnosticsDTO { + private BundleDTO bundle; + private Set loadedFiles; + private ClassLoaderDiagnosticsDTO parentClassLoader; + + @ApiModelProperty("Information about the Bundle that the ClassLoader belongs to, if any") + public BundleDTO getBundle() { + return bundle; + } + + public void setBundle(BundleDTO bundle) { + this.bundle = bundle; + } + + @ApiModelProperty("A Set of filenames for all JAR's and resources that have been loaded by this ClassLoader") + public Set getLoadedFiles() { + return loadedFiles; + } + + public void setLoadedFiles(Set loadedFiles) { + this.loadedFiles = loadedFiles; + } + + @ApiModelProperty("A ClassLoaderDiagnosticsDTO that provides information about the parent ClassLoader") + public ClassLoaderDiagnosticsDTO getParentClassLoader() { + return parentClassLoader; + } + + public void setParentClassLoader(ClassLoaderDiagnosticsDTO parentClassLoader) { + this.parentClassLoader = parentClassLoader; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ConnectionDiagnosticsDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ConnectionDiagnosticsDTO.java new file mode 100644 index 000000000000..951ac414b21d --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ConnectionDiagnosticsDTO.java @@ -0,0 +1,148 @@ +/* + * 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.web.api.dto.diagnostics; + +import javax.xml.bind.annotation.XmlType; + +import org.apache.nifi.web.api.dto.ConnectionDTO; + +import io.swagger.annotations.ApiModelProperty; + +@XmlType(name = "connectionDiagnostics") +public class ConnectionDiagnosticsDTO { + private ConnectionDTO connection; + private int totalFlowFileCount; + private long totalByteCount; + private int activeQueueFlowFileCount; + private long activeQueueByteCount; + private int swapFlowFileCount; + private long swapByteCount; + private int swapFiles; + private int inFlightFlowFileCount; + private long inFlightByteCount; + private Boolean allActiveQueueFlowFilesPenalized; + private Boolean anyActiveQueueFlowFilesPenalized; + + @ApiModelProperty("Information about the Connection") + public ConnectionDTO getConnection() { + return connection; + } + + public void setConnection(ConnectionDTO connection) { + this.connection = connection; + } + + @ApiModelProperty("Total number of FlowFiles owned by the Connection") + public int getTotalFlowFileCount() { + return totalFlowFileCount; + } + + public void setTotalFlowFileCount(int totalFlowFileCount) { + this.totalFlowFileCount = totalFlowFileCount; + } + + @ApiModelProperty("Total number of bytes that make up the content for the FlowFiles owned by this Connection") + public long getTotalByteCount() { + return totalByteCount; + } + + public void setTotalByteCount(long totalByteCount) { + this.totalByteCount = totalByteCount; + } + + @ApiModelProperty("Total number of FlowFiles that exist in the Connection's Active Queue, immediately available to be offered up to a component") + public int getActiveQueueFlowFileCount() { + return activeQueueFlowFileCount; + } + + public void setActiveQueueFlowFileCount(int activeQueueFlowFileCount) { + this.activeQueueFlowFileCount = activeQueueFlowFileCount; + } + + @ApiModelProperty("Total number of bytes that make up the content for the FlowFiles that are present in the Connection's Active Queue") + public long getActiveQueueByteCount() { + return activeQueueByteCount; + } + + public void setActiveQueueByteCount(long activeQueueByteCount) { + this.activeQueueByteCount = activeQueueByteCount; + } + + @ApiModelProperty("The total number of FlowFiles that are swapped out for this Connection") + public int getSwapFlowFileCount() { + return swapFlowFileCount; + } + + public void setSwapFlowFileCount(int swapFlowFileCount) { + this.swapFlowFileCount = swapFlowFileCount; + } + + @ApiModelProperty("Total number of bytes that make up the content for the FlowFiles that are swapped out to disk for the Connection") + public long getSwapByteCount() { + return swapByteCount; + } + + public void setSwapByteCount(long swapByteCount) { + this.swapByteCount = swapByteCount; + } + + @ApiModelProperty("The number of Swap Files that exist for this Connection") + public int getSwapFiles() { + return swapFiles; + } + + public void setSwapFiles(int swapFiles) { + this.swapFiles = swapFiles; + } + + @ApiModelProperty("The number of In-Flight FlowFiles for this Connection. These are FlowFiles that belong to the connection but are currently being operated on by a Processor, Port, etc.") + public int getInFlightFlowFileCount() { + return inFlightFlowFileCount; + } + + public void setInFlightFlowFileCount(int inFlightFlowFileCount) { + this.inFlightFlowFileCount = inFlightFlowFileCount; + } + + @ApiModelProperty("The number bytes that make up the content of the FlowFiles that are In-Flight") + public long getInFlightByteCount() { + return inFlightByteCount; + } + + public void setInFlightByteCount(long inFlightByteCount) { + this.inFlightByteCount = inFlightByteCount; + } + + @ApiModelProperty("Whether or not all of the FlowFiles in the Active Queue are penalized") + public Boolean getAllActiveQueueFlowFilesPenalized() { + return allActiveQueueFlowFilesPenalized; + } + + public void setAllActiveQueueFlowFilesPenalized(Boolean allFlowFilesPenalized) { + this.allActiveQueueFlowFilesPenalized = allFlowFilesPenalized; + } + + @ApiModelProperty("Whether or not any of the FlowFiles in the Active Queue are penalized") + public Boolean getAnyActiveQueueFlowFilesPenalized() { + return anyActiveQueueFlowFilesPenalized; + } + + public void setAnyActiveQueueFlowFilesPenalized(Boolean anyFlowFilesPenalized) { + this.anyActiveQueueFlowFilesPenalized = anyFlowFilesPenalized; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ControllerServiceDiagnosticsDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ControllerServiceDiagnosticsDTO.java new file mode 100644 index 000000000000..063b3b4c7cf6 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ControllerServiceDiagnosticsDTO.java @@ -0,0 +1,48 @@ +/* + * 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.web.api.dto.diagnostics; + +import javax.xml.bind.annotation.XmlType; + +import org.apache.nifi.web.api.entity.ControllerServiceEntity; + +import io.swagger.annotations.ApiModelProperty; + +@XmlType(name = "controllerServiceDiagnostics") +public class ControllerServiceDiagnosticsDTO { + private ControllerServiceEntity controllerService; + private ClassLoaderDiagnosticsDTO classLoaderDiagnostics; + + public void setControllerService(final ControllerServiceEntity controllerService) { + this.controllerService = controllerService; + } + + @ApiModelProperty("The Controller Service") + public ControllerServiceEntity getControllerService() { + return controllerService; + } + + public void setClassLoaderDiagnostics(ClassLoaderDiagnosticsDTO classLoaderDiagnostics) { + this.classLoaderDiagnostics = classLoaderDiagnostics; + } + + @ApiModelProperty("Information about the Controller Service's Class Loader") + public ClassLoaderDiagnosticsDTO getClassLoaderDiagnostics() { + return classLoaderDiagnostics; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/GCDiagnosticsSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/GCDiagnosticsSnapshotDTO.java new file mode 100644 index 000000000000..d9abeec7d68f --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/GCDiagnosticsSnapshotDTO.java @@ -0,0 +1,67 @@ +/* + * 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.web.api.dto.diagnostics; + +import java.util.Date; + +import javax.xml.bind.annotation.XmlType; + +import io.swagger.annotations.ApiModelProperty; + +@XmlType(name = "gcDiagnosticsSnapshot") +public class GCDiagnosticsSnapshotDTO implements Cloneable { + private Date timestamp; + private Long collectionCount; + private Long collectionMillis; + + @ApiModelProperty("The timestamp of when the Snapshot was taken") + public Date getTimestamp() { + return timestamp; + } + + public void setTimestamp(Date timestamp) { + this.timestamp = timestamp; + } + + @ApiModelProperty("The number of times that Garbage Collection has occurred") + public Long getCollectionCount() { + return collectionCount; + } + + public void setCollectionCount(Long collectionCount) { + this.collectionCount = collectionCount; + } + + @ApiModelProperty("The number of milliseconds that the Garbage Collector spent performing Garbage Collection duties") + public Long getCollectionMillis() { + return collectionMillis; + } + + public void setCollectionMillis(Long collectionMillis) { + this.collectionMillis = collectionMillis; + } + + @Override + public GCDiagnosticsSnapshotDTO clone() { + final GCDiagnosticsSnapshotDTO clone = new GCDiagnosticsSnapshotDTO(); + clone.timestamp = timestamp; + clone.collectionCount = collectionCount; + clone.collectionMillis = collectionMillis; + return clone; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/GarbageCollectionDiagnosticsDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/GarbageCollectionDiagnosticsDTO.java new file mode 100644 index 000000000000..330fd4de9ea2 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/GarbageCollectionDiagnosticsDTO.java @@ -0,0 +1,61 @@ +/* + * 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.web.api.dto.diagnostics; + +import java.util.List; +import java.util.stream.Collectors; + +import javax.xml.bind.annotation.XmlType; + +import io.swagger.annotations.ApiModelProperty; + +@XmlType(name = "garbageCollectionDiagnostics") +public class GarbageCollectionDiagnosticsDTO implements Cloneable { + private String memoryManagerName; + private List snapshots; + + @ApiModelProperty("The name of the Memory Manager that this Garbage Collection information pertains to") + public String getMemoryManagerName() { + return memoryManagerName; + } + + public void setMemoryManagerName(String memoryManagerName) { + this.memoryManagerName = memoryManagerName; + } + + @ApiModelProperty("A list of snapshots that have been taken to determine the health of the JVM's heap") + public List getSnapshots() { + return snapshots; + } + + public void setSnapshots(List snapshots) { + this.snapshots = snapshots; + } + + @Override + protected GarbageCollectionDiagnosticsDTO clone() { + final GarbageCollectionDiagnosticsDTO clone = new GarbageCollectionDiagnosticsDTO(); + clone.memoryManagerName = memoryManagerName; + if (snapshots != null) { + clone.snapshots = snapshots.stream() + .map(GCDiagnosticsSnapshotDTO::clone) + .collect(Collectors.toList()); + } + return clone; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/JVMDiagnosticsDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/JVMDiagnosticsDTO.java new file mode 100644 index 000000000000..3e63f89b860b --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/JVMDiagnosticsDTO.java @@ -0,0 +1,68 @@ +/* + * 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.web.api.dto.diagnostics; + +import java.util.List; + +import javax.xml.bind.annotation.XmlType; + +import io.swagger.annotations.ApiModelProperty; + +@XmlType(name = "jvmDiagnostics") +public class JVMDiagnosticsDTO { + private Boolean clustered; + private Boolean connected; + private JVMDiagnosticsSnapshotDTO aggregateSnapshot; + private List nodeSnapshots; + + @ApiModelProperty("Whether or not the NiFi instance is clustered") + public Boolean getClustered() { + return clustered; + } + + public void setClustered(Boolean clustered) { + this.clustered = clustered; + } + + @ApiModelProperty("Whether or not the node is connected to the cluster") + public Boolean getConnected() { + return connected; + } + + public void setConnected(Boolean connected) { + this.connected = connected; + } + + @ApiModelProperty("Aggregate JVM diagnostic information about the entire cluster") + public JVMDiagnosticsSnapshotDTO getAggregateSnapshot() { + return aggregateSnapshot; + } + + public void setAggregateSnapshot(JVMDiagnosticsSnapshotDTO aggregateSnapshot) { + this.aggregateSnapshot = aggregateSnapshot; + } + + @ApiModelProperty("Node-wise breakdown of JVM diagnostic information") + public List getNodeSnapshots() { + return nodeSnapshots; + } + + public void setNodeSnapshots(List nodeSnapshots) { + this.nodeSnapshots = nodeSnapshots; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/JVMDiagnosticsSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/JVMDiagnosticsSnapshotDTO.java new file mode 100644 index 000000000000..4b77791908ac --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/JVMDiagnosticsSnapshotDTO.java @@ -0,0 +1,382 @@ +/* + * 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.web.api.dto.diagnostics; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import javax.xml.bind.annotation.XmlType; + +import org.apache.nifi.web.api.dto.BundleDTO; + +import io.swagger.annotations.ApiModelProperty; + +@XmlType(name = "jvmDiagnosticsSnapshot") +public class JVMDiagnosticsSnapshotDTO implements Cloneable { + // JVM/NiFi instance specific + private Boolean primaryNode; + private Boolean clusterCoordinator; + private String uptime; + private String timeZone; + private Integer maxTimerDrivenThreads; + private Integer maxEventDrivenThreads; + private Integer activeTimerDrivenThreads; + private Integer activeEventDrivenThreads; + private Set bundlesLoaded; + private RepositoryUsageDTO flowFileRepositoryStorageUsage; + private Set contentRepositoryStorageUsage; + private Set provenanceRepositoryStorageUsage; + private Long maxHeapBytes; + private String maxHeap; + private List garbageCollectionDiagnostics; + + // System Specific + private Integer cpuCores; + private Double cpuLoadAverage; + private Long physicalMemoryBytes; + private String physicalMemory; + + // Only available if we get OS MXBean and can create class com.sun.management.UnixOperatingSystemMXBean and the OS MXBean + // is of this type. + private Long openFileDescriptors; + private Long maxOpenFileDescriptors; + + @ApiModelProperty("Whether or not this node is primary node") + public Boolean getPrimaryNode() { + return primaryNode; + } + + public void setPrimaryNode(Boolean primaryNode) { + this.primaryNode = primaryNode; + } + + @ApiModelProperty("Whether or not this node is cluster coordinator") + public Boolean getClusterCoordinator() { + return clusterCoordinator; + } + + public void setClusterCoordinator(Boolean clusterCoordinator) { + this.clusterCoordinator = clusterCoordinator; + } + + @ApiModelProperty("How long this node has been running, formatted as hours:minutes:seconds.milliseconds") + public String getUptime() { + return uptime; + } + + public void setUptime(String uptime) { + this.uptime = uptime; + } + + @ApiModelProperty("The name of the Time Zone that is configured, if available") + public String getTimeZone() { + return timeZone; + } + + public void setTimeZone(String timeZone) { + this.timeZone = timeZone; + } + + @ApiModelProperty("The maximum number of timer-driven threads") + public Integer getMaxTimerDrivenThreads() { + return maxTimerDrivenThreads; + } + + public void setMaxTimerDrivenThreads(Integer maxTimerDrivenThreads) { + this.maxTimerDrivenThreads = maxTimerDrivenThreads; + } + + @ApiModelProperty("The maximum number of event-driven threads") + public Integer getMaxEventDrivenThreads() { + return maxEventDrivenThreads; + } + + public void setMaxEventDrivenThreads(Integer maxEventDrivenThreads) { + this.maxEventDrivenThreads = maxEventDrivenThreads; + } + + @ApiModelProperty("The number of timer-driven threads that are active") + public Integer getActiveTimerDrivenThreads() { + return activeTimerDrivenThreads; + } + + public void setActiveTimerDrivenThreads(Integer activeTimerDrivenThreads) { + this.activeTimerDrivenThreads = activeTimerDrivenThreads; + } + + @ApiModelProperty("The number of event-driven threads that are active") + public Integer getActiveEventDrivenThreads() { + return activeEventDrivenThreads; + } + + public void setActiveEventDrivenThreads(Integer activeEventDrivenThreads) { + this.activeEventDrivenThreads = activeEventDrivenThreads; + } + + @ApiModelProperty("The NiFi Bundles (NARs) that are loaded by NiFi") + public Set getBundlesLoaded() { + return bundlesLoaded; + } + + public void setBundlesLoaded(Set bundlesLoaded) { + this.bundlesLoaded = bundlesLoaded; + } + + @ApiModelProperty("Information about the FlowFile Repository's usage") + public RepositoryUsageDTO getFlowFileRepositoryStorageUsage() { + return flowFileRepositoryStorageUsage; + } + + public void setFlowFileRepositoryStorageUsage(RepositoryUsageDTO flowFileRepositoryStorageUsage) { + this.flowFileRepositoryStorageUsage = flowFileRepositoryStorageUsage; + } + + @ApiModelProperty("Information about the Content Repository's usage") + public Set getContentRepositoryStorageUsage() { + return contentRepositoryStorageUsage; + } + + public void setContentRepositoryStorageUsage(Set contentRepositoryStorageUsage) { + this.contentRepositoryStorageUsage = contentRepositoryStorageUsage; + } + + @ApiModelProperty("Information about the Provenance Repository's usage") + public Set getProvenanceRepositoryStorageUsage() { + return provenanceRepositoryStorageUsage; + } + + public void setProvenanceRepositoryStorageUsage(Set provenanceRepositoryStorageUsage) { + this.provenanceRepositoryStorageUsage = provenanceRepositoryStorageUsage; + } + + @ApiModelProperty("The maximum number of bytes that the JVM heap is configured to use for heap") + public Long getMaxHeapBytes() { + return maxHeapBytes; + } + + public void setMaxHeapBytes(Long heapBytes) { + this.maxHeapBytes = heapBytes; + } + + @ApiModelProperty("The maximum number of bytes that the JVM heap is configured to use, as a human-readable value") + public String getMaxHeap() { + return maxHeap; + } + + public void setMaxHeap(String maxHeap) { + this.maxHeap = maxHeap; + } + + @ApiModelProperty("The number of CPU Cores available on the system") + public Integer getCpuCores() { + return cpuCores; + } + + public void setCpuCores(Integer cpuCores) { + this.cpuCores = cpuCores; + } + + @ApiModelProperty("The 1-minute CPU Load Average") + public Double getCpuLoadAverage() { + return cpuLoadAverage; + } + + public void setCpuLoadAverage(Double cpuLoadAverage) { + this.cpuLoadAverage = cpuLoadAverage; + } + + @ApiModelProperty("The number of bytes of RAM available on the system") + public Long getPhysicalMemoryBytes() { + return physicalMemoryBytes; + } + + public void setPhysicalMemoryBytes(Long memoryBytes) { + this.physicalMemoryBytes = memoryBytes; + } + + @ApiModelProperty("The number of bytes of RAM available on the system as a human-readable value") + public String getPhysicalMemory() { + return physicalMemory; + } + + public void setPhysicalMemory(String memory) { + this.physicalMemory = memory; + } + + @ApiModelProperty("The number of files that are open by the NiFi process") + public Long getOpenFileDescriptors() { + return openFileDescriptors; + } + + public void setOpenFileDescriptors(Long openFileDescriptors) { + this.openFileDescriptors = openFileDescriptors; + } + + @ApiModelProperty("The maximum number of open file descriptors that are available to each process") + public Long getMaxOpenFileDescriptors() { + return maxOpenFileDescriptors; + } + + public void setMaxOpenFileDescriptors(Long maxOpenFileDescriptors) { + this.maxOpenFileDescriptors = maxOpenFileDescriptors; + } + + @ApiModelProperty("Diagnostic information about the JVM's garbage collections") + public List getGarbageCollectionDiagnostics() { + return garbageCollectionDiagnostics; + } + + public void setGarbageCollectionDiagnostics(List garbageCollectionDiagnostics) { + this.garbageCollectionDiagnostics = garbageCollectionDiagnostics; + } + + + @Override + public JVMDiagnosticsSnapshotDTO clone() { + final JVMDiagnosticsSnapshotDTO clone = new JVMDiagnosticsSnapshotDTO(); + clone.activeEventDrivenThreads = activeEventDrivenThreads; + clone.activeTimerDrivenThreads = activeTimerDrivenThreads; + clone.bundlesLoaded = bundlesLoaded == null ? null : new HashSet<>(bundlesLoaded); + clone.clusterCoordinator = clusterCoordinator; + clone.contentRepositoryStorageUsage = cloneRepoUsage(contentRepositoryStorageUsage); + clone.cpuCores = cpuCores; + clone.cpuLoadAverage = cpuLoadAverage; + clone.flowFileRepositoryStorageUsage = flowFileRepositoryStorageUsage == null ? null : flowFileRepositoryStorageUsage.clone(); + clone.maxEventDrivenThreads = maxEventDrivenThreads; + clone.maxHeap = maxHeap; + clone.maxHeapBytes = maxHeapBytes; + clone.maxOpenFileDescriptors = maxOpenFileDescriptors; + clone.maxTimerDrivenThreads = maxTimerDrivenThreads; + clone.openFileDescriptors = openFileDescriptors; + clone.physicalMemory = physicalMemory; + clone.physicalMemoryBytes = physicalMemoryBytes; + clone.primaryNode = primaryNode; + clone.provenanceRepositoryStorageUsage = cloneRepoUsage(provenanceRepositoryStorageUsage); + clone.timeZone = timeZone; + clone.uptime = uptime; + + if (garbageCollectionDiagnostics != null) { + clone.garbageCollectionDiagnostics = garbageCollectionDiagnostics.stream() + .map(gcDiag -> gcDiag.clone()) + .collect(Collectors.toList()); + } + + return clone; + } + + private static Set cloneRepoUsage(final Set repoUsage) { + if (repoUsage == null) { + return null; + } + + return repoUsage.stream() + .map(usage -> usage.clone()) + .collect(Collectors.toSet()); + } + + + @XmlType(name = "versionInfo") + public static class VersionInfoDTO implements Cloneable { + + private String niFiVersion; + private String javaVendor; + private String javaVersion; + private String javaVmVendor; + private String osName; + private String osVersion; + private String osArchitecture; + + @ApiModelProperty("The version of this NiFi.") + public String getNiFiVersion() { + return niFiVersion; + } + + public void setNiFiVersion(String niFiVersion) { + this.niFiVersion = niFiVersion; + } + + @ApiModelProperty("Java vendor") + public String getJavaVendor() { + return javaVendor; + } + + public void setJavaVendor(String javaVendor) { + this.javaVendor = javaVendor; + } + + @ApiModelProperty("Java VM Vendor") + public String getJavaVmVendor() { + return javaVmVendor; + } + + public void setJavaVmVendor(String javaVmVendor) { + this.javaVmVendor = javaVmVendor; + } + + @ApiModelProperty("Java version") + public String getJavaVersion() { + return javaVersion; + } + + public void setJavaVersion(String javaVersion) { + this.javaVersion = javaVersion; + } + + @ApiModelProperty("Host operating system name") + public String getOsName() { + return osName; + } + + public void setOsName(String osName) { + this.osName = osName; + } + + @ApiModelProperty("Host operating system version") + public String getOsVersion() { + return osVersion; + } + + public void setOsVersion(String osVersion) { + this.osVersion = osVersion; + } + + @ApiModelProperty("Host operating system architecture") + public String getOsArchitecture() { + return osArchitecture; + } + + public void setOsArchitecture(String osArchitecture) { + this.osArchitecture = osArchitecture; + } + + + @Override + public VersionInfoDTO clone() { + final VersionInfoDTO other = new VersionInfoDTO(); + other.setNiFiVersion(getNiFiVersion()); + other.setJavaVendor(getJavaVendor()); + other.setJavaVersion(getJavaVersion()); + other.setOsName(getOsName()); + other.setOsVersion(getOsVersion()); + other.setOsArchitecture(getOsArchitecture()); + return other; + } + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/NodeGCDiagnosticsSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/NodeGCDiagnosticsSnapshotDTO.java new file mode 100644 index 000000000000..008dc992e683 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/NodeGCDiagnosticsSnapshotDTO.java @@ -0,0 +1,67 @@ +/* + * 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.web.api.dto.diagnostics; + +import javax.xml.bind.annotation.XmlType; + +import io.swagger.annotations.ApiModelProperty; + +@XmlType(name = "nodeGCDiagnosticsSnapshot") +public class NodeGCDiagnosticsSnapshotDTO { + private String nodeId; + private String address; + private Integer apiPort; + + private GCDiagnosticsSnapshotDTO snapshot; + + @ApiModelProperty("The unique ID that identifies the node") + public String getNodeId() { + return nodeId; + } + + public void setNodeId(String nodeId) { + this.nodeId = nodeId; + } + + @ApiModelProperty("The API address of the node") + public String getAddress() { + return address; + } + + public void setAddress(String address) { + this.address = address; + } + + @ApiModelProperty("The API port used to communicate with the node") + public Integer getApiPort() { + return apiPort; + } + + public void setApiPort(Integer apiPort) { + this.apiPort = apiPort; + } + + @ApiModelProperty("The Garbage Collection Diagnostics Snapshot") + public GCDiagnosticsSnapshotDTO getSnapshot() { + return snapshot; + } + + public void setSnapshot(GCDiagnosticsSnapshotDTO snapshot) { + this.snapshot = snapshot; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/NodeJVMDiagnosticsSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/NodeJVMDiagnosticsSnapshotDTO.java new file mode 100644 index 000000000000..ceca0aec0241 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/NodeJVMDiagnosticsSnapshotDTO.java @@ -0,0 +1,68 @@ +/* + * 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.web.api.dto.diagnostics; + +import javax.xml.bind.annotation.XmlType; + +import io.swagger.annotations.ApiModelProperty; + +@XmlType(name = "nodeJVMDiagnosticsSnapshot") +public class NodeJVMDiagnosticsSnapshotDTO { + private String nodeId; + private String address; + private Integer apiPort; + + private JVMDiagnosticsSnapshotDTO snapshot; + + @ApiModelProperty("The unique ID that identifies the node") + public String getNodeId() { + return nodeId; + } + + public void setNodeId(String nodeId) { + this.nodeId = nodeId; + } + + @ApiModelProperty("The API address of the node") + public String getAddress() { + return address; + } + + public void setAddress(String address) { + this.address = address; + } + + @ApiModelProperty("The API port used to communicate with the node") + public Integer getApiPort() { + return apiPort; + } + + public void setApiPort(Integer apiPort) { + this.apiPort = apiPort; + } + + @ApiModelProperty("The JVM Diagnostics Snapshot") + public JVMDiagnosticsSnapshotDTO getSnapshot() { + return snapshot; + } + + public void setSnapshot(JVMDiagnosticsSnapshotDTO snapshot) { + this.snapshot = snapshot; + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ProcessorDiagnosticsDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ProcessorDiagnosticsDTO.java new file mode 100644 index 000000000000..77f5499e9d1c --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ProcessorDiagnosticsDTO.java @@ -0,0 +1,113 @@ +/* + * 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.web.api.dto.diagnostics; + +import java.util.List; +import java.util.Set; + +import javax.xml.bind.annotation.XmlType; + +import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO; + +import io.swagger.annotations.ApiModelProperty; + +@XmlType(name = "processorDiagnostics") +public class ProcessorDiagnosticsDTO { + private ProcessorDTO processor; + private ProcessorStatusDTO processorStatus; + private Set referencedControllerServices; + private Set incomingConnections; + private Set outgoingConnections; + private JVMDiagnosticsDTO jvmDiagnostics; + private List threadDumps; + private ClassLoaderDiagnosticsDTO classLoaderDiagnostics; + + + @ApiModelProperty("Information about the Processor for which the Diagnostic Report is generated") + public ProcessorDTO getProcessor() { + return processor; + } + + public void setProcessor(ProcessorDTO processor) { + this.processor = processor; + } + + @ApiModelProperty("The Status for the Processor for which the Diagnostic Report is generated") + public ProcessorStatusDTO getProcessorStatus() { + return processorStatus; + } + + public void setProcessorStatus(ProcessorStatusDTO processorStatus) { + this.processorStatus = processorStatus; + } + + @ApiModelProperty("Diagnostic Information about all Controller Services that the Processor is referencing") + public Set getReferencedControllerServices() { + return referencedControllerServices; + } + + public void setReferencedControllerServices(Set referencedControllerServices) { + this.referencedControllerServices = referencedControllerServices; + } + + @ApiModelProperty("Diagnostic Information about all incoming Connections") + public Set getIncomingConnections() { + return incomingConnections; + } + + public void setIncomingConnections(Set incomingConnections) { + this.incomingConnections = incomingConnections; + } + + @ApiModelProperty("Diagnostic Information about all outgoing Connections") + public Set getOutgoingConnections() { + return outgoingConnections; + } + + public void setOutgoingConnections(Set outgoingConnections) { + this.outgoingConnections = outgoingConnections; + } + + @ApiModelProperty("Diagnostic Information about the JVM and system-level diagnostics") + public JVMDiagnosticsDTO getJvmDiagnostics() { + return jvmDiagnostics; + } + + public void setJvmDiagnostics(JVMDiagnosticsDTO jvmDiagnostics) { + this.jvmDiagnostics = jvmDiagnostics; + } + + @ApiModelProperty("Thread Dumps that were taken of the threads that are active in the Processor") + public List getThreadDumps() { + return threadDumps; + } + + public void setThreadDumps(List threadDumps) { + this.threadDumps = threadDumps; + } + + @ApiModelProperty("Information about the Controller Service's Class Loader") + public ClassLoaderDiagnosticsDTO getClassLoaderDiagnostics() { + return classLoaderDiagnostics; + } + + public void setClassLoaderDiagnostics(ClassLoaderDiagnosticsDTO classLoaderDiagnostics) { + this.classLoaderDiagnostics = classLoaderDiagnostics; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/RepositoryUsageDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/RepositoryUsageDTO.java new file mode 100644 index 000000000000..55558ce3aa47 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/RepositoryUsageDTO.java @@ -0,0 +1,111 @@ +/* + * 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.web.api.dto.diagnostics; + +import javax.xml.bind.annotation.XmlType; + +import io.swagger.annotations.ApiModelProperty; + +@XmlType(name = "fileStoreUsage") +public class RepositoryUsageDTO implements Cloneable { + private String name; + private String fileStore; + + private String freeSpace; + private String totalSpace; + private Long freeSpaceBytes; + private Long totalSpaceBytes; + private String utilization; + + + @ApiModelProperty("The name of the repository") + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + @ApiModelProperty("The File Store that is used to store the repository's data") + public String getFileStore() { + return fileStore; + } + + public void setFileStore(String fileStore) { + this.fileStore = fileStore; + } + + @ApiModelProperty("Amount of free space.") + public String getFreeSpace() { + return freeSpace; + } + + public void setFreeSpace(String freeSpace) { + this.freeSpace = freeSpace; + } + + @ApiModelProperty("Amount of total space.") + public String getTotalSpace() { + return totalSpace; + } + + public void setTotalSpace(String totalSpace) { + this.totalSpace = totalSpace; + } + + @ApiModelProperty("Utilization of this storage location.") + public String getUtilization() { + return utilization; + } + + public void setUtilization(String utilization) { + this.utilization = utilization; + } + + @ApiModelProperty("The number of bytes of free space.") + public Long getFreeSpaceBytes() { + return freeSpaceBytes; + } + + public void setFreeSpaceBytes(Long freeSpaceBytes) { + this.freeSpaceBytes = freeSpaceBytes; + } + + @ApiModelProperty("The number of bytes of total space.") + public Long getTotalSpaceBytes() { + return totalSpaceBytes; + } + + public void setTotalSpaceBytes(Long totalSpaceBytes) { + this.totalSpaceBytes = totalSpaceBytes; + } + + @Override + public RepositoryUsageDTO clone() { + final RepositoryUsageDTO clone = new RepositoryUsageDTO(); + clone.fileStore = fileStore; + clone.freeSpace = freeSpace; + clone.freeSpaceBytes = freeSpaceBytes; + clone.name = name; + clone.totalSpace = totalSpace; + clone.totalSpaceBytes = totalSpaceBytes; + clone.utilization = utilization; + return clone; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ThreadDumpDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ThreadDumpDTO.java new file mode 100644 index 000000000000..be0b7ac3f21b --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ThreadDumpDTO.java @@ -0,0 +1,86 @@ +/* + * 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.web.api.dto.diagnostics; + +import javax.xml.bind.annotation.XmlType; + +import io.swagger.annotations.ApiModelProperty; + +@XmlType(name = "threadDump") +public class ThreadDumpDTO { + private String nodeId; + private String nodeAddress; + private Integer apiPort; + private String stackTrace; + private String threadName; + private long threadActiveMillis; + + @ApiModelProperty("The ID of the node in the cluster") + public String getNodeId() { + return nodeId; + } + + public void setNodeId(String nodeId) { + this.nodeId = nodeId; + } + + @ApiModelProperty("The address of the node in the cluster") + public String getNodeAddress() { + return nodeAddress; + } + + public void setNodeAddress(String nodeAddress) { + this.nodeAddress = nodeAddress; + } + + @ApiModelProperty("The port the node is listening for API requests.") + public Integer getApiPort() { + return apiPort; + } + + public void setApiPort(Integer port) { + this.apiPort = port; + } + + @ApiModelProperty("The stack trace for the thread") + public String getStackTrace() { + return stackTrace; + } + + public void setStackTrace(String stackTrace) { + this.stackTrace = stackTrace; + } + + @ApiModelProperty("The name of the thread") + public String getThreadName() { + return threadName; + } + + public void setThreadName(String threadName) { + this.threadName = threadName; + } + + @ApiModelProperty("The number of milliseconds that the thread has been executing in the Processor") + public long getThreadActiveMillis() { + return threadActiveMillis; + } + + public void setThreadActiveMillis(long threadActiveMillis) { + this.threadActiveMillis = threadActiveMillis; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorDiagnosticsEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorDiagnosticsEntity.java new file mode 100644 index 000000000000..603a15782a7f --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorDiagnosticsEntity.java @@ -0,0 +1,40 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; + +import org.apache.nifi.web.api.dto.diagnostics.ProcessorDiagnosticsDTO; + +import io.swagger.annotations.ApiModelProperty; + +@XmlRootElement(name = "processorDiagnostics") +public class ProcessorDiagnosticsEntity extends ComponentEntity implements Permissible { + private ProcessorDiagnosticsDTO processorDiagnostics; + + @Override + @ApiModelProperty("The Processor Diagnostics") + public ProcessorDiagnosticsDTO getComponent() { + return processorDiagnostics; + } + + @Override + public void setComponent(ProcessorDiagnosticsDTO processorDiagnostics) { + this.processorDiagnostics = processorDiagnostics; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java index fa23603937d7..96f2592b93db 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java @@ -49,6 +49,7 @@ import org.apache.nifi.cluster.coordination.http.endpoints.PrioritizerTypesEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ProcessGroupEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ProcessGroupsEndpointMerger; +import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorDiagnosticsEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorStatusEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorTypesEndpointMerger; @@ -155,6 +156,7 @@ public StandardHttpResponseMapper(final NiFiProperties nifiProperties) { endpointMergers.add(new AccessPolicyEndpointMerger()); endpointMergers.add(new SearchUsersEndpointMerger()); endpointMergers.add(new VariableRegistryEndpointMerger()); + endpointMergers.add(new ProcessorDiagnosticsEndpointMerger(snapshotMillis)); } @Override diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProcessorDiagnosticsEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProcessorDiagnosticsEndpointMerger.java new file mode 100644 index 000000000000..154059969301 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProcessorDiagnosticsEndpointMerger.java @@ -0,0 +1,65 @@ +/* + * 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.cluster.coordination.http.endpoints; + +import java.net.URI; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.manager.ProcessorDiagnosticsEntityMerger; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.entity.ProcessorDiagnosticsEntity; + +public class ProcessorDiagnosticsEndpointMerger implements EndpointResponseMerger { + public static final Pattern PROCESSOR_DIAGNOSTICS_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}/diagnostics"); + + private final ProcessorDiagnosticsEntityMerger diagnosticsEntityMerger; + + public ProcessorDiagnosticsEndpointMerger(final long componentStatusSnapshotMillis) { + diagnosticsEntityMerger = new ProcessorDiagnosticsEntityMerger(componentStatusSnapshotMillis); + } + + @Override + public boolean canHandle(final URI uri, final String method) { + if (!"GET".equalsIgnoreCase(method)) { + return false; + } + + return PROCESSOR_DIAGNOSTICS_URI_PATTERN.matcher(uri.getPath()).matches(); + } + + @Override + public NodeResponse merge(URI uri, String method, Set successfulResponses, Set problematicResponses, NodeResponse clientResponse) { + final ProcessorDiagnosticsEntity clientEntity = clientResponse.getClientResponse().readEntity(ProcessorDiagnosticsEntity.class); + + // Unmarshall each response into an entity. + final Map entityMap = new HashMap<>(); + for (final NodeResponse nodeResponse : successfulResponses) { + final ProcessorDiagnosticsEntity nodeResponseEntity = nodeResponse == clientResponse ? clientEntity : nodeResponse.getClientResponse().readEntity(ProcessorDiagnosticsEntity.class); + entityMap.put(nodeResponse.getNodeId(), nodeResponseEntity); + } + + diagnosticsEntityMerger.merge(clientEntity, entityMap); + return new NodeResponse(clientResponse, clientEntity); + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ProcessorDiagnosticsEntityMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ProcessorDiagnosticsEntityMerger.java new file mode 100644 index 000000000000..1f163f4702b9 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ProcessorDiagnosticsEntityMerger.java @@ -0,0 +1,119 @@ +/* + * 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.cluster.manager; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.dto.diagnostics.ControllerServiceDiagnosticsDTO; +import org.apache.nifi.web.api.dto.diagnostics.JVMDiagnosticsSnapshotDTO; +import org.apache.nifi.web.api.dto.diagnostics.NodeJVMDiagnosticsSnapshotDTO; +import org.apache.nifi.web.api.dto.diagnostics.ProcessorDiagnosticsDTO; +import org.apache.nifi.web.api.dto.diagnostics.ThreadDumpDTO; +import org.apache.nifi.web.api.entity.ControllerServiceEntity; +import org.apache.nifi.web.api.entity.ProcessorDiagnosticsEntity; + +public class ProcessorDiagnosticsEntityMerger implements ComponentEntityMerger { + private final long componentStatusSnapshotMillis; + + public ProcessorDiagnosticsEntityMerger(final long componentStatusSnapshotMillis) { + this.componentStatusSnapshotMillis = componentStatusSnapshotMillis; + } + + @Override + public void mergeComponents(final ProcessorDiagnosticsEntity clientEntity, final Map entityMap) { + final ProcessorDiagnosticsDTO clientDto = clientEntity.getComponent(); + + final List nodeJvmDiagnosticsSnapshots = new ArrayList<>(entityMap.size()); + + // Merge the Processor Statuses and create a separate NodeJVMDiagnosticsSnapshotDTO for each. We do both of these + // together simply because we are already iterating over the entityMap and we have to create the Node-specific JVM diagnostics + // before we start merging the values, in the second iteration over the map. + for (final Map.Entry entry : entityMap.entrySet()) { + final NodeIdentifier nodeId = entry.getKey(); + final ProcessorDiagnosticsEntity diagnosticsEntity = entry.getValue(); + final ProcessorDiagnosticsDTO diagnosticsDto = diagnosticsEntity.getComponent(); + + StatusMerger.merge(clientDto.getProcessorStatus(), clientEntity.getPermissions().getCanRead(), + diagnosticsDto.getProcessorStatus(), diagnosticsEntity.getPermissions().getCanRead(), + nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort()); + + final NodeJVMDiagnosticsSnapshotDTO nodeJvmDiagnosticsSnapshot = new NodeJVMDiagnosticsSnapshotDTO(); + nodeJvmDiagnosticsSnapshot.setAddress(nodeId.getApiAddress()); + nodeJvmDiagnosticsSnapshot.setApiPort(nodeId.getApiPort()); + nodeJvmDiagnosticsSnapshot.setNodeId(nodeId.getId()); + nodeJvmDiagnosticsSnapshot.setSnapshot(diagnosticsDto.getJvmDiagnostics().getAggregateSnapshot()); + nodeJvmDiagnosticsSnapshots.add(nodeJvmDiagnosticsSnapshot); + } + clientDto.getJvmDiagnostics().setNodeSnapshots(nodeJvmDiagnosticsSnapshots); + + // Merge JVM Diagnostics and thread dumps + final JVMDiagnosticsSnapshotDTO mergedJvmDiagnosticsSnapshot = clientDto.getJvmDiagnostics().getAggregateSnapshot().clone(); + for (final Map.Entry entry : entityMap.entrySet()) { + final NodeIdentifier nodeId = entry.getKey(); + final ProcessorDiagnosticsEntity diagnosticsEntity = entry.getValue(); + + if (diagnosticsEntity == clientEntity) { + for (final ThreadDumpDTO threadDump : clientDto.getThreadDumps()) { + threadDump.setNodeAddress(nodeId.getApiAddress()); + threadDump.setApiPort(nodeId.getApiPort()); + threadDump.setNodeId(nodeId.getId()); + } + + continue; + } + + final ProcessorDiagnosticsDTO diagnosticsDto = diagnosticsEntity.getComponent(); + final JVMDiagnosticsSnapshotDTO snapshot = diagnosticsDto.getJvmDiagnostics().getAggregateSnapshot(); + StatusMerger.merge(mergedJvmDiagnosticsSnapshot, snapshot, componentStatusSnapshotMillis); + + final List threadDumps = diagnosticsEntity.getComponent().getThreadDumps(); + for (final ThreadDumpDTO threadDump : threadDumps) { + threadDump.setNodeAddress(nodeId.getApiAddress()); + threadDump.setApiPort(nodeId.getApiPort()); + threadDump.setNodeId(nodeId.getId()); + clientDto.getThreadDumps().add(threadDump); + } + } + clientDto.getJvmDiagnostics().setAggregateSnapshot(mergedJvmDiagnosticsSnapshot); + + // Merge permissions on referenced controller services + final Map serviceEntityById = clientDto.getReferencedControllerServices().stream() + .map(diagnosticsDto -> diagnosticsDto.getControllerService()) + .collect(Collectors.toMap(ControllerServiceEntity::getId, Function.identity())); + + for (final Map.Entry entry : entityMap.entrySet()) { + final ProcessorDiagnosticsEntity procDiagnostics = entry.getValue(); + final Set serviceDtos = procDiagnostics.getComponent().getReferencedControllerServices(); + + for (final ControllerServiceDiagnosticsDTO serviceDto : serviceDtos) { + final ControllerServiceEntity serviceEntity = serviceDto.getControllerService(); + final ControllerServiceEntity targetEntity = serviceEntityById.get(serviceEntity.getId()); + if (targetEntity != null) { + PermissionsDtoMerger.mergePermissions(targetEntity.getPermissions(), serviceEntity.getPermissions()); + } + } + } + + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java index 9c940b470a58..50e63c7c0787 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java @@ -17,6 +17,16 @@ package org.apache.nifi.cluster.manager; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; + import org.apache.nifi.controller.status.RunStatus; import org.apache.nifi.controller.status.TransmissionStatus; import org.apache.nifi.registry.flow.VersionedFlowState; @@ -30,6 +40,9 @@ import org.apache.nifi.web.api.dto.SystemDiagnosticsSnapshotDTO; import org.apache.nifi.web.api.dto.SystemDiagnosticsSnapshotDTO.GarbageCollectionDTO; import org.apache.nifi.web.api.dto.SystemDiagnosticsSnapshotDTO.StorageUsageDTO; +import org.apache.nifi.web.api.dto.diagnostics.GCDiagnosticsSnapshotDTO; +import org.apache.nifi.web.api.dto.diagnostics.GarbageCollectionDiagnosticsDTO; +import org.apache.nifi.web.api.dto.diagnostics.JVMDiagnosticsSnapshotDTO; import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO; import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO; import org.apache.nifi.web.api.dto.status.ControllerStatusDTO; @@ -52,15 +65,6 @@ import org.apache.nifi.web.api.entity.ProcessorStatusSnapshotEntity; import org.apache.nifi.web.api.entity.RemoteProcessGroupStatusSnapshotEntity; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.TimeUnit; - public class StatusMerger { private static final String ZERO_COUNT = "0"; private static final String ZERO_BYTES = "0 bytes"; @@ -626,6 +630,91 @@ public static void merge(final SystemDiagnosticsSnapshotDTO target, final System updatePrettyPrintedFields(target); } + public static void merge(final JVMDiagnosticsSnapshotDTO target, final JVMDiagnosticsSnapshotDTO toMerge, final long numMillis) { + if (target == null || toMerge == null) { + return; + } + + target.setActiveEventDrivenThreads(add(target.getActiveEventDrivenThreads(), toMerge.getActiveEventDrivenThreads())); + target.setActiveTimerDrivenThreads(add(target.getActiveTimerDrivenThreads(), toMerge.getActiveTimerDrivenThreads())); + target.setBundlesLoaded(null); + target.setClusterCoordinator(null); + target.setContentRepositoryStorageUsage(null); + target.setCpuCores(add(target.getCpuCores(), toMerge.getCpuCores())); + target.setCpuLoadAverage(add(target.getCpuLoadAverage(), toMerge.getCpuLoadAverage())); + target.setFlowFileRepositoryStorageUsage(null); + target.setMaxEventDrivenThreads(add(target.getMaxEventDrivenThreads(), toMerge.getMaxEventDrivenThreads())); + target.setMaxHeapBytes(add(target.getMaxHeapBytes(), toMerge.getMaxHeapBytes())); + target.setMaxHeap(FormatUtils.formatDataSize(target.getMaxHeapBytes())); + target.setMaxOpenFileDescriptors(add(target.getMaxOpenFileDescriptors(), toMerge.getMaxOpenFileDescriptors())); + target.setMaxTimerDrivenThreads(add(target.getMaxTimerDrivenThreads(), toMerge.getMaxTimerDrivenThreads())); + target.setOpenFileDescriptors(add(target.getOpenFileDescriptors(), toMerge.getOpenFileDescriptors())); + target.setPhysicalMemoryBytes(add(target.getPhysicalMemoryBytes(), toMerge.getPhysicalMemoryBytes())); + target.setPhysicalMemory(FormatUtils.formatDataSize(target.getPhysicalMemoryBytes())); + target.setPrimaryNode(null); + target.setProvenanceRepositoryStorageUsage(null); + if (target.getTimeZone() == null || toMerge.getTimeZone() == null || !target.getTimeZone().equals(toMerge.getTimeZone())) { + target.setTimeZone(null); + } + target.setUptime(null); + + mergeGarbageCollectionDiagnostics(target.getGarbageCollectionDiagnostics(), toMerge.getGarbageCollectionDiagnostics(), numMillis); + } + + private static void mergeGarbageCollectionDiagnostics(final List target, final List toMerge, final long numMillis) { + final Map> metricsByMemoryMgr = new HashMap<>(); + merge(target, metricsByMemoryMgr, numMillis); + merge(toMerge, metricsByMemoryMgr, numMillis); + } + + private static void merge(final List toMerge, final Map> metricsByMemoryMgr, final long numMillis) { + for (final GarbageCollectionDiagnosticsDTO gcDiagnostics : toMerge) { + final String memoryManagerName = gcDiagnostics.getMemoryManagerName(); + final Map metricsByDate = metricsByMemoryMgr.computeIfAbsent(memoryManagerName, key -> new HashMap<>()); + + for (final GCDiagnosticsSnapshotDTO snapshot : gcDiagnostics.getSnapshots()) { + final long timestamp = snapshot.getTimestamp().getTime(); + final Date normalized = new Date(timestamp - timestamp % numMillis); + + final GCDiagnosticsSnapshotDTO aggregate = metricsByDate.computeIfAbsent(normalized, key -> new GCDiagnosticsSnapshotDTO()); + aggregate.setCollectionCount(add(aggregate.getCollectionCount(), snapshot.getCollectionCount())); + aggregate.setCollectionMillis(add(aggregate.getCollectionMillis(), snapshot.getCollectionMillis())); + aggregate.setTimestamp(normalized); + } + } + } + + + private static Integer add(final Integer a, final Integer b) { + if (a == null) { + return b; + } + if (b == null) { + return a; + } + return a + b; + } + + private static Double add(final Double a, final Double b) { + if (a == null) { + return b; + } + if (b == null) { + return a; + } + return a + b; + } + + private static Long add(final Long a, final Long b) { + if (a == null) { + return b; + } + if (b == null) { + return a; + } + return a + b; + } + public static void updatePrettyPrintedFields(final SystemDiagnosticsSnapshotDTO target) { // heap target.setMaxHeap(FormatUtils.formatDataSize(target.getMaxHeapBytes())); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ActiveThreadInfo.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ActiveThreadInfo.java new file mode 100644 index 000000000000..94a0b73e0a58 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ActiveThreadInfo.java @@ -0,0 +1,42 @@ +/* + * 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; + +public class ActiveThreadInfo { + private final String threadName; + private final String stackTrace; + private final long activeMillis; + + public ActiveThreadInfo(final String threadName, final String stackTrace, final long activeMillis) { + this.threadName = threadName; + this.stackTrace = stackTrace; + this.activeMillis = activeMillis; + } + + public String getThreadName() { + return threadName; + } + + public String getStackTrace() { + return stackTrace; + } + + public long getActiveMillis() { + return activeMillis; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java index 7aad6b450a10..a52271815782 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java @@ -16,6 +16,7 @@ */ package org.apache.nifi.controller; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -68,6 +69,8 @@ public ProcessorNode(final String id, public abstract Requirement getInputRequirement(); + public abstract List getActiveThreads(); + @Override public abstract boolean isValid(); 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 e68000f9c83d..34d5b2aa6001 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 @@ -122,6 +122,9 @@ import org.apache.nifi.controller.status.RunStatus; import org.apache.nifi.controller.status.TransmissionStatus; import org.apache.nifi.controller.status.history.ComponentStatusRepository; +import org.apache.nifi.controller.status.history.GarbageCollectionHistory; +import org.apache.nifi.controller.status.history.GarbageCollectionStatus; +import org.apache.nifi.controller.status.history.StandardGarbageCollectionStatus; import org.apache.nifi.controller.status.history.StatusHistoryUtil; import org.apache.nifi.controller.tasks.ExpireFlowFiles; import org.apache.nifi.diagnostics.SystemDiagnostics; @@ -230,6 +233,8 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.lang.management.GarbageCollectorMXBean; +import java.lang.management.ManagementFactory; import java.net.URL; import java.util.ArrayList; import java.util.Collection; @@ -612,7 +617,7 @@ private FlowController( timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() { @Override public void run() { - componentStatusRepository.capture(getControllerStatus()); + componentStatusRepository.capture(getControllerStatus(), getGarbageCollectionStatus()); } }, snapshotMillis, snapshotMillis, TimeUnit.MILLISECONDS); @@ -1639,6 +1644,14 @@ public int getMaxEventDrivenThreadCount() { return maxEventDrivenThreads.get(); } + public int getActiveEventDrivenThreadCount() { + return eventDrivenEngineRef.get().getActiveCount(); + } + + public int getActiveTimerDrivenThreadCount() { + return timerDrivenEngineRef.get().getActiveCount(); + } + public void setMaxTimerDrivenThreadCount(final int maxThreadCount) { writeLock.lock(); try { @@ -1718,6 +1731,18 @@ public SystemDiagnostics getSystemDiagnostics() { return factory.create(flowFileRepository, contentRepository, provenanceRepository); } + public String getContentRepoFileStoreName(final String containerName) { + return contentRepository.getContainerFileStoreName(containerName); + } + + public String getFlowRepoFileStoreName() { + return flowFileRepository.getFileStoreName(); + } + + public String getProvenanceRepoFileStoreName(final String containerName) { + return provenanceRepository.getContainerFileStoreName(containerName); + } + // // ProcessGroup access // @@ -2641,6 +2666,26 @@ public void onFunnelRemoved(final Funnel funnel) { public Funnel getFunnel(final String id) { return allFunnels.get(id); } + + public List getGarbageCollectionStatus() { + final List statuses = new ArrayList<>(); + + final Date now = new Date(); + for (final GarbageCollectorMXBean mbean : ManagementFactory.getGarbageCollectorMXBeans()) { + final String managerName = mbean.getName(); + final long count = mbean.getCollectionCount(); + final long millis = mbean.getCollectionTime(); + final GarbageCollectionStatus status = new StandardGarbageCollectionStatus(managerName, now, count, millis); + statuses.add(status); + } + + return statuses; + } + + public GarbageCollectionHistory getGarbageCollectionHistory() { + return componentStatusRepository.getGarbageCollectionHistory(new Date(0L), new Date()); + } + /** * Returns the status of all components in the controller. This request is * not in the context of a user so the results will be unfiltered. @@ -4062,6 +4107,10 @@ public boolean isPrimary() { return isClustered() && leaderElectionManager != null && leaderElectionManager.isLeader(ClusterRoles.PRIMARY_NODE); } + public boolean isClusterCoordinator() { + return isClustered() && leaderElectionManager != null && leaderElectionManager.isLeader(ClusterRoles.CLUSTER_COORDINATOR); + } + public void setPrimary(final boolean primary) { final PrimaryNodeState nodeState = primary ? PrimaryNodeState.ELECTED_PRIMARY_NODE : PrimaryNodeState.PRIMARY_NODE_REVOKED; final ProcessGroup rootGroup = getGroup(getRootGroupId()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java index 3d378b47c5ca..804463a2319a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java @@ -218,10 +218,56 @@ public boolean isActiveQueueEmpty() { return queueSize.activeQueueCount == 0 && queueSize.swappedCount == 0; } + @Override public QueueSize getActiveQueueSize() { return size.get().activeQueueSize(); } + @Override + public QueueSize getSwapQueueSize() { + return size.get().swapQueueSize(); + } + + @Override + public int getSwapFileCount() { + readLock.lock(); + try { + return this.swapLocations.size(); + } finally { + readLock.unlock("getSwapFileCount"); + } + } + + @Override + public boolean isAllActiveFlowFilesPenalized() { + readLock.lock(); + try { + // If there are no elements then we return false + if (activeQueue.isEmpty()) { + return false; + } + + // If the first element on the queue is penalized, then we know they all are, + // because our Comparator will put Penalized FlowFiles at the end. If the first + // FlowFile is not penalized, then we also know that they are not all penalized, + // so we can simplify this by looking solely at the first FlowFile in the queue. + final FlowFileRecord first = activeQueue.peek(); + return first.isPenalized(); + } finally { + readLock.unlock("isAllActiveFlowFilesPenalized"); + } + } + + @Override + public boolean isAnyActiveFlowFilePenalized() { + readLock.lock(); + try { + return activeQueue.stream().anyMatch(FlowFileRecord::isPenalized); + } finally { + readLock.unlock("isAnyActiveFlowFilePenalized"); + } + } + @Override public void acknowledge(final FlowFileRecord flowFile) { incrementUnacknowledgedQueueSize(-1, -flowFile.getSize()); @@ -1369,7 +1415,6 @@ public QueueSize getUnacknowledgedQueueSize() { return size.get().unacknowledgedQueueSize(); } - private void incrementActiveQueueSize(final int count, final long bytes) { boolean updated = false; while (!updated) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index 12f4b1e1ed5d..b4e40c87f8f4 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -18,6 +18,9 @@ import static java.util.Objects.requireNonNull; +import java.lang.management.ManagementFactory; +import java.lang.management.ThreadInfo; +import java.lang.management.ThreadMXBean; import java.net.URL; import java.util.ArrayList; import java.util.Collection; @@ -38,6 +41,9 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.commons.lang3.builder.EqualsBuilder; import org.apache.commons.lang3.builder.HashCodeBuilder; @@ -86,6 +92,7 @@ import org.apache.nifi.util.FormatUtils; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.ReflectionUtils; +import org.apache.nifi.util.ThreadUtils; import org.quartz.CronExpression; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -134,6 +141,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable // ??????? NOT any more private ExecutionNode executionNode; private final long onScheduleTimeoutMillis; + private final Map activeThreads = new HashMap<>(48); public StandardProcessorNode(final LoggableComponent processor, final String uuid, final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler, @@ -1119,8 +1127,12 @@ public synchronized void setProcessGroup(final ProcessGroup group) { @Override public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) { final Processor processor = processorRef.get().getProcessor(); + + activateThread(); try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) { processor.onTrigger(context, sessionFactory); + } finally { + deactivateThread(); } } @@ -1321,6 +1333,44 @@ public void start(final ScheduledExecutorService taskScheduler, final long admin } } + private synchronized void activateThread() { + final Thread thread = Thread.currentThread(); + final Long timestamp = System.currentTimeMillis(); + activeThreads.put(thread, timestamp); + } + + private synchronized void deactivateThread() { + activeThreads.remove(Thread.currentThread()); + } + + @Override + public synchronized List getActiveThreads() { + final long now = System.currentTimeMillis(); + final ThreadMXBean mbean = ManagementFactory.getThreadMXBean(); + final ThreadInfo[] infos = mbean.dumpAllThreads(true, true); + final long[] deadlockedThreadIds = mbean.findDeadlockedThreads(); + final long[] monitorDeadlockThreadIds = mbean.findMonitorDeadlockedThreads(); + + final Map threadInfoMap = Stream.of(infos) + .collect(Collectors.toMap(info -> info.getThreadId(), Function.identity(), (a, b) -> a)); + + final List threadList = new ArrayList<>(activeThreads.size()); + for (final Map.Entry entry : activeThreads.entrySet()) { + final Thread thread = entry.getKey(); + final Long timestamp = entry.getValue(); + final long activeMillis = now - timestamp; + final ThreadInfo threadInfo = threadInfoMap.get(thread.getId()); + + final String stackTrace = ThreadUtils.createStackTrace(thread, threadInfo, deadlockedThreadIds, monitorDeadlockThreadIds, activeMillis); + + final ActiveThreadInfo activeThreadInfo = new ActiveThreadInfo(thread.getName(), stackTrace, activeMillis); + threadList.add(activeThreadInfo); + } + + return threadList; + } + + private void initiateStart(final ScheduledExecutorService taskScheduler, final long administrativeYieldMillis, final ProcessContext processContext, final SchedulingAgentCallback schedulingAgentCallback) { @@ -1335,7 +1385,12 @@ private void initiateStart(final ScheduledExecutorService taskScheduler, final l try (final NarCloseable nc = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) { try { - ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, processor, processContext); + activateThread(); + try { + ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, processor, processContext); + } finally { + deactivateThread(); + } if (scheduledState.compareAndSet(ScheduledState.STARTING, ScheduledState.RUNNING)) { LOG.debug("Successfully completed the @OnScheduled methods of {}; will now start triggering processor to run", processor); @@ -1344,10 +1399,12 @@ private void initiateStart(final ScheduledExecutorService taskScheduler, final l LOG.debug("Successfully invoked @OnScheduled methods of {} but scheduled state is no longer STARTING so will stop processor now", processor); // can only happen if stopProcessor was called before service was transitioned to RUNNING state + activateThread(); try { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, processor, processContext); } finally { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, processor, processContext); + deactivateThread(); } scheduledState.set(ScheduledState.STOPPED); @@ -1361,10 +1418,12 @@ private void initiateStart(final ScheduledExecutorService taskScheduler, final l // If processor's task completed Exceptionally, then we want to retry initiating the start (if Processor is still scheduled to run). try (final NarCloseable nc = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) { + activateThread(); try { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, processor, processContext); } finally { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, processor, processContext); + deactivateThread(); } } @@ -1452,8 +1511,12 @@ public void run() { try { if (scheduleState.isScheduled()) { schedulingAgent.unschedule(StandardProcessorNode.this, scheduleState); + + activateThread(); try (final NarCloseable nc = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, processor, processContext); + } finally { + deactivateThread(); } } @@ -1461,8 +1524,11 @@ public void run() { // performing the lifecycle actions counts as 1 thread. final boolean allThreadsComplete = scheduleState.getActiveThreadCount() == 1; if (allThreadsComplete) { + activateThread(); try (final NarCloseable nc = NarCloseable.withComponentNarLoader(processor.getClass(), processor.getIdentifier())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, processor, processContext); + } finally { + deactivateThread(); } scheduleState.decrementActiveThreadCount(); 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 6a8d314aa86f..256dba94e5d2 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 @@ -430,6 +430,16 @@ public long getContainerUsableSpace(String containerName) throws IOException { return FileUtils.getContainerUsableSpace(path); } + @Override + public String getContainerFileStoreName(final String containerName) { + final Path path = containers.get(containerName); + try { + return Files.getFileStore(path).name(); + } catch (IOException e) { + return null; + } + } + @Override public void cleanup() { for (final Map.Entry entry : containers.entrySet()) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java index f7ff7b842255..5863fa80c1c8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java @@ -196,6 +196,11 @@ public long getContainerUsableSpace(String containerName) throws IOException { return maxBytes - repoSize.get(); } + @Override + public String getContainerFileStoreName(String containerName) { + return null; + } + @Override public ContentClaim create(boolean lossTolerant) throws IOException { if (lossTolerant) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileFlowFileRepository.java index ae8824a249a3..dee5346553f7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileFlowFileRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileFlowFileRepository.java @@ -56,6 +56,11 @@ public long getUsableStorageSpace() throws IOException { return 0L; } + @Override + public String getFileStoreName() { + return null; + } + @Override public void close() throws IOException { } 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 3901029695e9..1d843c004ab8 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 @@ -195,6 +195,17 @@ public long getUsableStorageSpace() throws IOException { return Files.getFileStore(flowFileRepositoryPath.toPath()).getUsableSpace(); } + @Override + public String getFileStoreName() { + final Path path = flowFileRepositoryPaths.iterator().next(); + + try { + return Files.getFileStore(path).name(); + } catch (IOException e) { + return null; + } + } + @Override public void updateRepository(final Collection records) throws IOException { updateRepository(records, alwaysSync); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StandardGarbageCollectionHistory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StandardGarbageCollectionHistory.java new file mode 100644 index 000000000000..b18ac05a11c5 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StandardGarbageCollectionHistory.java @@ -0,0 +1,50 @@ +/* + * 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.status.history; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class StandardGarbageCollectionHistory implements GarbageCollectionHistory { + private final Map> statusesByManagerName = new HashMap<>(); + + @Override + public Set getMemoryManagerNames() { + return statusesByManagerName.keySet(); + } + + @Override + public List getGarbageCollectionStatuses(final String memoryManagerName) { + final List statuses = statusesByManagerName.get(memoryManagerName); + if (statuses == null) { + return Collections.emptyList(); + } + + return Collections.unmodifiableList(statuses); + } + + public void addGarbageCollectionStatus(final GarbageCollectionStatus status) { + final String managerName = status.getMemoryManagerName(); + final List statuses = statusesByManagerName.computeIfAbsent(managerName, key -> new ArrayList<>()); + statuses.add(status); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StandardGarbageCollectionStatus.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StandardGarbageCollectionStatus.java new file mode 100644 index 000000000000..3746912ea118 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StandardGarbageCollectionStatus.java @@ -0,0 +1,54 @@ +/* + * 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.status.history; + +import java.util.Date; + +public class StandardGarbageCollectionStatus implements GarbageCollectionStatus { + private final String managerName; + private final Date timestamp; + private final long collectionCount; + private final long collectionMillis; + + public StandardGarbageCollectionStatus(final String managerName, final Date timestamp, final long collectionCount, final long collectionMillis) { + this.managerName = managerName; + this.timestamp = timestamp; + this.collectionCount = collectionCount; + this.collectionMillis = collectionMillis; + } + + @Override + public Date getTimestamp() { + return timestamp; + } + + @Override + public String getMemoryManagerName() { + return managerName; + } + + @Override + public long getCollectionCount() { + return collectionCount; + } + + @Override + public long getCollectionMillis() { + return collectionMillis; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java index 50a5123bd953..6b9003087a74 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java @@ -16,6 +16,10 @@ */ package org.apache.nifi.controller.status.history; +import java.util.Date; +import java.util.List; +import java.util.Map; + import org.apache.nifi.controller.status.ConnectionStatus; import org.apache.nifi.controller.status.ProcessGroupStatus; import org.apache.nifi.controller.status.ProcessorStatus; @@ -29,9 +33,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Date; -import java.util.Map; - public class VolatileComponentStatusRepository implements ComponentStatusRepository { public static final String NUM_DATA_POINTS_PROPERTY = "nifi.components.status.repository.buffer.size"; @@ -51,19 +52,20 @@ public VolatileComponentStatusRepository(){ public VolatileComponentStatusRepository(final NiFiProperties nifiProperties) { final int numDataPoints = nifiProperties.getIntegerProperty(NUM_DATA_POINTS_PROPERTY, DEFAULT_NUM_DATA_POINTS); - captures = new RingBuffer<>(numDataPoints); } @Override - public void capture(final ProcessGroupStatus rootGroupStatus) { - capture(rootGroupStatus, new Date()); + public void capture(final ProcessGroupStatus rootGroupStatus, final List gcStatus) { + capture(rootGroupStatus, gcStatus, new Date()); } @Override - public synchronized void capture(final ProcessGroupStatus rootGroupStatus, final Date timestamp) { - captures.add(new Capture(timestamp, ComponentStatusReport.fromProcessGroupStatus(rootGroupStatus, ComponentType.PROCESSOR, - ComponentType.CONNECTION, ComponentType.PROCESS_GROUP, ComponentType.REMOTE_PROCESS_GROUP))); + public synchronized void capture(final ProcessGroupStatus rootGroupStatus, final List gcStatus, final Date timestamp) { + final ComponentStatusReport statusReport = ComponentStatusReport.fromProcessGroupStatus(rootGroupStatus, ComponentType.PROCESSOR, + ComponentType.CONNECTION, ComponentType.PROCESS_GROUP, ComponentType.REMOTE_PROCESS_GROUP); + + captures.add(new Capture(timestamp, statusReport, gcStatus)); logger.debug("Captured metrics for {}", this); lastCaptureTime = Math.max(lastCaptureTime, timestamp.getTime()); } @@ -221,15 +223,41 @@ public boolean evaluate(final Capture capture) { return history; } + @Override + public GarbageCollectionHistory getGarbageCollectionHistory(final Date start, final Date end) { + final StandardGarbageCollectionHistory history = new StandardGarbageCollectionHistory(); + + captures.forEach(new ForEachEvaluator() { + @Override + public boolean evaluate(final Capture capture) { + if (capture.getCaptureDate().before(start)) { + return true; + } + if (capture.getCaptureDate().after(end)) { + return false; + } + + final List statuses = capture.getGarbageCollectionStatus(); + if (statuses != null) { + statuses.stream().forEach(history::addGarbageCollectionStatus); + } + return true; + } + }); + + return history; + } private static class Capture { private final Date captureDate; private final ComponentStatusReport statusReport; + private final List gcStatus; - public Capture(final Date date, final ComponentStatusReport statusReport) { + public Capture(final Date date, final ComponentStatusReport statusReport, final List gcStatus) { this.captureDate = date; this.statusReport = statusReport; + this.gcStatus = gcStatus; } public Date getCaptureDate() { @@ -239,5 +267,9 @@ public Date getCaptureDate() { public ComponentStatusReport getStatusReport() { return statusReport; } + + public List getGarbageCollectionStatus() { + return gcStatus; + } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/SystemDiagnostics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/SystemDiagnostics.java index b5d159719040..11908452e1cd 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/SystemDiagnostics.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/SystemDiagnostics.java @@ -39,6 +39,10 @@ public class SystemDiagnostics implements Cloneable { private int totalThreads; private int daemonThreads; + private Long totalPhysicalMemory; + private Long maxOpenFileHandles; + private Long openFileHandles; + private long uptime; private StorageUsage flowFileRepositoryStorageUsage; @@ -200,6 +204,30 @@ public void setUptime(long uptime) { this.uptime = uptime; } + public long getTotalPhysicalMemory() { + return totalPhysicalMemory; + } + + public void setTotalPhysicalMemory(long totalPhysicalMemory) { + this.totalPhysicalMemory = totalPhysicalMemory; + } + + public long getMaxOpenFileHandles() { + return maxOpenFileHandles; + } + + public void setMaxOpenFileHandles(long maxOpenFileHandles) { + this.maxOpenFileHandles = maxOpenFileHandles; + } + + public long getOpenFileHandles() { + return openFileHandles; + } + + public void setOpenFileHandles(long openFileHandles) { + this.openFileHandles = openFileHandles; + } + @Override public SystemDiagnostics clone() { final SystemDiagnostics clonedObj = new SystemDiagnostics(); @@ -239,6 +267,9 @@ public SystemDiagnostics clone() { clonedObj.usedNonHeap = usedNonHeap; clonedObj.creationTimestamp = creationTimestamp; clonedObj.uptime = uptime; + clonedObj.totalPhysicalMemory = totalPhysicalMemory; + clonedObj.openFileHandles = openFileHandles; + clonedObj.maxOpenFileHandles = maxOpenFileHandles; return clonedObj; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/SystemDiagnosticsFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/SystemDiagnosticsFactory.java index 45a8af740ebd..5c864951e84d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/SystemDiagnosticsFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/SystemDiagnosticsFactory.java @@ -24,6 +24,7 @@ import java.lang.management.OperatingSystemMXBean; import java.lang.management.RuntimeMXBean; import java.lang.management.ThreadMXBean; +import java.lang.reflect.Method; import java.util.Date; import java.util.LinkedHashMap; import java.util.List; @@ -154,6 +155,30 @@ public SystemDiagnostics create(final FlowFileRepository flowFileRepo, final Con } systemDiagnostics.setGarbageCollection(garbageCollection); + // This information is available only for *nix systems. + final OperatingSystemMXBean osStats = ManagementFactory.getOperatingSystemMXBean(); + try { + final Class unixOsMxBeanClass = Class.forName("com.sun.management.UnixOperatingSystemMXBean"); + if (unixOsMxBeanClass.isAssignableFrom(osStats.getClass())) { + final Method totalPhysicalMemory = unixOsMxBeanClass.getMethod("getTotalPhysicalMemorySize"); + totalPhysicalMemory.setAccessible(true); + final Long ramBytes = (Long) totalPhysicalMemory.invoke(osStats); + systemDiagnostics.setTotalPhysicalMemory(ramBytes); + + final Method maxFileDescriptors = unixOsMxBeanClass.getMethod("getMaxFileDescriptorCount"); + maxFileDescriptors.setAccessible(true); + final Long maxOpenFileDescriptors = (Long) maxFileDescriptors.invoke(osStats); + systemDiagnostics.setMaxOpenFileHandles(maxOpenFileDescriptors); + + final Method openFileDescriptors = unixOsMxBeanClass.getMethod("getOpenFileDescriptorCount"); + openFileDescriptors.setAccessible(true); + final Long openDescriptorCount = (Long) openFileDescriptors.invoke(osStats); + systemDiagnostics.setOpenFileHandles(openDescriptorCount); + } + } catch (final Throwable t) { + // Ignore. This will throw either ClassNotFound or NoClassDefFoundError if unavailable in this JVM. + } + // set the creation timestamp systemDiagnostics.setCreationTimestamp(new Date().getTime()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/ThreadUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/ThreadUtils.java new file mode 100644 index 000000000000..780c90738462 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/ThreadUtils.java @@ -0,0 +1,90 @@ +/* + * 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.util; + +import java.lang.management.LockInfo; +import java.lang.management.MonitorInfo; +import java.lang.management.ThreadInfo; + +public class ThreadUtils { + + public static String createStackTrace(final Thread thread, final ThreadInfo threadInfo, final long[] deadlockedThreadIds, final long[] monitorDeadlockThreadIds, final long activeMillis) { + final StringBuilder sb = new StringBuilder(); + sb.append("\"").append(threadInfo.getThreadName()).append("\" Id="); + sb.append(threadInfo.getThreadId()).append(" "); + sb.append(threadInfo.getThreadState().toString()).append(" "); + + switch (threadInfo.getThreadState()) { + case BLOCKED: + case TIMED_WAITING: + case WAITING: + sb.append(" on "); + sb.append(threadInfo.getLockInfo()); + break; + default: + break; + } + + if (threadInfo.isSuspended()) { + sb.append(" (suspended)"); + } + if (threadInfo.isInNative()) { + sb.append(" (in native code)"); + } + + if (deadlockedThreadIds != null && deadlockedThreadIds.length > 0) { + for (final long id : deadlockedThreadIds) { + if (id == threadInfo.getThreadId()) { + sb.append(" ** DEADLOCKED THREAD **"); + } + } + } + + if (monitorDeadlockThreadIds != null && monitorDeadlockThreadIds.length > 0) { + for (final long id : monitorDeadlockThreadIds) { + if (id == threadInfo.getThreadId()) { + sb.append(" ** MONITOR-DEADLOCKED THREAD **"); + } + } + } + + final StackTraceElement[] stackTraces = threadInfo.getStackTrace(); + for (final StackTraceElement element : stackTraces) { + sb.append("\n\tat ").append(element); + + final MonitorInfo[] monitors = threadInfo.getLockedMonitors(); + for (final MonitorInfo monitor : monitors) { + if (monitor.getLockedStackFrame().equals(element)) { + sb.append("\n\t- waiting on ").append(monitor); + } + } + } + + final LockInfo[] lockInfos = threadInfo.getLockedSynchronizers(); + if (lockInfos.length > 0) { + sb.append("\n\t"); + sb.append("Number of Locked Synchronizers: ").append(lockInfos.length); + for (final LockInfo lockInfo : lockInfos) { + sb.append("\n\t- ").append(lockInfo.toString()); + } + } + + sb.append("\n"); + return sb.toString(); + } +} 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 cf91ae2a0411..22a013174946 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 @@ -1971,6 +1971,11 @@ public long getUsableStorageSpace() throws IOException { return 0; } + @Override + public String getFileStoreName() { + return null; + } + @Override public boolean isVolatile() { return false; @@ -2090,6 +2095,11 @@ public long getContainerUsableSpace(String containerName) throws IOException { return 0; } + @Override + public String getContainerFileStoreName(String containerName) { + return null; + } + @Override public int decrementClaimantCount(ContentClaim claim) { if (claim == null) { 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 329b26819e04..c1e2d5ca73da 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 @@ -112,6 +112,11 @@ public SwapSummary recoverSwappedFlowFiles() { public void purgeSwapFiles() { } + @Override + public int getSwapFileCount() { + return 0; + } + @Override public void setPriorities(List newPriorities) { } @@ -154,6 +159,16 @@ public QueueSize getUnacknowledgedQueueSize() { return null; } + @Override + public QueueSize getActiveQueueSize() { + return size(); + } + + @Override + public QueueSize getSwapQueueSize() { + return null; + } + @Override public void acknowledge(FlowFileRecord flowFile) { } @@ -162,6 +177,16 @@ public void acknowledge(FlowFileRecord flowFile) { public void acknowledge(Collection flowFiles) { } + @Override + public boolean isAllActiveFlowFilesPenalized() { + return false; + } + + @Override + public boolean isAnyActiveFlowFilePenalized() { + return false; + } + @Override public boolean isFull() { return false; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java index 8326889f2408..350135190696 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/ExtensionManager.java @@ -95,6 +95,12 @@ public class ExtensionManager { definitionMap.put(StateProvider.class, new HashSet<>()); } + public static Set getAllBundles() { + return classNameBundleLookup.values().stream() + .flatMap(List::stream) + .collect(Collectors.toSet()); + } + /** * Loads all FlowFileProcessor, FlowFileComparator, ReportingTask class types that can be found on the bootstrap classloader and by creating classloaders for all NARs found within the classpath. * @param narBundles the bundles to scan through in search of extensions diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java index 009b096f15b3..ce9444065b1e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java @@ -95,6 +95,7 @@ import org.apache.nifi.web.api.entity.ProcessGroupEntity; import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity; import org.apache.nifi.web.api.entity.ProcessGroupStatusEntity; +import org.apache.nifi.web.api.entity.ProcessorDiagnosticsEntity; import org.apache.nifi.web.api.entity.ProcessorEntity; import org.apache.nifi.web.api.entity.ProcessorStatusEntity; import org.apache.nifi.web.api.entity.RegistryClientEntity; @@ -524,6 +525,14 @@ Set getControllerServiceTypes(final String serviceType, final */ ProcessorEntity getProcessor(String id); + /** + * Gets the Diagnostic information for the Processor with the specified id + * + * @param id the id of the processor + * @return the diagnostics information for the processor + */ + ProcessorDiagnosticsEntity getProcessorDiagnostics(String id); + /** * Gets the Processor transfer object for the specified id, as it is visible to the given user * diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java index f57f628cefe2..80283f7d9c37 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java @@ -78,6 +78,7 @@ import org.apache.nifi.controller.service.ControllerServiceReference; import org.apache.nifi.controller.service.ControllerServiceState; import org.apache.nifi.controller.status.ProcessGroupStatus; +import org.apache.nifi.controller.status.ProcessorStatus; import org.apache.nifi.diagnostics.SystemDiagnostics; import org.apache.nifi.events.BulletinFactory; import org.apache.nifi.groups.ProcessGroup; @@ -137,6 +138,7 @@ import org.apache.nifi.web.api.dto.ComponentHistoryDTO; import org.apache.nifi.web.api.dto.ComponentReferenceDTO; import org.apache.nifi.web.api.dto.ComponentStateDTO; +import org.apache.nifi.web.api.dto.ConnectableDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; import org.apache.nifi.web.api.dto.ControllerConfigurationDTO; import org.apache.nifi.web.api.dto.ControllerDTO; @@ -180,6 +182,11 @@ import org.apache.nifi.web.api.dto.VersionedFlowDTO; import org.apache.nifi.web.api.dto.action.HistoryDTO; import org.apache.nifi.web.api.dto.action.HistoryQueryDTO; +import org.apache.nifi.web.api.dto.diagnostics.ConnectionDiagnosticsDTO; +import org.apache.nifi.web.api.dto.diagnostics.ControllerServiceDiagnosticsDTO; +import org.apache.nifi.web.api.dto.diagnostics.JVMDiagnosticsDTO; +import org.apache.nifi.web.api.dto.diagnostics.JVMDiagnosticsSnapshotDTO; +import org.apache.nifi.web.api.dto.diagnostics.ProcessorDiagnosticsDTO; import org.apache.nifi.web.api.dto.flow.FlowDTO; import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO; import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO; @@ -222,6 +229,7 @@ import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity; import org.apache.nifi.web.api.entity.ProcessGroupStatusEntity; import org.apache.nifi.web.api.entity.ProcessGroupStatusSnapshotEntity; +import org.apache.nifi.web.api.entity.ProcessorDiagnosticsEntity; import org.apache.nifi.web.api.entity.ProcessorEntity; import org.apache.nifi.web.api.entity.ProcessorStatusEntity; import org.apache.nifi.web.api.entity.RegistryClientEntity; @@ -294,6 +302,7 @@ import java.util.Set; import java.util.UUID; import java.util.function.Function; +import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -4506,6 +4515,123 @@ public ComponentHistoryDTO getComponentHistory(final String componentId) { return history; } + private ControllerServiceEntity createControllerServiceEntity(final String serviceId, final NiFiUser user) { + final ControllerServiceNode serviceNode = controllerServiceDAO.getControllerService(serviceId); + return createControllerServiceEntity(serviceNode, Collections.emptySet(), user); + } + + @Override + public ProcessorDiagnosticsEntity getProcessorDiagnostics(final String id) { + final ProcessorNode processor = processorDAO.getProcessor(id); + final ProcessorStatus processorStatus = controllerFacade.getProcessorStatus(id); + + // Generate Processor Diagnostics + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + final ProcessorDiagnosticsDTO dto = controllerFacade.getProcessorDiagnostics(processor, processorStatus, bulletinRepository, serviceId -> createControllerServiceEntity(serviceId, user)); + + // Filter anything out of diagnostics that the user is not authorized to see. + final List jvmDiagnosticsSnaphots = new ArrayList<>(); + final JVMDiagnosticsDTO jvmDiagnostics = dto.getJvmDiagnostics(); + jvmDiagnosticsSnaphots.add(jvmDiagnostics.getAggregateSnapshot()); + + // filter controller-related information + final boolean canReadController = authorizableLookup.getController().isAuthorized(authorizer, RequestAction.READ, user); + if (!canReadController) { + for (final JVMDiagnosticsSnapshotDTO snapshot : jvmDiagnosticsSnaphots) { + snapshot.setMaxEventDrivenThreads(null); + snapshot.setMaxTimerDrivenThreads(null); + snapshot.setBundlesLoaded(null); + } + } + + // filter system diagnostics information + final boolean canReadSystem = authorizableLookup.getSystem().isAuthorized(authorizer, RequestAction.READ, user); + if (!canReadSystem) { + for (final JVMDiagnosticsSnapshotDTO snapshot : jvmDiagnosticsSnaphots) { + snapshot.setContentRepositoryStorageUsage(null); + snapshot.setCpuCores(null); + snapshot.setCpuLoadAverage(null); + snapshot.setFlowFileRepositoryStorageUsage(null); + snapshot.setMaxHeap(null); + snapshot.setMaxHeapBytes(null); + snapshot.setProvenanceRepositoryStorageUsage(null); + snapshot.setPhysicalMemory(null); + snapshot.setPhysicalMemoryBytes(null); + snapshot.setGarbageCollectionDiagnostics(null); + } + } + + // filter connections + final Predicate connectionAuthorized = connectionDiagnostics -> { + final String connectionId = connectionDiagnostics.getConnection().getId(); + return authorizableLookup.getConnection(connectionId).getAuthorizable().isAuthorized(authorizer, RequestAction.READ, user); + }; + + // Function that can be used to remove the Source or Destination of a ConnectionDTO, if the user is not authorized. + final Function filterSourceDestination = connectionDiagnostics -> { + final ConnectionDTO connection = connectionDiagnostics.getConnection(); + final ConnectableDTO sourceDto = connection.getSource(); + final Authorizable sourceAuthorizable = authorizableLookup.getLocalConnectable(sourceDto.getId()); + if (sourceAuthorizable == null || !sourceAuthorizable.isAuthorized(authorizer, RequestAction.READ, user)) { + connection.setSource(null); + } + + final ConnectableDTO destinationDto = connection.getDestination(); + final Authorizable destinationAuthorizable = authorizableLookup.getLocalConnectable(destinationDto.getId()); + if (destinationAuthorizable == null || !destinationAuthorizable.isAuthorized(authorizer, RequestAction.READ, user)) { + connection.setDestination(null); + } + + return connectionDiagnostics; + }; + + // Filter incoming connections by what user is authorized to READ + final Set incoming = dto.getIncomingConnections(); + final Set filteredIncoming = incoming.stream() + .filter(connectionAuthorized) + .map(filterSourceDestination) + .collect(Collectors.toSet()); + + dto.setIncomingConnections(filteredIncoming); + + // Filter outgoing connections by what user is authorized to READ + final Set outgoing = dto.getOutgoingConnections(); + final Set filteredOutgoing = outgoing.stream() + .filter(connectionAuthorized) + .map(filterSourceDestination) + .collect(Collectors.toSet()); + dto.setOutgoingConnections(filteredOutgoing); + + // Filter out any controller services that are referenced by the Processor + final Set referencedServices = dto.getReferencedControllerServices(); + final Set filteredReferencedServices = referencedServices.stream() + .filter(csDiagnostics -> { + final String csId = csDiagnostics.getControllerService().getId(); + return authorizableLookup.getControllerService(csId).getAuthorizable().isAuthorized(authorizer, RequestAction.READ, user); + }) + .map(csDiagnostics -> { + // Filter out any referencing components because those are generally not relevant from this context. + final ControllerServiceDTO serviceDto = csDiagnostics.getControllerService().getComponent(); + if (serviceDto != null) { + serviceDto.setReferencingComponents(null); + } + return csDiagnostics; + }) + .collect(Collectors.toSet()); + dto.setReferencedControllerServices(filteredReferencedServices); + + final Revision revision = revisionManager.getRevision(id); + final RevisionDTO revisionDto = dtoFactory.createRevisionDTO(revision); + final PermissionsDTO permissionsDto = dtoFactory.createPermissionsDto(processor); + final List bulletins = bulletinRepository.findBulletinsForSource(id).stream() + .map(bulletin -> dtoFactory.createBulletinDto(bulletin)) + .map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissionsDto.getCanRead())) + .collect(Collectors.toList()); + + final ProcessorStatusDTO processorStatusDto = dtoFactory.createProcessorStatusDto(controllerFacade.getProcessorStatus(processor.getIdentifier())); + return entityFactory.createProcessorDiagnosticsEntity(dto, revisionDto, permissionsDto, processorStatusDto, bulletins); + } + @Override public boolean isClustered() { return controllerFacade.isClustered(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java index 460ed37dc2b6..fcf1a9b30383 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java @@ -42,6 +42,7 @@ import org.apache.nifi.web.api.dto.ProcessorDTO; import org.apache.nifi.web.api.dto.PropertyDescriptorDTO; import org.apache.nifi.web.api.entity.ComponentStateEntity; +import org.apache.nifi.web.api.entity.ProcessorDiagnosticsEntity; import org.apache.nifi.web.api.entity.ProcessorEntity; import org.apache.nifi.web.api.entity.PropertyDescriptorEntity; import org.apache.nifi.web.api.request.ClientIdParameter; @@ -110,6 +111,22 @@ public ProcessorEntity populateRemainingProcessorEntityContent(ProcessorEntity p return processorEntity; } + /** + * Populate the uri's for the specified processors and their relationships. + * + * @param processorDiagnosticsEntity processor's diagnostics entity + * @return processor diagnostics entity + */ + public ProcessorDiagnosticsEntity populateRemainingProcessorDiagnosticsEntityContent(ProcessorDiagnosticsEntity processorDiagnosticsEntity) { + processorDiagnosticsEntity.setUri(generateResourceUri("processors", processorDiagnosticsEntity.getId(), "diagnostics")); + + // populate remaining content + if (processorDiagnosticsEntity.getComponent() != null && processorDiagnosticsEntity.getComponent().getProcessor() != null) { + populateRemainingProcessorContent(processorDiagnosticsEntity.getComponent().getProcessor()); + } + return processorDiagnosticsEntity; + } + /** * Populate the uri's for the specified processor and its relationships. */ @@ -192,6 +209,43 @@ public Response getProcessor( return generateOkResponse(entity).build(); } + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/diagnostics") + @ApiOperation(value = "Gets diagnostics information about a processor", + response = ProcessorEntity.class, + notes = NON_GUARANTEED_ENDPOINT, + authorizations = { @Authorization(value = "Read - /processors/{uuid}")} + ) + @ApiResponses(value = { + @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(code = 401, message = "Client could not be authenticated."), + @ApiResponse(code = 403, message = "Client is not authorized to make this request."), + @ApiResponse(code = 404, message = "The specified resource could not be found."), + @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.") + }) + public Response getProcessorDiagnostics( + @ApiParam(value = "The processor id.", required = true) @PathParam("id") final String id) throws InterruptedException { + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + // authorize access + serviceFacade.authorizeAccess(lookup -> { + final Authorizable processor = lookup.getProcessor(id).getAuthorizable(); + processor.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); + }); + + // get the specified processor's diagnostics + final ProcessorDiagnosticsEntity entity = serviceFacade.getProcessorDiagnostics(id); + populateRemainingProcessorDiagnosticsEntityContent(entity); + + // generate the response + return generateOkResponse(entity).build(); + } + /** * Returns the descriptor for the specified property. * diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java index 359b5241b76d..b0ef2493f127 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java @@ -16,6 +16,8 @@ */ package org.apache.nifi.web.api.dto; +import javax.ws.rs.WebApplicationException; + import org.apache.commons.lang3.ClassUtils; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.action.Action; @@ -68,9 +70,11 @@ import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.Port; import org.apache.nifi.connectable.Position; +import org.apache.nifi.controller.ActiveThreadInfo; import org.apache.nifi.controller.ConfiguredComponent; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.Counter; +import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.Snippet; @@ -78,6 +82,7 @@ import org.apache.nifi.controller.label.Label; 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.queue.FlowFileSummary; import org.apache.nifi.controller.queue.ListFlowFileState; import org.apache.nifi.controller.queue.ListFlowFileStatus; @@ -93,6 +98,8 @@ import org.apache.nifi.controller.status.ProcessGroupStatus; import org.apache.nifi.controller.status.ProcessorStatus; import org.apache.nifi.controller.status.RemoteProcessGroupStatus; +import org.apache.nifi.controller.status.history.GarbageCollectionHistory; +import org.apache.nifi.controller.status.history.GarbageCollectionStatus; import org.apache.nifi.diagnostics.GarbageCollection; import org.apache.nifi.diagnostics.StorageUsage; import org.apache.nifi.diagnostics.SystemDiagnostics; @@ -153,6 +160,16 @@ import org.apache.nifi.web.api.dto.action.details.ConnectDetailsDTO; import org.apache.nifi.web.api.dto.action.details.MoveDetailsDTO; import org.apache.nifi.web.api.dto.action.details.PurgeDetailsDTO; +import org.apache.nifi.web.api.dto.diagnostics.ClassLoaderDiagnosticsDTO; +import org.apache.nifi.web.api.dto.diagnostics.ConnectionDiagnosticsDTO; +import org.apache.nifi.web.api.dto.diagnostics.ControllerServiceDiagnosticsDTO; +import org.apache.nifi.web.api.dto.diagnostics.GCDiagnosticsSnapshotDTO; +import org.apache.nifi.web.api.dto.diagnostics.GarbageCollectionDiagnosticsDTO; +import org.apache.nifi.web.api.dto.diagnostics.JVMDiagnosticsDTO; +import org.apache.nifi.web.api.dto.diagnostics.JVMDiagnosticsSnapshotDTO; +import org.apache.nifi.web.api.dto.diagnostics.ProcessorDiagnosticsDTO; +import org.apache.nifi.web.api.dto.diagnostics.RepositoryUsageDTO; +import org.apache.nifi.web.api.dto.diagnostics.ThreadDumpDTO; import org.apache.nifi.web.api.dto.flow.FlowBreadcrumbDTO; import org.apache.nifi.web.api.dto.flow.FlowDTO; import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO; @@ -193,8 +210,10 @@ import org.apache.nifi.web.controller.ControllerFacade; import org.apache.nifi.web.revision.RevisionManager; -import javax.ws.rs.WebApplicationException; +import java.net.URL; +import java.net.URLClassLoader; import java.text.Collator; +import java.text.NumberFormat; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -3116,6 +3135,312 @@ public ResourceDTO createResourceDto(final Resource resource) { return dto; } + /** + * Creates a ProcessorDiagnosticsDTO from the given Processor and status information with some additional supporting information + * + * @param procNode the processor to create diagnostics for + * @param procStatus the status of given processor + * @param bulletinRepo the bulletin repository + * @param flowController flowController + * @param serviceEntityFactory function for creating a ControllerServiceEntity from a given ID + * @return ProcessorDiagnosticsDTO for the given Processor + */ + public ProcessorDiagnosticsDTO createProcessorDiagnosticsDto(final ProcessorNode procNode, final ProcessorStatus procStatus, final BulletinRepository bulletinRepo, + final FlowController flowController, final Function serviceEntityFactory) { + + final ProcessorDiagnosticsDTO procDiagnostics = new ProcessorDiagnosticsDTO(); + + procDiagnostics.setClassLoaderDiagnostics(createClassLoaderDiagnosticsDto(procNode)); + procDiagnostics.setIncomingConnections(procNode.getIncomingConnections().stream() + .map(this::createConnectionDiagnosticsDto) + .collect(Collectors.toSet())); + procDiagnostics.setOutgoingConnections(procNode.getConnections().stream() + .map(this::createConnectionDiagnosticsDto) + .collect(Collectors.toSet())); + procDiagnostics.setJvmDiagnostics(createJvmDiagnosticsDto(flowController)); + procDiagnostics.setProcessor(createProcessorDto(procNode)); + procDiagnostics.setProcessorStatus(createProcessorStatusDto(procStatus)); + procDiagnostics.setThreadDumps(createThreadDumpDtos(procNode)); + + final Set referencedServiceDiagnostics = createReferencedServiceDiagnostics(procNode.getProperties(), flowController, serviceEntityFactory); + procDiagnostics.setReferencedControllerServices(referencedServiceDiagnostics); + + return procDiagnostics; + } + + private Set createReferencedServiceDiagnostics(final Map properties, final ControllerServiceProvider serviceProvider, + final Function serviceEntityFactory) { + + final Set referencedServiceDiagnostics = new HashSet<>(); + for (final Map.Entry entry : properties.entrySet()) { + final PropertyDescriptor descriptor = entry.getKey(); + if (descriptor.getControllerServiceDefinition() == null) { + continue; + } + + final String serviceId = entry.getValue(); + final ControllerServiceNode serviceNode = serviceProvider.getControllerServiceNode(serviceId); + if (serviceNode == null) { + continue; + } + + final ControllerServiceDiagnosticsDTO serviceDiagnostics = createControllerServiceDiagnosticsDto(serviceNode, serviceEntityFactory, serviceProvider); + if (serviceDiagnostics != null) { + referencedServiceDiagnostics.add(serviceDiagnostics); + } + } + + return referencedServiceDiagnostics; + } + + /** + * Creates a ControllerServiceDiagnosticsDTO from the given Controller Service with some additional supporting information + * + * @param serviceNode the controller service to create diagnostics for + * @param bulletinRepo the bulletin repository + * @param serviceProvider the controller service provider + * @return ControllerServiceDiagnosticsDTO for the given Controller Service + */ + public ControllerServiceDiagnosticsDTO createControllerServiceDiagnosticsDto(final ControllerServiceNode serviceNode, final Function serviceEntityFactory, + final ControllerServiceProvider serviceProvider) { + + final ControllerServiceDiagnosticsDTO serviceDiagnostics = new ControllerServiceDiagnosticsDTO(); + final ControllerServiceEntity serviceEntity = serviceEntityFactory.apply(serviceNode.getIdentifier()); + serviceDiagnostics.setControllerService(serviceEntity); + + serviceDiagnostics.setClassLoaderDiagnostics(createClassLoaderDiagnosticsDto(serviceNode)); + return serviceDiagnostics; + } + + + private ClassLoaderDiagnosticsDTO createClassLoaderDiagnosticsDto(final ControllerServiceNode serviceNode) { + ClassLoader componentClassLoader = ExtensionManager.getInstanceClassLoader(serviceNode.getIdentifier()); + if (componentClassLoader == null) { + componentClassLoader = serviceNode.getControllerServiceImplementation().getClass().getClassLoader(); + } + + return createClassLoaderDiagnosticsDto(componentClassLoader); + } + + + private ClassLoaderDiagnosticsDTO createClassLoaderDiagnosticsDto(final ProcessorNode procNode) { + ClassLoader componentClassLoader = ExtensionManager.getInstanceClassLoader(procNode.getIdentifier()); + if (componentClassLoader == null) { + componentClassLoader = procNode.getProcessor().getClass().getClassLoader(); + } + + return createClassLoaderDiagnosticsDto(componentClassLoader); + } + + private ClassLoaderDiagnosticsDTO createClassLoaderDiagnosticsDto(final ClassLoader classLoader) { + final ClassLoaderDiagnosticsDTO dto = new ClassLoaderDiagnosticsDTO(); + + final Bundle bundle = ExtensionManager.getBundle(classLoader); + if (bundle != null) { + dto.setBundle(createBundleDto(bundle.getBundleDetails().getCoordinate())); + } + + final Set filesLoaded = new HashSet<>(); + if (classLoader instanceof URLClassLoader) { + final URLClassLoader urlClassLoader = (URLClassLoader) classLoader; + final URL[] urls = urlClassLoader.getURLs(); + + for (final URL url : urls) { + final String path = url.getPath(); + final int lastSlash = path.lastIndexOf("/"); + final String filename; + if (lastSlash >= 0) { + filename = path.substring(lastSlash + 1); + } else { + filename = path; + } + + if (!filename.isEmpty()) { // filter out directories + filesLoaded.add(filename); + } + } + } + + dto.setLoadedFiles(filesLoaded); + + final ClassLoader parentClassLoader = classLoader.getParent(); + if (parentClassLoader != null) { + dto.setParentClassLoader(createClassLoaderDiagnosticsDto(parentClassLoader)); + } + + return dto; + } + + private ConnectionDiagnosticsDTO createConnectionDiagnosticsDto(final Connection connection) { + final ConnectionDiagnosticsDTO dto = new ConnectionDiagnosticsDTO(); + dto.setConnection(createConnectionDto(connection)); + + final FlowFileQueue queue = connection.getFlowFileQueue(); + final QueueSize totalSize = queue.size(); + dto.setTotalByteCount(totalSize.getByteCount()); + dto.setTotalFlowFileCount(totalSize.getObjectCount()); + + final QueueSize activeSize = queue.getActiveQueueSize(); + dto.setActiveQueueByteCount(activeSize.getByteCount()); + dto.setActiveQueueFlowFileCount(activeSize.getObjectCount()); + + final QueueSize inFlightSize = queue.getUnacknowledgedQueueSize(); + dto.setInFlightByteCount(inFlightSize.getByteCount()); + dto.setInFlightFlowFileCount(inFlightSize.getObjectCount()); + + final QueueSize swapSize = queue.getSwapQueueSize(); + dto.setSwapByteCount(swapSize.getByteCount()); + dto.setSwapFlowFileCount(swapSize.getObjectCount()); + + dto.setSwapFiles(queue.getSwapFileCount()); + dto.setAllActiveQueueFlowFilesPenalized(queue.isAllActiveFlowFilesPenalized()); + dto.setAnyActiveQueueFlowFilesPenalized(queue.isAnyActiveFlowFilePenalized()); + + return dto; + } + + private JVMDiagnosticsDTO createJvmDiagnosticsDto(final FlowController flowController) { + final JVMDiagnosticsDTO dto = new JVMDiagnosticsDTO(); + dto.setAggregateSnapshot(createJvmDiagnosticsSnapshotDto(flowController)); + dto.setClustered(flowController.isClustered()); + dto.setConnected(flowController.isConnected()); + return dto; + } + + private JVMDiagnosticsSnapshotDTO createJvmDiagnosticsSnapshotDto(final FlowController flowController) { + final JVMDiagnosticsSnapshotDTO dto = new JVMDiagnosticsSnapshotDTO(); + + final SystemDiagnostics systemDiagnostics = flowController.getSystemDiagnostics(); + + final Set bundlesLoaded = ExtensionManager.getAllBundles().stream() + .map(bundle -> bundle.getBundleDetails().getCoordinate()) + .sorted((a, b) -> a.getCoordinate().compareTo(b.getCoordinate())) + .map(this::createBundleDto) + .collect(Collectors.toCollection(LinkedHashSet::new)); + + dto.setMaxOpenFileDescriptors(systemDiagnostics.getMaxOpenFileHandles()); + dto.setOpenFileDescriptors(systemDiagnostics.getOpenFileHandles()); + dto.setPhysicalMemoryBytes(systemDiagnostics.getTotalPhysicalMemory()); + dto.setPhysicalMemory(FormatUtils.formatDataSize(systemDiagnostics.getTotalPhysicalMemory())); + + final NumberFormat percentageFormat = NumberFormat.getPercentInstance(); + percentageFormat.setMaximumFractionDigits(2); + + final Set contentRepoUsage = new HashSet<>(); + for (final Map.Entry entry : systemDiagnostics.getContentRepositoryStorageUsage().entrySet()) { + final String repoName = entry.getKey(); + final StorageUsage usage = entry.getValue(); + + final RepositoryUsageDTO usageDto = new RepositoryUsageDTO(); + usageDto.setName(repoName); + + usageDto.setFileStore(flowController.getContentRepoFileStoreName(repoName)); + usageDto.setFreeSpace(FormatUtils.formatDataSize(usage.getFreeSpace())); + usageDto.setFreeSpaceBytes(usage.getFreeSpace()); + usageDto.setTotalSpace(FormatUtils.formatDataSize(usage.getTotalSpace())); + usageDto.setTotalSpaceBytes(usage.getTotalSpace()); + + final double usedPercentage = (usage.getTotalSpace() - usage.getFreeSpace()) / (double) usage.getTotalSpace(); + final String utilization = percentageFormat.format(usedPercentage); + usageDto.setUtilization(utilization); + contentRepoUsage.add(usageDto); + } + + final Set provRepoUsage = new HashSet<>(); + for (final Map.Entry entry : systemDiagnostics.getProvenanceRepositoryStorageUsage().entrySet()) { + final String repoName = entry.getKey(); + final StorageUsage usage = entry.getValue(); + + final RepositoryUsageDTO usageDto = new RepositoryUsageDTO(); + usageDto.setName(repoName); + + usageDto.setFileStore(flowController.getProvenanceRepoFileStoreName(repoName)); + usageDto.setFreeSpace(FormatUtils.formatDataSize(usage.getFreeSpace())); + usageDto.setFreeSpaceBytes(usage.getFreeSpace()); + usageDto.setTotalSpace(FormatUtils.formatDataSize(usage.getTotalSpace())); + usageDto.setTotalSpaceBytes(usage.getTotalSpace()); + + final double usedPercentage = (usage.getTotalSpace() - usage.getFreeSpace()) / (double) usage.getTotalSpace(); + final String utilization = percentageFormat.format(usedPercentage); + usageDto.setUtilization(utilization); + provRepoUsage.add(usageDto); + } + + final RepositoryUsageDTO flowFileRepoUsage = new RepositoryUsageDTO(); + for (final Map.Entry entry : systemDiagnostics.getProvenanceRepositoryStorageUsage().entrySet()) { + final String repoName = entry.getKey(); + final StorageUsage usage = entry.getValue(); + + flowFileRepoUsage.setName(repoName); + + flowFileRepoUsage.setFileStore(flowController.getFlowRepoFileStoreName()); + flowFileRepoUsage.setFreeSpace(FormatUtils.formatDataSize(usage.getFreeSpace())); + flowFileRepoUsage.setFreeSpaceBytes(usage.getFreeSpace()); + flowFileRepoUsage.setTotalSpace(FormatUtils.formatDataSize(usage.getTotalSpace())); + flowFileRepoUsage.setTotalSpaceBytes(usage.getTotalSpace()); + + final double usedPercentage = (usage.getTotalSpace() - usage.getFreeSpace()) / (double) usage.getTotalSpace(); + final String utilization = percentageFormat.format(usedPercentage); + flowFileRepoUsage.setUtilization(utilization); + } + + dto.setActiveEventDrivenThreads(flowController.getActiveEventDrivenThreadCount()); + dto.setActiveTimerDrivenThreads(flowController.getActiveTimerDrivenThreadCount()); + dto.setBundlesLoaded(bundlesLoaded); + dto.setClusterCoordinator(flowController.isClusterCoordinator()); + dto.setContentRepositoryStorageUsage(contentRepoUsage); + dto.setCpuCores(systemDiagnostics.getAvailableProcessors()); + dto.setCpuLoadAverage(systemDiagnostics.getProcessorLoadAverage()); + dto.setFlowFileRepositoryStorageUsage(flowFileRepoUsage); + dto.setMaxHeapBytes(systemDiagnostics.getMaxHeap()); + dto.setMaxHeap(FormatUtils.formatDataSize(systemDiagnostics.getMaxHeap())); + dto.setMaxEventDrivenThreads(flowController.getMaxEventDrivenThreadCount()); + dto.setMaxTimerDrivenThreads(flowController.getMaxTimerDrivenThreadCount()); + dto.setPrimaryNode(flowController.isPrimary()); + dto.setProvenanceRepositoryStorageUsage(provRepoUsage); + dto.setTimeZone(System.getProperty("user.timezone")); + dto.setUptime(FormatUtils.formatHoursMinutesSeconds(systemDiagnostics.getUptime(), TimeUnit.MILLISECONDS)); + + // Create the Garbage Collection History info + final GarbageCollectionHistory gcHistory = flowController.getGarbageCollectionHistory(); + final List gcDiagnostics = new ArrayList<>(); + for (final String memoryManager : gcHistory.getMemoryManagerNames()) { + final List statuses = gcHistory.getGarbageCollectionStatuses(memoryManager); + + final List gcSnapshots = new ArrayList<>(); + for (final GarbageCollectionStatus status : statuses) { + final GCDiagnosticsSnapshotDTO snapshotDto = new GCDiagnosticsSnapshotDTO(); + snapshotDto.setTimestamp(status.getTimestamp()); + snapshotDto.setCollectionCount(status.getCollectionCount()); + snapshotDto.setCollectionMillis(status.getCollectionMillis()); + gcSnapshots.add(snapshotDto); + } + + final GarbageCollectionDiagnosticsDTO gcDto = new GarbageCollectionDiagnosticsDTO(); + gcDto.setMemoryManagerName(memoryManager); + gcDto.setSnapshots(gcSnapshots); + gcDiagnostics.add(gcDto); + } + + dto.setGarbageCollectionDiagnostics(gcDiagnostics); + return dto; + } + + private List createThreadDumpDtos(final ProcessorNode procNode) { + final List threadDumps = new ArrayList<>(); + + final List activeThreads = procNode.getActiveThreads(); + for (final ActiveThreadInfo threadInfo : activeThreads) { + final ThreadDumpDTO dto = new ThreadDumpDTO(); + dto.setStackTrace(threadInfo.getStackTrace()); + dto.setThreadActiveMillis(threadInfo.getActiveMillis()); + dto.setThreadName(threadInfo.getThreadName()); + threadDumps.add(dto); + } + + return threadDumps; + } + /** * Creates a ProcessorConfigDTO from the specified ProcessorNode. * diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java index 566b519b35c8..c506415df357 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java @@ -17,6 +17,7 @@ package org.apache.nifi.web.api.dto; import org.apache.nifi.web.api.dto.action.ActionDTO; +import org.apache.nifi.web.api.dto.diagnostics.ProcessorDiagnosticsDTO; import org.apache.nifi.web.api.dto.flow.FlowBreadcrumbDTO; import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO; import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO; @@ -54,6 +55,7 @@ import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity; import org.apache.nifi.web.api.entity.ProcessGroupStatusEntity; import org.apache.nifi.web.api.entity.ProcessGroupStatusSnapshotEntity; +import org.apache.nifi.web.api.entity.ProcessorDiagnosticsEntity; import org.apache.nifi.web.api.entity.ProcessorEntity; import org.apache.nifi.web.api.entity.ProcessorStatusEntity; import org.apache.nifi.web.api.entity.ProcessorStatusSnapshotEntity; @@ -77,6 +79,27 @@ public final class EntityFactory { + public ProcessorDiagnosticsEntity createProcessorDiagnosticsEntity(final ProcessorDiagnosticsDTO dto, final RevisionDTO revision, final PermissionsDTO processorPermissions, + final ProcessorStatusDTO status, final List bulletins) { + final ProcessorDiagnosticsEntity entity = new ProcessorDiagnosticsEntity(); + entity.setRevision(revision); + if (dto != null) { + entity.setPermissions(processorPermissions); + entity.setId(dto.getProcessor().getId()); + if (processorPermissions != null && processorPermissions.getCanRead()) { + entity.setComponent(dto); + entity.setBulletins(bulletins); + } + } + + entity.setBulletins(bulletins); + return entity; + } + + private void pairDownDiagnostics(final ProcessorDiagnosticsDTO dto, final PermissionsDTO controllerPermissions) { + + } + public StatusHistoryEntity createStatusHistoryEntity(final StatusHistoryDTO statusHistory, final PermissionsDTO permissions) { final StatusHistoryEntity entity = new StatusHistoryEntity(); entity.setCanRead(permissions.getCanRead()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java index 6cef8415153d..03d52c7bc708 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java @@ -16,6 +16,10 @@ */ package org.apache.nifi.web.controller; +import static org.apache.nifi.controller.FlowController.ROOT_GROUP_ID_ALIAS; + +import javax.ws.rs.WebApplicationException; + import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.ClassUtils; import org.apache.commons.lang3.StringUtils; @@ -86,6 +90,7 @@ import org.apache.nifi.registry.flow.VersionedProcessGroup; import org.apache.nifi.remote.RemoteGroupPort; import org.apache.nifi.remote.RootGroupPort; +import org.apache.nifi.reporting.BulletinRepository; import org.apache.nifi.reporting.ReportingTask; import org.apache.nifi.scheduling.ExecutionNode; import org.apache.nifi.scheduling.SchedulingStrategy; @@ -102,6 +107,7 @@ import org.apache.nifi.web.api.dto.BundleDTO; import org.apache.nifi.web.api.dto.DocumentedTypeDTO; import org.apache.nifi.web.api.dto.DtoFactory; +import org.apache.nifi.web.api.dto.diagnostics.ProcessorDiagnosticsDTO; import org.apache.nifi.web.api.dto.provenance.AttributeDTO; import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO; import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO; @@ -116,10 +122,10 @@ import org.apache.nifi.web.api.dto.search.SearchResultsDTO; import org.apache.nifi.web.api.dto.status.ControllerStatusDTO; import org.apache.nifi.web.api.dto.status.StatusHistoryDTO; +import org.apache.nifi.web.api.entity.ControllerServiceEntity; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.ws.rs.WebApplicationException; import java.io.IOException; import java.io.InputStream; import java.text.Collator; @@ -139,10 +145,9 @@ import java.util.TreeSet; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; +import java.util.function.Function; import java.util.stream.Collectors; -import static org.apache.nifi.controller.FlowController.ROOT_GROUP_ID_ALIAS; - public class ControllerFacade implements Authorizable { private static final Logger logger = LoggerFactory.getLogger(ControllerFacade.class); @@ -1927,6 +1932,11 @@ private void addIfAppropriate(final String searchStr, final String value, final } } + public ProcessorDiagnosticsDTO getProcessorDiagnostics(final ProcessorNode processor, final ProcessorStatus processorStatus, final BulletinRepository bulletinRepository, + final Function serviceEntityFactory) { + return dtoFactory.createProcessorDiagnosticsDto(processor, processorStatus, bulletinRepository, flowController, serviceEntityFactory); + } + /* * setters */ diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java index 604bb3ff8e9a..0c969b991bff 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/PersistentProvenanceRepository.java @@ -544,6 +544,21 @@ public long getContainerCapacity(final String containerName) throws IOException } } + @Override + public String getContainerFileStoreName(final String containerName) { + final Map map = configuration.getStorageDirectories(); + final File container = map.get(containerName); + if (container == null) { + return null; + } + + try { + return Files.getFileStore(container.toPath()).name(); + } catch (IOException e) { + return null; + } + } + @Override public long getContainerUsableSpace(String containerName) throws IOException { Map map = configuration.getStorageDirectories(); @@ -2475,7 +2490,7 @@ private long getFirstEventId(final File logFile) { public Collection getAllLogFiles() { final SortedMap map = idToPathMap.get(); - return map == null ? new ArrayList() : map.values(); + return map == null ? new ArrayList<>() : map.values(); } private static class PathMapComparator implements Comparator { diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/WriteAheadProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/WriteAheadProvenanceRepository.java index e1299d0c3000..28c1a7f14df0 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/WriteAheadProvenanceRepository.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/main/java/org/apache/nifi/provenance/WriteAheadProvenanceRepository.java @@ -52,6 +52,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -321,6 +322,21 @@ public long getContainerCapacity(final String containerName) throws IOException } } + @Override + public String getContainerFileStoreName(final String containerName) { + final Map map = config.getStorageDirectories(); + final File container = map.get(containerName); + if (container == null) { + return null; + } + + try { + return Files.getFileStore(container.toPath()).name(); + } catch (IOException e) { + return null; + } + } + @Override public long getContainerUsableSpace(String containerName) throws IOException { Map map = config.getStorageDirectories(); diff --git a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java index dbac23ab9c64..d29b1df3e3af 100644 --- a/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java +++ b/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/main/java/org/apache/nifi/provenance/VolatileProvenanceRepository.java @@ -612,6 +612,11 @@ public Set getContainerNames() { public long getContainerUsableSpace(String containerName) throws IOException { return maxSize - ringBuffer.getSize(); } + + public String getContainerFileStoreName(String containerName) { + return null; + } + private AsyncLineageSubmission submitLineageComputation(final Collection flowFileUuids, final NiFiUser user, final LineageComputationType computationType, final Long eventId) { final String userId = user.getIdentity(); From 06dd683b6b0dab96df48a027618eed71be6f116f Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Fri, 16 Feb 2018 15:27:30 -0500 Subject: [PATCH 2/2] NIFI-4895: Added backend code to give users the ability to forcibly terminate a processor if the processor does not complete after clicking Stop --- .../nifi/processor/AbstractProcessor.java | 1 - .../exception/TerminatedTaskException.java | 38 ++ .../api/dto/diagnostics/ThreadDumpDTO.java | 11 + .../apache/nifi/connectable/Connectable.java | 22 ++ .../apache/nifi/controller/AbstractPort.java | 6 + .../nifi/controller/ActiveThreadInfo.java | 8 +- .../nifi/controller/ConfiguredComponent.java | 15 +- .../nifi/controller/LoggableComponent.java | 7 +- .../nifi/controller/ProcessScheduler.java | 24 ++ .../apache/nifi/controller/ProcessorNode.java | 39 +- .../nifi/controller/StandardFunnel.java | 6 + .../controller/TerminationAwareLogger.java | 313 ++++++++++++++++ .../ActiveProcessSessionFactory.java | 24 ++ ...ScheduleState.java => LifecycleState.java} | 43 ++- .../scheduling/SchedulingAgent.java | 10 +- .../org/apache/nifi/groups/ProcessGroup.java | 8 + .../nifi/controller/FlowController.java | 44 ++- .../nifi/controller/ProcessorDetails.java | 5 +- .../controller/StandardProcessorNode.java | 83 ++++- .../controller/lifecycle/TaskTermination.java | 26 ++ .../TaskTerminationAwareStateManager.java | 66 ++++ .../reporting/AbstractReportingTaskNode.java | 3 +- .../reporting/ReportingTaskDetails.java | 6 +- ...essContext.java => RepositoryContext.java} | 4 +- .../repository/StandardProcessSession.java | 247 ++++++++----- .../StandardProcessSessionFactory.java | 9 +- .../WeakHashMapProcessSessionFactory.java | 60 +++ .../WriteAheadFlowFileRepository.java | 2 +- .../claim/ContentClaimWriteCache.java | 4 +- .../io/TaskTerminationInputStream.java | 125 +++++++ .../io/TaskTerminationOutputStream.java | 100 +++++ .../scheduling/AbstractSchedulingAgent.java | 38 +- .../EventDrivenSchedulingAgent.java | 63 ++-- .../scheduling/QuartzSchedulingAgent.java | 51 +-- ...ory.java => RepositoryContextFactory.java} | 10 +- .../scheduling/StandardProcessScheduler.java | 161 +++++--- .../TimerDrivenSchedulingAgent.java | 203 +++++------ .../service/ControllerServiceDetails.java | 6 +- .../StandardControllerServiceNode.java | 3 +- .../StandardControllerServiceProvider.java | 9 +- .../nifi/controller/tasks/ActiveTask.java | 39 ++ .../controller/tasks/ConnectableTask.java | 259 +++++++++++++ .../tasks/ContinuallyRunConnectableTask.java | 112 ------ .../tasks/ContinuallyRunProcessorTask.java | 225 ------------ .../controller/tasks/ExpireFlowFiles.java | 12 +- .../controller/tasks/InvocationResult.java | 51 +++ .../tasks/ReportingTaskWrapper.java | 10 +- .../nifi/groups/StandardProcessGroup.java | 25 +- .../nifi/processor/SimpleProcessLogger.java | 4 +- .../processor/StandardProcessContext.java | 37 +- .../nifi/controller/TestFlowController.java | 6 +- .../controller/TestStandardProcessorNode.java | 19 +- .../TestStandardProcessSession.java | 93 +++-- .../scheduling/DummyScheduleState.java | 2 +- .../TestStandardProcessScheduler.java | 6 +- .../service/mock/MockProcessGroup.java | 4 + ...ssorTask.java => TestConnectableTask.java} | 48 ++- .../TestContinuallyRunConnectableTask.java | 72 ---- .../apache/nifi/web/NiFiServiceFacade.java | 15 + .../nifi/web/StandardNiFiServiceFacade.java | 11 + .../nifi/web/api/ProcessorResource.java | 90 +++-- .../apache/nifi/web/api/dto/DtoFactory.java | 3 +- .../org/apache/nifi/web/dao/ProcessorDAO.java | 14 + .../web/dao/impl/StandardProcessorDAO.java | 13 + .../nifi/processors/standard/DebugFlow.java | 345 ++++++++++-------- 65 files changed, 2333 insertions(+), 1085 deletions(-) create mode 100644 nifi-api/src/main/java/org/apache/nifi/processor/exception/TerminatedTaskException.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/TerminationAwareLogger.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/repository/ActiveProcessSessionFactory.java rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/{ScheduleState.java => LifecycleState.java} (71%) create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/lifecycle/TaskTermination.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/lifecycle/TaskTerminationAwareStateManager.java rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/{ProcessContext.java => RepositoryContext.java} (98%) create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WeakHashMapProcessSessionFactory.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/TaskTerminationInputStream.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/TaskTerminationOutputStream.java rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/{ProcessContextFactory.java => RepositoryContextFactory.java} (80%) create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ActiveTask.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/InvocationResult.java rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/{TestContinuallyRunProcessorTask.java => TestConnectableTask.java} (58%) delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestContinuallyRunConnectableTask.java diff --git a/nifi-api/src/main/java/org/apache/nifi/processor/AbstractProcessor.java b/nifi-api/src/main/java/org/apache/nifi/processor/AbstractProcessor.java index 7ff568e3d8f6..4efae5641aeb 100644 --- a/nifi-api/src/main/java/org/apache/nifi/processor/AbstractProcessor.java +++ b/nifi-api/src/main/java/org/apache/nifi/processor/AbstractProcessor.java @@ -27,7 +27,6 @@ public final void onTrigger(final ProcessContext context, final ProcessSessionFa onTrigger(context, session); session.commit(); } catch (final Throwable t) { - getLogger().error("{} failed to process due to {}; rolling back session", new Object[]{this, t}); session.rollback(true); throw t; } diff --git a/nifi-api/src/main/java/org/apache/nifi/processor/exception/TerminatedTaskException.java b/nifi-api/src/main/java/org/apache/nifi/processor/exception/TerminatedTaskException.java new file mode 100644 index 000000000000..602ad1d224ab --- /dev/null +++ b/nifi-api/src/main/java/org/apache/nifi/processor/exception/TerminatedTaskException.java @@ -0,0 +1,38 @@ +/* + * 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.processor.exception; + +import java.io.InputStream; +import java.io.OutputStream; + +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.ProcessSessionFactory; + +/** + * This Exception is thrown whenever a user terminates a Processor and the + * Processor subsequently attempts to call a method on a {@link ProcessSession}, + * {@link ProcessSessionFactory}, {@link ProcessContext}, or an + * {@link InputStream} or {@link OutputStream} that were generated from the associated + * Process Sessions + */ +public class TerminatedTaskException extends ProcessException { + public TerminatedTaskException() { + super(); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ThreadDumpDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ThreadDumpDTO.java index be0b7ac3f21b..096d679bd02f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ThreadDumpDTO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/diagnostics/ThreadDumpDTO.java @@ -29,6 +29,7 @@ public class ThreadDumpDTO { private String stackTrace; private String threadName; private long threadActiveMillis; + private boolean taskTerminated; @ApiModelProperty("The ID of the node in the cluster") public String getNodeId() { @@ -83,4 +84,14 @@ public long getThreadActiveMillis() { public void setThreadActiveMillis(long threadActiveMillis) { this.threadActiveMillis = threadActiveMillis; } + + public void setTaskTerminated(final boolean terminated) { + this.taskTerminated = terminated; + } + + @ApiModelProperty("Indicates whether or not the user has requested that the task be terminated. If this is true, it may indicate that " + + "the thread is in a state where it will continue running indefinitely without returning.") + public boolean isTaskTerminated() { + return taskTerminated; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Connectable.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Connectable.java index a91ce97dee68..26fa0f8308ac 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Connectable.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Connectable.java @@ -235,6 +235,8 @@ public interface Connectable extends Triggerable, ComponentAuthorizable, Positio */ void yield(); + void yield(long yieldDuration, TimeUnit timeUnit); + /** * @return the time in milliseconds since Epoch at which this Connectable * should no longer yield its threads @@ -247,6 +249,26 @@ public interface Connectable extends Triggerable, ComponentAuthorizable, Positio */ boolean isSideEffectFree(); + default boolean isIsolated() { + return false; + } + + default boolean isTriggerWhenAnyDestinationAvailable() { + return false; + } + + default long getRunDuration(TimeUnit timeUnit) { + return 0L; + } + + default Object getRunnableComponent() { + return this; + } + + default boolean isSessionBatchingSupported() { + return false; + } + void verifyCanDelete() throws IllegalStateException; void verifyCanDelete(boolean ignoreConnections) throws IllegalStateException; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java index 02406488668f..ebbccc454339 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java @@ -497,6 +497,12 @@ public void yield() { yieldExpiration.set(Math.max(yieldExpiration.get(), System.currentTimeMillis() + yieldMillis)); } + @Override + public void yield(final long yieldDuration, final TimeUnit timeUnit) { + final long yieldMillis = timeUnit.toMillis(yieldDuration); + yieldExpiration.set(Math.max(yieldExpiration.get(), System.currentTimeMillis() + yieldMillis)); + } + @Override public long getYieldExpiration() { return yieldExpiration.get(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ActiveThreadInfo.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ActiveThreadInfo.java index 94a0b73e0a58..75f77952f74e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ActiveThreadInfo.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ActiveThreadInfo.java @@ -21,11 +21,13 @@ public class ActiveThreadInfo { private final String threadName; private final String stackTrace; private final long activeMillis; + private final boolean terminated; - public ActiveThreadInfo(final String threadName, final String stackTrace, final long activeMillis) { + public ActiveThreadInfo(final String threadName, final String stackTrace, final long activeMillis, final boolean terminated) { this.threadName = threadName; this.stackTrace = stackTrace; this.activeMillis = activeMillis; + this.terminated = terminated; } public String getThreadName() { @@ -39,4 +41,8 @@ public String getStackTrace() { public long getActiveMillis() { return activeMillis; } + + public boolean isTerminated() { + return terminated; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java index b0f65a7a28ec..3561cef8cdb3 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java @@ -16,6 +16,12 @@ */ package org.apache.nifi.controller; +import java.net.URL; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; + import org.apache.nifi.authorization.AccessDeniedException; import org.apache.nifi.authorization.AuthorizationResult; import org.apache.nifi.authorization.AuthorizationResult.Result; @@ -28,15 +34,8 @@ import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationResult; -import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.registry.ComponentVariableRegistry; -import java.net.URL; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Set; - public interface ConfiguredComponent extends ComponentAuthorizable { @Override @@ -72,7 +71,7 @@ public default void setProperties(Map properties) { ConfigurableComponent getComponent(); - ComponentLog getLogger(); + TerminationAwareLogger getLogger(); boolean isExtensionMissing(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/LoggableComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/LoggableComponent.java index 6d036759c50f..303690cd3db7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/LoggableComponent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/LoggableComponent.java @@ -18,7 +18,6 @@ import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.components.ConfigurableComponent; -import org.apache.nifi.logging.ComponentLog; /** * Holder to pass around a ConfigurableComponent with its coordinate and logger. @@ -31,9 +30,9 @@ public class LoggableComponent { private final BundleCoordinate bundleCoordinate; - private final ComponentLog logger; + private final TerminationAwareLogger logger; - public LoggableComponent(final T component, final BundleCoordinate bundleCoordinate, final ComponentLog logger) { + public LoggableComponent(final T component, final BundleCoordinate bundleCoordinate, final TerminationAwareLogger logger) { this.component = component; this.bundleCoordinate = bundleCoordinate; this.logger = logger; @@ -47,7 +46,7 @@ public BundleCoordinate getBundleCoordinate() { return bundleCoordinate; } - public ComponentLog getLogger() { + public TerminationAwareLogger getLogger() { return logger; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java index 4aa406668351..9231382f13c3 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java @@ -16,6 +16,8 @@ */ package org.apache.nifi.controller; +import java.io.InputStream; +import java.io.OutputStream; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; @@ -24,6 +26,9 @@ import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.Port; import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.ProcessSessionFactory; import org.apache.nifi.scheduling.SchedulingStrategy; public interface ProcessScheduler { @@ -59,6 +64,25 @@ public interface ProcessScheduler { */ Future stopProcessor(ProcessorNode procNode); + /** + * Interrupts all threads that are currently active in the Processor in an attempt to + * regain the threads and stop running the tasks in the Processor. All instances of + * {@link ProcessSession}, {@link ProcessSessionFactory}, {@link ProcessContext}, and + * the {@link InputStream}s and {@link OutputStream}s that were generated from the associated + * Process Sessions will also be poisoned, meaning that any calls to those objects will result + * in a {@link TerminatedTaskException} being thrown. In addition, the number of active threads + * will immediately be set to 0 so that the Processor can be modified. Note, however, that if + * the threads do not return, they cannot be returned to the Thread Pool. As such, invoking this + * method many times or when many threads are active and deadlocked/livelocked can result in + * thread pool exhaustion. If the given Processor is not in a Scheduled State of STOPPED, then this + * method does nothing. + * + * @param procNode the Processor to terminate + * + * @throws IllegalStateException if the Processor's Scheduled State is not currently STOPPED + */ + void terminateProcessor(ProcessorNode procNode); + /** * Starts scheduling the given Port to run. If the Port is already scheduled * to run, does nothing. diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java index a52271815782..25974b063aeb 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java @@ -26,7 +26,7 @@ import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; import org.apache.nifi.connectable.Connectable; -import org.apache.nifi.controller.scheduling.ScheduleState; +import org.apache.nifi.controller.scheduling.LifecycleState; import org.apache.nifi.controller.scheduling.SchedulingAgent; import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; @@ -54,8 +54,10 @@ public ProcessorNode(final String id, this.scheduledState = new AtomicReference<>(ScheduledState.STOPPED); } + @Override public abstract boolean isIsolated(); + @Override public abstract boolean isTriggerWhenAnyDestinationAvailable(); @Override @@ -65,12 +67,21 @@ public ProcessorNode(final String id, public abstract boolean isEventDrivenSupported(); - public abstract boolean isHighThroughputSupported(); - public abstract Requirement getInputRequirement(); public abstract List getActiveThreads(); + /** + * Returns the number of threads that are still 'active' in this Processor but have been terminated + * via {@link #terminate()}. To understand more about these threads, such as their stack traces and + * how long they have been active, one can use {@link #getActiveThreads()} and then filter the results + * to include only those {@link ActiveThreadInfo} objects for which the thread is terminated. For example: + * {@code getActiveThreads().stream().filter(ActiveThreadInfo::isTerminated).collect(Collectors.toList());} + * + * @return the number of threads that are still 'active' in this Processor but have been terminated. + */ + public abstract int getTerminatedThreadCount(); + @Override public abstract boolean isValid(); @@ -82,6 +93,7 @@ public ProcessorNode(final String id, public abstract void setProcessor(LoggableComponent processor); + @Override public abstract void yield(long period, TimeUnit timeUnit); public abstract void setAutoTerminatedRelationships(Set relationships); @@ -99,6 +111,7 @@ public ProcessorNode(final String id, public abstract void setRunDuration(long duration, TimeUnit timeUnit); + @Override public abstract long getRunDuration(TimeUnit timeUnit); public abstract Map getStyle(); @@ -123,6 +136,8 @@ public ProcessorNode(final String id, */ public abstract void verifyCanStart(Set ignoredReferences); + public abstract void verifyCanTerminate(); + /** * */ @@ -193,7 +208,23 @@ public abstract void start(ScheduledExecutorService scheduler, * as well as the active thread counts are kept in sync */ public abstract CompletableFuture stop(ProcessScheduler processScheduler, ScheduledExecutorService executor, - ProcessContext processContext, SchedulingAgent schedulingAgent, ScheduleState scheduleState); + ProcessContext processContext, SchedulingAgent schedulingAgent, LifecycleState scheduleState); + + /** + * Marks all active tasks as terminated and interrupts all active threads + * + * @return the number of active tasks that were terminated + */ + public abstract int terminate(); + + /** + * Determines whether or not the task associated with the given thread is terminated + * + * @param thread the thread + * @return true if there is a task associated with the given thread and that task was terminated, false if + * the given thread is not an active thread, or if the active thread has not been terminated + */ + public abstract boolean isTerminated(Thread thread); /** * Will set the state of the processor to STOPPED which essentially implies diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java index 4b3507c5df84..1d598c8db61c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java @@ -459,6 +459,12 @@ public String getPenalizationPeriod() { @Override public void yield() { final long yieldMillis = getYieldPeriod(TimeUnit.MILLISECONDS); + yield(yieldMillis, TimeUnit.MILLISECONDS); + } + + @Override + public void yield(final long yieldDuration, final TimeUnit timeUnit) { + final long yieldMillis = timeUnit.toMillis(yieldDuration); yieldExpiration.set(Math.max(yieldExpiration.get(), System.currentTimeMillis() + yieldMillis)); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/TerminationAwareLogger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/TerminationAwareLogger.java new file mode 100644 index 000000000000..0a4fbd78f2df --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/TerminationAwareLogger.java @@ -0,0 +1,313 @@ +/* + * 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; + +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.logging.LogLevel; + +public class TerminationAwareLogger implements ComponentLog { + private final ComponentLog logger; + private final String TERMINATED_TASK_PREFIX = "[Terminated Process] - "; + private volatile boolean terminated = false; + + public TerminationAwareLogger(final ComponentLog logger) { + this.logger = logger; + } + + public void terminate() { + this.terminated = true; + } + + private boolean isTerminated() { + return terminated; + } + + private String getMessage(String originalMessage, LogLevel logLevel) { + return TERMINATED_TASK_PREFIX + logLevel.name() + " - " + originalMessage; + } + + @Override + public void warn(String msg, Throwable t) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.WARN), t); + return; + } + + logger.warn(msg, t); + } + + @Override + public void warn(String msg, Object[] os) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.WARN), os); + return; + } + + logger.warn(msg, os); + } + + @Override + public void warn(String msg, Object[] os, Throwable t) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.WARN), os, t); + return; + } + + logger.warn(msg, os, t); + } + + @Override + public void warn(String msg) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.WARN)); + return; + } + + logger.warn(msg); + } + + @Override + public void trace(String msg, Throwable t) { + if (isTerminated()) { + logger.trace(getMessage(msg, LogLevel.TRACE), t); + return; + } + + logger.trace(msg, t); + } + + @Override + public void trace(String msg, Object[] os) { + if (isTerminated()) { + logger.trace(getMessage(msg, LogLevel.TRACE), os); + return; + } + + logger.trace(msg, os); + } + + @Override + public void trace(String msg) { + if (isTerminated()) { + logger.trace(getMessage(msg, LogLevel.TRACE)); + return; + } + + logger.trace(msg); + } + + @Override + public void trace(String msg, Object[] os, Throwable t) { + if (isTerminated()) { + logger.trace(getMessage(msg, LogLevel.TRACE), os, t); + return; + } + + logger.trace(msg, os, t); + } + + @Override + public boolean isWarnEnabled() { + return logger.isWarnEnabled(); + } + + @Override + public boolean isTraceEnabled() { + return logger.isTraceEnabled(); + } + + @Override + public boolean isInfoEnabled() { + return logger.isInfoEnabled(); + } + + @Override + public boolean isErrorEnabled() { + return logger.isErrorEnabled(); + } + + @Override + public boolean isDebugEnabled() { + return logger.isDebugEnabled(); + } + + @Override + public void info(String msg, Throwable t) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.INFO), t); + return; + } + + logger.info(msg, t); + } + + @Override + public void info(String msg, Object[] os) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.INFO), os); + return; + } + + logger.info(msg, os); + } + + @Override + public void info(String msg) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.INFO)); + return; + } + + logger.info(msg); + } + + @Override + public void info(String msg, Object[] os, Throwable t) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.INFO), os, t); + return; + } + + logger.info(msg, os, t); + } + + @Override + public String getName() { + return logger.getName(); + } + + @Override + public void error(String msg, Throwable t) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.ERROR), t); + return; + } + + logger.error(msg, t); + } + + @Override + public void error(String msg, Object[] os) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.ERROR), os); + return; + } + + logger.error(msg, os); + } + + @Override + public void error(String msg) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.ERROR)); + return; + } + + logger.error(msg); + } + + @Override + public void error(String msg, Object[] os, Throwable t) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.ERROR), os, t); + return; + } + + logger.error(msg, os, t); + } + + @Override + public void debug(String msg, Throwable t) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.DEBUG), t); + return; + } + + logger.debug(msg, t); + } + + @Override + public void debug(String msg, Object[] os) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.DEBUG), os); + return; + } + + logger.debug(msg, os); + } + + @Override + public void debug(String msg, Object[] os, Throwable t) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.DEBUG), os, t); + return; + } + + logger.debug(msg, os, t); + } + + @Override + public void debug(String msg) { + if (isTerminated()) { + logger.debug(getMessage(msg, LogLevel.DEBUG)); + return; + } + + logger.debug(msg); + } + + @Override + public void log(LogLevel level, String msg, Throwable t) { + if (isTerminated()) { + logger.debug(getMessage(msg, level), t); + return; + } + + logger.log(level, msg, t); + } + + @Override + public void log(LogLevel level, String msg, Object[] os) { + if (isTerminated()) { + logger.debug(getMessage(msg, level), os); + return; + } + + logger.log(level, msg, os); + } + + @Override + public void log(LogLevel level, String msg) { + if (isTerminated()) { + logger.debug(getMessage(msg, level)); + return; + } + + logger.log(level, msg); + } + + @Override + public void log(LogLevel level, String msg, Object[] os, Throwable t) { + if (isTerminated()) { + logger.debug(getMessage(msg, level), os, t); + return; + } + + logger.log(level, msg, os, t); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/repository/ActiveProcessSessionFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/repository/ActiveProcessSessionFactory.java new file mode 100644 index 000000000000..7632215a3e61 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/repository/ActiveProcessSessionFactory.java @@ -0,0 +1,24 @@ +/* + * 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.processor.ProcessSessionFactory; + +public interface ActiveProcessSessionFactory extends ProcessSessionFactory { + void terminateActiveSessions(); +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/LifecycleState.java similarity index 71% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/LifecycleState.java index 207d30809d2e..bcc7321a1365 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/LifecycleState.java @@ -25,20 +25,40 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.controller.repository.ActiveProcessSessionFactory; +import org.apache.nifi.processor.exception.TerminatedTaskException; -public class ScheduleState { +public class LifecycleState { private final AtomicInteger activeThreadCount = new AtomicInteger(0); private final AtomicBoolean scheduled = new AtomicBoolean(false); private final Set> futures = new HashSet<>(); private final AtomicBoolean mustCallOnStoppedMethods = new AtomicBoolean(false); private volatile long lastStopTime = -1; + private volatile boolean terminated = false; + private final Set activeProcessSessionFactories = Collections.synchronizedSet(new HashSet<>()); + + public synchronized int incrementActiveThreadCount(final ActiveProcessSessionFactory sessionFactory) { + if (terminated) { + throw new TerminatedTaskException(); + } + + if (sessionFactory != null) { + activeProcessSessionFactories.add(sessionFactory); + } - public int incrementActiveThreadCount() { return activeThreadCount.incrementAndGet(); } - public int decrementActiveThreadCount() { + public synchronized int decrementActiveThreadCount(final ActiveProcessSessionFactory sessionFactory) { + if (terminated) { + return activeThreadCount.get(); + } + + if (sessionFactory != null) { + activeProcessSessionFactories.remove(sessionFactory); + } + return activeThreadCount.decrementAndGet(); } @@ -98,4 +118,21 @@ public synchronized void replaceFuture(final ScheduledFuture oldFuture, final public synchronized Set> getFutures() { return Collections.unmodifiableSet(futures); } + + public synchronized void terminate() { + this.terminated = true; + activeThreadCount.set(0); + + for (final ActiveProcessSessionFactory factory : activeProcessSessionFactories) { + factory.terminateActiveSessions(); + } + } + + public void clearTerminationFlag() { + this.terminated = false; + } + + public boolean isTerminated() { + return this.terminated; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/SchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/SchedulingAgent.java index c48e13fb4cd1..8dcdaa5a9ff0 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/SchedulingAgent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/scheduling/SchedulingAgent.java @@ -23,18 +23,20 @@ public interface SchedulingAgent { - void schedule(Connectable connectable, ScheduleState scheduleState); + void schedule(Connectable connectable, LifecycleState scheduleState); - void unschedule(Connectable connectable, ScheduleState scheduleState); + void unschedule(Connectable connectable, LifecycleState scheduleState); void onEvent(Connectable connectable); - void schedule(ReportingTaskNode taskNode, ScheduleState scheduleState); + void schedule(ReportingTaskNode taskNode, LifecycleState scheduleState); - void unschedule(ReportingTaskNode taskNode, ScheduleState scheduleState); + void unschedule(ReportingTaskNode taskNode, LifecycleState scheduleState); void setMaxThreadCount(int maxThreads); + void incrementMaxThreadCount(int toAdd); + void setAdministrativeYieldDuration(String duration); String getAdministrativeYieldDuration(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java index 3f32580fc731..d7a9e8b62a28 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java @@ -202,6 +202,14 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable, Versi */ CompletableFuture stopProcessor(ProcessorNode processor); + /** + * Terminates the given Processor + * + * @param processor processor to Terminate + * @throws IllegalStateException if the Processor's Scheduled State is not STOPPED. + */ + void terminateProcessor(ProcessorNode processor); + /** * Stops the given Port * 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 34d5b2aa6001..9956fa84827f 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 @@ -50,6 +50,7 @@ import org.apache.nifi.cluster.protocol.UnknownServiceAddressException; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.state.StateManager; import org.apache.nifi.components.state.StateManagerProvider; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; @@ -98,7 +99,7 @@ import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager; import org.apache.nifi.controller.repository.io.LimitedInputStream; import org.apache.nifi.controller.scheduling.EventDrivenSchedulingAgent; -import org.apache.nifi.controller.scheduling.ProcessContextFactory; +import org.apache.nifi.controller.scheduling.RepositoryContextFactory; import org.apache.nifi.controller.scheduling.QuartzSchedulingAgent; import org.apache.nifi.controller.scheduling.StandardProcessScheduler; import org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent; @@ -517,7 +518,7 @@ private FlowController( processScheduler = new StandardProcessScheduler(timerDrivenEngineRef.get(), this, encryptor, stateManagerProvider, this.nifiProperties); eventDrivenWorkerQueue = new EventDrivenWorkerQueue(false, false, processScheduler); - final ProcessContextFactory contextFactory = new ProcessContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository); + final RepositoryContextFactory contextFactory = new RepositoryContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository); eventDrivenSchedulingAgent = new EventDrivenSchedulingAgent( eventDrivenEngineRef.get(), this, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor); @@ -745,7 +746,7 @@ public void initializeFlow() throws IOException { flowFileRepository.loadFlowFiles(this, maxIdFromSwapFiles + 1); // Begin expiring FlowFiles that are old - final ProcessContextFactory contextFactory = new ProcessContextFactory(contentRepository, flowFileRepository, + final RepositoryContextFactory contextFactory = new RepositoryContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository); processScheduler.scheduleFrameworkTask(new ExpireFlowFiles(this, contextFactory), "Expire FlowFiles", 30L, 30L, TimeUnit.SECONDS); @@ -1224,12 +1225,13 @@ private LoggableComponent instantiateProcessor(final String type, fin final Processor processor = processorClass.newInstance(); final ComponentLog componentLogger = new SimpleProcessLogger(identifier, processor); - final ProcessorInitializationContext ctx = new StandardProcessorInitializationContext(identifier, componentLogger, this, this, nifiProperties); + final TerminationAwareLogger terminationAwareLogger = new TerminationAwareLogger(componentLogger); + final ProcessorInitializationContext ctx = new StandardProcessorInitializationContext(identifier, terminationAwareLogger, this, this, nifiProperties); processor.initialize(ctx); - LogRepositoryFactory.getRepository(identifier).setLogger(componentLogger); + LogRepositoryFactory.getRepository(identifier).setLogger(terminationAwareLogger); - return new LoggableComponent<>(processor, bundleCoordinate, componentLogger); + return new LoggableComponent<>(processor, bundleCoordinate, terminationAwareLogger); } catch (final Throwable t) { throw new ProcessorInstantiationException(type, t); } finally { @@ -1263,15 +1265,19 @@ public void reload(final ProcessorNode existingNode, final String newType, final // call OnRemoved for the existing processor using the previous instance class loader try (final NarCloseable x = NarCloseable.withComponentNarLoader(existingInstanceClassLoader)) { - final StandardProcessContext processContext = new StandardProcessContext( - existingNode, controllerServiceProvider, encryptor, getStateManagerProvider().getStateManager(id)); + final StateManager stateManager = getStateManagerProvider().getStateManager(id); + final StandardProcessContext processContext = new StandardProcessContext(existingNode, controllerServiceProvider, encryptor, stateManager, () -> false); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getProcessor(), processContext); } finally { ExtensionManager.closeURLClassLoader(id, existingInstanceClassLoader); } // set the new processor in the existing node - final LoggableComponent newProcessor = new LoggableComponent<>(newNode.getProcessor(), newNode.getBundleCoordinate(), newNode.getLogger()); + final ComponentLog componentLogger = new SimpleProcessLogger(id, newNode.getProcessor()); + final TerminationAwareLogger terminationAwareLogger = new TerminationAwareLogger(componentLogger); + LogRepositoryFactory.getRepository(id).setLogger(terminationAwareLogger); + + final LoggableComponent newProcessor = new LoggableComponent<>(newNode.getProcessor(), newNode.getBundleCoordinate(), terminationAwareLogger); existingNode.setProcessor(newProcessor); existingNode.setExtensionMissing(newNode.isExtensionMissing()); @@ -1672,8 +1678,7 @@ public void setMaxEventDrivenThreadCount(final int maxThreadCount) { } /** - * Updates the number of threads that can be simultaneously used for - * executing processors. + * Updates the number of threads that can be simultaneously used for executing processors. * This method must be called while holding the write lock! * * @param maxThreadCount max number of threads @@ -3433,8 +3438,9 @@ private LoggableComponent instantiateReportingTask(final String t final ReportingTask reportingTask = reportingTaskClass.cast(reportingTaskObj); final ComponentLog componentLog = new SimpleProcessLogger(id, reportingTask); + final TerminationAwareLogger terminationAwareLogger = new TerminationAwareLogger(componentLog); - return new LoggableComponent<>(reportingTask, bundleCoordinate, componentLog); + return new LoggableComponent<>(reportingTask, bundleCoordinate, terminationAwareLogger); } catch (final Exception e) { throw new ReportingTaskInstantiationException(type, e); } finally { @@ -3474,7 +3480,11 @@ public void reload(final ReportingTaskNode existingNode, final String newType, f } // set the new reporting task into the existing node - final LoggableComponent newReportingTask = new LoggableComponent<>(newNode.getReportingTask(), newNode.getBundleCoordinate(), newNode.getLogger()); + final ComponentLog componentLogger = new SimpleProcessLogger(id, existingNode.getReportingTask()); + final TerminationAwareLogger terminationAwareLogger = new TerminationAwareLogger(componentLogger); + LogRepositoryFactory.getRepository(id).setLogger(terminationAwareLogger); + + final LoggableComponent newReportingTask = new LoggableComponent<>(newNode.getReportingTask(), newNode.getBundleCoordinate(), terminationAwareLogger); existingNode.setReportingTask(newReportingTask); existingNode.setExtensionMissing(newNode.isExtensionMissing()); @@ -3602,8 +3612,12 @@ public void reload(final ControllerServiceNode existingNode, final String newTyp invocationHandler.setServiceNode(existingNode); // create LoggableComponents for the proxy and implementation - final LoggableComponent loggableProxy = new LoggableComponent<>(newNode.getProxiedControllerService(), bundleCoordinate, newNode.getLogger()); - final LoggableComponent loggableImplementation = new LoggableComponent<>(newNode.getControllerServiceImplementation(), bundleCoordinate, newNode.getLogger()); + final ComponentLog componentLogger = new SimpleProcessLogger(id, newNode.getControllerServiceImplementation()); + final TerminationAwareLogger terminationAwareLogger = new TerminationAwareLogger(componentLogger); + LogRepositoryFactory.getRepository(id).setLogger(terminationAwareLogger); + + final LoggableComponent loggableProxy = new LoggableComponent<>(newNode.getProxiedControllerService(), bundleCoordinate, terminationAwareLogger); + final LoggableComponent loggableImplementation = new LoggableComponent<>(newNode.getControllerServiceImplementation(), bundleCoordinate, terminationAwareLogger); // set the new impl, proxy, and invocation handler into the existing node existingNode.setControllerServiceAndProxy(loggableImplementation, loggableProxy, invocationHandler); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/ProcessorDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/ProcessorDetails.java index 687d5c417cf3..2a3e06239ac9 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/ProcessorDetails.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/ProcessorDetails.java @@ -24,7 +24,6 @@ import org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable; import org.apache.nifi.annotation.behavior.TriggerWhenEmpty; import org.apache.nifi.bundle.BundleCoordinate; -import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.Processor; /** @@ -41,7 +40,7 @@ public class ProcessorDetails { private final boolean eventDrivenSupported; private final boolean batchSupported; private final InputRequirement.Requirement inputRequirement; - private final ComponentLog componentLog; + private final TerminationAwareLogger componentLog; private final BundleCoordinate bundleCoordinate; public ProcessorDetails(final LoggableComponent processor) { @@ -101,7 +100,7 @@ public InputRequirement.Requirement getInputRequirement() { return inputRequirement; } - public ComponentLog getComponentLog() { + public TerminationAwareLogger getComponentLog() { return componentLog; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index b4e40c87f8f4..2853ebef2ed0 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -71,10 +71,11 @@ import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Position; import org.apache.nifi.controller.exception.ProcessorInstantiationException; -import org.apache.nifi.controller.scheduling.ScheduleState; +import org.apache.nifi.controller.scheduling.LifecycleState; import org.apache.nifi.controller.scheduling.SchedulingAgent; import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.controller.tasks.ActiveTask; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.LogLevel; @@ -141,7 +142,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable // ??????? NOT any more private ExecutionNode executionNode; private final long onScheduleTimeoutMillis; - private final Map activeThreads = new HashMap<>(48); + private final Map activeThreads = new HashMap<>(48); public StandardProcessorNode(final LoggableComponent processor, final String uuid, final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler, @@ -220,10 +221,15 @@ public ConfigurableComponent getComponent() { } @Override - public ComponentLog getLogger() { + public TerminationAwareLogger getLogger() { return processorRef.get().getComponentLog(); } + @Override + public Object getRunnableComponent() { + return getProcessor(); + } + @Override public BundleCoordinate getBundleCoordinate() { return processorRef.get().getBundleCoordinate(); @@ -331,7 +337,7 @@ public boolean isSideEffectFree() { } @Override - public boolean isHighThroughputSupported() { + public boolean isSessionBatchingSupported() { return processorRef.get().isBatchSupported(); } @@ -1336,7 +1342,7 @@ public void start(final ScheduledExecutorService taskScheduler, final long admin private synchronized void activateThread() { final Thread thread = Thread.currentThread(); final Long timestamp = System.currentTimeMillis(); - activeThreads.put(thread, timestamp); + activeThreads.put(thread, new ActiveTask(timestamp)); } private synchronized void deactivateThread() { @@ -1355,21 +1361,72 @@ public synchronized List getActiveThreads() { .collect(Collectors.toMap(info -> info.getThreadId(), Function.identity(), (a, b) -> a)); final List threadList = new ArrayList<>(activeThreads.size()); - for (final Map.Entry entry : activeThreads.entrySet()) { + for (final Map.Entry entry : activeThreads.entrySet()) { final Thread thread = entry.getKey(); - final Long timestamp = entry.getValue(); + final ActiveTask activeTask = entry.getValue(); + final Long timestamp = activeTask.getStartTime(); final long activeMillis = now - timestamp; final ThreadInfo threadInfo = threadInfoMap.get(thread.getId()); final String stackTrace = ThreadUtils.createStackTrace(thread, threadInfo, deadlockedThreadIds, monitorDeadlockThreadIds, activeMillis); - final ActiveThreadInfo activeThreadInfo = new ActiveThreadInfo(thread.getName(), stackTrace, activeMillis); + final ActiveThreadInfo activeThreadInfo = new ActiveThreadInfo(thread.getName(), stackTrace, activeMillis, activeTask.isTerminated()); threadList.add(activeThreadInfo); } return threadList; } + @Override + public synchronized int getTerminatedThreadCount() { + return (int) activeThreads.values().stream() + .filter(ActiveTask::isTerminated) + .count(); + } + + + @Override + public int terminate() { + verifyCanTerminate(); + + int count = 0; + for (final Map.Entry entry : activeThreads.entrySet()) { + final Thread thread = entry.getKey(); + final ActiveTask activeTask = entry.getValue(); + + if (!activeTask.isTerminated()) { + activeTask.terminate(); + + thread.setName(thread.getName() + " "); + count++; + } + + thread.interrupt(); + } + + getLogger().terminate(); + scheduledState.set(ScheduledState.STOPPED); + + return count; + } + + @Override + public boolean isTerminated(final Thread thread) { + final ActiveTask activeTask = activeThreads.get(thread); + if (activeTask == null) { + return false; + } + + return activeTask.isTerminated(); + } + + @Override + public void verifyCanTerminate() { + if (getScheduledState() != ScheduledState.STOPPED) { + throw new IllegalStateException("Processor is not stopped"); + } + } + private void initiateStart(final ScheduledExecutorService taskScheduler, final long administrativeYieldMillis, final ProcessContext processContext, final SchedulingAgentCallback schedulingAgentCallback) { @@ -1493,7 +1550,7 @@ public void run() { */ @Override public CompletableFuture stop(final ProcessScheduler processScheduler, final ScheduledExecutorService executor, final ProcessContext processContext, - final SchedulingAgent schedulingAgent, final ScheduleState scheduleState) { + final SchedulingAgent schedulingAgent, final LifecycleState scheduleState) { final Processor processor = processorRef.get().getProcessor(); LOG.info("Stopping processor: " + processor.getClass()); @@ -1501,7 +1558,7 @@ public CompletableFuture stop(final ProcessScheduler processScheduler, fin final CompletableFuture future = new CompletableFuture<>(); if (this.scheduledState.compareAndSet(ScheduledState.RUNNING, ScheduledState.STOPPING)) { // will ensure that the Processor represented by this node can only be stopped once - scheduleState.incrementActiveThreadCount(); + scheduleState.incrementActiveThreadCount(null); // will continue to monitor active threads, invoking OnStopped once there are no // active threads (with the exception of the thread performing shutdown operations) @@ -1531,10 +1588,14 @@ public void run() { deactivateThread(); } - scheduleState.decrementActiveThreadCount(); + scheduleState.decrementActiveThreadCount(null); scheduledState.set(ScheduledState.STOPPED); future.complete(null); + // This can happen only when we join a cluster. In such a case, we can inherit a flow from the cluster that says that + // the Processor is to be running. However, if the Processor is already in the process of stopping, we cannot immediately + // start running the Processor. As a result, we check here, since the Processor is stopped, and then immediately start the + // Processor if need be. if (desiredState == ScheduledState.RUNNING) { processScheduler.startProcessor(StandardProcessorNode.this, true); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/lifecycle/TaskTermination.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/lifecycle/TaskTermination.java new file mode 100644 index 000000000000..752f367ed95d --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/lifecycle/TaskTermination.java @@ -0,0 +1,26 @@ +/* + * 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.lifecycle; + +public interface TaskTermination { + /** + * @return true if the task that is being executed by the current Thread has been terminated, + * false otherwise. + */ + boolean isTerminated(); +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/lifecycle/TaskTerminationAwareStateManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/lifecycle/TaskTerminationAwareStateManager.java new file mode 100644 index 000000000000..63d4e0e75389 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/lifecycle/TaskTerminationAwareStateManager.java @@ -0,0 +1,66 @@ +/* + * 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.lifecycle; + +import java.io.IOException; +import java.util.Map; + +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateManager; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.processor.exception.TerminatedTaskException; + +public class TaskTerminationAwareStateManager implements StateManager { + private final StateManager stateManager; + private final TaskTermination taskTermination; + + public TaskTerminationAwareStateManager(final StateManager stateManager, final TaskTermination taskTermination) { + this.stateManager = stateManager; + this.taskTermination = taskTermination; + } + + private void verifyNotTerminated() { + if (taskTermination.isTerminated()) { + throw new TerminatedTaskException(); + } + } + + @Override + public void clear(final Scope scope) throws IOException { + verifyNotTerminated(); + stateManager.clear(scope); + } + + @Override + public StateMap getState(final Scope scope) throws IOException { + verifyNotTerminated(); + return stateManager.getState(scope); + } + + @Override + public boolean replace(final StateMap oldValue, final Map newValue, final Scope scope) throws IOException { + verifyNotTerminated(); + return stateManager.replace(oldValue, newValue, scope); + } + + @Override + public void setState(final Map state, final Scope scope) throws IOException { + verifyNotTerminated(); + stateManager.setState(state, scope); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java index 07923c61874f..b066616163b9 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java @@ -36,6 +36,7 @@ import org.apache.nifi.controller.ReloadComponent; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.TerminationAwareLogger; import org.apache.nifi.controller.ValidationContextFactory; import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; @@ -114,7 +115,7 @@ public BundleCoordinate getBundleCoordinate() { } @Override - public ComponentLog getLogger() { + public TerminationAwareLogger getLogger() { return reportingTaskRef.get().getComponentLog(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskDetails.java index 3561c071554d..a88ba17a7ae5 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskDetails.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskDetails.java @@ -18,7 +18,7 @@ import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.controller.LoggableComponent; -import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.controller.TerminationAwareLogger; import org.apache.nifi.reporting.ReportingTask; /** @@ -27,7 +27,7 @@ class ReportingTaskDetails { private final ReportingTask reportingTask; - private final ComponentLog componentLog; + private final TerminationAwareLogger componentLog; private final BundleCoordinate bundleCoordinate; public ReportingTaskDetails(final LoggableComponent reportingTask) { @@ -40,7 +40,7 @@ public ReportingTask getReportingTask() { return reportingTask; } - public ComponentLog getComponentLog() { + public TerminationAwareLogger getComponentLog() { return componentLog; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/ProcessContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryContext.java similarity index 98% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/ProcessContext.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryContext.java index 4d0d850716b7..a4073716cb00 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/ProcessContext.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryContext.java @@ -35,7 +35,7 @@ /** * */ -public class ProcessContext { +public class RepositoryContext { private final Connectable connectable; private final ContentRepository contentRepo; @@ -45,7 +45,7 @@ public class ProcessContext { private final ProvenanceEventRepository provenanceRepo; private final AtomicLong connectionIndex; - public ProcessContext(final Connectable connectable, final AtomicLong connectionIndex, final ContentRepository contentRepository, + public RepositoryContext(final Connectable connectable, final AtomicLong connectionIndex, final ContentRepository contentRepository, final FlowFileRepository flowFileRepository, final FlowFileEventRepository flowFileEventRepository, final CounterRepository counterRepository, final ProvenanceEventRepository provenanceRepository) { this.connectable = connectable; 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 24fbf7231273..c863e30a5eab 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 @@ -16,10 +16,38 @@ */ package org.apache.nifi.controller.repository; -import org.apache.commons.io.IOUtils; +import java.io.BufferedOutputStream; +import java.io.ByteArrayInputStream; +import java.io.Closeable; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Objects; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connection; import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.lifecycle.TaskTermination; import org.apache.nifi.controller.queue.FlowFileQueue; import org.apache.nifi.controller.queue.QueueSize; import org.apache.nifi.controller.repository.claim.ContentClaim; @@ -30,6 +58,8 @@ import org.apache.nifi.controller.repository.io.FlowFileAccessInputStream; import org.apache.nifi.controller.repository.io.FlowFileAccessOutputStream; import org.apache.nifi.controller.repository.io.LimitedInputStream; +import org.apache.nifi.controller.repository.io.TaskTerminationInputStream; +import org.apache.nifi.controller.repository.io.TaskTerminationOutputStream; import org.apache.nifi.controller.repository.metrics.StandardFlowFileEvent; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; @@ -40,6 +70,7 @@ import org.apache.nifi.processor.exception.FlowFileHandlingException; import org.apache.nifi.processor.exception.MissingFlowFileException; import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.exception.TerminatedTaskException; import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.io.StreamCallback; @@ -55,33 +86,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedOutputStream; -import java.io.ByteArrayInputStream; -import java.io.Closeable; -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Objects; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.regex.Pattern; -import java.util.stream.Collectors; - /** *

* Provides a ProcessSession that ensures all accesses, changes and transfers @@ -106,11 +110,12 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE private static final Logger claimLog = LoggerFactory.getLogger(StandardProcessSession.class.getSimpleName() + ".claims"); private static final int MAX_ROLLBACK_FLOWFILES_TO_LOG = 5; - private final Map records = new HashMap<>(); - private final Map connectionCounts = new HashMap<>(); - private final Map> unacknowledgedFlowFiles = new HashMap<>(); - private final Map appendableStreams = new HashMap<>(); - private final ProcessContext context; + private final Map records = new ConcurrentHashMap<>(); + private final Map connectionCounts = new ConcurrentHashMap<>(); + private final Map> unacknowledgedFlowFiles = new ConcurrentHashMap<>(); + private final Map appendableStreams = new ConcurrentHashMap<>(); + private final RepositoryContext context; + private final TaskTermination taskTermination; private final Map readRecursionSet = new HashMap<>();// set used to track what is currently being operated on to prevent logic failures if recursive calls occurring private final Set writeRecursionSet = new HashSet<>(); private final Map deleteOnCommit = new HashMap<>(); @@ -137,9 +142,9 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE private long processingStartTime; // List of InputStreams that have been opened by calls to {@link #read(FlowFile)} and not yet closed - private final Map openInputStreams = new HashMap<>(); + private final Map openInputStreams = new ConcurrentHashMap<>(); // List of OutputStreams that have been opened by calls to {@link #write(FlowFile)} and not yet closed - private final Map openOutputStreams = new HashMap<>(); + private final Map openOutputStreams = new ConcurrentHashMap<>(); // maps a FlowFile to all Provenance Events that were generated for that FlowFile. // we do this so that if we generate a Fork event, for example, and then remove the event in the same @@ -153,8 +158,9 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE private Checkpoint checkpoint = new Checkpoint(); private final ContentClaimWriteCache claimCache; - public StandardProcessSession(final ProcessContext context) { + public StandardProcessSession(final RepositoryContext context, final TaskTermination taskTermination) { this.context = context; + this.taskTermination = taskTermination; final Connectable connectable = context.getConnectable(); final String componentType; @@ -194,6 +200,13 @@ public StandardProcessSession(final ProcessContext context) { processingStartTime = System.nanoTime(); } + private void verifyTaskActive() { + if (taskTermination.isTerminated()) { + rollback(false, true); + throw new TerminatedTaskException(); + } + } + private void closeStreams(final Map streamMap, final String action, final String streamType) { final Map openStreamCopy = new HashMap<>(streamMap); // avoid ConcurrentModificationException by creating a copy of the List for (final Map.Entry entry : openStreamCopy.entrySet()) { @@ -212,7 +225,7 @@ private void closeStreams(final Map streamMap, fi } public void checkpoint() { - + verifyTaskActive(); resetWriteClaims(false); closeStreams(openInputStreams, "committed", "input"); @@ -318,6 +331,7 @@ public void checkpoint() { @Override public void commit() { + verifyTaskActive(); checkpoint(); commit(this.checkpoint); this.checkpoint = null; @@ -783,6 +797,8 @@ private void updateEventContentClaims(final ProvenanceEventBuilder builder, fina @Override public StandardProvenanceEventRecord enrich(final ProvenanceEventRecord rawEvent, final FlowFile flowFile) { + verifyTaskActive(); + final StandardRepositoryRecord repoRecord = records.get(flowFile); if (repoRecord == null) { throw new FlowFileHandlingException(flowFile + " is not known in this session (" + toString() + ")"); @@ -927,6 +943,7 @@ public void rollback() { @Override public void rollback(final boolean penalize) { rollback(penalize, false); + verifyTaskActive(); } private void rollback(final boolean penalize, final boolean rollbackCheckpoint) { @@ -1153,6 +1170,8 @@ private void acknowledgeRecords() { @Override public void migrate(final ProcessSession newOwner, final Collection flowFiles) { + verifyTaskActive(); + if (Objects.requireNonNull(newOwner) == this) { throw new IllegalArgumentException("Cannot migrate FlowFiles from a Process Session to itself"); } @@ -1469,6 +1488,8 @@ private void registerDequeuedRecord(final FlowFileRecord flowFile, final Connect @Override public void adjustCounter(final String name, final long delta, final boolean immediate) { + verifyTaskActive(); + final Map counters; if (immediate) { if (immediateCounters == null) { @@ -1501,6 +1522,7 @@ private void adjustCounter(final String name, final long delta, final Map connections = context.getPollableConnections(); final int numConnections = connections.size(); for (int numAttempts = 0; numAttempts < numConnections; numAttempts++) { @@ -1520,6 +1542,8 @@ public FlowFile get() { @Override public List get(final int maxResults) { + verifyTaskActive(); + if (maxResults < 0) { throw new IllegalArgumentException(); } @@ -1554,6 +1578,8 @@ public FlowFileFilterResult filter(final FlowFile flowFile) { @Override public List get(final FlowFileFilter filter) { + verifyTaskActive(); + return get(new ConnectionPoller() { @Override public List poll(final Connection connection, final Set expiredRecords) { @@ -1562,31 +1588,6 @@ public List poll(final Connection connection, final Set get(final Connection connection, final ConnectionPoller poller, final boolean lockQueue) { - if (lockQueue) { - connection.lock(); - } - - try { - final Set expired = new HashSet<>(); - final List newlySelected = poller.poll(connection, expired); - removeExpired(expired, connection); - - if (newlySelected.isEmpty() && expired.isEmpty()) { - return new ArrayList<>(); - } - - for (final FlowFileRecord flowFile : newlySelected) { - registerDequeuedRecord(flowFile, connection); - } - - return new ArrayList<>(newlySelected); - } finally { - if (lockQueue) { - connection.unlock(); - } - } - } private List get(final ConnectionPoller poller, final boolean lockAllQueues) { final List connections = context.getPollableConnections(); @@ -1630,6 +1631,8 @@ private List get(final ConnectionPoller poller, final boolean lockAllQ @Override public QueueSize getQueueSize() { + verifyTaskActive(); + int flowFileCount = 0; long byteCount = 0L; for (final Connection conn : context.getPollableConnections()) { @@ -1642,6 +1645,8 @@ public QueueSize getQueueSize() { @Override public FlowFile create() { + verifyTaskActive(); + final Map attrs = new HashMap<>(); attrs.put(CoreAttributes.FILENAME.key(), String.valueOf(System.nanoTime())); attrs.put(CoreAttributes.PATH.key(), DEFAULT_FLOWFILE_PATH); @@ -1659,12 +1664,15 @@ public FlowFile create() { @Override public FlowFile clone(FlowFile example) { + verifyTaskActive(); example = validateRecordState(example); return clone(example, 0L, example.getSize()); } @Override public FlowFile clone(FlowFile example, final long offset, final long size) { + verifyTaskActive(); + example = validateRecordState(example); final StandardRepositoryRecord exampleRepoRecord = records.get(example); final FlowFileRecord currRec = exampleRepoRecord.getCurrent(); @@ -1734,6 +1742,8 @@ private void registerJoinEvent(final FlowFile child, final Collection @Override public FlowFile penalize(FlowFile flowFile) { + verifyTaskActive(); + flowFile = validateRecordState(flowFile); final StandardRepositoryRecord record = records.get(flowFile); final long expirationEpochMillis = System.currentTimeMillis() + context.getConnectable().getPenalizationPeriod(TimeUnit.MILLISECONDS); @@ -1744,6 +1754,7 @@ public FlowFile penalize(FlowFile flowFile) { @Override public FlowFile putAttribute(FlowFile flowFile, final String key, final String value) { + verifyTaskActive(); flowFile = validateRecordState(flowFile); if (CoreAttributes.UUID.key().equals(key)) { @@ -1759,6 +1770,8 @@ public FlowFile putAttribute(FlowFile flowFile, final String key, final String v @Override public FlowFile putAllAttributes(FlowFile flowFile, final Map attributes) { + verifyTaskActive(); + flowFile = validateRecordState(flowFile); final StandardRepositoryRecord record = records.get(flowFile); @@ -1779,6 +1792,7 @@ public FlowFile putAllAttributes(FlowFile flowFile, final Map at @Override public FlowFile removeAttribute(FlowFile flowFile, final String key) { + verifyTaskActive(); flowFile = validateRecordState(flowFile); if (CoreAttributes.UUID.key().equals(key)) { @@ -1793,6 +1807,7 @@ public FlowFile removeAttribute(FlowFile flowFile, final String key) { @Override public FlowFile removeAllAttributes(FlowFile flowFile, final Set keys) { + verifyTaskActive(); flowFile = validateRecordState(flowFile); if (keys == null) { @@ -1817,6 +1832,8 @@ public FlowFile removeAllAttributes(FlowFile flowFile, final Set keys) { @Override public FlowFile removeAllAttributes(FlowFile flowFile, final Pattern keyPattern) { + verifyTaskActive(); + flowFile = validateRecordState(flowFile); final StandardRepositoryRecord record = records.get(flowFile); final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).removeAttributes(keyPattern).build(); @@ -1851,6 +1868,7 @@ private void updateLastQueuedDate(final StandardRepositoryRecord record) { @Override public void transfer(FlowFile flowFile, final Relationship relationship) { + verifyTaskActive(); flowFile = validateRecordState(flowFile); final int numDestinations = context.getConnections(relationship).size(); final int multiplier = Math.max(1, numDestinations); @@ -1881,6 +1899,8 @@ public void transfer(FlowFile flowFile, final Relationship relationship) { @Override public void transfer(FlowFile flowFile) { + verifyTaskActive(); + flowFile = validateRecordState(flowFile); final StandardRepositoryRecord record = records.get(flowFile); if (record.getOriginalQueue() == null) { @@ -1899,6 +1919,7 @@ public void transfer(final Collection flowFiles) { @Override public void transfer(Collection flowFiles, final Relationship relationship) { + verifyTaskActive(); flowFiles = validateRecordState(flowFiles); boolean autoTerminated = false; @@ -1936,6 +1957,8 @@ public void transfer(Collection flowFiles, final Relationship relation @Override public void remove(FlowFile flowFile) { + verifyTaskActive(); + flowFile = validateRecordState(flowFile); final StandardRepositoryRecord record = records.get(flowFile); record.markForDelete(); @@ -1957,6 +1980,8 @@ public void remove(FlowFile flowFile) { @Override public void remove(Collection flowFiles) { + verifyTaskActive(); + flowFiles = validateRecordState(flowFiles); for (final FlowFile flowFile : flowFiles) { final StandardRepositoryRecord record = records.get(flowFile); @@ -2126,7 +2151,12 @@ private InputStream getInputStream(final FlowFile flowFile, final ContentClaim c try { StreamUtils.skip(rawInStream, offset); } catch(IOException ioe) { - IOUtils.closeQuietly(rawInStream); + try { + rawInStream.close(); + } catch (final Exception e) { + ioe.addSuppressed(ioe); + } + throw ioe; } return rawInStream; @@ -2147,6 +2177,8 @@ public void read(final FlowFile source, final InputStreamCallback reader) { @Override public void read(FlowFile source, boolean allowSessionStreamManagement, InputStreamCallback reader) { + verifyTaskActive(); + source = validateRecordState(source, true); final StandardRepositoryRecord record = records.get(source); @@ -2172,7 +2204,7 @@ public void read(FlowFile source, boolean allowSessionStreamManagement, InputStr try { incrementReadCount(source); - reader.process(ffais); + reader.process(createTaskTerminationStream(ffais)); // Allow processors to close the file after reading to avoid too many files open or do smart session stream management. if (this.currentReadClaimStream != null && !allowSessionStreamManagement) { @@ -2201,6 +2233,8 @@ public void read(FlowFile source, boolean allowSessionStreamManagement, InputStr @Override public InputStream read(FlowFile source) { + verifyTaskActive(); + source = validateRecordState(source, true); final StandardRepositoryRecord record = records.get(source); @@ -2302,7 +2336,16 @@ public String toString() { incrementReadCount(sourceFlowFile); openInputStreams.put(sourceFlowFile, errorHandlingStream); - return errorHandlingStream; + + return createTaskTerminationStream(errorHandlingStream); + } + + private InputStream createTaskTerminationStream(final InputStream delegate) { + return new TaskTerminationInputStream(delegate, taskTermination, () -> rollback(false, true)); + } + + private OutputStream createTaskTerminationStream(final OutputStream delegate) { + return new TaskTerminationOutputStream(delegate, taskTermination, () -> rollback(false, true)); } private void incrementReadCount(final FlowFile flowFile) { @@ -2325,11 +2368,15 @@ private void decrementReadCount(final FlowFile flowFile) { @Override public FlowFile merge(final Collection sources, final FlowFile destination) { + verifyTaskActive(); + return merge(sources, destination, null, null, null); } @Override public FlowFile merge(Collection sources, FlowFile destination, final byte[] header, final byte[] footer, final byte[] demarcator) { + verifyTaskActive(); + sources = validateRecordState(sources); destination = validateRecordState(destination); if (sources.contains(destination)) { @@ -2431,6 +2478,7 @@ private void ensureNotAppending(final ContentClaim claim) throws IOException { @Override public OutputStream write(FlowFile source) { + verifyTaskActive(); source = validateRecordState(source); final StandardRepositoryRecord record = records.get(source); @@ -2530,7 +2578,7 @@ public void close() throws IOException { writeRecursionSet.add(source); openOutputStreams.put(source, errorHandlingOutputStream); - return errorHandlingOutputStream; + return createTaskTerminationStream(errorHandlingOutputStream); } catch (final ContentNotFoundException nfe) { resetWriteClaims(); // need to reset write claim before we can remove the claim destroyContent(newClaim); @@ -2553,6 +2601,7 @@ public void close() throws IOException { @Override public FlowFile write(FlowFile source, final OutputStreamCallback writer) { + verifyTaskActive(); source = validateRecordState(source); final StandardRepositoryRecord record = records.get(source); @@ -2568,7 +2617,8 @@ public FlowFile write(FlowFile source, final OutputStreamCallback writer) { final ByteCountingOutputStream countingOut = new ByteCountingOutputStream(disableOnClose)) { try { writeRecursionSet.add(source); - writer.process(new FlowFileAccessOutputStream(countingOut, source)); + final OutputStream ffaos = new FlowFileAccessOutputStream(countingOut, source); + writer.process(createTaskTerminationStream(ffaos)); } finally { writtenToFlowFile = countingOut.getBytesWritten(); bytesWritten += countingOut.getBytesWritten(); @@ -2609,6 +2659,8 @@ public FlowFile write(FlowFile source, final OutputStreamCallback writer) { @Override public FlowFile append(FlowFile source, final OutputStreamCallback writer) { + verifyTaskActive(); + source = validateRecordState(source); final StandardRepositoryRecord record = records.get(source); long newSize = 0L; @@ -2789,6 +2841,7 @@ private void resetReadClaim() { @Override public FlowFile write(FlowFile source, final StreamCallback writer) { + verifyTaskActive(); source = validateRecordState(source); final StandardRepositoryRecord record = records.get(source); final ContentClaim currClaim = record.getCurrentClaim(); @@ -2821,10 +2874,11 @@ public FlowFile write(FlowFile source, final StreamCallback writer) { // ContentNotFoundException because if it is thrown, the Processor code may catch it and do something else with it // but in reality, if it is thrown, we want to know about it and handle it, even if the Processor code catches it. final FlowFileAccessInputStream ffais = new FlowFileAccessInputStream(countingIn, source, currClaim); + final FlowFileAccessOutputStream ffaos = new FlowFileAccessOutputStream(countingOut, source); boolean cnfeThrown = false; try { - writer.process(ffais, new FlowFileAccessOutputStream(countingOut, source)); + writer.process(createTaskTerminationStream(ffais), createTaskTerminationStream(ffaos)); } catch (final ContentNotFoundException cnfe) { cnfeThrown = true; throw cnfe; @@ -2868,6 +2922,8 @@ public FlowFile write(FlowFile source, final StreamCallback writer) { @Override public FlowFile importFrom(final Path source, final boolean keepSourceFile, FlowFile destination) { + verifyTaskActive(); + destination = validateRecordState(destination); // TODO: find a better solution. With Windows 7 and Java 7 (very early update, at least), Files.isWritable(source.getParent()) returns false, even when it should be true. if (!keepSourceFile && !Files.isWritable(source.getParent()) && !source.getParent().toFile().canWrite()) { @@ -2918,6 +2974,8 @@ public FlowFile importFrom(final Path source, final boolean keepSourceFile, Flow @Override public FlowFile importFrom(final InputStream source, FlowFile destination) { + verifyTaskActive(); + destination = validateRecordState(destination); final StandardRepositoryRecord record = records.get(destination); ContentClaim newClaim = null; @@ -2929,7 +2987,7 @@ public FlowFile importFrom(final InputStream source, FlowFile destination) { newClaim = context.getContentRepository().create(context.getConnectable().isLossTolerant()); claimLog.debug("Creating ContentClaim {} for 'importFrom' for {}", newClaim, destination); - newSize = context.getContentRepository().importFrom(source, newClaim); + newSize = context.getContentRepository().importFrom(createTaskTerminationStream(source), newClaim); bytesWritten += newSize; } catch (final IOException e) { throw new FlowFileAccessException("Unable to create ContentClaim due to " + e.toString(), e); @@ -2955,6 +3013,7 @@ public FlowFile importFrom(final InputStream source, FlowFile destination) { @Override public void exportTo(FlowFile source, final Path destination, final boolean append) { + verifyTaskActive(); source = validateRecordState(source); final StandardRepositoryRecord record = records.get(source); try { @@ -2973,6 +3032,7 @@ public void exportTo(FlowFile source, final Path destination, final boolean appe @Override public void exportTo(FlowFile source, final OutputStream destination) { + verifyTaskActive(); source = validateRecordState(source); final StandardRepositoryRecord record = records.get(source); @@ -2997,25 +3057,24 @@ public void exportTo(FlowFile source, final OutputStream destination) { // and translates into either FlowFileAccessException or ContentNotFoundException. We keep track of any // ContentNotFoundException because if it is thrown, the Processor code may catch it and do something else with it // but in reality, if it is thrown, we want to know about it and handle it, even if the Processor code catches it. - final FlowFileAccessInputStream ffais = new FlowFileAccessInputStream(countingStream, source, record.getCurrentClaim()); - boolean cnfeThrown = false; + try (final FlowFileAccessInputStream ffais = new FlowFileAccessInputStream(countingStream, source, record.getCurrentClaim())) { + boolean cnfeThrown = false; - try { - incrementReadCount(source); - StreamUtils.copy(ffais, destination, source.getSize()); - } catch (final ContentNotFoundException cnfe) { - cnfeThrown = true; - throw cnfe; - } finally { - decrementReadCount(source); + try { + incrementReadCount(source); + StreamUtils.copy(ffais, createTaskTerminationStream(destination), source.getSize()); + } catch (final ContentNotFoundException cnfe) { + cnfeThrown = true; + throw cnfe; + } finally { + decrementReadCount(source); - IOUtils.closeQuietly(ffais); - // if cnfeThrown is true, we don't need to re-throw the Exception; it will propagate. - if (!cnfeThrown && ffais.getContentNotFoundException() != null) { - throw ffais.getContentNotFoundException(); + // if cnfeThrown is true, we don't need to re-throw the Exception; it will propagate. + if (!cnfeThrown && ffais.getContentNotFoundException() != null) { + throw ffais.getContentNotFoundException(); + } } } - } catch (final ContentNotFoundException nfe) { handleContentNotFound(nfe, record); } catch (final IOException ex) { @@ -3106,6 +3165,7 @@ private FlowFile getMostRecent(final FlowFile flowFile) { @Override public FlowFile create(FlowFile parent) { + verifyTaskActive(); parent = getMostRecent(parent); final Map newAttributes = new HashMap<>(3); @@ -3144,6 +3204,8 @@ public FlowFile create(FlowFile parent) { @Override public FlowFile create(Collection parents) { + verifyTaskActive(); + parents = parents.stream().map(this::getMostRecent).collect(Collectors.toList()); final Map newAttributes = intersectAttributes(parents); @@ -3229,12 +3291,13 @@ private static Map intersectAttributes(final Collection autoTerminatedEvents = new ArrayList<>(); private final Set reportedEvents = new LinkedHashSet<>(); - private final Map records = new HashMap<>(); - private final Map connectionCounts = new HashMap<>(); - private final Map> unacknowledgedFlowFiles = new HashMap<>(); + private final Map records = new ConcurrentHashMap<>(); + private final Map connectionCounts = new ConcurrentHashMap<>(); + private final Map> unacknowledgedFlowFiles = new ConcurrentHashMap<>(); private Map countersOnCommit = new HashMap<>(); private Map immediateCounters = new HashMap<>(); @@ -3294,18 +3357,10 @@ private void checkpoint(final StandardProcessSession session, final List(); - } - this.countersOnCommit.putAll(session.countersOnCommit); } if (session.immediateCounters != null) { - if (this.immediateCounters == null) { - this.immediateCounters = new HashMap<>(); - } - this.immediateCounters.putAll(session.immediateCounters); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSessionFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSessionFactory.java index 76cecbb49715..7e4d5b047f51 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSessionFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSessionFactory.java @@ -16,18 +16,21 @@ */ package org.apache.nifi.controller.repository; +import org.apache.nifi.controller.lifecycle.TaskTermination; import org.apache.nifi.processor.ProcessSessionFactory; public class StandardProcessSessionFactory implements ProcessSessionFactory { - private final ProcessContext context; + private final RepositoryContext context; + private final TaskTermination taskTermination; - public StandardProcessSessionFactory(final ProcessContext context) { + public StandardProcessSessionFactory(final RepositoryContext context, final TaskTermination taskTermination) { this.context = context; + this.taskTermination = taskTermination; } @Override public StandardProcessSession createSession() { - return new StandardProcessSession(context); + return new StandardProcessSession(context, taskTermination); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WeakHashMapProcessSessionFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WeakHashMapProcessSessionFactory.java new file mode 100644 index 000000000000..cda65bb4b3e9 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WeakHashMapProcessSessionFactory.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.controller.repository; + +import java.util.Map; +import java.util.WeakHashMap; + +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.ProcessSessionFactory; +import org.apache.nifi.processor.exception.TerminatedTaskException; + +public class WeakHashMapProcessSessionFactory implements ActiveProcessSessionFactory { + private final ProcessSessionFactory delegate; + private final Map sessionMap = new WeakHashMap<>(); + private boolean terminated = false; + + public WeakHashMapProcessSessionFactory(final ProcessSessionFactory delegate) { + this.delegate = delegate; + } + + @Override + public synchronized ProcessSession createSession() { + if (terminated) { + throw new TerminatedTaskException(); + } + + final ProcessSession session = delegate.createSession(); + sessionMap.put(session, Boolean.TRUE); + return session; + } + + @Override + public synchronized void terminateActiveSessions() { + terminated = true; + for (final ProcessSession session : sessionMap.keySet()) { + try { + session.rollback(); + } catch (final TerminatedTaskException tte) { + // ignore + } + } + + sessionMap.clear(); + } +} 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 1d843c004ab8..852b569b48e8 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 @@ -197,7 +197,7 @@ public long getUsableStorageSpace() throws IOException { @Override public String getFileStoreName() { - final Path path = flowFileRepositoryPaths.iterator().next(); + final Path path = flowFileRepositoryPath.toPath(); try { return Files.getFileStore(path).name(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaimWriteCache.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaimWriteCache.java index 6b608acc2197..bf3a87088227 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaimWriteCache.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/ContentClaimWriteCache.java @@ -20,16 +20,16 @@ import java.io.BufferedOutputStream; import java.io.IOException; import java.io.OutputStream; -import java.util.HashMap; import java.util.LinkedList; import java.util.Map; import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; import org.apache.nifi.controller.repository.ContentRepository; public class ContentClaimWriteCache { private final ContentRepository contentRepo; - private final Map streamMap = new HashMap<>(); + private final Map streamMap = new ConcurrentHashMap<>(); private final Queue queue = new LinkedList<>(); private final int bufferSize; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/TaskTerminationInputStream.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/TaskTerminationInputStream.java new file mode 100644 index 000000000000..ffbe59278199 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/TaskTerminationInputStream.java @@ -0,0 +1,125 @@ +/* + * 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.io; + +import java.io.IOException; +import java.io.InputStream; + +import org.apache.nifi.controller.lifecycle.TaskTermination; +import org.apache.nifi.processor.exception.TerminatedTaskException; + +public class TaskTerminationInputStream extends InputStream { + private final TaskTermination taskTermination; + private final InputStream delegate; + private final Runnable terminatedCallback; + + public TaskTerminationInputStream(final InputStream delegate, final TaskTermination taskTermination, final Runnable terminatedCallback) { + this.delegate = delegate; + this.taskTermination = taskTermination; + this.terminatedCallback = terminatedCallback; + } + + private void verifyNotTerminated() { + if (taskTermination.isTerminated()) { + final TerminatedTaskException tte = new TerminatedTaskException(); + + if (terminatedCallback != null) { + try { + terminatedCallback.run(); + } catch (final Exception e) { + tte.addSuppressed(e); + } + } + + throw tte; + } + } + + @Override + public int read() throws IOException { + verifyNotTerminated(); + return delegate.read(); + } + + @Override + public int read(byte[] b) throws IOException { + verifyNotTerminated(); + return delegate.read(b); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + verifyNotTerminated(); + return delegate.read(b, off, len); + } + + @Override + public int available() throws IOException { + verifyNotTerminated(); + return delegate.available(); + } + + @Override + public long skip(long n) throws IOException { + verifyNotTerminated(); + return delegate.skip(n); + } + + @Override + public synchronized void reset() throws IOException { + verifyNotTerminated(); + delegate.reset(); + } + + @Override + public synchronized void mark(int readlimit) { + verifyNotTerminated(); + delegate.mark(readlimit); + } + + @Override + public boolean markSupported() { + verifyNotTerminated(); + return delegate.markSupported(); + } + + @Override + public void close() throws IOException { + try { + delegate.close(); + } catch (final Exception e) { + if (taskTermination.isTerminated()) { + final TerminatedTaskException tte = new TerminatedTaskException(); + tte.addSuppressed(e); + + if (terminatedCallback != null) { + try { + terminatedCallback.run(); + } catch (final Exception callbackException) { + tte.addSuppressed(callbackException); + } + } + + throw tte; + } + } + + verifyNotTerminated(); + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/TaskTerminationOutputStream.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/TaskTerminationOutputStream.java new file mode 100644 index 000000000000..f76199d8a40e --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/TaskTerminationOutputStream.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.controller.repository.io; + +import java.io.IOException; +import java.io.OutputStream; + +import org.apache.nifi.controller.lifecycle.TaskTermination; +import org.apache.nifi.processor.exception.TerminatedTaskException; + +public class TaskTerminationOutputStream extends OutputStream { + private final TaskTermination taskTermination; + private final OutputStream delegate; + private final Runnable terminatedCallback; + + public TaskTerminationOutputStream(final OutputStream delegate, final TaskTermination taskTermination, final Runnable terminatedCallback) { + this.delegate = delegate; + this.taskTermination = taskTermination; + this.terminatedCallback = terminatedCallback; + } + + private void verifyNotTerminated() { + if (taskTermination.isTerminated()) { + final TerminatedTaskException tte = new TerminatedTaskException(); + + if (terminatedCallback != null) { + try { + terminatedCallback.run(); + } catch (final Exception e) { + tte.addSuppressed(e); + } + } + + throw tte; + } + } + + @Override + public void write(final int b) throws IOException { + verifyNotTerminated(); + delegate.write(b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + verifyNotTerminated(); + delegate.write(b, off, len); + } + + @Override + public void write(byte[] b) throws IOException { + verifyNotTerminated(); + delegate.write(b); + } + + @Override + public void flush() throws IOException { + verifyNotTerminated(); + delegate.flush(); + } + + @Override + public void close() throws IOException { + try { + delegate.close(); + } catch (final Exception e) { + if (taskTermination.isTerminated()) { + final TerminatedTaskException tte = new TerminatedTaskException(); + tte.addSuppressed(e); + + if (terminatedCallback != null) { + try { + terminatedCallback.run(); + } catch (final Exception callbackException) { + tte.addSuppressed(callbackException); + } + } + + throw tte; + } + } + + verifyNotTerminated(); + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/AbstractSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/AbstractSchedulingAgent.java index 8f36e1e85000..d6380ec71cb4 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/AbstractSchedulingAgent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/AbstractSchedulingAgent.java @@ -22,10 +22,10 @@ /** * Base implementation of the {@link SchedulingAgent} which encapsulates the - * updates to the {@link ScheduleState} based on invoked operation and then + * updates to the {@link LifecycleState} based on invoked operation and then * delegates to the corresponding 'do' methods. For example; By invoking - * {@link #schedule(Connectable, ScheduleState)} the - * {@link ScheduleState#setScheduled(boolean)} with value 'true' will be + * {@link #schedule(Connectable, LifecycleState)} the + * {@link LifecycleState#setScheduled(boolean)} with value 'true' will be * invoked. * * @see EventDrivenSchedulingAgent @@ -41,70 +41,70 @@ protected AbstractSchedulingAgent(FlowEngine flowEngine) { } @Override - public void schedule(Connectable connectable, ScheduleState scheduleState) { + public void schedule(Connectable connectable, LifecycleState scheduleState) { scheduleState.setScheduled(true); this.doSchedule(connectable, scheduleState); } @Override - public void unschedule(Connectable connectable, ScheduleState scheduleState) { + public void unschedule(Connectable connectable, LifecycleState scheduleState) { scheduleState.setScheduled(false); this.doUnschedule(connectable, scheduleState); } @Override - public void schedule(ReportingTaskNode taskNode, ScheduleState scheduleState) { + public void schedule(ReportingTaskNode taskNode, LifecycleState scheduleState) { scheduleState.setScheduled(true); this.doSchedule(taskNode, scheduleState); } @Override - public void unschedule(ReportingTaskNode taskNode, ScheduleState scheduleState) { + public void unschedule(ReportingTaskNode taskNode, LifecycleState scheduleState) { scheduleState.setScheduled(false); this.doUnschedule(taskNode, scheduleState); } /** - * Schedules the provided {@link Connectable}. Its {@link ScheduleState} + * Schedules the provided {@link Connectable}. Its {@link LifecycleState} * will be set to true * * @param connectable * the instance of {@link Connectable} * @param scheduleState - * the instance of {@link ScheduleState} + * the instance of {@link LifecycleState} */ - protected abstract void doSchedule(Connectable connectable, ScheduleState scheduleState); + protected abstract void doSchedule(Connectable connectable, LifecycleState scheduleState); /** - * Unschedules the provided {@link Connectable}. Its {@link ScheduleState} + * Unschedules the provided {@link Connectable}. Its {@link LifecycleState} * will be set to false * * @param connectable * the instance of {@link Connectable} * @param scheduleState - * the instance of {@link ScheduleState} + * the instance of {@link LifecycleState} */ - protected abstract void doUnschedule(Connectable connectable, ScheduleState scheduleState); + protected abstract void doUnschedule(Connectable connectable, LifecycleState scheduleState); /** * Schedules the provided {@link ReportingTaskNode}. Its - * {@link ScheduleState} will be set to true + * {@link LifecycleState} will be set to true * * @param connectable * the instance of {@link ReportingTaskNode} * @param scheduleState - * the instance of {@link ScheduleState} + * the instance of {@link LifecycleState} */ - protected abstract void doSchedule(ReportingTaskNode connectable, ScheduleState scheduleState); + protected abstract void doSchedule(ReportingTaskNode connectable, LifecycleState scheduleState); /** * Unschedules the provided {@link ReportingTaskNode}. Its - * {@link ScheduleState} will be set to false + * {@link LifecycleState} will be set to false * * @param connectable * the instance of {@link ReportingTaskNode} * @param scheduleState - * the instance of {@link ScheduleState} + * the instance of {@link LifecycleState} */ - protected abstract void doUnschedule(ReportingTaskNode connectable, ScheduleState scheduleState); + protected abstract void doUnschedule(ReportingTaskNode connectable, LifecycleState scheduleState); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java index 7b2e9662cd27..8b960fc3cf1f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java @@ -31,10 +31,13 @@ import org.apache.nifi.controller.EventDrivenWorkerQueue; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.lifecycle.TaskTerminationAwareStateManager; +import org.apache.nifi.controller.repository.ActiveProcessSessionFactory; import org.apache.nifi.controller.repository.BatchingSessionFactory; -import org.apache.nifi.controller.repository.ProcessContext; +import org.apache.nifi.controller.repository.RepositoryContext; import org.apache.nifi.controller.repository.StandardProcessSession; import org.apache.nifi.controller.repository.StandardProcessSessionFactory; +import org.apache.nifi.controller.repository.WeakHashMapProcessSessionFactory; import org.apache.nifi.controller.repository.metrics.StandardFlowFileEvent; import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.encrypt.StringEncryptor; @@ -58,7 +61,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent { private final ControllerServiceProvider serviceProvider; private final StateManagerProvider stateManagerProvider; private final EventDrivenWorkerQueue workerQueue; - private final ProcessContextFactory contextFactory; + private final RepositoryContextFactory contextFactory; private final AtomicInteger maxThreadCount; private final AtomicInteger activeThreadCount = new AtomicInteger(0); private final StringEncryptor encryptor; @@ -66,10 +69,10 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent { private volatile String adminYieldDuration = "1 sec"; private final ConcurrentMap connectionIndexMap = new ConcurrentHashMap<>(); - private final ConcurrentMap scheduleStates = new ConcurrentHashMap<>(); + private final ConcurrentMap scheduleStates = new ConcurrentHashMap<>(); public EventDrivenSchedulingAgent(final FlowEngine flowEngine, final ControllerServiceProvider serviceProvider, final StateManagerProvider stateManagerProvider, - final EventDrivenWorkerQueue workerQueue, final ProcessContextFactory contextFactory, final int maxThreadCount, final StringEncryptor encryptor) { + final EventDrivenWorkerQueue workerQueue, final RepositoryContextFactory contextFactory, final int maxThreadCount, final StringEncryptor encryptor) { super(flowEngine); this.serviceProvider = serviceProvider; this.stateManagerProvider = stateManagerProvider; @@ -98,24 +101,24 @@ public void shutdown() { } @Override - public void doSchedule(final ReportingTaskNode taskNode, ScheduleState scheduleState) { + public void doSchedule(final ReportingTaskNode taskNode, LifecycleState scheduleState) { throw new UnsupportedOperationException("ReportingTasks cannot be scheduled in Event-Driven Mode"); } @Override - public void doUnschedule(ReportingTaskNode taskNode, ScheduleState scheduleState) { + public void doUnschedule(ReportingTaskNode taskNode, LifecycleState scheduleState) { throw new UnsupportedOperationException("ReportingTasks cannot be scheduled in Event-Driven Mode"); } @Override - public void doSchedule(final Connectable connectable, final ScheduleState scheduleState) { + public void doSchedule(final Connectable connectable, final LifecycleState scheduleState) { workerQueue.resumeWork(connectable); logger.info("Scheduled {} to run in Event-Driven mode", connectable); scheduleStates.put(connectable, scheduleState); } @Override - public void doUnschedule(final Connectable connectable, final ScheduleState scheduleState) { + public void doUnschedule(final Connectable connectable, final LifecycleState scheduleState) { workerQueue.suspendWork(connectable); logger.info("Stopped scheduling {} to run", connectable); } @@ -138,6 +141,10 @@ public void setMaxThreadCount(final int maxThreadCount) { } } + @Override + public void incrementMaxThreadCount(int toAdd) { + } + @Override public void setAdministrativeYieldDuration(final String yieldDuration) { this.adminYieldDuration = yieldDuration; @@ -171,7 +178,7 @@ public void run() { continue; } final Connectable connectable = worker.getConnectable(); - final ScheduleState scheduleState = scheduleStates.get(connectable); + final LifecycleState scheduleState = scheduleStates.get(connectable); if (scheduleState == null) { // Component not yet scheduled to run but has received events continue; @@ -189,26 +196,29 @@ public void run() { } } - final ProcessContext context = contextFactory.newProcessContext(connectable, connectionIndex); + final RepositoryContext context = contextFactory.newProcessContext(connectable, connectionIndex); if (connectable instanceof ProcessorNode) { final ProcessorNode procNode = (ProcessorNode) connectable; - final StandardProcessContext standardProcessContext = new StandardProcessContext(procNode, serviceProvider, encryptor, getStateManager(connectable.getIdentifier())); + final StateManager stateManager = new TaskTerminationAwareStateManager(getStateManager(connectable.getIdentifier()), scheduleState::isTerminated); + final StandardProcessContext standardProcessContext = new StandardProcessContext(procNode, serviceProvider, encryptor, stateManager, scheduleState::isTerminated); final long runNanos = procNode.getRunDuration(TimeUnit.NANOSECONDS); final ProcessSessionFactory sessionFactory; final StandardProcessSession rawSession; final boolean batch; - if (procNode.isHighThroughputSupported() && runNanos > 0L) { - rawSession = new StandardProcessSession(context); + if (procNode.isSessionBatchingSupported() && runNanos > 0L) { + rawSession = new StandardProcessSession(context, scheduleState::isTerminated); sessionFactory = new BatchingSessionFactory(rawSession); batch = true; } else { rawSession = null; - sessionFactory = new StandardProcessSessionFactory(context); + sessionFactory = new StandardProcessSessionFactory(context, scheduleState::isTerminated); batch = false; } + final ActiveProcessSessionFactory activeSessionFactory = new WeakHashMapProcessSessionFactory(sessionFactory); + final long startNanos = System.nanoTime(); final long finishNanos = startNanos + runNanos; int invocationCount = 0; @@ -216,7 +226,7 @@ public void run() { try { while (shouldRun) { - trigger(procNode, context, scheduleState, standardProcessContext, sessionFactory); + trigger(procNode, context, scheduleState, standardProcessContext, activeSessionFactory); invocationCount++; if (!batch) { @@ -266,9 +276,10 @@ public void run() { onEvent(procNode); } } else { - final ProcessSessionFactory sessionFactory = new StandardProcessSessionFactory(context); + final ProcessSessionFactory sessionFactory = new StandardProcessSessionFactory(context, scheduleState::isTerminated); + final ActiveProcessSessionFactory activeSessionFactory = new WeakHashMapProcessSessionFactory(sessionFactory); final ConnectableProcessContext connectableProcessContext = new ConnectableProcessContext(connectable, encryptor, getStateManager(connectable.getIdentifier())); - trigger(connectable, scheduleState, connectableProcessContext, sessionFactory); + trigger(connectable, scheduleState, connectableProcessContext, activeSessionFactory); // See explanation above for the ProcessorNode as to why we do this. if (Connectables.flowFilesQueued(connectable)) { @@ -281,15 +292,15 @@ public void run() { } } - private void trigger(final Connectable worker, final ScheduleState scheduleState, final ConnectableProcessContext processContext, final ProcessSessionFactory sessionFactory) { - final int newThreadCount = scheduleState.incrementActiveThreadCount(); + private void trigger(final Connectable worker, final LifecycleState scheduleState, final ConnectableProcessContext processContext, final ActiveProcessSessionFactory sessionFactory) { + final int newThreadCount = scheduleState.incrementActiveThreadCount(sessionFactory); if (newThreadCount > worker.getMaxConcurrentTasks() && worker.getMaxConcurrentTasks() > 0) { // its possible that the worker queue could give us a worker node that is eligible to run based // on the number of threads but another thread has already incremented the thread count, result in // reaching the maximum number of threads. we won't know this until we atomically increment the thread count // on the Schedule State, so we check it here. in this case, we cannot trigger the Processor, as doing so would // result in using more than the maximum number of defined threads - scheduleState.decrementActiveThreadCount(); + scheduleState.decrementActiveThreadCount(sessionFactory); return; } @@ -317,20 +328,20 @@ private void trigger(final Connectable worker, final ScheduleState scheduleState } } - scheduleState.decrementActiveThreadCount(); + scheduleState.decrementActiveThreadCount(sessionFactory); } } - private void trigger(final ProcessorNode worker, final ProcessContext context, final ScheduleState scheduleState, - final StandardProcessContext processContext, final ProcessSessionFactory sessionFactory) { - final int newThreadCount = scheduleState.incrementActiveThreadCount(); + private void trigger(final ProcessorNode worker, final RepositoryContext context, final LifecycleState scheduleState, + final StandardProcessContext processContext, final ActiveProcessSessionFactory sessionFactory) { + final int newThreadCount = scheduleState.incrementActiveThreadCount(sessionFactory); if (newThreadCount > worker.getMaxConcurrentTasks() && worker.getMaxConcurrentTasks() > 0) { // its possible that the worker queue could give us a worker node that is eligible to run based // on the number of threads but another thread has already incremented the thread count, result in // reaching the maximum number of threads. we won't know this until we atomically increment the thread count // on the Schedule State, so we check it here. in this case, we cannot trigger the Processor, as doing so would // result in using more than the maximum number of defined threads - scheduleState.decrementActiveThreadCount(); + scheduleState.decrementActiveThreadCount(sessionFactory); return; } @@ -359,7 +370,7 @@ private void trigger(final ProcessorNode worker, final ProcessContext context, f } } - scheduleState.decrementActiveThreadCount(); + scheduleState.decrementActiveThreadCount(sessionFactory); } } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java index eb855d403f43..c3f39895f6d3 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java @@ -21,22 +21,16 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.nifi.components.state.StateManager; import org.apache.nifi.connectable.Connectable; -import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.controller.FlowController; -import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ReportingTaskNode; -import org.apache.nifi.controller.tasks.ContinuallyRunConnectableTask; -import org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask; +import org.apache.nifi.controller.tasks.ConnectableTask; import org.apache.nifi.controller.tasks.ReportingTaskWrapper; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.engine.FlowEngine; -import org.apache.nifi.processor.StandardProcessContext; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.util.FormatUtils; import org.quartz.CronExpression; @@ -48,29 +42,25 @@ public class QuartzSchedulingAgent extends AbstractSchedulingAgent { private final Logger logger = LoggerFactory.getLogger(QuartzSchedulingAgent.class); private final FlowController flowController; - private final ProcessContextFactory contextFactory; + private final RepositoryContextFactory contextFactory; private final StringEncryptor encryptor; private volatile String adminYieldDuration = "1 sec"; private final Map> canceledTriggers = new HashMap<>(); - public QuartzSchedulingAgent(final FlowController flowController, final FlowEngine flowEngine, final ProcessContextFactory contextFactory, final StringEncryptor enryptor) { + public QuartzSchedulingAgent(final FlowController flowController, final FlowEngine flowEngine, final RepositoryContextFactory contextFactory, final StringEncryptor enryptor) { super(flowEngine); this.flowController = flowController; this.contextFactory = contextFactory; this.encryptor = enryptor; } - private StateManager getStateManager(final String componentId) { - return flowController.getStateManagerProvider().getStateManager(componentId); - } - @Override public void shutdown() { } @Override - public void doSchedule(final ReportingTaskNode taskNode, final ScheduleState scheduleState) { + public void doSchedule(final ReportingTaskNode taskNode, final LifecycleState scheduleState) { final List existingTriggers = canceledTriggers.get(taskNode); if (existingTriggers != null) { throw new IllegalStateException("Cannot schedule " + taskNode.getReportingTask().getIdentifier() + " because it is already scheduled to run"); @@ -120,7 +110,7 @@ public void run() { } @Override - public synchronized void doSchedule(final Connectable connectable, final ScheduleState scheduleState) { + public synchronized void doSchedule(final Connectable connectable, final LifecycleState scheduleState) { final List existingTriggers = canceledTriggers.get(connectable); if (existingTriggers != null) { throw new IllegalStateException("Cannot schedule " + connectable + " because it is already scheduled to run"); @@ -136,18 +126,7 @@ public synchronized void doSchedule(final Connectable connectable, final Schedul final List triggers = new ArrayList<>(); for (int i = 0; i < connectable.getMaxConcurrentTasks(); i++) { - final Callable continuallyRunTask; - - if (connectable.getConnectableType() == ConnectableType.PROCESSOR) { - final ProcessorNode procNode = (ProcessorNode) connectable; - - final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier())); - ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController, contextFactory, scheduleState, standardProcContext); - continuallyRunTask = runnableTask; - } else { - final ConnectableProcessContext connProcContext = new ConnectableProcessContext(connectable, encryptor, getStateManager(connectable.getIdentifier())); - continuallyRunTask = new ContinuallyRunConnectableTask(contextFactory, connectable, scheduleState, connProcContext); - } + final ConnectableTask continuallyRunTask = new ConnectableTask(this, connectable, flowController, contextFactory, scheduleState, encryptor); final AtomicBoolean canceled = new AtomicBoolean(false); final Runnable command = new Runnable() { @@ -158,7 +137,7 @@ public void run() { } try { - continuallyRunTask.call(); + continuallyRunTask.invoke(); } catch (final RuntimeException re) { throw re; } catch (final Exception e) { @@ -188,16 +167,16 @@ public void run() { } @Override - public synchronized void doUnschedule(final Connectable connectable, final ScheduleState scheduleState) { + public synchronized void doUnschedule(final Connectable connectable, final LifecycleState scheduleState) { unschedule((Object) connectable, scheduleState); } @Override - public synchronized void doUnschedule(final ReportingTaskNode taskNode, final ScheduleState scheduleState) { + public synchronized void doUnschedule(final ReportingTaskNode taskNode, final LifecycleState scheduleState) { unschedule((Object) taskNode, scheduleState); } - private void unschedule(final Object scheduled, final ScheduleState scheduleState) { + private void unschedule(final Object scheduled, final LifecycleState scheduleState) { final List triggers = canceledTriggers.remove(scheduled); if (triggers == null) { throw new IllegalStateException("Cannot unschedule " + scheduled + " because it was not scheduled to run"); @@ -233,4 +212,14 @@ public String getAdministrativeYieldDuration() { public long getAdministrativeYieldDuration(final TimeUnit timeUnit) { return FormatUtils.getTimeDuration(adminYieldDuration, timeUnit); } + + @Override + public void incrementMaxThreadCount(int toAdd) { + final int corePoolSize = flowEngine.getCorePoolSize(); + if (corePoolSize - toAdd < 1) { + throw new IllegalStateException("Cannot remove " + (-toAdd) + " threads from pool because there are only " + corePoolSize + " threads in the pool"); + } + + flowEngine.setCorePoolSize(corePoolSize + toAdd); + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ProcessContextFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/RepositoryContextFactory.java similarity index 80% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ProcessContextFactory.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/RepositoryContextFactory.java index 9454a4b23084..4cffedabdee5 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ProcessContextFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/RepositoryContextFactory.java @@ -23,10 +23,10 @@ import org.apache.nifi.controller.repository.CounterRepository; import org.apache.nifi.controller.repository.FlowFileEventRepository; import org.apache.nifi.controller.repository.FlowFileRepository; -import org.apache.nifi.controller.repository.ProcessContext; +import org.apache.nifi.controller.repository.RepositoryContext; import org.apache.nifi.provenance.ProvenanceEventRepository; -public class ProcessContextFactory { +public class RepositoryContextFactory { private final ContentRepository contentRepo; private final FlowFileRepository flowFileRepo; @@ -34,7 +34,7 @@ public class ProcessContextFactory { private final CounterRepository counterRepo; private final ProvenanceEventRepository provenanceRepo; - public ProcessContextFactory(final ContentRepository contentRepository, final FlowFileRepository flowFileRepository, + public RepositoryContextFactory(final ContentRepository contentRepository, final FlowFileRepository flowFileRepository, final FlowFileEventRepository flowFileEventRepository, final CounterRepository counterRepository, final ProvenanceEventRepository provenanceRepository) { @@ -45,7 +45,7 @@ public ProcessContextFactory(final ContentRepository contentRepository, final Fl this.provenanceRepo = provenanceRepository; } - public ProcessContext newProcessContext(final Connectable connectable, final AtomicLong connectionIndex) { - return new ProcessContext(connectable, connectionIndex, contentRepo, flowFileRepo, flowFileEventRepo, counterRepo, provenanceRepo); + public RepositoryContext newProcessContext(final Connectable connectable, final AtomicLong connectionIndex) { + return new RepositoryContext(connectable, connectionIndex, contentRepo, flowFileRepo, flowFileEventRepo, counterRepo, provenanceRepo); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java index 1155bfe11b3d..e5601b4378c8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java @@ -19,6 +19,7 @@ import static java.util.Objects.requireNonNull; import java.lang.reflect.InvocationTargetException; +import java.util.Collections; import java.util.List; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -38,12 +39,14 @@ import org.apache.nifi.connectable.Port; import org.apache.nifi.controller.AbstractPort; import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.SchedulingAgentCallback; import org.apache.nifi.controller.StandardProcessorNode; +import org.apache.nifi.controller.exception.ProcessorInstantiationException; import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.encrypt.StringEncryptor; @@ -70,11 +73,12 @@ public final class StandardProcessScheduler implements ProcessScheduler { private static final Logger LOG = LoggerFactory.getLogger(StandardProcessScheduler.class); private final ControllerServiceProvider controllerServiceProvider; + private final FlowController flowController; private final long administrativeYieldMillis; private final String administrativeYieldDuration; private final StateManagerProvider stateManagerProvider; - private final ConcurrentMap scheduleStates = new ConcurrentHashMap<>(); + private final ConcurrentMap lifecycleStates = new ConcurrentHashMap<>(); private final ScheduledExecutorService frameworkTaskExecutor; private final ConcurrentMap strategyAgentMap = new ConcurrentHashMap<>(); @@ -84,15 +88,11 @@ public final class StandardProcessScheduler implements ProcessScheduler { private final StringEncryptor encryptor; - public StandardProcessScheduler( - final FlowEngine componentLifecycleThreadPool, - final ControllerServiceProvider controllerServiceProvider, - final StringEncryptor encryptor, - final StateManagerProvider stateManagerProvider, - final NiFiProperties nifiProperties - ) { + public StandardProcessScheduler(final FlowEngine componentLifecycleThreadPool, final FlowController flowController, final StringEncryptor encryptor, + final StateManagerProvider stateManagerProvider, final NiFiProperties nifiProperties) { this.componentLifeCycleThreadPool = componentLifecycleThreadPool; - this.controllerServiceProvider = controllerServiceProvider; + this.controllerServiceProvider = flowController; + this.flowController = flowController; this.encryptor = encryptor; this.stateManagerProvider = stateManagerProvider; @@ -138,7 +138,6 @@ public void setMaxThreadCount(final SchedulingStrategy schedulingStrategy, final return; } - agent.setMaxThreadCount(maxThreadCount); } public void setSchedulingAgent(final SchedulingStrategy strategy, final SchedulingAgent agent) { @@ -170,12 +169,12 @@ public void shutdown() { @Override public void schedule(final ReportingTaskNode taskNode) { - final ScheduleState scheduleState = getScheduleState(requireNonNull(taskNode)); - if (scheduleState.isScheduled()) { + final LifecycleState lifecycleState = getLifecycleState(requireNonNull(taskNode), true); + if (lifecycleState.isScheduled()) { return; } - final int activeThreadCount = scheduleState.getActiveThreadCount(); + final int activeThreadCount = lifecycleState.getActiveThreadCount(); if (activeThreadCount > 0) { throw new IllegalStateException("Reporting Task " + taskNode.getName() + " cannot be started because it has " + activeThreadCount + " threads still running"); } @@ -185,25 +184,25 @@ public void schedule(final ReportingTaskNode taskNode) { } final SchedulingAgent agent = getSchedulingAgent(taskNode.getSchedulingStrategy()); - scheduleState.setScheduled(true); + lifecycleState.setScheduled(true); final Runnable startReportingTaskRunnable = new Runnable() { @Override public void run() { - final long lastStopTime = scheduleState.getLastStopTime(); + final long lastStopTime = lifecycleState.getLastStopTime(); final ReportingTask reportingTask = taskNode.getReportingTask(); // Continually attempt to start the Reporting Task, and if we fail sleep for a bit each time. while (true) { try { - synchronized (scheduleState) { + synchronized (lifecycleState) { // if no longer scheduled to run, then we're finished. This can happen, for example, // if the @OnScheduled method throws an Exception and the user stops the reporting task // while we're administratively yielded. // we also check if the schedule state's last start time is equal to what it was before. // if not, then means that the reporting task has been stopped and started again, so we should just // bail; another thread will be responsible for invoking the @OnScheduled methods. - if (!scheduleState.isScheduled() || scheduleState.getLastStopTime() != lastStopTime) { + if (!lifecycleState.isScheduled() || lifecycleState.getLastStopTime() != lastStopTime) { return; } @@ -211,7 +210,7 @@ public void run() { ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, reportingTask, taskNode.getConfigurationContext()); } - agent.schedule(taskNode, scheduleState); + agent.schedule(taskNode, lifecycleState); return; } } catch (final Exception e) { @@ -241,8 +240,8 @@ public void run() { @Override public void unschedule(final ReportingTaskNode taskNode) { - final ScheduleState scheduleState = getScheduleState(requireNonNull(taskNode)); - if (!scheduleState.isScheduled()) { + final LifecycleState lifecycleState = getLifecycleState(requireNonNull(taskNode), false); + if (!lifecycleState.isScheduled()) { return; } @@ -256,8 +255,8 @@ public void unschedule(final ReportingTaskNode taskNode) { public void run() { final ConfigurationContext configurationContext = taskNode.getConfigurationContext(); - synchronized (scheduleState) { - scheduleState.setScheduled(false); + synchronized (lifecycleState) { + lifecycleState.setScheduled(false); try { try (final NarCloseable x = NarCloseable.withComponentNarLoader(reportingTask.getClass(), reportingTask.getIdentifier())) { @@ -278,9 +277,9 @@ public void run() { } } - agent.unschedule(taskNode, scheduleState); + agent.unschedule(taskNode, lifecycleState); - if (scheduleState.getActiveThreadCount() == 0 && scheduleState.mustCallOnStoppedMethods()) { + if (lifecycleState.getActiveThreadCount() == 0 && lifecycleState.mustCallOnStoppedMethods()) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, reportingTask, configurationContext); } } @@ -299,27 +298,29 @@ public void run() { */ @Override public synchronized CompletableFuture startProcessor(final ProcessorNode procNode, final boolean failIfStopping) { + final LifecycleState lifecycleState = getLifecycleState(requireNonNull(procNode), true); + final StandardProcessContext processContext = new StandardProcessContext(procNode, this.controllerServiceProvider, - this.encryptor, getStateManager(procNode.getIdentifier())); - final ScheduleState scheduleState = getScheduleState(requireNonNull(procNode)); + this.encryptor, getStateManager(procNode.getIdentifier()), lifecycleState::isTerminated); final CompletableFuture future = new CompletableFuture<>(); final SchedulingAgentCallback callback = new SchedulingAgentCallback() { @Override public void trigger() { - getSchedulingAgent(procNode).schedule(procNode, scheduleState); + lifecycleState.clearTerminationFlag(); + getSchedulingAgent(procNode).schedule(procNode, lifecycleState); future.complete(null); } @Override - public Future scheduleTask(Callable task) { - scheduleState.incrementActiveThreadCount(); - return componentLifeCycleThreadPool.submit(task); + public Future scheduleTask(final Callable task) { + lifecycleState.incrementActiveThreadCount(null); + return componentMonitoringThreadPool.submit(task); } @Override public void onTaskComplete() { - scheduleState.decrementActiveThreadCount(); + lifecycleState.decrementActiveThreadCount(null); } }; @@ -330,19 +331,49 @@ public void onTaskComplete() { /** * Stops the given {@link Processor} by invoking its - * {@link ProcessorNode#stop(ScheduledExecutorService, org.apache.nifi.processor.ProcessContext, SchedulingAgent, ScheduleState)} + * {@link ProcessorNode#stop(ScheduledExecutorService, org.apache.nifi.processor.ProcessContext, SchedulingAgent, LifecycleState)} * method. * - * @see StandardProcessorNode#stop(ScheduledExecutorService, org.apache.nifi.processor.ProcessContext, SchedulingAgent, ScheduleState) + * @see StandardProcessorNode#stop(ScheduledExecutorService, org.apache.nifi.processor.ProcessContext, SchedulingAgent, LifecycleState) */ @Override public synchronized CompletableFuture stopProcessor(final ProcessorNode procNode) { + final LifecycleState lifecycleState = getLifecycleState(procNode, false); + StandardProcessContext processContext = new StandardProcessContext(procNode, this.controllerServiceProvider, - this.encryptor, getStateManager(procNode.getIdentifier())); - final ScheduleState state = getScheduleState(procNode); + this.encryptor, getStateManager(procNode.getIdentifier()), lifecycleState::isTerminated); LOG.info("Stopping {}", procNode); - return procNode.stop(this, this.componentLifeCycleThreadPool, processContext, getSchedulingAgent(procNode), state); + return procNode.stop(this, this.componentLifeCycleThreadPool, processContext, getSchedulingAgent(procNode), lifecycleState); + } + + @Override + public synchronized void terminateProcessor(final ProcessorNode procNode) { + if (procNode.getScheduledState() != ScheduledState.STOPPED) { + throw new IllegalStateException("Cannot terminate " + procNode + " because it is not currently stopped"); + } + + final LifecycleState state = getLifecycleState(procNode, false); + if (state.getActiveThreadCount() == 0) { + LOG.debug("Will not terminate {} because it has no active threads", procNode); + return; + } + + LOG.debug("Terminating {}", procNode); + + final int tasksTerminated = procNode.terminate(); + state.terminate(); + + getSchedulingAgent(procNode).incrementMaxThreadCount(tasksTerminated); + + try { + flowController.reload(procNode, procNode.getProcessor().getClass().getName(), procNode.getBundleCoordinate(), Collections.emptySet()); + } catch (final ProcessorInstantiationException e) { + // This shouldn't happen because we already have been able to instantiate the processor before + LOG.error("Failed to replace instance of Processor for {} when terminating Processor", procNode); + } + + LOG.info("Successfully terminated {} with {} active threads", procNode, tasksTerminated); } @Override @@ -378,7 +409,7 @@ public void registerEvent(final Connectable worker) { @Override public int getActiveThreadCount(final Object scheduled) { - return getScheduleState(scheduled).getActiveThreadCount(); + return getLifecycleState(scheduled, false).getActiveThreadCount(); } @Override @@ -414,22 +445,23 @@ private synchronized void startConnectable(final Connectable connectable) { throw new IllegalStateException(connectable.getIdentifier() + " is disabled, so it cannot be started"); } - final ScheduleState scheduleState = getScheduleState(requireNonNull(connectable)); - if (scheduleState.isScheduled()) { + final LifecycleState lifecycleState = getLifecycleState(requireNonNull(connectable), true); + if (lifecycleState.isScheduled()) { return; } - final int activeThreads = scheduleState.getActiveThreadCount(); + final int activeThreads = lifecycleState.getActiveThreadCount(); if (activeThreads > 0) { throw new IllegalStateException("Port cannot be scheduled to run until its last " + activeThreads + " threads finish"); } - getSchedulingAgent(connectable).schedule(connectable, scheduleState); - scheduleState.setScheduled(true); + lifecycleState.clearTerminationFlag(); + getSchedulingAgent(connectable).schedule(connectable, lifecycleState); + lifecycleState.setScheduled(true); } private synchronized void stopConnectable(final Connectable connectable) { - final ScheduleState state = getScheduleState(requireNonNull(connectable)); + final LifecycleState state = getLifecycleState(requireNonNull(connectable), false); if (!state.isScheduled()) { return; } @@ -516,25 +548,48 @@ public synchronized void disableReportingTask(final ReportingTaskNode taskNode) @Override public boolean isScheduled(final Object scheduled) { - final ScheduleState scheduleState = scheduleStates.get(scheduled); - return scheduleState == null ? false : scheduleState.isScheduled(); + final LifecycleState lifecycleState = lifecycleStates.get(scheduled); + return lifecycleState == null ? false : lifecycleState.isScheduled(); } /** - * Returns the ScheduleState that is registered for the given component; if - * no ScheduleState current is registered, one is created and registered + * Returns the LifecycleState that is registered for the given component; if + * no LifecycleState current is registered, one is created and registered * atomically, and then that value is returned. * * @param schedulable schedulable * @return scheduled state */ - private ScheduleState getScheduleState(final Object schedulable) { - ScheduleState scheduleState = this.scheduleStates.get(schedulable); - if (scheduleState == null) { - scheduleState = new ScheduleState(); - this.scheduleStates.putIfAbsent(schedulable, scheduleState); + private LifecycleState getLifecycleState(final Object schedulable, final boolean replaceTerminatedState) { + LifecycleState lifecycleState; + while (true) { + lifecycleState = this.lifecycleStates.get(schedulable); + + if (lifecycleState == null) { + lifecycleState = new LifecycleState(); + final LifecycleState existing = this.lifecycleStates.putIfAbsent(schedulable, lifecycleState); + + if (existing == null) { + break; + } else { + continue; + } + } else if (replaceTerminatedState && lifecycleState.isTerminated()) { + final LifecycleState newLifecycleState = new LifecycleState(); + final boolean replaced = this.lifecycleStates.replace(schedulable, lifecycleState, newLifecycleState); + + if (replaced) { + lifecycleState = newLifecycleState; + break; + } else { + continue; + } + } else { + break; + } } - return scheduleState; + + return lifecycleState; } @Override 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 9dc329a76a55..cb498bfe5e45 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 @@ -18,25 +18,18 @@ import java.util.ArrayList; import java.util.List; -import java.util.concurrent.Callable; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import org.apache.nifi.components.state.StateManager; import org.apache.nifi.connectable.Connectable; -import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.controller.FlowController; -import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ReportingTaskNode; -import org.apache.nifi.controller.tasks.ContinuallyRunConnectableTask; -import org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask; +import org.apache.nifi.controller.tasks.ConnectableTask; +import org.apache.nifi.controller.tasks.InvocationResult; import org.apache.nifi.controller.tasks.ReportingTaskWrapper; import org.apache.nifi.encrypt.StringEncryptor; import org.apache.nifi.engine.FlowEngine; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.StandardProcessContext; -import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.util.FormatUtils; import org.apache.nifi.util.NiFiProperties; import org.slf4j.Logger; @@ -48,17 +41,13 @@ public class TimerDrivenSchedulingAgent extends AbstractSchedulingAgent { private final long noWorkYieldNanos; private final FlowController flowController; - private final ProcessContextFactory contextFactory; + private final RepositoryContextFactory contextFactory; private final StringEncryptor encryptor; private volatile String adminYieldDuration = "1 sec"; - public TimerDrivenSchedulingAgent( - final FlowController flowController, - final FlowEngine flowEngine, - final ProcessContextFactory contextFactory, - final StringEncryptor encryptor, - final NiFiProperties nifiProperties) { + public TimerDrivenSchedulingAgent(final FlowController flowController, final FlowEngine flowEngine, final RepositoryContextFactory contextFactory, + final StringEncryptor encryptor, final NiFiProperties nifiProperties) { super(flowEngine); this.flowController = flowController; this.contextFactory = contextFactory; @@ -72,17 +61,13 @@ public TimerDrivenSchedulingAgent( } } - private StateManager getStateManager(final String componentId) { - return flowController.getStateManagerProvider().getStateManager(componentId); - } - @Override public void shutdown() { flowEngine.shutdown(); } @Override - public void doSchedule(final ReportingTaskNode taskNode, final ScheduleState scheduleState) { + public void doSchedule(final ReportingTaskNode taskNode, final LifecycleState scheduleState) { final Runnable reportingTaskWrapper = new ReportingTaskWrapper(taskNode, scheduleState); final long schedulingNanos = taskNode.getSchedulingPeriod(TimeUnit.NANOSECONDS); @@ -95,100 +80,19 @@ public void doSchedule(final ReportingTaskNode taskNode, final ScheduleState sch } @Override - public void doSchedule(final Connectable connectable, final ScheduleState scheduleState) { - + public void doSchedule(final Connectable connectable, final LifecycleState scheduleState) { final List> futures = new ArrayList<>(); - for (int i = 0; i < connectable.getMaxConcurrentTasks(); i++) { - final Callable continuallyRunTask; - final ProcessContext processContext; + final ConnectableTask connectableTask = new ConnectableTask(this, connectable, flowController, contextFactory, scheduleState, encryptor); + for (int i = 0; i < connectable.getMaxConcurrentTasks(); i++) { // Determine the task to run and create it. - if (connectable.getConnectableType() == ConnectableType.PROCESSOR) { - final ProcessorNode procNode = (ProcessorNode) connectable; - final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier())); - final ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController, - contextFactory, scheduleState, standardProcContext); - - continuallyRunTask = runnableTask; - processContext = standardProcContext; - } else { - processContext = new ConnectableProcessContext(connectable, encryptor, getStateManager(connectable.getIdentifier())); - continuallyRunTask = new ContinuallyRunConnectableTask(contextFactory, connectable, scheduleState, processContext); - } - final AtomicReference> futureRef = new AtomicReference<>(); - final Runnable yieldDetectionRunnable = new Runnable() { - @Override - public void run() { - // Call the continually run task. It will return a boolean indicating whether or not we should yield - // based on a lack of work for to do for the component. - final boolean shouldYield; - try { - shouldYield = continuallyRunTask.call(); - } catch (final RuntimeException re) { - throw re; - } catch (final Exception e) { - throw new ProcessException(e); - } - - // If the component is yielded, cancel its future and re-submit it to run again - // after the yield has expired. - final long newYieldExpiration = connectable.getYieldExpiration(); - final long now = System.currentTimeMillis(); - if (newYieldExpiration > now) { - final long yieldMillis = newYieldExpiration - now; - final long scheduleMillis = connectable.getSchedulingPeriod(TimeUnit.MILLISECONDS); - final ScheduledFuture scheduledFuture = futureRef.get(); - if (scheduledFuture == null) { - return; - } - - // If we are able to cancel the future, create a new one and update the ScheduleState so that it has - // an accurate accounting of which futures are outstanding; we must then also update the futureRef - // so that we can do this again the next time that the component is yielded. - if (scheduledFuture.cancel(false)) { - final long yieldNanos = Math.max(TimeUnit.MILLISECONDS.toNanos(scheduleMillis), TimeUnit.MILLISECONDS.toNanos(yieldMillis)); - - synchronized (scheduleState) { - if (scheduleState.isScheduled()) { - final ScheduledFuture newFuture = flowEngine.scheduleWithFixedDelay(this, yieldNanos, - connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS); - - scheduleState.replaceFuture(scheduledFuture, newFuture); - futureRef.set(newFuture); - } - } - } - } else if (noWorkYieldNanos > 0L && shouldYield) { - // Component itself didn't yield but there was no work to do, so the framework will choose - // to yield the component automatically for a short period of time. - final ScheduledFuture scheduledFuture = futureRef.get(); - if (scheduledFuture == null) { - return; - } - - // If we are able to cancel the future, create a new one and update the ScheduleState so that it has - // an accurate accounting of which futures are outstanding; we must then also update the futureRef - // so that we can do this again the next time that the component is yielded. - if (scheduledFuture.cancel(false)) { - synchronized (scheduleState) { - if (scheduleState.isScheduled()) { - final ScheduledFuture newFuture = flowEngine.scheduleWithFixedDelay(this, noWorkYieldNanos, - connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS); - - scheduleState.replaceFuture(scheduledFuture, newFuture); - futureRef.set(newFuture); - } - } - } - } - } - }; + final Runnable trigger = createTrigger(connectableTask, scheduleState, futureRef); // Schedule the task to run - final ScheduledFuture future = flowEngine.scheduleWithFixedDelay(yieldDetectionRunnable, 0L, - connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS); + final ScheduledFuture future = flowEngine.scheduleWithFixedDelay(trigger, 0L, + connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS); // now that we have the future, set the atomic reference so that if the component is yielded we // are able to then cancel this future. @@ -202,8 +106,78 @@ public void run() { logger.info("Scheduled {} to run with {} threads", connectable, connectable.getMaxConcurrentTasks()); } + + private Runnable createTrigger(final ConnectableTask connectableTask, final LifecycleState scheduleState, final AtomicReference> futureRef) { + final Connectable connectable = connectableTask.getConnectable(); + final Runnable yieldDetectionRunnable = new Runnable() { + @Override + public void run() { + // Call the task. It will return a boolean indicating whether or not we should yield + // based on a lack of work for to do for the component. + final InvocationResult invocationResult = connectableTask.invoke(); + if (invocationResult.isYield()) { + logger.debug("Yielding {} due to {}", connectable, invocationResult.getYieldExplanation()); + } + + // If the component is yielded, cancel its future and re-submit it to run again + // after the yield has expired. + final long newYieldExpiration = connectable.getYieldExpiration(); + final long now = System.currentTimeMillis(); + if (newYieldExpiration > now) { + final long yieldMillis = newYieldExpiration - now; + final long scheduleMillis = connectable.getSchedulingPeriod(TimeUnit.MILLISECONDS); + final ScheduledFuture scheduledFuture = futureRef.get(); + if (scheduledFuture == null) { + return; + } + + // If we are able to cancel the future, create a new one and update the ScheduleState so that it has + // an accurate accounting of which futures are outstanding; we must then also update the futureRef + // so that we can do this again the next time that the component is yielded. + if (scheduledFuture.cancel(false)) { + final long yieldNanos = Math.max(TimeUnit.MILLISECONDS.toNanos(scheduleMillis), TimeUnit.MILLISECONDS.toNanos(yieldMillis)); + + synchronized (scheduleState) { + if (scheduleState.isScheduled()) { + final long schedulingNanos = connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS); + final ScheduledFuture newFuture = flowEngine.scheduleWithFixedDelay(this, yieldNanos, schedulingNanos, TimeUnit.NANOSECONDS); + + scheduleState.replaceFuture(scheduledFuture, newFuture); + futureRef.set(newFuture); + } + } + } + } else if (noWorkYieldNanos > 0L && invocationResult.isYield()) { + // Component itself didn't yield but there was no work to do, so the framework will choose + // to yield the component automatically for a short period of time. + final ScheduledFuture scheduledFuture = futureRef.get(); + if (scheduledFuture == null) { + return; + } + + // If we are able to cancel the future, create a new one and update the ScheduleState so that it has + // an accurate accounting of which futures are outstanding; we must then also update the futureRef + // so that we can do this again the next time that the component is yielded. + if (scheduledFuture.cancel(false)) { + synchronized (scheduleState) { + if (scheduleState.isScheduled()) { + final ScheduledFuture newFuture = flowEngine.scheduleWithFixedDelay(this, noWorkYieldNanos, + connectable.getSchedulingPeriod(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS); + + scheduleState.replaceFuture(scheduledFuture, newFuture); + futureRef.set(newFuture); + } + } + } + } + } + }; + + return yieldDetectionRunnable; + } + @Override - public void doUnschedule(final Connectable connectable, final ScheduleState scheduleState) { + public void doUnschedule(final Connectable connectable, final LifecycleState scheduleState) { for (final ScheduledFuture future : scheduleState.getFutures()) { // stop scheduling to run but do not interrupt currently running tasks. future.cancel(false); @@ -213,7 +187,7 @@ public void doUnschedule(final Connectable connectable, final ScheduleState sche } @Override - public void doUnschedule(final ReportingTaskNode taskNode, final ScheduleState scheduleState) { + public void doUnschedule(final ReportingTaskNode taskNode, final LifecycleState scheduleState) { for (final ScheduledFuture future : scheduleState.getFutures()) { // stop scheduling to run but do not interrupt currently running tasks. future.cancel(false); @@ -245,4 +219,13 @@ public void onEvent(final Connectable connectable) { public void setMaxThreadCount(final int maxThreads) { } + @Override + public void incrementMaxThreadCount(int toAdd) { + final int corePoolSize = flowEngine.getCorePoolSize(); + if (corePoolSize - toAdd < 1) { + throw new IllegalStateException("Cannot remove " + (-toAdd) + " threads from pool because there are only " + corePoolSize + " threads in the pool"); + } + + flowEngine.setCorePoolSize(corePoolSize + toAdd); + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceDetails.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceDetails.java index 3db77c97c12d..b02136947602 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceDetails.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceDetails.java @@ -19,7 +19,7 @@ import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.LoggableComponent; -import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.controller.TerminationAwareLogger; /** * Holder for StandardControllerServiceNode to atomically swap out the component. @@ -28,7 +28,7 @@ public class ControllerServiceDetails { private final ControllerService proxiedControllerService; private final ControllerService implementation; - private final ComponentLog componentLog; + private final TerminationAwareLogger componentLog; private final BundleCoordinate bundleCoordinate; private final ControllerServiceInvocationHandler invocationHandler; @@ -50,7 +50,7 @@ public ControllerService getImplementation() { return implementation; } - public ComponentLog getComponentLog() { + public TerminationAwareLogger getComponentLog() { return componentLog; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java index 53fd166a6cd8..cba26a662c68 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java @@ -54,6 +54,7 @@ import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.LoggableComponent; import org.apache.nifi.controller.ReloadComponent; +import org.apache.nifi.controller.TerminationAwareLogger; import org.apache.nifi.controller.ValidationContextFactory; import org.apache.nifi.controller.exception.ControllerServiceInstantiationException; import org.apache.nifi.groups.ProcessGroup; @@ -111,7 +112,7 @@ public ConfigurableComponent getComponent() { } @Override - public ComponentLog getLogger() { + public TerminationAwareLogger getLogger() { return controllerServiceHolder.get().getComponentLog(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java index b4d9e8bbf650..978fcb94c43f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java @@ -54,6 +54,7 @@ import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.TerminationAwareLogger; import org.apache.nifi.controller.ValidationContextFactory; import org.apache.nifi.controller.exception.ComponentLifeCycleException; import org.apache.nifi.controller.exception.ControllerServiceInstantiationException; @@ -146,12 +147,14 @@ public ControllerServiceNode createControllerService(final String type, final St logger.info("Created Controller Service of type {} with identifier {}", type, id); final ComponentLog serviceLogger = new SimpleProcessLogger(id, originalService); - originalService.initialize(new StandardControllerServiceInitializationContext(id, serviceLogger, this, getStateManager(id), nifiProperties)); + final TerminationAwareLogger terminationAwareLogger = new TerminationAwareLogger(serviceLogger); + + originalService.initialize(new StandardControllerServiceInitializationContext(id, terminationAwareLogger, this, getStateManager(id), nifiProperties)); final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this, variableRegistry); - final LoggableComponent originalLoggableComponent = new LoggableComponent<>(originalService, bundleCoordinate, serviceLogger); - final LoggableComponent proxiedLoggableComponent = new LoggableComponent<>(proxiedService, bundleCoordinate, serviceLogger); + final LoggableComponent originalLoggableComponent = new LoggableComponent<>(originalService, bundleCoordinate, terminationAwareLogger); + final LoggableComponent proxiedLoggableComponent = new LoggableComponent<>(proxiedService, bundleCoordinate, terminationAwareLogger); final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry); final ControllerServiceNode serviceNode = new StandardControllerServiceNode(originalLoggableComponent, proxiedLoggableComponent, invocationHandler, diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ActiveTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ActiveTask.java new file mode 100644 index 000000000000..614995b62909 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ActiveTask.java @@ -0,0 +1,39 @@ +/* + * 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.tasks; + +public class ActiveTask { + private final long startTime; + private volatile boolean terminated; + + public ActiveTask(final long startTime) { + this.startTime = startTime; + } + + public long getStartTime() { + return startTime; + } + + public boolean isTerminated() { + return terminated; + } + + public void terminate() { + this.terminated = true; + } +} 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 new file mode 100644 index 000000000000..b0ba44648452 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ConnectableTask.java @@ -0,0 +1,259 @@ +/* + * 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.tasks; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.nifi.components.state.StateManager; +import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.lifecycle.TaskTerminationAwareStateManager; +import org.apache.nifi.controller.repository.ActiveProcessSessionFactory; +import org.apache.nifi.controller.repository.BatchingSessionFactory; +import org.apache.nifi.controller.repository.RepositoryContext; +import org.apache.nifi.controller.repository.StandardProcessSession; +import org.apache.nifi.controller.repository.StandardProcessSessionFactory; +import org.apache.nifi.controller.repository.WeakHashMapProcessSessionFactory; +import org.apache.nifi.controller.repository.metrics.StandardFlowFileEvent; +import org.apache.nifi.controller.scheduling.ConnectableProcessContext; +import org.apache.nifi.controller.scheduling.RepositoryContextFactory; +import org.apache.nifi.controller.scheduling.LifecycleState; +import org.apache.nifi.controller.scheduling.SchedulingAgent; +import org.apache.nifi.encrypt.StringEncryptor; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.nar.NarCloseable; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSessionFactory; +import org.apache.nifi.processor.SimpleProcessLogger; +import org.apache.nifi.processor.StandardProcessContext; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.exception.TerminatedTaskException; +import org.apache.nifi.util.Connectables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Continually runs a processor as long as the processor has work to do. {@link #call()} will return true if the processor should be yielded, false otherwise. + */ +public class ConnectableTask { + + private static final Logger logger = LoggerFactory.getLogger(ConnectableTask.class); + + private final SchedulingAgent schedulingAgent; + private final Connectable connectable; + private final RepositoryContext repositoryContext; + private final LifecycleState scheduleState; + private final ProcessContext processContext; + private final FlowController flowController; + private final int numRelationships; + private final boolean hasNonLoopConnection; + + + public ConnectableTask(final SchedulingAgent schedulingAgent, final Connectable connectable, + final FlowController flowController, final RepositoryContextFactory contextFactory, final LifecycleState scheduleState, + final StringEncryptor encryptor) { + + this.schedulingAgent = schedulingAgent; + this.connectable = connectable; + this.scheduleState = scheduleState; + this.numRelationships = connectable.getRelationships().size(); + this.flowController = flowController; + this.hasNonLoopConnection = Connectables.hasNonLoopConnection(connectable); + + final StateManager stateManager = new TaskTerminationAwareStateManager(flowController.getStateManagerProvider().getStateManager(connectable.getIdentifier()), scheduleState::isTerminated); + if (connectable instanceof ProcessorNode) { + processContext = new StandardProcessContext((ProcessorNode) connectable, flowController, encryptor, stateManager, scheduleState::isTerminated); + } else { + processContext = new ConnectableProcessContext(connectable, encryptor, stateManager); + } + + repositoryContext = contextFactory.newProcessContext(connectable, new AtomicLong(0L)); + } + + public Connectable getConnectable() { + return connectable; + } + + private boolean isRunOnCluster(final FlowController flowController) { + return !connectable.isIsolated() || !flowController.isConfiguredForClustering() || flowController.isPrimary(); + } + + private boolean isYielded() { + // after one yield period, the scheduling agent could call this again when + // yieldExpiration == currentTime, and we don't want that to still be considered 'yielded' + // so this uses ">" instead of ">=" + return connectable.getYieldExpiration() > System.currentTimeMillis(); + } + + private boolean isWorkToDo() { + return connectable.isTriggerWhenEmpty() || !connectable.hasIncomingConnection() || !hasNonLoopConnection || Connectables.flowFilesQueued(connectable); + } + + private boolean isBackPressureEngaged() { + return connectable.getIncomingConnections().stream() + .filter(con -> con.getSource() == connectable) + .map(con -> con.getFlowFileQueue()) + .anyMatch(queue -> queue.isFull()); + } + + public InvocationResult invoke() { + if (scheduleState.isTerminated()) { + return InvocationResult.DO_NOT_YIELD; + } + + // make sure processor is not yielded + if (isYielded()) { + 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)) { + return InvocationResult.DO_NOT_YIELD; + } + + // Make sure processor has work to do. This means that it meets one of these criteria: + // * It is annotated with @TriggerWhenEmpty + // * It has data in an incoming Connection + // * It has no incoming connections + // * All incoming connections are self-loops + if (!isWorkToDo()) { + return InvocationResult.yield("No work to do"); + } + + if (numRelationships > 0) { + final int requiredNumberOfAvailableRelationships = connectable.isTriggerWhenAnyDestinationAvailable() ? 1 : numRelationships; + if (!repositoryContext.isRelationshipAvailabilitySatisfied(requiredNumberOfAvailableRelationships)) { + return InvocationResult.yield("Backpressure Applied"); + } + } + + final long batchNanos = connectable.getRunDuration(TimeUnit.NANOSECONDS); + final ProcessSessionFactory sessionFactory; + final StandardProcessSession rawSession; + final boolean batch; + if (connectable.isSessionBatchingSupported() && batchNanos > 0L) { + rawSession = new StandardProcessSession(repositoryContext, scheduleState::isTerminated); + sessionFactory = new BatchingSessionFactory(rawSession); + batch = true; + } else { + rawSession = null; + sessionFactory = new StandardProcessSessionFactory(repositoryContext, scheduleState::isTerminated); + batch = false; + } + + final ActiveProcessSessionFactory activeSessionFactory = new WeakHashMapProcessSessionFactory(sessionFactory); + scheduleState.incrementActiveThreadCount(activeSessionFactory); + + final long startNanos = System.nanoTime(); + final long finishIfBackpressureEngaged = startNanos + (batchNanos / 25L); + final long finishNanos = startNanos + batchNanos; + int invocationCount = 0; + + final String originalThreadName = Thread.currentThread().getName(); + try { + try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(connectable.getRunnableComponent().getClass(), connectable.getIdentifier())) { + boolean shouldRun = connectable.getScheduledState() == ScheduledState.RUNNING; + while (shouldRun) { + connectable.onTrigger(processContext, activeSessionFactory); + invocationCount++; + + if (!batch) { + return InvocationResult.DO_NOT_YIELD; + } + + final long nanoTime = System.nanoTime(); + if (nanoTime > finishNanos) { + return InvocationResult.DO_NOT_YIELD; + } + + if (nanoTime > finishIfBackpressureEngaged && isBackPressureEngaged()) { + return InvocationResult.DO_NOT_YIELD; + } + + if (connectable.getScheduledState() != ScheduledState.RUNNING) { + break; + } + + if (!isWorkToDo()) { + break; + } + if (isYielded()) { + break; + } + + if (numRelationships > 0) { + final int requiredNumberOfAvailableRelationships = connectable.isTriggerWhenAnyDestinationAvailable() ? 1 : numRelationships; + shouldRun = repositoryContext.isRelationshipAvailabilitySatisfied(requiredNumberOfAvailableRelationships); + } + } + } catch (final TerminatedTaskException tte) { + final ComponentLog procLog = new SimpleProcessLogger(connectable.getIdentifier(), connectable.getRunnableComponent()); + procLog.info("Failed to process session due to task being terminated", new Object[] {tte}); + } catch (final ProcessException pe) { + final ComponentLog procLog = new SimpleProcessLogger(connectable.getIdentifier(), connectable.getRunnableComponent()); + procLog.error("Failed to process session due to {}", new Object[] {pe}); + } catch (final Throwable t) { + // Use ComponentLog to log the event so that a bulletin will be created for this processor + final ComponentLog procLog = new SimpleProcessLogger(connectable.getIdentifier(), connectable.getRunnableComponent()); + procLog.error("{} failed to process session due to {}; Processor Administratively Yielded for {}", + new Object[] {connectable.getRunnableComponent(), t, schedulingAgent.getAdministrativeYieldDuration()}, t); + logger.warn("Administratively Yielding {} due to uncaught Exception: {}", connectable.getRunnableComponent(), t.toString(), t); + + connectable.yield(schedulingAgent.getAdministrativeYieldDuration(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS); + } + } finally { + try { + if (batch) { + try { + rawSession.commit(); + } catch (final Exception e) { + 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); + + try { + rawSession.rollback(true); + } catch (final Exception e1) { + procLog.error("Failed to roll back session {} due to {}", new Object[] { rawSession, e.toString() }, e); + } + } + } + + final long processingNanos = System.nanoTime() - startNanos; + + try { + final StandardFlowFileEvent procEvent = new StandardFlowFileEvent(connectable.getIdentifier()); + procEvent.setProcessingNanos(processingNanos); + procEvent.setInvocations(invocationCount); + repositoryContext.getFlowFileEventRepository().updateRepository(procEvent); + } catch (final IOException e) { + logger.error("Unable to update FlowFileEvent Repository for {}; statistics may be inaccurate. Reason for failure: {}", connectable.getRunnableComponent(), e.toString()); + logger.error("", e); + } + } finally { + scheduleState.decrementActiveThreadCount(activeSessionFactory); + Thread.currentThread().setName(originalThreadName); + } + } + + return InvocationResult.DO_NOT_YIELD; + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java deleted file mode 100644 index daf21a58320c..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.tasks; - -import java.util.concurrent.Callable; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.nifi.annotation.lifecycle.OnStopped; -import org.apache.nifi.connectable.Connectable; -import org.apache.nifi.connectable.ConnectableType; -import org.apache.nifi.controller.repository.StandardProcessSessionFactory; -import org.apache.nifi.controller.scheduling.ProcessContextFactory; -import org.apache.nifi.controller.scheduling.ScheduleState; -import org.apache.nifi.nar.NarCloseable; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.ProcessSessionFactory; -import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.util.Connectables; -import org.apache.nifi.util.ReflectionUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Continually runs a Connectable as long as the processor has work to do. {@link #call()} will return true if the Connectable should be yielded, false otherwise. - */ -public class ContinuallyRunConnectableTask implements Callable { - - private static final Logger logger = LoggerFactory.getLogger(ContinuallyRunConnectableTask.class); - - private final Connectable connectable; - private final ScheduleState scheduleState; - private final ProcessSessionFactory sessionFactory; - private final ProcessContext processContext; - - public ContinuallyRunConnectableTask(final ProcessContextFactory contextFactory, final Connectable connectable, final ScheduleState scheduleState, final ProcessContext processContext) { - this.connectable = connectable; - this.scheduleState = scheduleState; - this.sessionFactory = new StandardProcessSessionFactory(contextFactory.newProcessContext(connectable, new AtomicLong(0L))); - this.processContext = processContext; - } - - @Override - public Boolean call() { - if (!scheduleState.isScheduled()) { - return false; - } - - // Connectable should run if the following conditions are met: - // 1. It is not yielded. - // 2. It has incoming connections with FlowFiles queued or doesn't expect incoming connections - // 3. If it is a funnel, it has an outgoing connection (this is needed because funnels are "always on"; other - // connectable components cannot even be started if they need an outbound connection and don't have one) - // 4. There is a connection for each relationship. - final boolean triggerWhenEmpty = connectable.isTriggerWhenEmpty(); - boolean flowFilesQueued = true; - boolean funnelWithoutConnections = false; - boolean relationshipAvailable = true; - final boolean shouldRun = (connectable.getYieldExpiration() < System.currentTimeMillis()) - && (triggerWhenEmpty || (flowFilesQueued = Connectables.flowFilesQueued(connectable))) - && (connectable.getConnectableType() != ConnectableType.FUNNEL || !(funnelWithoutConnections = connectable.getConnections().isEmpty())) - && (connectable.getRelationships().isEmpty() || (relationshipAvailable = Connectables.anyRelationshipAvailable(connectable))); - - if (shouldRun) { - scheduleState.incrementActiveThreadCount(); - try { - try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(connectable.getClass(), connectable.getIdentifier())) { - connectable.onTrigger(processContext, sessionFactory); - } catch (final ProcessException pe) { - logger.error("{} failed to process session due to {}", connectable, pe.toString()); - } catch (final Throwable t) { - logger.error("{} failed to process session due to {}", connectable, t.toString()); - logger.error("", t); - - logger.warn("{} Administratively Pausing for 10 seconds due to processing failure: {}", connectable, t.toString(), t); - try { - Thread.sleep(10000L); - } catch (final InterruptedException e) { - } - - } - } finally { - if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) { - try (final NarCloseable x = NarCloseable.withComponentNarLoader(connectable.getClass(), connectable.getIdentifier())) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, connectable, processContext); - } - } - - scheduleState.decrementActiveThreadCount(); - } - } else if (!flowFilesQueued || funnelWithoutConnections || !relationshipAvailable) { - // Either there are no FlowFiles queued, it's a funnel without outgoing connections, or the relationship is not available (i.e., backpressure is applied). - // We will yield for just a bit. - return true; - } - - return false; // do not yield - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java deleted file mode 100644 index 8c2707860a12..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java +++ /dev/null @@ -1,225 +0,0 @@ -/* - * 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.tasks; - -import java.io.IOException; -import java.util.concurrent.Callable; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.nifi.controller.FlowController; -import org.apache.nifi.controller.ProcessorNode; -import org.apache.nifi.controller.repository.BatchingSessionFactory; -import org.apache.nifi.controller.repository.ProcessContext; -import org.apache.nifi.controller.repository.StandardProcessSession; -import org.apache.nifi.controller.repository.StandardProcessSessionFactory; -import org.apache.nifi.controller.repository.metrics.StandardFlowFileEvent; -import org.apache.nifi.controller.scheduling.ProcessContextFactory; -import org.apache.nifi.controller.scheduling.ScheduleState; -import org.apache.nifi.controller.scheduling.SchedulingAgent; -import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.nar.NarCloseable; -import org.apache.nifi.processor.ProcessSessionFactory; -import org.apache.nifi.processor.SimpleProcessLogger; -import org.apache.nifi.processor.StandardProcessContext; -import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.util.Connectables; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Continually runs a processor as long as the processor has work to do. {@link #call()} will return true if the processor should be yielded, false otherwise. - */ -public class ContinuallyRunProcessorTask implements Callable { - - private static final Logger logger = LoggerFactory.getLogger(ContinuallyRunProcessorTask.class); - - private final SchedulingAgent schedulingAgent; - private final ProcessorNode procNode; - private final ProcessContext context; - private final ScheduleState scheduleState; - private final StandardProcessContext processContext; - private final FlowController flowController; - private final int numRelationships; - - public ContinuallyRunProcessorTask(final SchedulingAgent schedulingAgent, final ProcessorNode procNode, - final FlowController flowController, final ProcessContextFactory contextFactory, final ScheduleState scheduleState, - final StandardProcessContext processContext) { - - this.schedulingAgent = schedulingAgent; - this.procNode = procNode; - this.scheduleState = scheduleState; - this.numRelationships = procNode.getRelationships().size(); - this.flowController = flowController; - - context = contextFactory.newProcessContext(procNode, new AtomicLong(0L)); - this.processContext = processContext; - } - - static boolean isRunOnCluster(final ProcessorNode procNode, FlowController flowController) { - return !procNode.isIsolated() || !flowController.isConfiguredForClustering() || flowController.isPrimary(); - } - - static boolean isYielded(final ProcessorNode procNode) { - // after one yield period, the scheduling agent could call this again when - // yieldExpiration == currentTime, and we don't want that to still be considered 'yielded' - // so this uses ">" instead of ">=" - return procNode.getYieldExpiration() > System.currentTimeMillis(); - } - - static boolean isWorkToDo(final ProcessorNode procNode) { - return procNode.isTriggerWhenEmpty() || !procNode.hasIncomingConnection() || !Connectables.hasNonLoopConnection(procNode) || Connectables.flowFilesQueued(procNode); - } - - private boolean isBackPressureEngaged() { - return procNode.getIncomingConnections().stream() - .filter(con -> con.getSource() == procNode) - .map(con -> con.getFlowFileQueue()) - .anyMatch(queue -> queue.isFull()); - } - - @Override - public Boolean call() { - // make sure processor is not yielded - if (isYielded(procNode)) { - return false; - } - - // make sure that either we're not clustered or this processor runs on all nodes or that this is the primary node - if (!isRunOnCluster(procNode, flowController)) { - return false; - } - - // Make sure processor has work to do. This means that it meets one of these criteria: - // * It is annotated with @TriggerWhenEmpty - // * It has data in an incoming Connection - // * It has no incoming connections - // * All incoming connections are self-loops - if (!isWorkToDo(procNode)) { - return true; - } - - if (numRelationships > 0) { - final int requiredNumberOfAvailableRelationships = procNode.isTriggerWhenAnyDestinationAvailable() ? 1 : numRelationships; - if (!context.isRelationshipAvailabilitySatisfied(requiredNumberOfAvailableRelationships)) { - return true; - } - } - - final long batchNanos = procNode.getRunDuration(TimeUnit.NANOSECONDS); - final ProcessSessionFactory sessionFactory; - final StandardProcessSession rawSession; - final boolean batch; - if (procNode.isHighThroughputSupported() && batchNanos > 0L) { - rawSession = new StandardProcessSession(context); - sessionFactory = new BatchingSessionFactory(rawSession); - batch = true; - } else { - rawSession = null; - sessionFactory = new StandardProcessSessionFactory(context); - batch = false; - } - - scheduleState.incrementActiveThreadCount(); - - final long startNanos = System.nanoTime(); - final long finishIfBackpressureEngaged = startNanos + (batchNanos / 25L); - final long finishNanos = startNanos + batchNanos; - int invocationCount = 0; - try { - try (final AutoCloseable ncl = NarCloseable.withComponentNarLoader(procNode.getProcessor().getClass(), procNode.getIdentifier())) { - boolean shouldRun = true; - while (shouldRun) { - procNode.onTrigger(processContext, sessionFactory); - invocationCount++; - - if (!batch) { - return false; - } - - final long nanoTime = System.nanoTime(); - if (nanoTime > finishNanos) { - return false; - } - - if (nanoTime > finishIfBackpressureEngaged && isBackPressureEngaged()) { - return false; - } - - - if (!isWorkToDo(procNode)) { - break; - } - if (isYielded(procNode)) { - break; - } - - if (numRelationships > 0) { - final int requiredNumberOfAvailableRelationships = procNode.isTriggerWhenAnyDestinationAvailable() ? 1 : numRelationships; - shouldRun = context.isRelationshipAvailabilitySatisfied(requiredNumberOfAvailableRelationships); - } - } - } catch (final ProcessException pe) { - final ComponentLog procLog = new SimpleProcessLogger(procNode.getIdentifier(), procNode.getProcessor()); - procLog.error("Failed to process session due to {}", new Object[]{pe}); - } catch (final Throwable t) { - // Use ComponentLog to log the event so that a bulletin will be created for this processor - final ComponentLog procLog = new SimpleProcessLogger(procNode.getIdentifier(), procNode.getProcessor()); - procLog.error("{} failed to process session due to {}", new Object[]{procNode.getProcessor(), t}); - procLog.warn("Processor Administratively Yielded for {} due to processing failure", new Object[]{schedulingAgent.getAdministrativeYieldDuration()}); - logger.warn("Administratively Yielding {} due to uncaught Exception: {}", procNode.getProcessor(), t.toString()); - logger.warn("", t); - - procNode.yield(schedulingAgent.getAdministrativeYieldDuration(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS); - } - } finally { - try { - if (batch) { - try { - rawSession.commit(); - } catch (final Exception e) { - final ComponentLog procLog = new SimpleProcessLogger(procNode.getIdentifier(), procNode.getProcessor()); - procLog.error("Failed to commit session {} due to {}; rolling back", new Object[] { rawSession, e.toString() }, e); - - try { - rawSession.rollback(true); - } catch (final Exception e1) { - procLog.error("Failed to roll back session {} due to {}", new Object[] { rawSession, e.toString() }, e); - } - } - } - - final long processingNanos = System.nanoTime() - startNanos; - - try { - final StandardFlowFileEvent procEvent = new StandardFlowFileEvent(procNode.getIdentifier()); - procEvent.setProcessingNanos(processingNanos); - procEvent.setInvocations(invocationCount); - context.getFlowFileEventRepository().updateRepository(procEvent); - } catch (final IOException e) { - logger.error("Unable to update FlowFileEvent Repository for {}; statistics may be inaccurate. Reason for failure: {}", procNode.getProcessor(), e.toString()); - logger.error("", e); - } - } finally { - scheduleState.decrementActiveThreadCount(); - } - } - - return false; - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java index 159f58f3922c..6e2ee4c3be5e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java @@ -25,10 +25,10 @@ import org.apache.nifi.connectable.Port; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.ProcessorNode; -import org.apache.nifi.controller.repository.ProcessContext; +import org.apache.nifi.controller.repository.RepositoryContext; import org.apache.nifi.controller.repository.StandardProcessSession; import org.apache.nifi.controller.repository.StandardProcessSessionFactory; -import org.apache.nifi.controller.scheduling.ProcessContextFactory; +import org.apache.nifi.controller.scheduling.RepositoryContextFactory; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.groups.RemoteProcessGroup; import org.apache.nifi.util.FormatUtils; @@ -42,9 +42,9 @@ public class ExpireFlowFiles implements Runnable { private static final Logger logger = LoggerFactory.getLogger(ExpireFlowFiles.class); private final FlowController flowController; - private final ProcessContextFactory contextFactory; + private final RepositoryContextFactory contextFactory; - public ExpireFlowFiles(final FlowController flowController, final ProcessContextFactory contextFactory) { + public ExpireFlowFiles(final FlowController flowController, final RepositoryContextFactory contextFactory) { this.flowController = flowController; this.contextFactory = contextFactory; } @@ -60,8 +60,8 @@ public void run() { } private StandardProcessSession createSession(final Connectable connectable) { - final ProcessContext context = contextFactory.newProcessContext(connectable, new AtomicLong(0L)); - final StandardProcessSessionFactory sessionFactory = new StandardProcessSessionFactory(context); + final RepositoryContext context = contextFactory.newProcessContext(connectable, new AtomicLong(0L)); + final StandardProcessSessionFactory sessionFactory = new StandardProcessSessionFactory(context, () -> false); return sessionFactory.createSession(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/InvocationResult.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/InvocationResult.java new file mode 100644 index 000000000000..2efff65eb0d3 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/InvocationResult.java @@ -0,0 +1,51 @@ +/* + * 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.tasks; + +public interface InvocationResult { + boolean isYield(); + + String getYieldExplanation(); + + + public static InvocationResult DO_NOT_YIELD = new InvocationResult() { + @Override + public boolean isYield() { + return false; + } + + @Override + public String getYieldExplanation() { + return null; + } + }; + + public static InvocationResult yield(final String explanation) { + return new InvocationResult() { + @Override + public boolean isYield() { + return true; + } + + @Override + public String getYieldExplanation() { + return explanation; + } + }; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java index 5f14d197b289..f89b920e0c97 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java @@ -18,7 +18,7 @@ import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.controller.ReportingTaskNode; -import org.apache.nifi.controller.scheduling.ScheduleState; +import org.apache.nifi.controller.scheduling.LifecycleState; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.processor.SimpleProcessLogger; @@ -27,16 +27,16 @@ public class ReportingTaskWrapper implements Runnable { private final ReportingTaskNode taskNode; - private final ScheduleState scheduleState; + private final LifecycleState scheduleState; - public ReportingTaskWrapper(final ReportingTaskNode taskNode, final ScheduleState scheduleState) { + public ReportingTaskWrapper(final ReportingTaskNode taskNode, final LifecycleState scheduleState) { this.taskNode = taskNode; this.scheduleState = scheduleState; } @Override public synchronized void run() { - scheduleState.incrementActiveThreadCount(); + scheduleState.incrementActiveThreadCount(null); try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(taskNode.getReportingTask().getClass(), taskNode.getIdentifier())) { taskNode.getReportingTask().onTrigger(taskNode.getReportingContext()); } catch (final Throwable t) { @@ -55,7 +55,7 @@ public synchronized void run() { } } } finally { - scheduleState.decrementActiveThreadCount(); + scheduleState.decrementActiveThreadCount(null); } } } 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 c738737eb04d..c4318af7c590 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 @@ -463,7 +463,7 @@ private StateManager getStateManager(final String componentId) { private void shutdown(final ProcessGroup procGroup) { for (final ProcessorNode node : procGroup.getProcessors()) { try (final NarCloseable x = NarCloseable.withComponentNarLoader(node.getProcessor().getClass(), node.getIdentifier())) { - final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor, getStateManager(node.getIdentifier())); + final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor, getStateManager(node.getIdentifier()), () -> false); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, node.getProcessor(), processContext); } } @@ -851,7 +851,7 @@ public void removeProcessor(final ProcessorNode processor) { } try (final NarCloseable x = NarCloseable.withComponentNarLoader(processor.getProcessor().getClass(), processor.getIdentifier())) { - final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor, getStateManager(processor.getIdentifier())); + final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor, getStateManager(processor.getIdentifier()), () -> false); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, processor.getProcessor(), processContext); } catch (final Exception e) { throw new ComponentLifeCycleException("Failed to invoke 'OnRemoved' methods of processor with id " + processor.getIdentifier(), e); @@ -1308,6 +1308,27 @@ public CompletableFuture stopProcessor(final ProcessorNode processor) { } } + @Override + public void terminateProcessor(final ProcessorNode processor) { + readLock.lock(); + try { + if (!processors.containsKey(processor.getIdentifier())) { + throw new IllegalStateException("No processor with ID " + processor.getIdentifier() + " belongs to this Process Group"); + } + + final ScheduledState state = processor.getScheduledState(); + if (state != ScheduledState.STOPPED) { + throw new IllegalStateException("Cannot terminate processor with ID " + processor.getIdentifier() + " because it is not stopped"); + } + + scheduler.terminateProcessor(processor); + } finally { + readLock.unlock(); + } + + } + + @Override public void stopInputPort(final Port port) { readLock.lock(); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java index 8e926040c571..deef9581b331 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java @@ -72,7 +72,7 @@ public void warn(String msg, Object[] os) { } if (lastArgIsException(os)) { - warn(msg, Arrays.copyOfRange(os, 0, os.length - 1), (Throwable) os[os.length - 1]); + warn(msg, os, (Throwable) os[os.length - 1]); } else { msg = "{} " + msg; os = addProcessor(os); @@ -265,7 +265,7 @@ public void error(String msg, Object[] os) { } if (lastArgIsException(os)) { - error(msg, Arrays.copyOfRange(os, 0, os.length - 1), (Throwable) os[os.length - 1]); + error(msg, os, (Throwable) os[os.length - 1]); } else { os = addProcessor(os); msg = "{} " + msg; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java index dfba330ce08a..167fa733b862 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java @@ -35,8 +35,10 @@ import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceLookup; import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.lifecycle.TaskTermination; import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.encrypt.StringEncryptor; +import org.apache.nifi.processor.exception.TerminatedTaskException; import org.apache.nifi.util.Connectables; public class StandardProcessContext implements ProcessContext, ControllerServiceLookup { @@ -46,12 +48,15 @@ public class StandardProcessContext implements ProcessContext, ControllerService private final Map preparedQueries; private final StringEncryptor encryptor; private final StateManager stateManager; + private final TaskTermination taskTermination; - public StandardProcessContext(final ProcessorNode processorNode, final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor, final StateManager stateManager) { + public StandardProcessContext(final ProcessorNode processorNode, final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor, final StateManager stateManager, + final TaskTermination taskTermination) { this.procNode = processorNode; this.controllerServiceProvider = controllerServiceProvider; this.encryptor = encryptor; this.stateManager = stateManager; + this.taskTermination = taskTermination; preparedQueries = new HashMap<>(); for (final Map.Entry entry : procNode.getProperties().entrySet()) { @@ -68,8 +73,15 @@ public StandardProcessContext(final ProcessorNode processorNode, final Controlle } } + private void verifyTaskActive() { + if (taskTermination.isTerminated()) { + throw new TerminatedTaskException(); + } + } + @Override public PropertyValue getProperty(final PropertyDescriptor descriptor) { + verifyTaskActive(); return getProperty(descriptor.getName()); } @@ -80,6 +92,7 @@ public PropertyValue getProperty(final PropertyDescriptor descriptor) { */ @Override public PropertyValue getProperty(final String propertyName) { + verifyTaskActive(); final Processor processor = procNode.getProcessor(); final PropertyDescriptor descriptor = processor.getPropertyDescriptor(propertyName); if (descriptor == null) { @@ -94,36 +107,43 @@ public PropertyValue getProperty(final String propertyName) { @Override public PropertyValue newPropertyValue(final String rawValue) { + verifyTaskActive(); return new StandardPropertyValue(rawValue, this, Query.prepare(rawValue), procNode.getVariableRegistry()); } @Override public void yield() { + verifyTaskActive(); procNode.yield(); } @Override public ControllerService getControllerService(final String serviceIdentifier) { + verifyTaskActive(); return controllerServiceProvider.getControllerServiceForComponent(serviceIdentifier, procNode.getIdentifier()); } @Override public int getMaxConcurrentTasks() { + verifyTaskActive(); return procNode.getMaxConcurrentTasks(); } @Override public String getAnnotationData() { + verifyTaskActive(); return procNode.getAnnotationData(); } @Override public Map getProperties() { + verifyTaskActive(); return procNode.getProperties(); } @Override public Map getAllProperties() { + verifyTaskActive(); final Map propValueMap = new LinkedHashMap<>(); for (final Map.Entry entry : getProperties().entrySet()) { propValueMap.put(entry.getKey().getName(), entry.getValue()); @@ -133,16 +153,19 @@ public Map getAllProperties() { @Override public String encrypt(final String unencrypted) { + verifyTaskActive(); return encryptor.encrypt(unencrypted); } @Override public String decrypt(final String encrypted) { + verifyTaskActive(); return encryptor.decrypt(encrypted); } @Override public Set getControllerServiceIdentifiers(final Class serviceType) { + verifyTaskActive(); if (!serviceType.isInterface()) { throw new IllegalArgumentException("ControllerServices may be referenced only via their interfaces; " + serviceType + " is not an interface"); } @@ -151,26 +174,31 @@ public Set getControllerServiceIdentifiers(final Class getAvailableRelationships() { + verifyTaskActive(); final Set set = new HashSet<>(); for (final Relationship relationship : procNode.getRelationships()) { final Collection connections = procNode.getConnections(relationship); @@ -195,27 +223,32 @@ public Set getAvailableRelationships() { @Override public String getControllerServiceName(final String serviceIdentifier) { + verifyTaskActive(); return controllerServiceProvider.getControllerServiceName(serviceIdentifier); } @Override public boolean hasIncomingConnection() { + verifyTaskActive(); return procNode.hasIncomingConnection(); } @Override public boolean hasNonLoopConnection() { + verifyTaskActive(); return Connectables.hasNonLoopConnection(procNode); } @Override public boolean hasConnection(final Relationship relationship) { + verifyTaskActive(); final Set connections = procNode.getConnections(relationship); return connections != null && !connections.isEmpty(); } @Override public boolean isExpressionLanguagePresent(final PropertyDescriptor property) { + verifyTaskActive(); if (property == null || !property.isExpressionLanguageSupported()) { return false; } @@ -226,11 +259,13 @@ public boolean isExpressionLanguagePresent(final PropertyDescriptor property) { @Override public StateManager getStateManager() { + verifyTaskActive(); return stateManager; } @Override public String getName() { + verifyTaskActive(); return procNode.getName(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java index d9728b24f791..58388710393b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java @@ -769,7 +769,7 @@ public void testInstantiateSnippetWhenProcessorMissingBundle() throws Exception processorDTO.setDescription("description"); processorDTO.setSupportsParallelProcessing(!processorNode.isTriggeredSerially()); processorDTO.setSupportsEventDriven(processorNode.isEventDrivenSupported()); - processorDTO.setSupportsBatching(processorNode.isHighThroughputSupported()); + processorDTO.setSupportsBatching(processorNode.isSessionBatchingSupported()); ProcessorConfigDTO configDTO = new ProcessorConfigDTO(); configDTO.setSchedulingPeriod(processorNode.getSchedulingPeriod()); @@ -822,7 +822,7 @@ public void testInstantiateSnippetWithProcessor() throws ProcessorInstantiationE processorDTO.setDescription("description"); processorDTO.setSupportsParallelProcessing(!processorNode.isTriggeredSerially()); processorDTO.setSupportsEventDriven(processorNode.isEventDrivenSupported()); - processorDTO.setSupportsBatching(processorNode.isHighThroughputSupported()); + processorDTO.setSupportsBatching(processorNode.isSessionBatchingSupported()); ProcessorConfigDTO configDTO = new ProcessorConfigDTO(); configDTO.setSchedulingPeriod(processorNode.getSchedulingPeriod()); @@ -877,7 +877,7 @@ public void testInstantiateSnippetWithDisabledProcessor() throws ProcessorInstan processorDTO.setDescription("description"); processorDTO.setSupportsParallelProcessing(!processorNode.isTriggeredSerially()); processorDTO.setSupportsEventDriven(processorNode.isEventDrivenSupported()); - processorDTO.setSupportsBatching(processorNode.isHighThroughputSupported()); + processorDTO.setSupportsBatching(processorNode.isSessionBatchingSupported()); ProcessorConfigDTO configDTO = new ProcessorConfigDTO(); configDTO.setSchedulingPeriod(processorNode.getSchedulingPeriod()); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java index 9bff6f6fd488..552db3db3339 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java @@ -51,7 +51,6 @@ import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.engine.FlowEngine; import org.apache.nifi.expression.ExpressionLanguageCompiler; -import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.nar.SystemBundle; @@ -102,7 +101,7 @@ public void testStart() throws InterruptedException { NiFiProperties.createBasicNiFiProperties(null, null), new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent); final ScheduledExecutorService taskScheduler = new FlowEngine(1, "TestClasspathResources", true); - final StandardProcessContext processContext = new StandardProcessContext(procNode, null, null, null); + final StandardProcessContext processContext = new StandardProcessContext(procNode, null, null, null, () -> false); final SchedulingAgentCallback schedulingAgentCallback = new SchedulingAgentCallback() { @Override public void onTaskComplete() { @@ -405,7 +404,7 @@ private StandardProcessorNode createProcessorNode(final Processor processor, fin final ValidationContextFactory validationContextFactory = createValidationContextFactory(); final NiFiProperties niFiProperties = NiFiProperties.createBasicNiFiProperties("src/test/resources/conf/nifi.properties", null); final ProcessScheduler processScheduler = Mockito.mock(ProcessScheduler.class); - final ComponentLog componentLog = Mockito.mock(ComponentLog.class); + final TerminationAwareLogger componentLog = Mockito.mock(TerminationAwareLogger.class); final Bundle systemBundle = SystemBundle.create(niFiProperties); ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet()); @@ -422,7 +421,6 @@ private StandardProcessorNode createProcessorNode(final Processor processor, fin private static class MockReloadComponent implements ReloadComponent { private String newType; - private BundleCoordinate bundleCoordinate; private final Set additionalUrls = new LinkedHashSet<>(); public Set getAdditionalUrls() { @@ -433,28 +431,23 @@ public String getNewType() { return newType; } - public BundleCoordinate getBundleCoordinate() { - return bundleCoordinate; - } - @Override public void reload(ProcessorNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set additionalUrls) throws ProcessorInstantiationException { - reload(newType, bundleCoordinate, additionalUrls); + reload(newType, additionalUrls); } @Override public void reload(ControllerServiceNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set additionalUrls) throws ControllerServiceInstantiationException { - reload(newType, bundleCoordinate, additionalUrls); + reload(newType, additionalUrls); } @Override public void reload(ReportingTaskNode existingNode, String newType, BundleCoordinate bundleCoordinate, Set additionalUrls) throws ReportingTaskInstantiationException { - reload(newType, bundleCoordinate, additionalUrls); + reload(newType, additionalUrls); } - private void reload(String newType, BundleCoordinate bundleCoordinate, Set additionalUrls) { + private void reload(String newType, Set additionalUrls) { this.newType = newType; - this.bundleCoordinate = bundleCoordinate; this.additionalUrls.clear(); if (additionalUrls != null) { this.additionalUrls.addAll(additionalUrls); 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 22a013174946..e28c2afd7096 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 @@ -16,6 +16,47 @@ */ package org.apache.nifi.controller.repository; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.notNull; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FilterOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.regex.Pattern; + import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; @@ -55,53 +96,12 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.FilterOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; -import java.util.regex.Pattern; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.notNull; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - public class TestStandardProcessSession { private StandardProcessSession session; private MockContentRepository contentRepo; private FlowFileQueue flowFileQueue; - private ProcessContext context; + private RepositoryContext context; private Connectable connectable; private ProvenanceEventRepository provenanceRepo; @@ -188,11 +188,10 @@ public Set answer(final InvocationOnMock invocation) throws Throwabl contentRepo.initialize(new StandardResourceClaimManager()); flowFileRepo = new MockFlowFileRepository(); - context = new ProcessContext(connectable, new AtomicLong(0L), contentRepo, flowFileRepo, flowFileEventRepo, counterRepo, provenanceRepo); - session = new StandardProcessSession(context); + context = new RepositoryContext(connectable, new AtomicLong(0L), contentRepo, flowFileRepo, flowFileEventRepo, counterRepo, provenanceRepo); + session = new StandardProcessSession(context, () -> false); } - @SuppressWarnings("unchecked") private Connection createConnection() { AtomicReference queueReference = new AtomicReference<>(flowFileQueue); Connection connection = createConnection(queueReference); @@ -1232,7 +1231,7 @@ public void testManyFilesOpened() throws IOException { StandardProcessSession[] standardProcessSessions = new StandardProcessSession[100000]; for (int i = 0; i < 70000; i++) { - standardProcessSessions[i] = new StandardProcessSession(context); + standardProcessSessions[i] = new StandardProcessSession(context, () -> false); FlowFile flowFile = standardProcessSessions[i].create(); final byte[] buff = new byte["Hello".getBytes().length]; @@ -1805,7 +1804,7 @@ public void testMigrateWithAppendableStream() throws IOException { flowFile = session.append(flowFile, out -> out.write("1".getBytes())); flowFile = session.append(flowFile, out -> out.write("2".getBytes())); - final StandardProcessSession newSession = new StandardProcessSession(context); + final StandardProcessSession newSession = new StandardProcessSession(context, () -> false); assertTrue(session.isFlowFileKnown(flowFile)); assertFalse(newSession.isFlowFileKnown(flowFile)); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/DummyScheduleState.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/DummyScheduleState.java index f70e188bb21e..70d9df74546b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/DummyScheduleState.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/DummyScheduleState.java @@ -16,7 +16,7 @@ */ package org.apache.nifi.controller.scheduling; -public class DummyScheduleState extends ScheduleState { +public class DummyScheduleState extends LifecycleState { public DummyScheduleState(boolean isScheduled) { setScheduled(isScheduled); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java index 2497ac7cab5e..ea721b8ae8d5 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java @@ -51,13 +51,13 @@ import org.apache.nifi.controller.ReloadComponent; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.StandardProcessorNode; +import org.apache.nifi.controller.TerminationAwareLogger; import org.apache.nifi.controller.ValidationContextFactory; import org.apache.nifi.controller.cluster.Heartbeater; import org.apache.nifi.controller.reporting.StandardReportingInitializationContext; import org.apache.nifi.controller.reporting.StandardReportingTaskNode; import org.apache.nifi.controller.scheduling.processors.FailOnScheduledProcessor; import org.apache.nifi.controller.service.ControllerServiceNode; -import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.controller.service.ControllerServiceState; import org.apache.nifi.controller.service.StandardControllerServiceNode; import org.apache.nifi.controller.service.StandardControllerServiceProvider; @@ -115,7 +115,7 @@ public void setup() throws InitializationException { systemBundle = SystemBundle.create(nifiProperties); ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet()); - scheduler = new StandardProcessScheduler(new FlowEngine(1, "Unit Test", true), Mockito.mock(ControllerServiceProvider.class), null, stateMgrProvider, nifiProperties); + scheduler = new StandardProcessScheduler(new FlowEngine(1, "Unit Test", true), Mockito.mock(FlowController.class), null, stateMgrProvider, nifiProperties); scheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, Mockito.mock(SchedulingAgent.class)); reportingTask = new TestReportingTask(); @@ -124,7 +124,7 @@ public void setup() throws InitializationException { reportingTask.initialize(config); final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(null, variableRegistry); - final ComponentLog logger = Mockito.mock(ComponentLog.class); + final TerminationAwareLogger logger = Mockito.mock(TerminationAwareLogger.class); final ReloadComponent reloadComponent = Mockito.mock(ReloadComponent.class); final LoggableComponent loggableComponent = new LoggableComponent<>(reportingTask, systemBundle.getBundleDetails().getCoordinate(), logger); taskNode = new StandardReportingTaskNode(loggableComponent, UUID.randomUUID().toString(), null, scheduler, validationContextFactory, diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java index 95e2d6a4b00f..08a71bcf0cd2 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java @@ -682,4 +682,8 @@ public void verifyCanShowLocalModifications() { @Override public void onComponentModified() { } + + @Override + public void terminateProcessor(ProcessorNode processor) { + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestContinuallyRunProcessorTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestConnectableTask.java similarity index 58% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestContinuallyRunProcessorTask.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestConnectableTask.java index 90e40a2d499b..3ff95800c738 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestContinuallyRunProcessorTask.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestConnectableTask.java @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.nifi.controller.tasks; import static org.junit.Assert.assertFalse; @@ -21,25 +22,49 @@ import static org.mockito.Mockito.when; import java.util.Collections; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.nifi.components.state.StateManagerProvider; +import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connection; +import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.queue.FlowFileQueue; -import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.controller.repository.FlowFileEventRepository; +import org.apache.nifi.controller.repository.RepositoryContext; +import org.apache.nifi.controller.scheduling.RepositoryContextFactory; +import org.apache.nifi.controller.scheduling.LifecycleState; +import org.apache.nifi.controller.scheduling.SchedulingAgent; +import org.apache.nifi.encrypt.StringEncryptor; +import org.apache.nifi.processor.Processor; import org.junit.Test; import org.mockito.Mockito; -public class TestContinuallyRunProcessorTask { - +public class TestConnectableTask { @Test public void testIsWorkToDo() { - System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, TestContinuallyRunProcessorTask.class.getResource("/conf/nifi.properties").getFile()); - final ProcessorNode procNode = Mockito.mock(ProcessorNode.class); Mockito.when(procNode.hasIncomingConnection()).thenReturn(false); + final Processor processor = Mockito.mock(Processor.class); + Mockito.when(procNode.getIdentifier()).thenReturn("123"); + Mockito.when(procNode.getRunnableComponent()).thenReturn(processor); + + final FlowController flowController = Mockito.mock(FlowController.class); + Mockito.when(flowController.getStateManagerProvider()).thenReturn(Mockito.mock(StateManagerProvider.class)); + + final RepositoryContext repoContext = Mockito.mock(RepositoryContext.class); + Mockito.when(repoContext.getFlowFileEventRepository()).thenReturn(Mockito.mock(FlowFileEventRepository.class)); + + final RepositoryContextFactory contextFactory = Mockito.mock(RepositoryContextFactory.class); + Mockito.when(contextFactory.newProcessContext(Mockito.any(Connectable.class), Mockito.any(AtomicLong.class))).thenReturn(repoContext); + + final LifecycleState scheduleState = new LifecycleState(); + final StringEncryptor encryptor = Mockito.mock(StringEncryptor.class); + ConnectableTask task = new ConnectableTask(Mockito.mock(SchedulingAgent.class), procNode, flowController, contextFactory, scheduleState, encryptor); + // There is work to do because there are no incoming connections. - assertTrue(ContinuallyRunProcessorTask.isWorkToDo(procNode)); + assertFalse(task.invoke().isYield()); // Test with only a single connection that is self-looping and empty final Connection selfLoopingConnection = Mockito.mock(Connection.class); @@ -48,7 +73,7 @@ public void testIsWorkToDo() { when(procNode.hasIncomingConnection()).thenReturn(true); when(procNode.getIncomingConnections()).thenReturn(Collections.singletonList(selfLoopingConnection)); - assertTrue(ContinuallyRunProcessorTask.isWorkToDo(procNode)); + assertFalse(task.invoke().isYield()); // Test with only a single connection that is self-looping and empty final FlowFileQueue flowFileQueue = Mockito.mock(FlowFileQueue.class); @@ -58,7 +83,7 @@ public void testIsWorkToDo() { when(nonEmptyQueue.isActiveQueueEmpty()).thenReturn(false); when(selfLoopingConnection.getFlowFileQueue()).thenReturn(nonEmptyQueue); - assertTrue(ContinuallyRunProcessorTask.isWorkToDo(procNode)); + assertFalse(task.invoke().isYield()); // Test with only a non-looping Connection that has no FlowFiles final Connection emptyConnection = Mockito.mock(Connection.class); @@ -67,7 +92,10 @@ public void testIsWorkToDo() { when(emptyConnection.getFlowFileQueue()).thenReturn(flowFileQueue); when(procNode.getIncomingConnections()).thenReturn(Collections.singletonList(emptyConnection)); - assertFalse(ContinuallyRunProcessorTask.isWorkToDo(procNode)); + + // Create a new ConnectableTask because we want to have a different value for the 'hasNonLoopConnection' value, which is calculated in the task's constructor. + task = new ConnectableTask(Mockito.mock(SchedulingAgent.class), procNode, flowController, contextFactory, scheduleState, encryptor); + assertTrue(task.invoke().isYield()); // test when the queue has data final Connection nonEmptyConnection = Mockito.mock(Connection.class); @@ -75,7 +103,7 @@ public void testIsWorkToDo() { when(nonEmptyConnection.getDestination()).thenReturn(procNode); when(nonEmptyConnection.getFlowFileQueue()).thenReturn(nonEmptyQueue); when(procNode.getIncomingConnections()).thenReturn(Collections.singletonList(nonEmptyConnection)); - assertTrue(ContinuallyRunProcessorTask.isWorkToDo(procNode)); + assertFalse(task.invoke().isYield()); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestContinuallyRunConnectableTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestContinuallyRunConnectableTask.java deleted file mode 100644 index 51339819c183..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestContinuallyRunConnectableTask.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * 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.tasks; - -import org.apache.nifi.connectable.ConnectableType; -import org.apache.nifi.connectable.Connection; -import org.apache.nifi.connectable.Funnel; -import org.apache.nifi.controller.queue.FlowFileQueue; -import org.apache.nifi.controller.scheduling.DummyScheduleState; -import org.apache.nifi.controller.scheduling.ProcessContextFactory; -import org.apache.nifi.processor.ProcessContext; -import org.apache.nifi.processor.Relationship; -import org.junit.Assert; -import org.junit.Test; -import org.mockito.Mockito; - -import java.util.Collections; - -public class TestContinuallyRunConnectableTask { - - @Test - public void funnelsShouldYieldWhenNoOutboundConnections() { - - // Incoming connection with FlowFile - final FlowFileQueue flowFileQueueNonEmpty = Mockito.mock(FlowFileQueue.class); - Mockito.when(flowFileQueueNonEmpty.isActiveQueueEmpty()).thenReturn(false); - - final Connection connectionNonEmpty = Mockito.mock(Connection.class); - Mockito.when(connectionNonEmpty.getFlowFileQueue()).thenReturn(flowFileQueueNonEmpty); - - // Create a Funnel with an inbound connection, and no outbound connections - final Funnel testFunnelNoOutbound = Mockito.mock(Funnel.class); - Mockito.when(testFunnelNoOutbound.getIncomingConnections()).thenReturn(Collections.singletonList(connectionNonEmpty)); - Mockito.when(testFunnelNoOutbound.getConnections()).thenReturn(Collections.emptySet()); - - // Set the Funnel to be yielding up to 5 seconds ago - Mockito.when(testFunnelNoOutbound.getYieldExpiration()).thenReturn(System.currentTimeMillis() - 5000); - // Set the Funnel 'isTriggeredWhenEmpty' to false (same as what 'StandardFunnel' returns) - Mockito.when(testFunnelNoOutbound.isTriggerWhenEmpty()).thenReturn(false); - // Set the Funnel connection type - Mockito.when(testFunnelNoOutbound.getConnectableType()).thenReturn(ConnectableType.FUNNEL); - // Set the Funnel relationships to Anonymous (same as what 'StandardFunnel' returns) - Mockito.when(testFunnelNoOutbound.getRelationships()).thenReturn(Collections.singletonList(Relationship.ANONYMOUS)); - - // Create Mock 'ProcessContextFactory', and 'ProcessContext' - final ProcessContextFactory pcf = Mockito.mock(ProcessContextFactory.class); - Mockito.when(pcf.newProcessContext(Mockito.any(), Mockito.any())).thenReturn(null); - - final ProcessContext pc = Mockito.mock(ProcessContext.class); - - // Create ContinuallyRunConnectableTask - ContinuallyRunConnectableTask crct = new ContinuallyRunConnectableTask(pcf, testFunnelNoOutbound, new DummyScheduleState(true), pc); - - // We should yield since this Funnel has no outbound connections. - Assert.assertTrue("Didn't yield when a Funnel has no outbound connections.", crct.call()); - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java index ce9444065b1e..0e3d06176bb7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java @@ -525,6 +525,21 @@ Set getControllerServiceTypes(final String serviceType, final */ ProcessorEntity getProcessor(String id); + /** + * Terminates the Processor with the specified id + * + * @param id Id of the processor to terminate + * @return The Processor transfer object + */ + ProcessorEntity terminateProcessor(String id); + + /** + * Verifies that the processor with the given id can be terminated at this time + * + * @param processorId the id of the processor + */ + void verifyTerminateProcessor(String processorId); + /** * Gets the Diagnostic information for the Processor with the specified id * diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java index 80283f7d9c37..8c28ce603980 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java @@ -1181,6 +1181,17 @@ public ProcessorEntity deleteProcessor(final Revision revision, final String pro return entityFactory.createProcessorEntity(snapshot, null, permissions, null, null); } + @Override + public ProcessorEntity terminateProcessor(final String processorId) { + processorDAO.terminate(processorId); + return getProcessor(processorId); + } + + @Override + public void verifyTerminateProcessor(final String processorId) { + processorDAO.verifyTerminate(processorId); + } + @Override public LabelEntity deleteLabel(final Revision revision, final String labelId) { final Label label = labelDAO.getLabel(labelId); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java index fcf1a9b30383..b155d37fe163 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessorResource.java @@ -16,12 +16,23 @@ */ package org.apache.nifi.web.api; -import io.swagger.annotations.Api; -import io.swagger.annotations.ApiOperation; -import io.swagger.annotations.ApiParam; -import io.swagger.annotations.ApiResponse; -import io.swagger.annotations.ApiResponses; -import io.swagger.annotations.Authorization; +import javax.servlet.ServletContext; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; +import javax.ws.rs.DefaultValue; +import javax.ws.rs.GET; +import javax.ws.rs.HttpMethod; +import javax.ws.rs.POST; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; + import org.apache.commons.lang3.StringUtils; import org.apache.nifi.authorization.AuthorizeControllerServiceReference; import org.apache.nifi.authorization.Authorizer; @@ -48,25 +59,16 @@ import org.apache.nifi.web.api.request.ClientIdParameter; import org.apache.nifi.web.api.request.LongParameter; -import javax.servlet.ServletContext; -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.Consumes; -import javax.ws.rs.DELETE; -import javax.ws.rs.DefaultValue; -import javax.ws.rs.GET; -import javax.ws.rs.HttpMethod; -import javax.ws.rs.POST; -import javax.ws.rs.PUT; -import javax.ws.rs.Path; -import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.QueryParam; -import javax.ws.rs.core.Context; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; import java.util.List; import java.util.Set; +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiParam; +import io.swagger.annotations.ApiResponse; +import io.swagger.annotations.ApiResponses; +import io.swagger.annotations.Authorization; + /** * RESTful endpoint for managing a Processor. */ @@ -209,6 +211,50 @@ public Response getProcessor( return generateOkResponse(entity).build(); } + @DELETE + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/threads") + @ApiOperation(value = "Terminates a processor, essentially \"deleting\" its threads and any active tasks", response = ProcessorEntity.class, authorizations = { + @Authorization(value = "Write - /processors/{uuid}") + }) + @ApiResponses(value = { + @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(code = 401, message = "Client could not be authenticated."), + @ApiResponse(code = 403, message = "Client is not authorized to make this request."), + @ApiResponse(code = 404, message = "The specified resource could not be found."), + @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.") + }) + public Response terminateProcessor( + @QueryParam(VERSION) final LongParameter version, + @ApiParam(value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.", required = false) @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId, + @ApiParam(value = "The processor id.", required = true) @PathParam("id") final String id) throws InterruptedException { + + if (isReplicateRequest()) { + return replicate(HttpMethod.POST); + } + + final Revision requestRevision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id); + final ProcessorEntity requestProcessorEntity = new ProcessorEntity(); + requestProcessorEntity.setId(id); + + return withWriteLock( + serviceFacade, + requestProcessorEntity, + requestRevision, + lookup -> { + final Authorizable authorizable = lookup.getProcessor(id).getAuthorizable(); + authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); + }, + () -> serviceFacade.verifyTerminateProcessor(id), + (revision, processorEntity) -> { + final ProcessorEntity entity = serviceFacade.terminateProcessor(requestProcessorEntity.getId()); + populateRemainingProcessorEntityContent(entity); + + return generateOkResponse(entity).build(); + }); + } + @GET @Consumes(MediaType.WILDCARD) @Produces(MediaType.APPLICATION_JSON) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java index b0ef2493f127..7c9eb4410f95 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java @@ -2753,7 +2753,7 @@ public int compare(final RelationshipDTO r1, final RelationshipDTO r2) { dto.setDescription(getCapabilityDescription(node.getClass())); dto.setSupportsParallelProcessing(!node.isTriggeredSerially()); dto.setSupportsEventDriven(node.isEventDrivenSupported()); - dto.setSupportsBatching(node.isHighThroughputSupported()); + dto.setSupportsBatching(node.isSessionBatchingSupported()); dto.setConfig(createProcessorConfigDto(node)); final Collection validationErrors = node.getValidationErrors(); @@ -3435,6 +3435,7 @@ private List createThreadDumpDtos(final ProcessorNode procNode) { dto.setStackTrace(threadInfo.getStackTrace()); dto.setThreadActiveMillis(threadInfo.getActiveMillis()); dto.setThreadName(threadInfo.getThreadName()); + dto.setTaskTerminated(threadInfo.isTerminated()); threadDumps.add(dto); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessorDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessorDAO.java index a1bf17012864..04a1440e82bb 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessorDAO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessorDAO.java @@ -71,6 +71,20 @@ public interface ProcessorDAO { */ void verifyUpdate(ProcessorDTO processorDTO); + /** + * Verifies that the specified processor can be terminated at this time + * + * @param processorId the id of the processor + */ + void verifyTerminate(String processorId); + + /** + * Terminates the processor with the given ID + * + * @param processorId + */ + void terminate(String processorId); + /** * Updates the configuration for the processor using the specified processorDTO. * 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/StandardProcessorDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java index 95d0b54b8494..5838f37ab0bb 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java @@ -317,6 +317,19 @@ public Set getProcessors(String groupId, boolean includeDescendan } } + @Override + public void verifyTerminate(final String processorId) { + final ProcessorNode processor = locateProcessor(processorId); + processor.verifyCanTerminate(); + } + + @Override + public void terminate(final String processorId) { + final ProcessorNode processor = locateProcessor(processorId); + processor.getProcessGroup().terminateProcessor(processor); + } + + @Override public void verifyUpdate(final ProcessorDTO processorDTO) { verifyUpdate(locateProcessor(processorDTO.getId()), processorDTO); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DebugFlow.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DebugFlow.java index 6e5515717535..93047b737fa6 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DebugFlow.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DebugFlow.java @@ -206,7 +206,22 @@ public class DebugFlow extends AbstractProcessor { .allowableValues("true", "false") .defaultValue("false") .build(); - + static final PropertyDescriptor ON_TRIGGER_SLEEP_TIME = new PropertyDescriptor.Builder() + .name("OnTrigger Pause Time") + .description("Specifies how long the processor should sleep in the onTrigger() method, so that the processor can be forced to take a long time to perform its task") + .required(true) + .defaultValue("0 sec") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .build(); + static final PropertyDescriptor IGNORE_INTERRUPTS = new PropertyDescriptor.Builder() + .name("Ignore Interrupts When Paused") + .description("If the Processor's thread(s) are sleeping (due to one of the \"Pause Time\" properties above), and the thread is interrupted, " + + "this indicates whether the Processor should ignore the interrupt and continue sleeping or if it should allow itself to be interrupted.") + .expressionLanguageSupported(false) + .allowableValues("true", "false") + .defaultValue("false") + .required(true) + .build(); private volatile Integer flowFileMaxSuccess = 0; private volatile Integer flowFileMaxFailure = 0; @@ -273,6 +288,8 @@ protected List getSupportedPropertyDescriptors() { propList.add(ON_UNSCHEDULED_FAIL); propList.add(ON_STOPPED_SLEEP_TIME); propList.add(ON_STOPPED_FAIL); + propList.add(ON_TRIGGER_SLEEP_TIME); + propList.add(IGNORE_INTERRUPTS); propertyDescriptors.compareAndSet(null, Collections.unmodifiableList(propList)); } @@ -297,25 +314,42 @@ public void onScheduled(ProcessContext context) throws ClassNotFoundException, I flowFileExceptionClass = (Class) Class.forName(context.getProperty(FF_EXCEPTION_CLASS).toString()); noFlowFileExceptionClass = (Class) Class.forName(context.getProperty(NO_FF_EXCEPTION_CLASS).toString()); - sleep(context.getProperty(ON_SCHEDULED_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS)); + sleep(context.getProperty(ON_SCHEDULED_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS), + context.getProperty(IGNORE_INTERRUPTS).asBoolean()); fail(context.getProperty(ON_SCHEDULED_FAIL).asBoolean(), OnScheduled.class); } @OnUnscheduled public void onUnscheduled(final ProcessContext context) throws InterruptedException { - sleep(context.getProperty(ON_UNSCHEDULED_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS)); + sleep(context.getProperty(ON_UNSCHEDULED_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS), + context.getProperty(IGNORE_INTERRUPTS).asBoolean()); fail(context.getProperty(ON_UNSCHEDULED_FAIL).asBoolean(), OnUnscheduled.class); } @OnStopped public void onStopped(final ProcessContext context) throws InterruptedException { - sleep(context.getProperty(ON_STOPPED_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS)); + sleep(context.getProperty(ON_STOPPED_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS), + context.getProperty(IGNORE_INTERRUPTS).asBoolean()); + fail(context.getProperty(ON_STOPPED_FAIL).asBoolean(), OnStopped.class); } - private void sleep(final long millis) throws InterruptedException { + private void sleep(final long millis, final boolean ignoreInterrupts) throws InterruptedException { if (millis > 0L) { - Thread.sleep(millis); + final long endSleep = System.currentTimeMillis() + millis; + while (System.currentTimeMillis() < endSleep) { + // Use Math.max(1, ) here in case + // System.currentTimeMillis() has changed since the check above + // and subtracting it from endSleep would now result in a value <= 0. + try { + Thread.sleep(Math.max(1L, endSleep - System.currentTimeMillis())); + } catch (final InterruptedException ie) { + if (!ignoreInterrupts) { + Thread.currentThread().interrupt(); + throw ie; + } + } + } } } @@ -332,164 +366,181 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro FlowFile ff = session.get(); - // Make up to 2 passes to allow rollover from last cycle to first. - // (This could be "while(true)" since responses should break out if selected, but this - // prevents endless loops in the event of unexpected errors or future changes.) - for (int pass = 2; pass > 0; pass--) { - if (ff == null) { - if (curr_noff_resp.state() == NoFlowFileResponseState.NO_FF_SKIP_RESPONSE) { - if (noFlowFileCurrSkip < noFlowFileMaxSkip) { - noFlowFileCurrSkip += 1; - logger.info("DebugFlow skipping with no flow file"); - return; - } else { - noFlowFileCurrSkip = 0; - curr_noff_resp.getNextCycle(); + try { + // Make up to 2 passes to allow rollover from last cycle to first. + // (This could be "while(true)" since responses should break out if selected, but this + // prevents endless loops in the event of unexpected errors or future changes.) + for (int pass = 2; pass > 0; pass--) { + if (ff == null) { + if (curr_noff_resp.state() == NoFlowFileResponseState.NO_FF_SKIP_RESPONSE) { + if (noFlowFileCurrSkip < noFlowFileMaxSkip) { + noFlowFileCurrSkip += 1; + logger.info("DebugFlow skipping with no flow file"); + return; + } else { + noFlowFileCurrSkip = 0; + curr_noff_resp.getNextCycle(); + } } - } - if (curr_noff_resp.state() == NoFlowFileResponseState.NO_FF_EXCEPTION_RESPONSE) { - if (noFlowFileCurrException < noFlowFileMaxException) { - noFlowFileCurrException += 1; - logger.info("DebugFlow throwing NPE with no flow file"); - String message = "forced by " + this.getClass().getName(); - RuntimeException rte; - try { - rte = noFlowFileExceptionClass.getConstructor(String.class).newInstance(message); - throw rte; - } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { - if (logger.isErrorEnabled()) { - logger.error("{} unexpected exception throwing DebugFlow exception: {}", - new Object[]{this, e}); + + if (curr_noff_resp.state() == NoFlowFileResponseState.NO_FF_EXCEPTION_RESPONSE) { + if (noFlowFileCurrException < noFlowFileMaxException) { + noFlowFileCurrException += 1; + logger.info("DebugFlow throwing NPE with no flow file"); + String message = "forced by " + this.getClass().getName(); + RuntimeException rte; + try { + rte = noFlowFileExceptionClass.getConstructor(String.class).newInstance(message); + throw rte; + } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { + if (logger.isErrorEnabled()) { + logger.error("{} unexpected exception throwing DebugFlow exception: {}", + new Object[] {this, e}); + } } + } else { + noFlowFileCurrException = 0; + curr_noff_resp.getNextCycle(); } - } else { - noFlowFileCurrException = 0; - curr_noff_resp.getNextCycle(); } - } - if (curr_noff_resp.state() == NoFlowFileResponseState.NO_FF_YIELD_RESPONSE) { - if (noFlowFileCurrYield < noFlowFileMaxYield) { - noFlowFileCurrYield += 1; - logger.info("DebugFlow yielding with no flow file"); - context.yield(); - break; - } else { - noFlowFileCurrYield = 0; - curr_noff_resp.getNextCycle(); + + if (curr_noff_resp.state() == NoFlowFileResponseState.NO_FF_YIELD_RESPONSE) { + if (noFlowFileCurrYield < noFlowFileMaxYield) { + noFlowFileCurrYield += 1; + logger.info("DebugFlow yielding with no flow file"); + context.yield(); + break; + } else { + noFlowFileCurrYield = 0; + curr_noff_resp.getNextCycle(); + } } - } - return; - } else { - final int writeIterations = context.getProperty(WRITE_ITERATIONS).asInteger(); - if (writeIterations > 0 && pass == 1) { - final Random random = new Random(); - - for (int i = 0; i < writeIterations; i++) { - final byte[] data = new byte[context.getProperty(CONTENT_SIZE).asDataSize(DataUnit.B).intValue()]; - random.nextBytes(data); - - ff = session.write(ff, new OutputStreamCallback() { - @Override - public void process(final OutputStream out) throws IOException { - out.write(data); - } - }); + + return; + } else { + final int writeIterations = context.getProperty(WRITE_ITERATIONS).asInteger(); + if (writeIterations > 0 && pass == 1) { + final Random random = new Random(); + + for (int i = 0; i < writeIterations; i++) { + final byte[] data = new byte[context.getProperty(CONTENT_SIZE).asDataSize(DataUnit.B).intValue()]; + random.nextBytes(data); + + ff = session.write(ff, new OutputStreamCallback() { + @Override + public void process(final OutputStream out) throws IOException { + out.write(data); + } + }); + } } - } - if (curr_ff_resp.state() == FlowFileResponseState.FF_SUCCESS_RESPONSE) { - if (flowFileCurrSuccess < flowFileMaxSuccess) { - flowFileCurrSuccess += 1; - logger.info("DebugFlow transferring to success file={} UUID={}", - new Object[]{ff.getAttribute(CoreAttributes.FILENAME.key()), - ff.getAttribute(CoreAttributes.UUID.key())}); - session.transfer(ff, REL_SUCCESS); - session.commit(); - break; - } else { - flowFileCurrSuccess = 0; - curr_ff_resp.getNextCycle(); + if (curr_ff_resp.state() == FlowFileResponseState.FF_SUCCESS_RESPONSE) { + if (flowFileCurrSuccess < flowFileMaxSuccess) { + flowFileCurrSuccess += 1; + logger.info("DebugFlow transferring to success file={} UUID={}", + new Object[] {ff.getAttribute(CoreAttributes.FILENAME.key()), + ff.getAttribute(CoreAttributes.UUID.key())}); + session.transfer(ff, REL_SUCCESS); + break; + } else { + flowFileCurrSuccess = 0; + curr_ff_resp.getNextCycle(); + } } - } - if (curr_ff_resp.state() == FlowFileResponseState.FF_FAILURE_RESPONSE) { - if (flowFileCurrFailure < flowFileMaxFailure) { - flowFileCurrFailure += 1; - logger.info("DebugFlow transferring to failure file={} UUID={}", - new Object[]{ff.getAttribute(CoreAttributes.FILENAME.key()), - ff.getAttribute(CoreAttributes.UUID.key())}); - session.transfer(ff, REL_FAILURE); - session.commit(); - break; - } else { - flowFileCurrFailure = 0; - curr_ff_resp.getNextCycle(); + + if (curr_ff_resp.state() == FlowFileResponseState.FF_FAILURE_RESPONSE) { + if (flowFileCurrFailure < flowFileMaxFailure) { + flowFileCurrFailure += 1; + logger.info("DebugFlow transferring to failure file={} UUID={}", + new Object[] {ff.getAttribute(CoreAttributes.FILENAME.key()), + ff.getAttribute(CoreAttributes.UUID.key())}); + session.transfer(ff, REL_FAILURE); + break; + } else { + flowFileCurrFailure = 0; + curr_ff_resp.getNextCycle(); + } } - } - if (curr_ff_resp.state() == FlowFileResponseState.FF_ROLLBACK_RESPONSE) { - if (flowFileCurrRollback < flowFileMaxRollback) { - flowFileCurrRollback += 1; - logger.info("DebugFlow rolling back (no penalty) file={} UUID={}", - new Object[]{ff.getAttribute(CoreAttributes.FILENAME.key()), - ff.getAttribute(CoreAttributes.UUID.key())}); - session.rollback(); - session.commit(); - break; - } else { - flowFileCurrRollback = 0; - curr_ff_resp.getNextCycle(); + + if (curr_ff_resp.state() == FlowFileResponseState.FF_ROLLBACK_RESPONSE) { + if (flowFileCurrRollback < flowFileMaxRollback) { + flowFileCurrRollback += 1; + logger.info("DebugFlow rolling back (no penalty) file={} UUID={}", + new Object[] {ff.getAttribute(CoreAttributes.FILENAME.key()), + ff.getAttribute(CoreAttributes.UUID.key())}); + session.rollback(); + break; + } else { + flowFileCurrRollback = 0; + curr_ff_resp.getNextCycle(); + } } - } - if (curr_ff_resp.state() == FlowFileResponseState.FF_YIELD_RESPONSE) { - if (flowFileCurrYield < flowFileMaxYield) { - flowFileCurrYield += 1; - logger.info("DebugFlow yielding file={} UUID={}", - new Object[]{ff.getAttribute(CoreAttributes.FILENAME.key()), - ff.getAttribute(CoreAttributes.UUID.key())}); - session.rollback(); - context.yield(); - return; - } else { - flowFileCurrYield = 0; - curr_ff_resp.getNextCycle(); + + if (curr_ff_resp.state() == FlowFileResponseState.FF_YIELD_RESPONSE) { + if (flowFileCurrYield < flowFileMaxYield) { + flowFileCurrYield += 1; + logger.info("DebugFlow yielding file={} UUID={}", + new Object[] {ff.getAttribute(CoreAttributes.FILENAME.key()), + ff.getAttribute(CoreAttributes.UUID.key())}); + session.rollback(); + context.yield(); + return; + } else { + flowFileCurrYield = 0; + curr_ff_resp.getNextCycle(); + } } - } - if (curr_ff_resp.state() == FlowFileResponseState.FF_PENALTY_RESPONSE) { - if (flowFileCurrPenalty < flowFileMaxPenalty) { - flowFileCurrPenalty += 1; - logger.info("DebugFlow rolling back (with penalty) file={} UUID={}", - new Object[]{ff.getAttribute(CoreAttributes.FILENAME.key()), - ff.getAttribute(CoreAttributes.UUID.key())}); - session.rollback(true); - session.commit(); - break; - } else { - flowFileCurrPenalty = 0; - curr_ff_resp.getNextCycle(); + + if (curr_ff_resp.state() == FlowFileResponseState.FF_PENALTY_RESPONSE) { + if (flowFileCurrPenalty < flowFileMaxPenalty) { + flowFileCurrPenalty += 1; + logger.info("DebugFlow rolling back (with penalty) file={} UUID={}", + new Object[] {ff.getAttribute(CoreAttributes.FILENAME.key()), + ff.getAttribute(CoreAttributes.UUID.key())}); + session.rollback(true); + break; + } else { + flowFileCurrPenalty = 0; + curr_ff_resp.getNextCycle(); + } } - } - if (curr_ff_resp.state() == FlowFileResponseState.FF_EXCEPTION_RESPONSE) { - if (flowFileCurrException < flowFileMaxException) { - flowFileCurrException += 1; - String message = "forced by " + this.getClass().getName(); - logger.info("DebugFlow throwing NPE file={} UUID={}", - new Object[]{ff.getAttribute(CoreAttributes.FILENAME.key()), - ff.getAttribute(CoreAttributes.UUID.key())}); - RuntimeException rte; - try { - rte = flowFileExceptionClass.getConstructor(String.class).newInstance(message); - throw rte; - } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { - if (logger.isErrorEnabled()) { - logger.error("{} unexpected exception throwing DebugFlow exception: {}", - new Object[]{this, e}); + + if (curr_ff_resp.state() == FlowFileResponseState.FF_EXCEPTION_RESPONSE) { + if (flowFileCurrException < flowFileMaxException) { + flowFileCurrException += 1; + String message = "forced by " + this.getClass().getName(); + logger.info("DebugFlow throwing NPE file={} UUID={}", + new Object[] {ff.getAttribute(CoreAttributes.FILENAME.key()), + ff.getAttribute(CoreAttributes.UUID.key())}); + RuntimeException rte; + try { + rte = flowFileExceptionClass.getConstructor(String.class).newInstance(message); + throw rte; + } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { + if (logger.isErrorEnabled()) { + logger.error("{} unexpected exception throwing DebugFlow exception: {}", + new Object[] {this, e}); + } } + } else { + flowFileCurrException = 0; + curr_ff_resp.getNextCycle(); } - } else { - flowFileCurrException = 0; - curr_ff_resp.getNextCycle(); } } } + } finally { + final long sleepMillis = context.getProperty(ON_TRIGGER_SLEEP_TIME).asTimePeriod(TimeUnit.MILLISECONDS); + + try { + if (sleepMillis > 0) { + sleep(sleepMillis, context.getProperty(IGNORE_INTERRUPTS).asBoolean()); + getLogger().info("DebugFlow finishes sleeping at completion of its onTrigger() method"); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } } }