From c5b2bd494edcd087e70a17c8585e1d11b3ec61a5 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Tue, 2 Sep 2025 18:22:14 -0400 Subject: [PATCH 1/4] NIFI-15258: Initial implementation of Connectors MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit NIFI-15258: Fixed vulnerable lz4 dependency NIFI-15258: Fixed checkstyle violations/pmd NIFI-15258: Addressed PMD findings NIFI-15259: Added REST API for Connectors Introduced comprehensive REST endpoints for managing Connectors through the NiFi web interface and API clients. Key capabilities: - Full lifecycle management (create, read, update, delete) - Start/stop control with run status endpoints - Multi-step configuration with property groups supporting string literals, asset references, and secret references - Asynchronous configuration verification with polling support - Access to connector's managed flow and process group status - Search functionality within connector's encapsulated flow Cluster support: - Request replication across cluster nodes - Response merging for dynamic property allowable values that may differ between nodes - Consistent configuration step and property descriptor handling Infrastructure: - Audit logging for all connector configuration changes - Authorization integration with NiFi's policy framework - UI extension support for connector-specific interfaces - Frontend updates for connector access policy management NIFI-15323: Adding operate permissions to the ConnectorEntity. (#10625) This closes #10625 NIFI-15322: Require all property descriptors within a PropertyGroup / ConfigurationStep are unique (#10626) * NIFI-15322: Huge refactoring of Connectors to require that tuple is unique. Requiring that is simple, but now that we enforce that, we can greatly simplify much of the code. * NIFI-15322: Added StepConfiguration to simplify API rather than using Map in some places * NIFI-15322: Now that we enforce unique property names per step, introduced StepConfigurationContext to further reduce API complexity Signed-off-by: Kevin Doran NIFI-15326: Adding support of configuration step documentation. (#10631) NIFI-15322: Adapting to new connector configuration step model. NIFI-15330: Allow GhostConnector to be updated (#10632) NIFI-15312: Implementation of Parameter Provider based Secrets Manager; updated mock framework to support secrets; validation cleanup (#10638) NIFI-15336: Created AuthorizableSecret and implemented ParameterProviderSecretProvider to provide the Parameter Provider as the associated Authorizable (#10644) NIFI-15258: Enable Custom UI to retrieve connector with flow contexts (#10652) NIFI-15352: Added ConnectorClient to toolkit-cli so that it can be us… (#10654) * NIFI-15352: Added ConnectorClient to toolkit-cli so that it can be used in system tests; created system tests for Connectors; bug fixes found running tests; updated Secret/SecretReference to support fully qualified names * NIFI-15352: Addressed review feedback NIFI-15343: Adding an endpoint to return available secrets to the con… (#10645) * NIFI-15343: Adding an endpoint to return available secrets to the connector configuration wizard. * NIFI-15343: Adding providerId to SecretDTO. * NIFI-15343: Skipping ParameterProviders that are invalid or validating when fetching secrets. * NIFI-15343: Adding fully qualified name to SecretDTO. - Fixing rebase issue. This closes #10645 NIFI-15315 Add support for assets in connectors (#10647) - Support multiple asset references - Clean up of assets in StandardConnectorDAO after apply and discard - Add endpoint for listing assets for a given connector - Add endpoint for discarding the working config of a connector - Add endpoint for retrieving asset content - Add system test for connectors that uploads an asset Signed-off-by: Kevin Doran * Fix issues after rebasing, enable ClusteredConnectorAssetsIT NIFI-15361: Allowing configuration step documentation to be returned … (#10667) * NIFI-15361: Allowing configuration step documentation to be returned independent of any connector instances. NIFI-15369: Allow ConfigurationStep to depend on another (ConfigurationStep,Property) tuple (#10669) NIFI-15367: Ensure that Connectors' implicit parameter contexts are n… (#10665) * NIFI-15367: Ensure that Connectors' implicit parameter contexts are not registered with ParameterContextManager; added system test to verify; fixed bug in ConnectorAssetsIT where it did not wait for connector to return to its 'STOPPED' state before attempting to delete * NIFI-15367: Bug fixes; review feedback NIFI-15370: Add Connector ID to process groups and do not register create process groups if they have a connector ID. (#10673) NIFI-15356: Adding authorization to the StandardNiFiConnectorWebContext. (#10660) NIFI-15353: Adding support for rendering Connector Documentation. (#10683) * NIFI-15353: Adding support for rendering Connector Documentation. * NIFI-15353: Addressing review feedback. * NIFI-15353: Using mergeMap instead of switchMap to allow for concurrent doc loading. This closes #10683 NIFI-15429: Adding an optional query parameter for specifying which p… (#10732) * NIFI-15429: Adding an optional query parameter for specifying which process group should be returned within the Connectors flow. * NIFI-15429: Including managed process group id in the ConnectorDTO. * NIFI-15429: Addressing review feedback. * NIFI-15429: Updating path for fetching a Connector flow to require the desired Process Group ID. * NIFI-15429: Fixing failing unit test. NIFI-15430: Ensure that if we fail to initialize a Connector, we crea… (#10733) * NIFI-15430: Ensure that if we fail to initialize a Connector, we create a GhostConnector instead and ensure that we also proivde the extensionMissing flag on ConnectorNode * NIFI-15430: Added extensionMissing flag to Connector DTO * NIFI-15430: If unable to load initial flow of a Connector, make ghosted connector instead NIFI-15427: Added abiliy to drop flowfiles / drain flowfiles from a C… (#10730) * NIFI-15427: Added abiliy to drop flowfiles / drain flowfiles from a Connector; added some system tests to verify existing behavior; fixed existing issue with IT * NIFI-15427: Fixed Exception message in case unable to purge FlowFiles due to Connector state * NIFI-15427: Ensured thread safety of state transitions when draining/purging FlowFiles * NIFI-15427: Addressed review feedback NIFI-15433: If connector validation throws an Exception keep trying u… (#10736) * NIFI-15433: If connector validation throws an Exception keep trying until it completes (with a timeout between retries); some bug fixes * NIFI-15433: Addressed review feedback * NIFI-15433: Fixed failing unit tests NIFI-15434: Ensure that we start or stop connectors on flow sync (#10737) NIFI-15445: Removed 'disabled' state from Connectors; also fixed bug … (#10749) * NIFI-15445: Removed 'disabled' state from Connectors; also fixed bug to ensure that if we fail to apply an update to a Connector that we stop it. * NIFI-15445: Addressed review feedback * NIFI-15445: Fixed checkstyle NIFI-15440: Implementation of ConnectorActions (#10748) * NIFI-15440: Implementation of ConnectorActions * NIFI-15440: Addressed review feedback * NIFI-15440: Addressed issues that occurred after rebase * NIFI-15440: Addressed review feedback NIFI-15439: Ensure that Process Groups are accessible from the approp… (#10742) * NIFI-15439: Ensure that Process Groups are accessible from the appropriate endpoint but not accessible elsewhere * NIFI-15439: FlowAnalyzingRegistryClient should use an explicit Connector ID of null when retrieving Process Group from FlowManager NIFI-15468: Ensure that we properly initialize ConnectorNode even if a Ghost Connector is created (#10775) This closes #10775 NIFI-15446: When invoking ConnectorMethod, make sure to serialize/des… (#10750) * NIFI-15446: When invoking ConnectorMethod, make sure to serialize/deserialize the arguments and return value * NIFI-15446: Addressed review feedback NIFI-15376: Adding dependencies to the configuration step dto. (#10674) NIFI-15453: Adding an endpoint to return all controller services for … (#10770) * NIFI-15453: Adding an endpoint to return all controller services for a process group in a connector. * NIFI-15453: Addressing review feedback. This closes #10770 NIFI-15472: Added addAsset methods to ConnectorTestRunner (#10776) NIFI-15482: MockConnectorAssetManager requires SPI file to start (#10783) NIFI-15451: Added ability for Connectors to retrieve bundles availabl… (#10756) * NIFI-15451: Added ability for Connectors to retrieve bundles available for component types and updated VersionedFlowUtils to make use of it to easily update Versioned components * NIFI-15451: Added additional unit tests to VersionedFlowUtils; updates to how we compare component versions * NIFI-15451: Fixed PMD violation NIFI-15481: Updated ConnectorActions to reflect that starting is not … (#10782) * NIFI-15481: Updated ConnectorActions to reflect that starting is not allowed when invalid. Also updated ValidationState to ensure that we don't return any ValidationResult with a valid flag of true as a 'validation error'. * NIFI-15481: Addressed review feedback NIFI-15258: Fixes necessary after rebasing against main NIFI-15258: Addressed checkstyle violations NIFI-15461: Added ability to initiate drainage of Connector's FlowFiles and calce; added tests to verify; some bug fixes (#10767) NIFI-15489: Addressed the 4 bugs called out in Jira: Wrong delimiter in StandardConnectorPropertyValue.asList, ConnectorValidationContextBridge.getProperty() does not resolve property value properly; StandaloneProcessGroupLifecycle.stop() does not await controller service disabling; switched hashmap in StandaloneProcessGroupLifecycle to ConcurrentHashMap (#10798) - Addressed another similar bug in which an invalid connector that is expected to be running would prevent NiFi from starting NIFI-15490: If any processors or controller services are missing, make Connector invalid (#10799) NIFI-15485: Include provider name when providing fqn for secrets (#10801) NIFI-15497: Fixed unit tests (#10805) NIFI-15258: Fixes for system tests that needed to be updated from 2.7.0-SNAPSHOT to 2.8.0-SNAPSHOT (#10807) NIFI-15495 Restart Connectors that reference assets that were synchro… (#10806) * NIFI-15495 Restart Connectors that reference assets that were synchronized - Ensure Connectors re-resolve property values before starting - Ensure asset clean up happens only after applyUpdate fully finishes - Add connector asset properties to default nifi.properties * Fix system test * Encapsulate restart logic in new method on ConnectorNode * Fix JavaDoc NIFI-15465 Allow client-specified connector component ids (#10768) Add verifyCreate to ConnectorDAO and test cases NIFI-15467: Ensure that Connectors' versions are automatically resolv… (#10811) * NIFI-15467: Ensure that Connectors' versions are automatically resolved on startup - Allow connectors to specify Bundle Compatability when updating flow - Default to BundleCompatability.RESOLVE_BUNDLE when connectors are updating flows - Bug fixes; updated MockConnectorInitializationContext to extend from StandardConnectorInitializationContext to cut down on code duplication * NIFI-15467: Addressed review feedback NIFI-15529: Fix to StatelessGroupLifecycle to ensure that we set the desired state of Stateless Group Node before attempting to schedule; updated StandardProcessScheduler.startStatelessGroup to make it more clear if this is left out again at some point (#10832) NIFI-15525: Ensure that we call onStepConfigured when we restore flow; ensure that we synchronized Parmaeter Providers before Connectors (#10835) NIFI-15509: Enable Component reload additional classpath resources if needed (#10812) NIFI-15536: Change Connector API to use DescribedValue vs AllowableValue (#10841) NIFI-15511: Added endpoints for purging all FlowFiles for a given Con… (#10833) * NIFI-15511: Added endpoints for purging all FlowFiles for a given Connector; added method to ConnectorClient to call these endpoints; updated system tests to use these endpoints when tearing down flows; some bug fixes * NIFI-15511: Added WRITE permission requirement for retrieving and deleting a Connector Purge request NIFI-15514: Ensure that Parameter Contexts are assigned to all Proces… (#10815) * NIFI-15514: Ensure that Parameter Contexts are assigned to all Process Groups in a Connector and not just the top-level group. Ensure that all components are started when Connector starts instead of just Processors and Controller Services - When Working Context is recreated, ensure that we appropriately apply Parameter Context to newly created Process Group(s) - Ensure that when we cleanup unused assets for Connectors that we consider any assets that are referenced in either the Working or Active context instead of just the Active context - Ensure that when we stop Process Group we call all tasks in background threads instead of calling .thenRun which could potentially run in the foreground thread * NIFI-15514: Update parameter context assignment to occur during sync. * NIFI-15514: Fixes around ensuring that processors/controller services are properly configured and notified of any configuration changes when parameters change - Removed the updateParameterContexts from ProcessGroup.updateFlow, which was added in a previous commit as we went a different direction for the fix --------- Co-authored-by: Bob Paulin NIFI-15538: When starting/stopping components allow specifying whethe… (#10843) * NIFI-15538: When starting/stopping components allow specifying whether or not the action should be recursive; code cleanup and simplification to use a Virtual Thread to execute code sequentially instead of chaining CompletableFutures. * NIFI-15538: Addressed review feedback --------- Co-authored-by: Mark Payne NIFI-15480 Added ability to drop FlowFiles selectively using a Predicate (#10848) * NIFI-15480: Added ability to drop FlowFiles selectively using a Predicate Added system tests to verify selective dropping of flowfiles from a Connector. Also made significant updates so that diagnostic bundles are captured when a system test fails and diagnostic bundle also contains details of each Connection's FlowFileQueue. This was necessary in aiding the debugging of the new feature but is also something that has been missing for some time. * NIFI-15480: Ensure that when we perform a selective drop on a FlowFileQueue that we pass appropriate SWAP_FILE_DELETED / SWAP_FILE_RENAMED events to the FlowFile Repository NIFI-15549: Adding endpoints for viewing and clearing state for compo… (#10855) * NIFI-15549: Adding endpoints for viewing and clearing state for components that are managed by a Connector. * NIFI-15549: Addressing review feedback. NIFI-15557: Allow Connectors to enable a Controller Service using overridden property values (#10862) NIFI-15488: Added significant number of debug log messages as well as some info/error for connector-related events (#10803) NIFI-15544: If a Connector requires processors or controller services that are unavailable, make Connector invalid but not ghosted (#10851) NIFI-15558: Sensitive Parameter References should resolve to value. (#10863) NIFI-15539: Updating the actions in FlowFileQueueResource to support … (#10846) * NIFI-15539: Updating the actions in FlowFileQueueResource to support interacting with queues in flows managed by a Connector. * NIFI-15539: Updating resource endpoint to include data policies for connectors. NIFI-15565: Code cleanup; improvements to .cursorrules to provide bet… (#10869) * NIFI-15565: Code cleanup; improvements to .cursorrules to provide better instructions regarding automated tests * NIFI-15565: Broke apart cursorrules file into the new format of .cursor/rules/* NIFI-15355 Connector Configuration Repository (#10876) * NIFI-15255 Introduce ConnectorConfigurationRepository Extension Interface * Add new property key to default nifi.properties file * Address peer review comments NIFI-15559: Enforce data permissions when purging data from a Connector. (#10867) This closes #10867 NIFI-15258: Fixes as a result of rebasing against main NIFI-15577: Including connector id in provenance event dto. (#10883) * NIFI-15577: Including connector id in provenance event dto. - Fixing go to routing logic in the provenance event table. * NIFI-15577: Fixing back navigation context when navigating to provenance from the flow designer. This closes #10883 NIFI-15581: Add support for ControllerService Mocks for Connectors. (#10885) NIFI-15579: When synchronizing PG when Versioned flow for stateless group node, do not synchronize parameter contexts if group is within a Connector (#10884) NIFI-15605 - Rebase Connectors branch with main for 2.9.0-SNAPSHOT NIFI-15606: Connector Mock War implementation. (#10907) * NIFI-15606: Connector Mock War implementation. * Load war files from Connector Nar * Mock Connector Web Context * Allow test runner to set http port * NIFI-15606: Connector Mock War implementation. * Code Review Feedback NIFI-15611: Fetchable Values should pull from current FlowContext (#10906) NIFI-15618: Bug fixes around handling of FlowFileActivity with child groups and stateless groups (#10912) NIFI-15598: When enabling referenced controller services in a connector, skip references from properties with unsatisfied dependencies (#10897) NIFI-15615: Ensure that we include necessary values in the ConnectorStatusDTO (#10913) NIFI-15610 Add asset management to ConnectorConfigurationProvider (#10909) NIFI-15621: Allow Configuration to Validate with ValidationContext (#10916) * Currently this is only implemented for processors in schedule periods * Allows connector to pass Validation Context to be used instead of current parameter context. NIFI-15640: NPE in the applyFlow should NOT crash the runtime. (#10931) * NIFI-15640: NPE in the applyFlow should NOT crash the runtime. * Leave the flow in a state of UPDATE_FAILED * With validationState of INVALID. * NIFI-15640: NPE in the applyFlow should NOT crash the runtime. * Code Review Feedback NIFI-15629 Add ConnectorRequestContext (#10924) Signed-off-by: David Handermann NIFI-15663: Update to use new ComponentHierarchyScope instead of on facade/lifecycle classes (#10958) NIFI-15680: Not including global parameter contexts and parameters in connector search results. (#10980) This closes #10980 NIFI-15604: Expose VersionedExternalFlow that represents the Active/Working flow from ConnectorTestRunner (#10901) Signed-off-by: Kevin Doran NIFI-15648 - Resolve SECRET property values before fetchAllowableValues and verify in working flow context (#10937) Co-authored-by: Bob Paulin Co-authored-by: Bryan Bende Co-authored-by: Kevin Doran Co-authored-by: Matt Gilman Co-authored-by: Pierre Villard Co-authored-by: Mark Payne --- .cursor/rules/building.mdc | 18 + .cursor/rules/code-style.mdc | 74 + .cursor/rules/ending-conditions.mdc | 30 + .cursor/rules/extension-development.mdc | 107 + .cursor/rules/extension-testing.mdc | 80 + .cursor/rules/framework-testing.mdc | 46 + .cursor/rules/persona.mdc | 16 + .cursor/rules/testing-standards.mdc | 81 + .gitignore | 2 +- .../component/api/ComponentManifest.java | 10 + .../component/api/ConfigurationStep.java | 81 + .../api/ConfigurationStepDependency.java | 61 + .../component/api/ConnectorDefinition.java | 41 + .../api/ConnectorPropertyDependency.java | 51 + .../api/ConnectorPropertyDescriptor.java | 111 + .../component/api/ConnectorPropertyGroup.java | 61 + .../component/api/ConnectorPropertyType.java | 43 + checkstyle.xml | 7 - .../AgentPropertyValidationContext.java | 5 + .../command/DefaultFlowStateStrategyTest.java | 9 +- nifi-assembly/pom.xml | 6 + nifi-commons/nifi-connector-utils/pom.xml | 42 + .../connector/util/VersionedFlowUtils.java | 592 ++++ .../util/TestVersionedFlowUtils.java | 285 ++ .../org/apache/nifi/util/NiFiProperties.java | 64 +- .../org/apache/nifi/wali/HashMapSnapshot.java | 35 +- .../nifi/wali/LengthDelimitedJournal.java | 43 +- .../src/main/java/org/wali/SerDe.java | 13 + .../src/main/java/org/wali/SerDeFactory.java | 13 + .../src/main/java/org/wali/UpdateType.java | 10 +- nifi-commons/pom.xml | 1 + .../nifi-connector-mock-api/pom.xml | 35 + .../ConnectorConfigVerificationResult.java | 31 + .../connector/server/ConnectorMockServer.java | 36 + .../connector/server/ConnectorTestRunner.java | 103 + .../nifi-connector-mock-server-nar/pom.xml | 55 + .../nifi-connector-mock-server/pom.xml | 80 + .../connector/server/MockAuditService.java | 63 + .../server/MockConnectorAssetManager.java | 179 ++ .../MockConnectorInitializationContext.java | 78 + .../server/MockConnectorRepository.java | 35 + .../MockExtensionDiscoveringManager.java | 74 + .../connector/server/MockExtensionMapper.java | 61 + .../server/MockNiFiConnectorWebContext.java | 52 + .../server/MockRuleViolationsManager.java | 79 + .../MockServerConfigVerificationResult.java | 52 + .../connector/server/MockStateManager.java | 89 + .../server/MockStateManagerProvider.java | 59 + .../mock/connector/server/MockStateMap.java | 56 + .../server/NopPropertyEncryptor.java | 32 + .../connector/server/PermitAllAuthorizer.java | 46 + .../server/StandardConnectorMockServer.java | 629 +++++ .../ConnectorTestRunnerAuthorizable.java | 34 + .../ConnectorTestRunnerSecretProvider.java | 94 + .../ConnectorTestRunnerSecretsManager.java | 99 + .../services/org.apache.nifi.NiFiServer | 16 + .../org.apache.nifi.asset.AssetManager | 16 + ...i.components.connector.ConnectorRepository | 16 + ...omponents.connector.secrets.SecretsManager | 16 + .../server/MockConnectorAssetManagerTest.java | 245 ++ .../MockNiFiConnectorWebContextTest.java | 80 + .../server/MockStateManagerProviderTest.java | 267 ++ .../StandardConnectorMockServerJettyTest.java | 186 ++ .../pom.xml | 130 + .../connectors/tests/AllowableValuesIT.java | 87 + .../connectors/tests/CreateConnectorIT.java | 92 + .../mock/connectors/tests/CronScheduleIT.java | 68 + .../tests/MockControllerServiceIT.java | 87 + .../connectors/tests/MockProcessorIT.java | 91 + .../pom.xml | 35 + .../pom.xml | 48 + .../connectors/AllowableValuesConnector.java | 106 + .../connectors/CronScheduleConnector.java | 106 + .../nifi/mock/connectors/GenerateAndLog.java | 55 + .../connectors/MissingBundleConnector.java | 108 + ...apache.nifi.components.connector.Connector | 19 + .../flows/Cron_Schedule_Connector.json | 410 +++ .../resources/flows/Generate_and_Update.json | 393 +++ .../nifi-connector-mock-test-bundle/pom.xml | 33 + .../nifi-connector-mock/pom.xml | 147 + .../StandardConnectorTestRunner.java | 282 ++ .../src/main/resources/nifi.properties | 165 ++ nifi-connector-mock-bundle/pom.xml | 34 + .../nifi-kafka-to-s3-connector/pom.xml | 40 + .../kafkas3/KafkaConnectionStep.java | 135 + .../nifi/connectors/kafkas3/KafkaToS3.java | 297 ++ .../kafkas3/KafkaToS3FlowBuilder.java | 213 ++ .../connectors/kafkas3/KafkaTopicsStep.java | 85 + .../nifi/connectors/kafkas3/S3Step.java | 156 ++ ...apache.nifi.components.connector.Connector | 17 + .../steps/Kafka_Connection.md | 27 + .../steps/Kafka_Topics.md | 34 + .../steps/S3_Configuration.md | 65 + .../src/main/resources/flows/Kafka_to_S3.json | 2105 ++++++++++++++ .../pom.xml | 238 ++ .../nifi/connectors/kafkas3/KafkaToS3IT.java | 758 ++++++ .../nifi-kafka-to-s3-nar/pom.xml | 36 + .../nifi-kafka-to-s3-bundle/pom.xml | 39 + nifi-connectors/pom.xml | 42 + .../main/asciidoc/administration-guide.adoc | 5 + .../aws/s3/AbstractS3Processor.java | 11 + .../processors/aws/region/RegionUtil.java | 2 +- .../nifi/kafka/processors/ConsumeKafka.java | 118 +- .../kafka/processors/ConsumeKafkaTest.java | 22 +- .../service/api/consumer/PollingContext.java | 4 +- .../service/Kafka3ConnectionService.java | 43 +- .../consumer/Kafka3AssignmentService.java | 100 + .../service/consumer/RecordIterable.java | 79 + .../script/impl/ValidationContextAdapter.java | 5 + .../standard/DuplicateFlowFile.java | 2 +- .../org/apache/nifi/asset/AssetManager.java | 30 +- .../ConnectorConfigurationProvider.java | 161 ++ ...nnectorConfigurationProviderException.java | 37 + ...gurationProviderInitializationContext.java | 50 + .../connector/ConnectorRequestContext.java | 88 + .../ConnectorRequestContextHolder.java | 64 + .../ConnectorWorkingConfiguration.java | 55 + .../StandardConnectorRequestContext.java | 84 + .../connector/secrets/AuthorizableSecret.java | 25 + .../controller/queue/DropFlowFileStatus.java | 7 + .../nifi/controller/queue/FlowFileQueue.java | 15 + .../queue/LocalQueuePartitionDiagnostics.java | 10 + .../repository/FlowFileSwapManager.java | 10 + .../repository/RepositoryRecord.java | 8 + .../repository/RepositoryRecordType.java | 2 +- .../org/apache/nifi/web/UiExtensionType.java | 1 + .../ConnectorRequestContextHolderTest.java | 69 + .../TestVolatileProvenanceRepository.java | 6 +- .../api/dto/ConfigVerificationResultDTO.java | 10 + .../ConfigurationStepConfigurationDTO.java | 96 + .../dto/ConfigurationStepDependencyDTO.java | 70 + .../nifi/web/api/dto/ConnectorActionDTO.java | 70 + .../api/dto/ConnectorConfigurationDTO.java | 43 + .../apache/nifi/web/api/dto/ConnectorDTO.java | 172 ++ .../dto/ConnectorPropertyDependencyDTO.java | 57 + .../dto/ConnectorPropertyDescriptorDTO.java | 137 + .../api/dto/ConnectorValueReferenceDTO.java | 120 + .../dto/PropertyGroupConfigurationDTO.java | 82 + .../apache/nifi/web/api/dto/SecretDTO.java | 89 + .../VerifyConnectorConfigStepRequestDTO.java | 69 + .../dto/provenance/ProvenanceEventDTO.java | 14 + .../api/dto/status/ConnectorStatusDTO.java | 186 ++ .../status/ConnectorStatusSnapshotDTO.java | 322 +++ .../NodeConnectorStatusSnapshotDTO.java | 79 + .../api/entity/ConfigurationStepEntity.java | 77 + .../entity/ConfigurationStepNamesEntity.java | 44 + .../nifi/web/api/entity/ConnectorEntity.java | 65 + ...onnectorPropertyAllowableValuesEntity.java | 71 + .../api/entity/ConnectorRunStatusEntity.java | 38 + .../web/api/entity/ConnectorTypesEntity.java | 44 + .../nifi/web/api/entity/ConnectorsEntity.java | 55 + .../web/api/entity/CurrentUserEntity.java | 13 + .../nifi/web/api/entity/NarDetailsEntity.java | 10 + .../nifi/web/api/entity/SecretsEntity.java | 43 + .../api/entity/StepDocumentationEntity.java | 35 + ...erifyConnectorConfigStepRequestEntity.java | 37 + ...constitutedSerializedRepositoryRecord.java | 13 + .../repository/RepositoryRecordSerde.java | 13 +- .../SchemaRepositoryRecordSerde.java | 34 +- .../SerializedRepositoryRecord.java | 8 + .../StandardRepositoryRecordSerdeFactory.java | 13 +- .../schema/RepositoryRecordFieldMap.java | 17 +- .../schema/RepositoryRecordSchema.java | 65 +- .../schema/RepositoryRecordUpdate.java | 15 +- .../resource/ResourceFactory.java | 27 + .../authorization/resource/ResourceType.java | 3 +- .../http/StandardHttpResponseMapper.java | 14 + .../http/endpoints/AssetsEndpointMerger.java | 2 +- .../ComponentStateEndpointMerger.java | 6 +- .../endpoints/ConnectorEndpointMerger.java | 60 + .../ConnectorFlowEndpointMerger.java | 56 + .../ConnectorPropertyGroupEndpointMerger.java | 51 + ...ectorPropertyGroupNamesEndpointMerger.java | 63 + .../ConnectorStatusEndpointMerger.java | 86 + .../endpoints/ConnectorsEndpointMerger.java | 66 + .../endpoints/CurrentUserEndpointMerger.java | 1 + .../http/endpoints/FlowMerger.java | 11 + ...rifyConnectorConfigStepEndpointMerger.java | 72 + .../ClusteredConnectorRequestReplicator.java | 131 + .../ThreadPoolRequestReplicator.java | 2 + .../ConfigurationStepEntityMerger.java | 160 ++ .../ConfigurationStepNamesEntityMerger.java | 62 + .../manager/ConnectorEntityMerger.java | 212 ++ .../ConnectorPropertyDescriptorDtoMerger.java | 64 + .../manager/ConnectorsEntityMerger.java | 40 + .../nifi/cluster/manager/StatusMerger.java | 95 + ...nnectorRequestReplicatorConfiguration.java | 50 + .../FrameworkClusterConfiguration.java | 9 +- .../CurrentUserEndpointMergerTest.java | 4 + .../TestThreadPoolRequestReplicator.java | 6 +- .../ConfigurationStepEntityMergerTest.java | 184 ++ .../manager/ConnectorEntityMergerTest.java | 404 +++ ...nectorPropertyDescriptorDtoMergerTest.java | 171 ++ .../org/apache/nifi/asset/StandardAsset.java | 13 +- .../nifi/asset/StandardAssetManager.java | 103 +- .../asset/StandardConnectorAssetManager.java | 225 ++ .../StandardConnectorValidationTrigger.java | 92 + .../ParameterProviderSecretProvider.java | 89 + .../ParameterProviderSecretsManager.java | 180 ++ .../connector/secrets/StandardSecret.java | 186 ++ ...rdSecretsManagerInitializationContext.java | 33 + .../controller/StandardProcessorNode.java | 246 +- .../controller/flow/AbstractFlowManager.java | 55 +- .../flowanalysis/FlowAnalysisUtil.java | 6 +- .../StandardParameterProviderNode.java | 24 +- .../repository/StandardProcessSession.java | 32 +- ...ardControllerServiceInvocationHandler.java | 40 +- .../StandardControllerServiceNode.java | 141 +- .../StandardControllerServiceProvider.java | 5 +- .../StandardControllerServiceResolver.java | 6 +- .../events/VolatileBulletinRepository.java | 1 + ...tandardVersionedComponentSynchronizer.java | 50 +- .../nifi/groups/StandardProcessGroup.java | 96 +- .../nifi/logging/ConnectorLogObserver.java | 49 + .../nifi/logging/ProcessorLogObserver.java | 2 +- .../parameter/StandardParameterContext.java | 28 +- .../StandardParameterReferenceManager.java | 17 +- .../processor/StandardValidationContext.java | 38 +- .../StandardValidationContextFactory.java | 6 +- .../flow/FlowAnalyzingRegistryClientNode.java | 16 +- .../flow/StandardFlowRegistryClientNode.java | 3 +- ...java => VersionedComponentFlowMapper.java} | 95 +- .../remote/StandardRemoteProcessGroup.java | 11 +- .../org.apache.nifi.asset.AssetManager | 3 +- ...omponents.connector.secrets.SecretsManager | 16 + .../StandardConnectorAssetManagerTest.java | 178 ++ .../TestParameterProviderSecretsManager.java | 374 +++ .../StandardProcessSessionTest.java | 7 + ...StandardControllerServiceResolverTest.java | 6 +- .../nifi/groups/StandardProcessGroupTest.java | 3 +- .../FlowAnalyzingRegistryClientNodeTest.java | 6 +- ... => TestVersionedComponentFlowMapper.java} | 10 +- .../connector/ConfigurationUpdateResult.java | 24 + .../components/connector/ConnectorAction.java | 49 + .../connector/ConnectorConfiguration.java | 65 + .../components/connector/ConnectorNode.java | 281 ++ .../connector/ConnectorRepository.java | 181 ++ ...nectorRepositoryInitializationContext.java | 50 + .../connector/ConnectorRequestReplicator.java | 30 + .../components/connector/ConnectorState.java | 31 + .../connector/ConnectorStateTransition.java | 82 + .../connector/ConnectorUpdateContext.java | 26 + .../connector/ConnectorValidationTrigger.java | 44 + .../connector/FlowContextFactory.java | 27 + ...ameworkConnectorInitializationContext.java | 29 + ...ConnectorInitializationContextBuilder.java | 39 + .../connector/FrameworkFlowContext.java | 32 + .../MutableConnectorConfigurationContext.java | 64 + .../connector/NamedStepConfiguration.java | 21 + .../ParameterContextFacadeFactory.java | 25 + .../connector/ProcessGroupFacadeFactory.java | 26 + .../connector/ProcessGroupFactory.java | 24 + .../connector/secrets/SecretProvider.java | 34 + .../connector/secrets/SecretsManager.java | 40 + .../SecretsManagerInitializationContext.java | 26 + .../DisabledServiceValidationResult.java | 14 + .../validation/ValidationState.java | 16 +- .../apache/nifi/connectable/Connectable.java | 2 + .../ConnectableFlowFileActivity.java | 70 + .../nifi/connectable/FlowFileActivity.java | 34 + .../connectable/FlowFileTransferCounts.java | 88 + .../ProcessGroupFlowFileActivity.java | 170 ++ .../controller/AbstractComponentNode.java | 69 +- .../apache/nifi/controller/AbstractPort.java | 10 + .../apache/nifi/controller/ComponentNode.java | 3 + .../controller/ParameterProviderNode.java | 13 + .../nifi/controller/ProcessScheduler.java | 22 +- .../apache/nifi/controller/ProcessorNode.java | 17 +- .../nifi/controller/StandardFunnel.java | 9 + .../controller/ValidationContextFactory.java | 4 +- .../nifi/controller/flow/FlowManager.java | 60 +- .../controller/flow/VersionedDataflow.java | 10 + .../controller/queue/DropFlowFileRequest.java | 15 + .../service/ControllerServiceNode.java | 24 +- .../service/ControllerServiceProvider.java | 4 +- .../apache/nifi/events/BulletinFactory.java | 6 + .../org/apache/nifi/groups/ProcessGroup.java | 31 +- .../nifi/groups/RemoteProcessGroup.java | 4 +- .../nifi/manifest/RuntimeManifestService.java | 16 +- .../ProcessGroupFlowFileActivityTest.java | 162 ++ .../controller/TestAbstractComponentNode.java | 2 +- .../nifi-framework-core/pom.xml | 10 +- .../nifi/asset/AssetsRestApiClient.java | 55 +- .../asset/StandardAssetComponentManager.java | 4 +- .../nifi/asset/StandardAssetSynchronizer.java | 4 +- .../StandardConnectorAssetSynchronizer.java | 280 ++ .../connector/ConnectorDetails.java | 45 + .../connector/ConnectorParameterLookup.java | 193 ++ .../ConnectorValidationContextBridge.java | 146 + .../connector/DescribedValueProvider.java | 28 + .../components/connector/GhostConnector.java | 138 + .../StandaloneConnectorRequestReplicator.java | 40 + .../StandardComponentBundleLookup.java | 172 ++ .../connector/StandardConnectorAction.java | 81 + ...StandardConnectorConfigurationContext.java | 323 +++ ...gurationProviderInitializationContext.java | 44 + ...tandardConnectorInitializationContext.java | 205 ++ .../connector/StandardConnectorNode.java | 1592 +++++++++++ .../StandardConnectorPropertyValue.java | 101 + ...ardConnectorRepoInitializationContext.java | 85 + .../StandardConnectorRepository.java | 594 ++++ .../StandardConnectorStateTransition.java | 113 + .../StandardConnectorValidationContext.java | 107 + .../connector/StandardFlowContext.java | 130 + .../StandardStepConfigurationContext.java | 59 + .../standalone/ComponentContextProvider.java | 41 + ...xecutionEngineStatelessGroupLifecycle.java | 46 + .../StandaloneConnectionFacade.java | 82 + .../StandaloneControllerServiceFacade.java | 243 ++ .../StandaloneControllerServiceLifecycle.java | 77 + .../StandaloneParameterContextFacade.java | 229 ++ .../StandaloneProcessGroupFacade.java | 424 +++ .../StandaloneProcessGroupLifecycle.java | 463 ++++ .../standalone/StandaloneProcessorFacade.java | 253 ++ .../StandaloneProcessorLifecycle.java | 93 + .../StandaloneStatelessGroupLifecycle.java | 54 + .../nifi/controller/ExtensionBuilder.java | 409 ++- .../controller/FileSystemSwapManager.java | 9 + .../nifi/controller/FlowController.java | 226 +- .../nifi/controller/StandardFlowService.java | 30 +- .../FlowControllerFlowContextFactory.java | 140 + ...owControllerProcessGroupFacadeFactory.java | 70 + .../StandardComponentContextProvider.java | 105 + .../controller/flow/StandardFlowManager.java | 147 +- .../StandardStatelessGroupNodeFactory.java | 23 +- .../StandardFlowAnalysisContext.java | 4 +- .../StandardFlowAnalysisRuleNode.java | 3 +- .../queue/AbstractFlowFileQueue.java | 34 +- .../controller/queue/SelectiveDropResult.java | 75 + .../queue/StandardFlowFileQueue.java | 57 + ...tandardLocalQueuePartitionDiagnostics.java | 17 +- .../queue/SwappablePriorityQueue.java | 254 +- .../SocketLoadBalancedFlowFileQueue.java | 66 + .../clustered/partition/QueuePartition.java | 13 + .../partition/RemoteQueuePartition.java | 9 + .../StandardRebalancingPartition.java | 9 + .../SwappablePriorityQueueLocalPartition.java | 8 + .../reporting/StandardReportingTaskNode.java | 3 +- .../LiveSerializedRepositoryRecord.java | 5 + .../WriteAheadFlowFileRepository.java | 33 +- .../scheduling/StandardProcessScheduler.java | 91 +- .../VersionedDataflowMapper.java | 20 +- .../VersionedFlowSynchronizer.java | 170 +- .../VersionedReportingTaskSnapshotMapper.java | 6 +- .../controller/tasks/StatelessFlowTask.java | 69 + .../BootstrapDiagnosticsFactory.java | 2 + .../tasks/ConnectionDiagnosticTask.java | 265 ++ .../flowanalysis/StandardFlowAnalyzer.java | 10 +- .../FlowControllerConfiguration.java | 52 +- .../groups/StandardStatelessGroupNode.java | 10 + .../StandardRuntimeManifestService.java | 45 + .../org/apache/nifi/nar/NarInstallTask.java | 4 +- ...i.components.connector.ConnectorRepository | 16 + .../connector/BlockingConnector.java | 132 + .../DynamicAllowableValuesConnector.java | 142 + .../connector/DynamicFlowConnector.java | 336 +++ .../connector/MissingBundleConnector.java | 111 + .../OnPropertyModifiedConnector.java | 88 + .../connector/ParameterConnector.java | 131 + .../connector/SleepingConnector.java | 126 + .../connector/StandardConnectorNodeIT.java | 675 +++++ .../TestStandardComponentBundleLookup.java | 246 ++ ...StandardConnectorConfigurationContext.java | 256 ++ ...tandardConnectorInitializationContext.java | 387 +++ .../connector/TestStandardConnectorNode.java | 848 ++++++ .../TestStandardConnectorRepository.java | 877 ++++++ .../TestStandaloneProcessGroupLifecycle.java | 292 ++ .../processors/CreateDummyFlowFile.java | 84 + .../processors/DuplicateFlowFile.java | 128 + .../processors/ExposeFileValues.java | 72 + .../processors/LogFlowFileContents.java | 58 + .../processors/OnPropertyModifiedTracker.java | 116 + .../processors/OverwriteFlowFile.java | 92 + .../connector/processors/Sleep.java | 151 + .../processors/TerminateFlowFile.java | 37 + .../connector/services/CounterService.java | 26 + .../services/impl/StandardCounterService.java | 37 + .../controller/MockStateManagerProvider.java | 72 + .../nifi/controller/MockSwapManager.java | 5 + .../controller/TestStandardFlowFileQueue.java | 118 +- .../nifi/controller/flow/NopConnector.java | 167 ++ .../flow/TestStandardFlowManager.java | 174 ++ .../TestSocketLoadBalancedFlowFileQueue.java | 117 +- .../clustered/TestSwappablePriorityQueue.java | 138 +- .../SchemaRepositoryRecordSerdeTest.java | 10 +- .../repository/StandardProcessSessionIT.java | 5 +- .../TestWriteAheadFlowFileRepository.java | 137 +- .../TestStandardProcessScheduler.java | 81 + .../VersionedFlowSynchronizerTest.java | 6 + ...TestStandardControllerServiceProvider.java | 42 +- .../service/mock/MockProcessGroup.java | 26 +- .../tasks/TestStatelessFlowTask.java | 140 +- .../StandardRuntimeManifestServiceTest.java | 48 + ... => VersionedComponentFlowMapperTest.java} | 4 +- ...apache.nifi.components.connector.Connector | 18 + ...g.apache.nifi.controller.ControllerService | 5 +- .../org.apache.nifi.processor.Processor | 11 +- .../Another_Test_Step.md | 4 + .../org.example.TestConnector/Test_Step.md | 8 + .../src/test/resources/colors.txt | 3 + .../test/resources/flows/choose-color.json | 76 + .../generate-and-log-with-parameter.json | 253 ++ .../flows/generate-duplicate-log-flow.json | 690 +++++ .../flows/on-property-modified-tracker.json | 98 + .../StandardExtensionDiscoveringManager.java | 20 +- .../org/apache/nifi/nar/DummyConnector.java | 120 + ...ensionDiscoveringManagerConnectorTest.java | 49 + ...apache.nifi.components.connector.Connector | 18 + .../nifi/headless/HeadlessNiFiServer.java | 17 +- .../java/org/apache/nifi/nar/NarUnpacker.java | 4 +- .../repository/StandardRepositoryRecord.java | 24 +- .../nifi-framework/nifi-resources/pom.xml | 2 + .../src/main/resources/conf/nifi.properties | 5 + .../apache/nifi/web/server/JettyServer.java | 117 +- .../server/StandardServerProviderTest.java | 8 +- .../apache/nifi/ui/extension/UiExtension.java | 16 + .../apache/nifi/audit/ConnectorAuditor.java | 416 +++ .../authorization/AuthorizableLookup.java | 31 + .../StandardAuthorizableLookup.java | 46 +- .../apache/nifi/web/NiFiServiceFacade.java | 187 ++ .../nifi/web/NiFiWebApiResourceConfig.java | 1 + .../nifi/web/StandardNiFiServiceFacade.java | 589 +++- .../nifi/web/api/ConnectorResource.java | 2419 +++++++++++++++++ .../nifi/web/api/FlowFileQueueResource.java | 12 +- .../org/apache/nifi/web/api/FlowResource.java | 210 +- .../web/api/ParameterContextResource.java | 4 +- .../nifi/web/api/ProcessGroupResource.java | 2 +- .../apache/nifi/web/api/dto/DtoFactory.java | 384 ++- .../nifi/web/api/dto/EntityFactory.java | 89 + .../WebApplicationConfiguration.java | 29 + .../StandardNiFiConnectorWebContext.java | 99 + .../AuthorizingConnectionFacade.java | 89 + ...AuthorizingConnectorInvocationHandler.java | 130 + .../AuthorizingControllerServiceFacade.java | 122 + ...AuthorizingControllerServiceLifecycle.java | 63 + .../authorization/AuthorizingFlowContext.java | 70 + .../AuthorizingParameterContextFacade.java | 95 + .../AuthorizingProcessGroupFacade.java | 178 ++ .../AuthorizingProcessGroupLifecycle.java | 129 + .../AuthorizingProcessorFacade.java | 122 + .../AuthorizingProcessorLifecycle.java | 80 + .../AuthorizingStatelessGroupLifecycle.java | 55 + .../ConnectorAuthorizationContext.java | 74 + .../nifi/web/controller/ControllerFacade.java | 115 +- .../apache/nifi/web/dao/ConnectionDAO.java | 9 + .../org/apache/nifi/web/dao/ConnectorDAO.java | 84 + .../apache/nifi/web/dao/ProcessGroupDAO.java | 9 + .../nifi/web/dao/impl/ComponentDAO.java | 31 +- .../web/dao/impl/StandardConnectionDAO.java | 52 +- .../web/dao/impl/StandardConnectorDAO.java | 284 ++ .../impl/StandardControllerServiceDAO.java | 2 +- .../dao/impl/StandardParameterContextDAO.java | 2 +- .../web/dao/impl/StandardProcessGroupDAO.java | 19 +- .../web/dao/impl/StandardProcessorDAO.java | 3 +- .../filter/ConnectorRequestContextFilter.java | 89 + .../src/main/webapp/WEB-INF/web.xml | 8 + .../apache/nifi/audit/TestLabelAuditor.java | 2 +- .../nifi/audit/TestProcessGroupAuditor.java | 6 +- .../nifi/audit/TestProcessorAuditor.java | 2 +- .../StandardAuthorizableLookupTest.java | 109 + .../web/StandardNiFiServiceFacadeTest.java | 372 ++- .../nifi/web/api/TestConnectorResource.java | 748 +++++ .../web/api/TestDataTransferResource.java | 18 +- .../apache/nifi/web/api/TestFlowResource.java | 46 +- .../StandardNiFiConnectorWebContextTest.java | 219 ++ ...orizingConnectorInvocationHandlerTest.java | 239 ++ .../AuthorizingFlowContextTest.java | 167 ++ ...AuthorizingParameterContextFacadeTest.java | 162 ++ .../web/controller/ControllerFacadeTest.java | 306 +++ .../dao/impl/StandardConnectionDAOTest.java | 196 ++ .../dao/impl/StandardConnectorDAOTest.java | 311 +++ .../dao/impl/StandardProcessGroupDAOTest.java | 130 + .../dao/impl/StandardProcessorDAOTest.java | 2 +- .../impl/TestStandardParameterContextDAO.java | 2 +- .../dao/impl/TestStandardProcessGroupDAO.java | 15 +- .../ConnectorRequestContextFilterTest.java | 178 ++ ...tandardClientRegistrationProviderTest.java | 2 +- ...andardRegistrationBuilderProviderTest.java | 4 +- ...elyingPartyRegistrationRepositoryTest.java | 4 +- .../x509/X509AuthenticationProviderTest.java | 2 +- .../component-access-policies.component.ts | 6 +- .../global-access-policies.component.ts | 1 + .../feature/documentation-routing.module.ts | 5 + .../feature/documentation.component.html | 13 + .../feature/documentation.component.ts | 8 + .../feature/documentation.module.ts | 2 + .../service/documentation.service.ts | 17 + .../connector-definition.actions.ts | 56 + .../connector-definition.effects.ts | 79 + .../connector-definition.reducer.ts | 92 + .../connector-definition.selectors.ts | 31 + .../state/connector-definition/index.ts | 73 + .../app/pages/documentation/state/index.ts | 4 + ...nnector-property-definition.component.html | 95 + ...nnector-property-definition.component.scss | 20 + ...ctor-property-definition.component.spec.ts | 176 ++ ...connector-property-definition.component.ts | 88 + .../connector-definition.component.html | 209 ++ .../connector-definition.component.scss | 20 + .../connector-definition.component.spec.ts | 415 +++ .../connector-definition.component.ts | 204 ++ .../feature/flow-designer.module.ts | 4 +- .../service/canvas-context-menu.service.ts | 3 +- .../flow-designer/state/flow/flow.actions.ts | 3 +- .../state/flow/flow.effects.spec.ts | 104 +- .../flow-designer/state/flow/flow.effects.ts | 10 +- .../state/provenance-event-listing/index.ts | 1 + .../provenance-event-listing.effects.ts | 18 +- .../provenance-event-table.component.ts | 3 +- ...flow-analysis-rule-table.component.spec.ts | 4 + ...ment-controller-services.component.spec.ts | 4 + ...arameter-providers-table.component.spec.ts | 4 + .../reporting-task-table.component.spec.ts | 4 + .../user-table/user-table.component.spec.ts | 4 + .../app/service/extension-types.service.ts | 4 + .../current-user/current-user.reducer.ts | 1 + .../nifi/src/app/state/current-user/index.ts | 1 + .../extension-types.effects.ts | 9 +- .../extension-types.reducer.ts | 3 + .../extension-types.selectors.ts | 8 +- .../src/app/state/extension-types/index.ts | 2 + .../src/services/nifi-common.service.ts | 5 + .../frontend/libs/shared/src/types/index.ts | 1 + .../extension/manifest/ConfigurationStep.java | 80 + .../manifest/ConfigurationStepDependency.java | 67 + .../extension/manifest/ConnectorProperty.java | 120 + .../manifest/ConnectorPropertyDependency.java | 57 + .../manifest/ConnectorPropertyGroup.java | 67 + .../manifest/ConnectorPropertyType.java | 43 + .../nifi/extension/manifest/Extension.java | 14 + .../extension/manifest/ExtensionType.java | 4 +- .../manifest/ComponentManifestBuilder.java | 7 + .../StandardComponentManifestBuilder.java | 12 + .../impl/StandardRuntimeManifestBuilder.java | 126 + ...rdRuntimeManifestBuilderConnectorTest.java | 349 +++ .../nifi/util/MockValidationContext.java | 8 + .../org/apache/nifi/nar/ExtensionMapping.java | 25 +- .../stateless/flow/StatelessDataflow.java | 9 + .../bootstrap/AllowListClassLoader.java | 5 + .../bootstrap/StatelessBootstrap.java | 2 +- .../reporting/StatelessReportingTaskNode.java | 3 +- .../scheduling/StatelessProcessScheduler.java | 25 +- .../engine/StatelessFlowManager.java | 21 +- .../stateless/flow/StandardStatelessFlow.java | 5 + .../queue/StatelessFlowFileQueue.java | 8 + .../nifi-system-test-extensions/pom.xml | 5 + .../tests/system/AssetConnector.java | 93 + .../system/BundleResolutionConnector.java | 163 ++ .../tests/system/CalculateConnector.java | 226 ++ .../system/ComponentLifecycleConnector.java | 177 ++ .../tests/system/DataQueuingConnector.java | 79 + .../system/GatedDataQueuingConnector.java | 122 + .../system/NestedProcessGroupConnector.java | 75 + .../connectors/tests/system/NopConnector.java | 133 + .../system/ParameterContextConnector.java | 286 ++ .../tests/system/SelectiveDropConnector.java | 150 + .../processors/tests/system/Calculate.java | 71 + .../tests/system/TerminateFlowFile.java | 28 + .../tests/system/UpdateContent.java | 41 +- ...apache.nifi.components.connector.Connector | 25 + .../org.apache.nifi.processor.Processor | 1 + .../nifi/tests/system/NiFiClientUtil.java | 315 +++ .../nifi/tests/system/NiFiSystemIT.java | 26 + .../SpawnedStandaloneNiFiInstanceFactory.java | 47 + .../system/TroubleshootingTestWatcher.java | 105 +- .../clustering/FlowSynchronizationIT.java | 3 + .../ClusteredConnectorAssetsIT.java | 158 ++ .../connectors/ClusteredConnectorDrainIT.java | 228 ++ .../connectors/ClusteredConnectorIT.java | 220 ++ .../system/connectors/ConnectorAssetsIT.java | 207 ++ .../system/connectors/ConnectorCrudIT.java | 280 ++ .../system/connectors/ConnectorDrainIT.java | 178 ++ .../connectors/ConnectorLifecycleIT.java | 235 ++ .../ConnectorMethodMarshallingIT.java | 71 + .../ConnectorParameterContextIT.java | 119 + .../ConnectorVersionResolutionIT.java | 97 + .../connectors/SelectiveDropConnectorIT.java | 188 ++ .../connector-version-mismatch/flow.json | 52 + .../flow.json | 2 +- .../flows/migrate-properties/flow.json | 56 +- .../cli/impl/client/NiFiClientFactory.java | 11 + .../nifi/toolkit/client/ConnectorClient.java | 419 +++ .../nifi/toolkit/client/FlowClient.java | 8 + .../nifi/toolkit/client/NiFiClient.java | 8 +- .../client/impl/JerseyConnectorClient.java | 673 +++++ .../toolkit/client/impl/JerseyFlowClient.java | 9 + .../toolkit/client/impl/JerseyNiFiClient.java | 13 +- pom.xml | 2 + 588 files changed, 56339 insertions(+), 959 deletions(-) create mode 100644 .cursor/rules/building.mdc create mode 100644 .cursor/rules/code-style.mdc create mode 100644 .cursor/rules/ending-conditions.mdc create mode 100644 .cursor/rules/extension-development.mdc create mode 100644 .cursor/rules/extension-testing.mdc create mode 100644 .cursor/rules/framework-testing.mdc create mode 100644 .cursor/rules/persona.mdc create mode 100644 .cursor/rules/testing-standards.mdc create mode 100644 c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConfigurationStep.java create mode 100644 c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConfigurationStepDependency.java create mode 100644 c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorDefinition.java create mode 100644 c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyDependency.java create mode 100644 c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyDescriptor.java create mode 100644 c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyGroup.java create mode 100644 c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyType.java create mode 100644 nifi-commons/nifi-connector-utils/pom.xml create mode 100644 nifi-commons/nifi-connector-utils/src/main/java/org/apache/nifi/components/connector/util/VersionedFlowUtils.java create mode 100644 nifi-commons/nifi-connector-utils/src/test/java/org/apache/nifi/components/connector/util/TestVersionedFlowUtils.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-api/pom.xml create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-api/src/main/java/org/apache/nifi/mock/connector/server/ConnectorConfigVerificationResult.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-api/src/main/java/org/apache/nifi/mock/connector/server/ConnectorMockServer.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-api/src/main/java/org/apache/nifi/mock/connector/server/ConnectorTestRunner.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server-nar/pom.xml create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/pom.xml create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockAuditService.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockConnectorAssetManager.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockConnectorInitializationContext.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockConnectorRepository.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockExtensionDiscoveringManager.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockExtensionMapper.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockNiFiConnectorWebContext.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockRuleViolationsManager.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockServerConfigVerificationResult.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockStateManager.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockStateManagerProvider.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockStateMap.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/NopPropertyEncryptor.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/PermitAllAuthorizer.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/StandardConnectorMockServer.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/secrets/ConnectorTestRunnerAuthorizable.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/secrets/ConnectorTestRunnerSecretProvider.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/secrets/ConnectorTestRunnerSecretsManager.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.NiFiServer create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.asset.AssetManager create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.components.connector.ConnectorRepository create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.components.connector.secrets.SecretsManager create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/MockConnectorAssetManagerTest.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/MockNiFiConnectorWebContextTest.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/MockStateManagerProviderTest.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/StandardConnectorMockServerJettyTest.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/pom.xml create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/AllowableValuesIT.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/CreateConnectorIT.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/CronScheduleIT.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/MockControllerServiceIT.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/MockProcessorIT.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors-nar/pom.xml create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/pom.xml create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/AllowableValuesConnector.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/CronScheduleConnector.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/GenerateAndLog.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/MissingBundleConnector.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/resources/META-INF/services/org.apache.nifi.components.connector.Connector create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/resources/flows/Cron_Schedule_Connector.json create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/resources/flows/Generate_and_Update.json create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/pom.xml create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock/pom.xml create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock/src/main/java/org/apache/nifi/mock/connector/StandardConnectorTestRunner.java create mode 100644 nifi-connector-mock-bundle/nifi-connector-mock/src/main/resources/nifi.properties create mode 100644 nifi-connector-mock-bundle/pom.xml create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/pom.xml create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaConnectionStep.java create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaToS3.java create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaToS3FlowBuilder.java create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaTopicsStep.java create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/S3Step.java create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/META-INF/services/org.apache.nifi.components.connector.Connector create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Connection.md create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Topics.md create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/S3_Configuration.md create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/flows/Kafka_to_S3.json create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-integration-tests/pom.xml create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-integration-tests/src/test/java/org/apache/nifi/connectors/kafkas3/KafkaToS3IT.java create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-nar/pom.xml create mode 100644 nifi-connectors/nifi-kafka-to-s3-bundle/pom.xml create mode 100644 nifi-connectors/pom.xml create mode 100644 nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-shared/src/main/java/org/apache/nifi/kafka/service/consumer/Kafka3AssignmentService.java create mode 100644 nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-shared/src/main/java/org/apache/nifi/kafka/service/consumer/RecordIterable.java create mode 100644 nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfigurationProvider.java create mode 100644 nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfigurationProviderException.java create mode 100644 nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfigurationProviderInitializationContext.java create mode 100644 nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorRequestContext.java create mode 100644 nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorRequestContextHolder.java create mode 100644 nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorWorkingConfiguration.java create mode 100644 nifi-framework-api/src/main/java/org/apache/nifi/components/connector/StandardConnectorRequestContext.java create mode 100644 nifi-framework-api/src/main/java/org/apache/nifi/components/connector/secrets/AuthorizableSecret.java create mode 100644 nifi-framework-api/src/test/java/org/apache/nifi/components/connector/ConnectorRequestContextHolderTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConfigurationStepConfigurationDTO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConfigurationStepDependencyDTO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorActionDTO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorConfigurationDTO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorDTO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorPropertyDependencyDTO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorPropertyDescriptorDTO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorValueReferenceDTO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyGroupConfigurationDTO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/SecretDTO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VerifyConnectorConfigStepRequestDTO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectorStatusDTO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectorStatusSnapshotDTO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectorStatusSnapshotDTO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConfigurationStepEntity.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConfigurationStepNamesEntity.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorEntity.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorPropertyAllowableValuesEntity.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorRunStatusEntity.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorTypesEntity.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorsEntity.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/SecretsEntity.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/StepDocumentationEntity.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VerifyConnectorConfigStepRequestEntity.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorEndpointMerger.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorFlowEndpointMerger.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorPropertyGroupEndpointMerger.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorPropertyGroupNamesEndpointMerger.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorStatusEndpointMerger.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorsEndpointMerger.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/VerifyConnectorConfigStepEndpointMerger.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ClusteredConnectorRequestReplicator.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConfigurationStepEntityMerger.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConfigurationStepNamesEntityMerger.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConnectorEntityMerger.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConnectorPropertyDescriptorDtoMerger.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConnectorsEntityMerger.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/framework/configuration/ConnectorRequestReplicatorConfiguration.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/ConfigurationStepEntityMergerTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/ConnectorEntityMergerTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/ConnectorPropertyDescriptorDtoMergerTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/asset/StandardConnectorAssetManager.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/StandardConnectorValidationTrigger.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/ParameterProviderSecretProvider.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/ParameterProviderSecretsManager.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/StandardSecret.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/StandardSecretsManagerInitializationContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/ConnectorLogObserver.java rename nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/{NiFiRegistryFlowMapper.java => VersionedComponentFlowMapper.java} (92%) create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/resources/META-INF/services/org.apache.nifi.components.connector.secrets.SecretsManager create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/asset/StandardConnectorAssetManagerTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/components/connector/secrets/TestParameterProviderSecretsManager.java rename nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/registry/flow/mapping/{TestNiFiRegistryFlowMapper.java => TestVersionedComponentFlowMapper.java} (95%) create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConfigurationUpdateResult.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorAction.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfiguration.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorNode.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorRepository.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorRepositoryInitializationContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorRequestReplicator.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorState.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorStateTransition.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorUpdateContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorValidationTrigger.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FlowContextFactory.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FrameworkConnectorInitializationContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FrameworkConnectorInitializationContextBuilder.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FrameworkFlowContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/MutableConnectorConfigurationContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/NamedStepConfiguration.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ParameterContextFacadeFactory.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ProcessGroupFacadeFactory.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ProcessGroupFactory.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/secrets/SecretProvider.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/secrets/SecretsManager.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/secrets/SecretsManagerInitializationContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/ConnectableFlowFileActivity.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/FlowFileActivity.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/FlowFileTransferCounts.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/ProcessGroupFlowFileActivity.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/connectable/ProcessGroupFlowFileActivityTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/StandardConnectorAssetSynchronizer.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/ConnectorDetails.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/ConnectorParameterLookup.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/ConnectorValidationContextBridge.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/DescribedValueProvider.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/GhostConnector.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandaloneConnectorRequestReplicator.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardComponentBundleLookup.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorAction.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorConfigurationContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorConfigurationProviderInitializationContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorInitializationContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorNode.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorPropertyValue.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorRepoInitializationContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorRepository.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorStateTransition.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorValidationContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardFlowContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardStepConfigurationContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/ComponentContextProvider.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/IllegalExecutionEngineStatelessGroupLifecycle.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneConnectionFacade.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneControllerServiceFacade.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneControllerServiceLifecycle.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneParameterContextFacade.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessGroupFacade.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessGroupLifecycle.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessorFacade.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessorLifecycle.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneStatelessGroupLifecycle.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/FlowControllerFlowContextFactory.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/FlowControllerProcessGroupFacadeFactory.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardComponentContextProvider.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/SelectiveDropResult.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/bootstrap/tasks/ConnectionDiagnosticTask.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.components.connector.ConnectorRepository create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/BlockingConnector.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/DynamicAllowableValuesConnector.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/DynamicFlowConnector.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/MissingBundleConnector.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/OnPropertyModifiedConnector.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/ParameterConnector.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/SleepingConnector.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/StandardConnectorNodeIT.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardComponentBundleLookup.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorConfigurationContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorInitializationContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorNode.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorRepository.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/facades/standalone/TestStandaloneProcessGroupLifecycle.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/CreateDummyFlowFile.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/DuplicateFlowFile.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/ExposeFileValues.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/LogFlowFileContents.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/OnPropertyModifiedTracker.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/OverwriteFlowFile.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/Sleep.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/TerminateFlowFile.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/services/CounterService.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/services/impl/StandardCounterService.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/MockStateManagerProvider.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/flow/NopConnector.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/flow/TestStandardFlowManager.java rename nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/{NiFiRegistryFlowMapperTest.java => VersionedComponentFlowMapperTest.java} (99%) create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.components.connector.Connector create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/TestRuntimeManifest/nifi-test-components-nar/META-INF/docs/steps/org.example.TestConnector/Another_Test_Step.md create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/TestRuntimeManifest/nifi-test-components-nar/META-INF/docs/steps/org.example.TestConnector/Test_Step.md create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/colors.txt create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/choose-color.json create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/generate-and-log-with-parameter.json create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/generate-duplicate-log-flow.json create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/on-property-modified-tracker.json create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/DummyConnector.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/StandardExtensionDiscoveringManagerConnectorTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/META-INF/services/org.apache.nifi.components.connector.Connector create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ConnectorAuditor.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectorResource.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/StandardNiFiConnectorWebContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingConnectionFacade.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingConnectorInvocationHandler.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingControllerServiceFacade.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingControllerServiceLifecycle.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingFlowContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingParameterContextFacade.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessGroupFacade.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessGroupLifecycle.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessorFacade.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessorLifecycle.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingStatelessGroupLifecycle.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/ConnectorAuthorizationContext.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ConnectorDAO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectorDAO.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/filter/ConnectorRequestContextFilter.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestConnectorResource.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/StandardNiFiConnectorWebContextTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/authorization/AuthorizingConnectorInvocationHandlerTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/authorization/AuthorizingFlowContextTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/authorization/AuthorizingParameterContextFacadeTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerFacadeTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardConnectionDAOTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardConnectorDAOTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAOTest.java create mode 100644 nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/filter/ConnectorRequestContextFilterTest.java create mode 100644 nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.actions.ts create mode 100644 nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.effects.ts create mode 100644 nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.reducer.ts create mode 100644 nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.selectors.ts create mode 100644 nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/index.ts create mode 100644 nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.html create mode 100644 nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.scss create mode 100644 nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.spec.ts create mode 100644 nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.ts create mode 100644 nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.html create mode 100644 nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.scss create mode 100644 nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.spec.ts create mode 100644 nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.ts create mode 100644 nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConfigurationStep.java create mode 100644 nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConfigurationStepDependency.java create mode 100644 nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorProperty.java create mode 100644 nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorPropertyDependency.java create mode 100644 nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorPropertyGroup.java create mode 100644 nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorPropertyType.java create mode 100644 nifi-manifest/nifi-runtime-manifest-core/src/test/java/org/apache/nifi/runtime/manifest/impl/StandardRuntimeManifestBuilderConnectorTest.java create mode 100644 nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/AssetConnector.java create mode 100644 nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/BundleResolutionConnector.java create mode 100644 nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/CalculateConnector.java create mode 100644 nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/ComponentLifecycleConnector.java create mode 100644 nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/DataQueuingConnector.java create mode 100644 nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/GatedDataQueuingConnector.java create mode 100644 nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/NestedProcessGroupConnector.java create mode 100644 nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/NopConnector.java create mode 100644 nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/ParameterContextConnector.java create mode 100644 nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/SelectiveDropConnector.java create mode 100644 nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/Calculate.java create mode 100644 nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.components.connector.Connector create mode 100644 nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ClusteredConnectorAssetsIT.java create mode 100644 nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ClusteredConnectorDrainIT.java create mode 100644 nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ClusteredConnectorIT.java create mode 100644 nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorAssetsIT.java create mode 100644 nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorCrudIT.java create mode 100644 nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorDrainIT.java create mode 100644 nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorLifecycleIT.java create mode 100644 nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorMethodMarshallingIT.java create mode 100644 nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorParameterContextIT.java create mode 100644 nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorVersionResolutionIT.java create mode 100644 nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/SelectiveDropConnectorIT.java create mode 100644 nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/connector-version-mismatch/flow.json create mode 100644 nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/ConnectorClient.java create mode 100644 nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/impl/JerseyConnectorClient.java diff --git a/.cursor/rules/building.mdc b/.cursor/rules/building.mdc new file mode 100644 index 000000000000..af4829a42628 --- /dev/null +++ b/.cursor/rules/building.mdc @@ -0,0 +1,18 @@ +--- +description: Maven build instructions for the NiFi codebase +alwaysApply: true +--- + +# Building + +NiFi is a complex Maven codebase. Never build code (testing or otherwise) using javac. +Always use `mvn` instead, or preferably the `.mvnw` wrapper script. + +Additionally, building a maven module using the also-make flag (`-am`) is often very +expensive and slow. Instead, only build the specific module you are modifying. Assume that +the user has already built the entire codebase and that only the specific module you are +modifying needs to be built again. If this fails, you can prompt the user to build the entire +codebase, but only after you have attempted to build the relevant modules yourself first. +It is important not to run `mvn clean` at the root level or at the `nifi-assembly` level without +the user's express permission, as this may delete a running instance of NiFi, causing permanent +loss of flows and configuration. diff --git a/.cursor/rules/code-style.mdc b/.cursor/rules/code-style.mdc new file mode 100644 index 000000000000..ca7e9cfed5a6 --- /dev/null +++ b/.cursor/rules/code-style.mdc @@ -0,0 +1,74 @@ +--- +description: Java code style conventions for the NiFi codebase +globs: "**/*.java" +alwaysApply: false +--- + +# Code Style + +NiFi adheres to a few code styles that are not necessarily common. Please ensure that you +observe these code styles. + +1. Any variable that can be marked `final` must be marked `final`. This includes + declarations of Exceptions, method arguments, local variables, member variables, etc. +2. Short-hand is highly discouraged in names of variables, classes, methods, etc., as well + as in documentation. Exceptions to this include in the framework, you may see references to + `procNode` for `ProcessorNode` or other such short-hand that is very difficult to confuse with + other terms, and it is used only when clearly defined such as `final ProcessorNode procNode = ...`. + Even though, however, we would not abbreviate `ControllerService` as `cs` because `cs` is too vague + and easily misunderstood. Instead, a value of `serviceNode` might be used. +3. Private / helper methods should not be placed before the first public/protected method + that calls it. +4. Unless the method is to be heavily reused, avoid creating trivial 1-2 line methods and + instead just place the code inline. +5. Code is allowed to be up to 200 characters wide. Avoid breaking lines into many short lines. +6. Avoid creating private methods that are called only once unless they are at least 10 + lines long or are complex. +7. It is never acceptable to use star imports. Import each individual class that is to be used. +8. Never use underscores in class names, variables, or filenames. +9. Never use System.out.println but instead use SLF4J Loggers. +10. Avoid excessive whitespace in method invocations. For example, instead of writing: + +```java +myObject.doSomething( + arg1, + arg2, + arg3, + arg4, + arg5 +); +``` + +Write this instead: + +```java +myObject.doSomething(arg1, arg2, arg3, arg4, arg5); +``` + +It is okay to use many newlines in a builder pattern, such as: +```java +final MyObject myObject = MyObject.builder() + .arg1(arg1) + .arg2(arg2) + .arg3(arg3) + .build(); +``` + +It is also acceptable when chaining methods in a functional style such as: +```java +final List result = myList.stream() + .filter(s -> s.startsWith("A")) + .map(String::toUpperCase) + .toList(); +``` + +11. When possible, prefer importing a class, rather than using fully qualified classname + inline in the code. +12. Avoid statically importing methods, except in methods that are frequently used in testing + frameworks, such as the `Assertions` and `Mockito` classes. +13. Avoid trailing whitespace at the end of lines, especially in blank lines. +14. The `var` keyword is never allowed in the codebase. Always explicitly declare the type of variables. +15. Prefer procedural code over functional code. For example, prefer using a for loop instead of a stream + when the logic is not simple and straightforward. The stream API is powerful but can be difficult to + read when overused or used in complex scenarios. Functional style is best used when the logic is simple + and chains together no more than 3-4 operations. diff --git a/.cursor/rules/ending-conditions.mdc b/.cursor/rules/ending-conditions.mdc new file mode 100644 index 000000000000..aa2b607801a1 --- /dev/null +++ b/.cursor/rules/ending-conditions.mdc @@ -0,0 +1,30 @@ +--- +description: Task completion checklist that must be verified before considering any task done +alwaysApply: true +--- + +# Ending Conditions + +When you have completed a task, ensure that you have verified the following: + +1. All code compiles and builds successfully using `mvn`. +2. All relevant unit tests pass successfully using `mvn`. +3. All code adheres to the Code Style rules. +4. Checkstyle and PMD pass successfully using + `mvn checkstyle:check pmd:check -T 1C` from the appropriate directory. +5. Unit tests have been added to verify the functionality of any sufficiently complex method. +6. A system test or an integration test has been added if the change makes significant + changes to the framework and the interaction between a significant number of classes. +7. You have performed a full review of the code to ensure that there are no logical errors + and that the code is not duplicative or difficult to understand. If you find any code that + is in need of refactoring due to clarity or duplication, you should report this to the user + and offer to make those changes as well. +8. If creating a new Processor or Controller Service, ensure that all relevant annotations + have been added, including `@Tags`, `@CapabilityDescription`, `@UseCase`, and + `@MultiProcessorUseCase` as appropriate. + + +Do not consider the task complete until all of the above conditions have been met. When you +do consider the task complete, provide a summary of what you changed and which tests were +added or modified and what the behavior is that they verify. Additionally, provide any feedback +about your work that may need further review or that is not entirely complete. diff --git a/.cursor/rules/extension-development.mdc b/.cursor/rules/extension-development.mdc new file mode 100644 index 000000000000..74e53f34d5f1 --- /dev/null +++ b/.cursor/rules/extension-development.mdc @@ -0,0 +1,107 @@ +--- +description: Development patterns for NiFi extensions (Processors, Controller Services, Connectors). Covers Property Descriptors, Relationships, and common patterns. +alwaysApply: false +--- + +# Extension Development + +This rule applies when developing NiFi extensions: Processors, Controller Services, and Connectors. + +## Property Descriptors + +Property Descriptors are defined as `static final` fields on the component class using +`PropertyDescriptor.Builder`. + +- **Naming:** Use clear, descriptive names. The `displayName` field should never be used. Make the + name itself clear and concise. Use Title Case for property names. +- **Required vs. optional:** Mark properties as `.required(true)` when the component cannot + function without them. Prefer sensible defaults via `.defaultValue(...)` when possible. + When a default value is provided, the property will always have a value. The `required` flag in this + case is more of a documentation aid to indicate the importance of the property. +- **Validators:** Always attach an appropriate `Validator` (e.g., `StandardValidators.NON_EMPTY_VALIDATOR`, + `StandardValidators.POSITIVE_INTEGER_VALIDATOR`). The Validator can be left off only when Allowable Values + are provided. In this case, do not include a Validator because it is redundant and confusing. +- **Expression Language:** If a property should support Expression Language, add + `.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)` or the + appropriate scope. Always document when Expression Language is supported in the property + description. Some developers tend to go overboard here and feel like Expression Language should be supported + everywhere, but this is a mistake! The default assumption should be that Expression Language is not supported + unless the value is expected to be different for every FlowFile that is processed. +- **Dependencies:** Use `.dependsOn(...)` to conditionally show properties based on the + values of other properties. This keeps the configuration UI clean and avoids exposing + irrelevant properties. If there is a dependency, it is important to understand that `.required(true)` means that + this property is required IF AND ONLY IF the dependency condition is met. + +## Processors + +- The `onTrigger` method should be focused on processing FlowFiles. Keep setup and teardown + logic in lifecycle methods when possible. +- Prefer `session.read()` and `session.write()` with callbacks over directly working with + streams to ensure proper resource management. +- Prefer `session.commitAsync()` over `session.commit()`. The `commit` method was the original implementation, + but it has now been deprecated in favor of `commitAsync`. The `commitAsync` call provide a clearer, cleaner + interface for handling post-commit actions including success and failure callbacks. In addition, the async + method allows Processors to be used much more efficiently in a Stateless NiFi flow. + +### Processor Lifecycle Annotations + +- Use `@OnScheduled` for setup that should happen once before the processor starts + running (e.g., creating clients, compiling patterns). +- Use `@OnStopped` for cleanup (e.g., closing clients, releasing resources). +- `@OnUnscheduled` is rarely used but can be used to interrupt long-running processes when the Processor is stopped. + Generally, though, it is preferable to write the Processor in such a way that long-running processes check `isScheduled()` + and stop gracefully if the return value is `false`. + +### Relationships +- **Declaration**: Relationships are defined as `static final` fields using `new Relationship.Builder()`. + Relationship names should generally be lowercase. +- **Success and Failure:** Most processors define at least a `success` and `failure` + relationship. Use `REL_SUCCESS` and `REL_FAILURE` as constant names. +- **Original relationship:** Processors that enrich or fork FlowFiles often include an + `original` relationship for the unmodified input FlowFile. + +### Use Case Documentation +The `@UseCase` and `@MultiProcessorUseCase` annotations help document common usage patterns for Processors. +This is helpful for users to understand when and how to use the component effectively. It is equally important +for Agents that can determine which components should be used for a given task. + +- Use `@UseCase` to document common use cases for the Processor. This helps users understand + when and how to use the component effectively. This is unnecessary for Processors that serve a single use case + that is clearly described by the component name and description. For example, a Processor that consumes messages + from a specific service likely does not need a `@UseCase` annotation because its purpose is clear. +- Use `@MultiProcessorUseCase` to document well-known patterns that involve multiple Processors working + together to achieve a common goal. Examples include List/Fetch patterns, Fork/Join patterns, etc. + The `@MultiProcessorUseCase` annotation should not be added to each individual Processor involved in the pattern. + Rather, the convention is to add the annotation to the last Processor in the flow that completes the pattern. + Some Processors will have one or more `@UseCase` annotations and no `@MultiProcessorUseCase` annotations, + while some will have one or more `@MultiProcessorUseCase` annotations and no `@UseCase` annotations. + + +## Controller Services + +Controller Services are objects that can be shared across multiple components. This is typically done for +clients that connect to external systems in order to avoid creating many connections, or in order to share +configuration across multiple components without the user having to duplicate configuration. Controller Services +can also be helpful for abstracting away some piece of functionality into a separate extension point so that the +implementation can be swapped out by the user. For example, Record Readers and Writers are implemented as Controller +Services so that the user can simply choose which format they want to read and write in a flexible and reusable way. + +That said, Controller Services can be more onerous to configure and maintain for the user, so they should +be used sparingly and only when there is a clear benefit to doing so. + +### Controller Service Lifecycle Annotations + +- Use `@OnScheduled` for setup that should happen once before the service is enabled (e.g., creating clients, compiling patterns). +- Use `@OnDisabled` for cleanup (e.g., closing clients, releasing resources). + + +## General Patterns + +- Use `ComponentLog` (obtained via `getLogger()`) for all logging, not SLF4J directly. + This ensures log messages are associated with the component instance and that they generate Bulletins. +- Use `@CapabilityDescription` to provide a clear and concise description of what the component does. This should not + be used for configuration details. +- Use `@Tags` to provide relevant keywords that help users find the component. +- Use `@SeeAlso` to reference related components. +- Use `@WritesAttributes` and `@ReadsAttributes` to document which FlowFile attributes are read and written by the component. +- Use `@DynamicProperty` to document any dynamic properties supported by the component. diff --git a/.cursor/rules/extension-testing.mdc b/.cursor/rules/extension-testing.mdc new file mode 100644 index 000000000000..97692657289b --- /dev/null +++ b/.cursor/rules/extension-testing.mdc @@ -0,0 +1,80 @@ +--- +description: Testing guidance for NiFi extensions (Processors, Controller Services, Connectors). Covers nifi-mock and TestRunner usage. +alwaysApply: false +--- + +# Extension Testing + +This rule applies when writing tests for NiFi extensions: Processors, Controller Services, and Connectors. + +## Unit Tests + +Unit tests should be used to test individual classes and methods in isolation. This often +will result in mocking dependency classes. However, if there already exists a Mock +implementation of an interface or dependency class, it is preferred to use the existing +Mock implementation. Similarly, for simple classes, it is preferable to make use of the +real implementation of a class rather than creating a Mock implementation. We are infinitely +more interested in having tests that are fast, reliable, correct, and easy to maintain than +we are in having tests that adhere to strict and arbitrary definitions of what constitutes +a "unit test." + +## Use nifi-mock + +Tests for extensions should always make use of the `nifi-mock` mocking framework. This is +done through the `TestRunner` interface and its standard implementation, obtained via +`TestRunners.newTestRunner(processor)`. + +The `TestRunner` provides methods for: +- Setting property values (`setProperty`) +- Enqueueing FlowFiles (`enqueue`) +- Running the processor (`run`) +- Asserting transfer to relationships (`assertTransferCount`, `assertAllFlowFilesTransferred`) +- Validating processor configuration (`assertValid`, `assertNotValid`) +- Asserting content and attributes of FlowFiles (`assertContentEquals`, `assertAttributeEquals`, etc.) + +## No System Tests for Extensions + +System tests are not expected for extensions. Extensions are tested at the unit level using +`nifi-mock`. The `nifi-mock` framework provides sufficient isolation and simulation of the +NiFi runtime environment. + +## What to Test + +- **Property validation:** If the extension has a custom Validator, it +- **customValidate:** If the extension overrides the `customValidate` method, test that it correctly + validates the configuration and produces appropriate validation results. +- **Relationship routing:** Verify that FlowFiles are routed to the correct relationship + based on input and configuration. +- **Content transformation:** For processors that modify FlowFile content, verify that + output content matches expectations. +- **Attribute handling:** Verify that expected attributes are set on output FlowFiles. +- **Error handling:** Verify that error conditions (bad input, misconfiguration, simulated + failures) are handled correctly, typically by routing to a failure relationship. + +## What NOT to Test + +- **NiFi framework behavior:** Do not attempt to test the behavior of the NiFi framework itself. + For example, do not test that `session.commitAsync()` actually commits a transaction. Instead, + focus on testing that your extension behaves correctly when `commitAsync` is called, and trust + that the NiFi framework will handle the commit correctly. +- **Validator behavior:** If a custom validator is used by an extension, that custom validator should + be tested separate as a unit test for the validator itself. However, if the extension point provides + a `customValidate` method, that should absolutely be tested as part of the extension's unit tests. +- **The PropertyDescriptors that are returned:** Do not test that the `getSupportedPropertyDescriptors` + method returns the expected PropertyDescriptors. This is an anti-pattern because it does not properly + test that the extension abides by the contract of the API. For example, if a new PropertyDescriptor is + added whose default is to behave the same way as the old behavior, the test should absolutely pass. + However, if the test is written to expect a specific set of PropertyDescriptors, then the test will fail, + leading to confusion and unnecessary maintenance. + +## Controller Service Testing + +When a processor depends on a Controller Service, use `TestRunner.addControllerService` +and `TestRunner.enableControllerService` to wire up either a real or mock implementation +of the service for testing. + +## TestContainers + +For Processors that interact with external systems, it can be helpful to use TestContainers to spin up +a temporary instance of the external system for testing. This allows for more realistic integration tests +without requiring the user to have the external system installed and running on their machine. diff --git a/.cursor/rules/framework-testing.mdc b/.cursor/rules/framework-testing.mdc new file mode 100644 index 000000000000..d25680b5021d --- /dev/null +++ b/.cursor/rules/framework-testing.mdc @@ -0,0 +1,46 @@ +--- +description: Testing guidance for NiFi framework code (not extensions). Covers when to use unit, integration, and system tests for framework classes. +alwaysApply: false +--- + +# Framework Testing + +This rule applies when working on NiFi framework code (not Processors, Controller +Services, or Connectors). + +## Unit Tests + +Unit tests should be used to test individual classes and methods in isolation. This often +will result in mocking dependency classes. However, if there already exists a Mock +implementation of an interface or dependency class, it is preferred to use the existing +Mock implementation. Similarly, for simple classes, it is preferable to make use of the +real implementation of a class rather than creating a Mock implementation. We are infinitely +more interested in having tests that are fast, reliable, correct, and easy to maintain than +we are in having tests that adhere to strict and arbitrary definitions of what constitutes +a "unit test." + +## Integration Tests + +When working in the framework, unit tests are still important, but integration tests and +system tests are often more important. Integration tests are still allowed to use mocks but +typically we prefer to use real implementations of classes in order to ensure a more +realistic and holistic test. + +## System Tests + +System tests live in the `nifi-system-tests` module and should be used for any changes +that make significant changes to the framework and the interaction between a significant +number of classes. They should also be used for any changes that may be fairly isolated but +which are in a critical path of the framework, especially those that affect how data is +persisted, processed, or accessed; or those that affect how components are created, +configured, scheduled, or executed. + +Good candidates for system tests include changes to `ProcessScheduler`, `ProcessorNode`, +`ControllerServiceNode`, `FlowController`, `FlowManager`, how Parameters are handled, flow +synchronization, the repositories, etc. + +## Escalation + +Any unit test that ends up requiring a large number of mocks is a good candidate for an +integration test, and any integration test that ends up requiring a large number of mocks +is a good candidate for a system test. diff --git a/.cursor/rules/persona.mdc b/.cursor/rules/persona.mdc new file mode 100644 index 000000000000..3e4a66ed6455 --- /dev/null +++ b/.cursor/rules/persona.mdc @@ -0,0 +1,16 @@ +--- +description: AI persona and general approach for working on the Apache NiFi codebase +alwaysApply: true +--- + +# AI Persona + +Act as an experienced Java software engineer. When considering how to implement a task, +first consider the big picture of what is being asked. Then determine which classes will +need to be updated. + +Quite often, a single request will require manipulating many different classes. Generally +speaking, it is best to avoid changing established interfaces, especially those in nifi-api. +It is acceptable when necessary, but any change in nifi-api needs to be backward compatible. +For example, you might introduce a new method with a default implementation, or add a new method +and deprecate an old one without removing it. diff --git a/.cursor/rules/testing-standards.mdc b/.cursor/rules/testing-standards.mdc new file mode 100644 index 000000000000..6cb98f4853c4 --- /dev/null +++ b/.cursor/rules/testing-standards.mdc @@ -0,0 +1,81 @@ +--- +description: Shared test coding conventions for all NiFi automated tests (framework and extensions) +globs: "**/src/test/**/*.java" +alwaysApply: false +--- + +# Testing Standards + +In addition to the general rules defined in the Code Style rule, follow these rules when +creating or manipulating automated tests. + +## Test Code Conventions + +1. NEVER add comments such as `// Given`, `// When`, `// Then`. These comments are + considered an anti-pattern and should be removed or replaced whenever they are + encountered. Instead, leave them out all together (preferred) or use comments that + clearly articulate what is happening, such as `// Setup`, `// Invoke method`, + `// Assert expected results`. One of the reasons that this is considered an anti-pattern + (in addition to the fact that the given/when/then nomenclature itself provides no + meaning) is that it assumes a very specific pattern in unit tests, that we will create a + bunch of objects, invoke the method we care about, make assertions, and then end. This + often results in many tests that are extremely repetitive. Instead, whenever it makes + sense to do so, create the prerequisite objects, invoke the method we care about with + appropriate arguments, make assertions, and then invoke again with a different set of + arguments, make assertions, etc. There is no need to have many repetitive methods that + each create many repetitive objects. + +2. Unit tests are Java. They are not English. As such, they should be written like Java. + Frameworks such as assertj that strive to make the unit tests look more "English-like" + should be avoided. Use of these frameworks sometimes works well but often quickly + devolves into automated tests that read like neither English nor Java. + +3. Like any other code, unit tests should be created using reusable methods where + appropriate. Do not create 15 methods that are all very similar and repetitive. Instead, + create reusable methods that can be called from each of the methods. + +4. Never use the `assert` keyword. Use JUnit assertions instead. + +5. Never create a test file that is centered around testing a method or capability. Unit + tests must always be named after the class they are testing. It is okay if a given unit + test class is very, very long. + +6. This is a Java project using the Maven structure. Java test files must always fall under + src/test/java of the appropriate sub-module. + +7. Never use pointless assertions such as assertDoesNotThrow. This adds nothing but + complexity. Just call the method, and if it throws an Exception, the test will fail. It + is assumed by default that each line does not throw an Exception. + +8. Avoid providing messages in assert statements when they do not offer benefits over the default message. + For example, rather than: +``` + assertEquals(3, processors.size(), "Expected 3 processors in the initial flow"); +``` + Just use: +``` + assertEquals(3, processors.size()); +``` +Because the expectation is already clear from assertion, and this message is actually harmful because it hides the size +of the `processors` Collection. + +## General Testing Philosophy + +- Unit tests should be used to verify any sufficiently complex method in a class. We should + *NOT* have unit tests for trivial methods such as getters and setters, or methods that are + only a few lines long and are not complex. A good general rule of thumb is that if a + person can understand a method and verify that it is correct in a few seconds, then it is + not necessary to have a unit test for that method. + +- Avoid adding multiple tests that are redundant. For example, if you have a method that + takes a single `String` argument it may make sense to test with `null`, a 0-length + string, and a long String. But do not test with 4 different Strings each with a different + value but which test the same lines of code. Avoid over-testing by adding multiple tests + whose differences are conditions that do not truly affect the behavior of the code. + +- Unit tests should always focus on the "contract" of the method. That is, given a certain + input, the test should assert that it receives the correct output. The test should NOT be + focused on the inner details of how the method works. Focusing on the inner details of + how the method works is an anti-pattern because it results in tests that are more brittle, + more difficult to maintain, and tests that fail when the method is improved or refactored + but still adheres to the same contract. diff --git a/.gitignore b/.gitignore index ff8a118e9a59..e299e15e43c4 100644 --- a/.gitignore +++ b/.gitignore @@ -21,7 +21,7 @@ nb-configuration.xml .java-version /nifi-nar-bundles/nifi-py4j-bundle/nifi-python-extension-api/src/main/python/dist/ __pycache__ -.cursor/ +.cursor/debug.log # Develocity .mvn/.develocity/ diff --git a/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ComponentManifest.java b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ComponentManifest.java index 1bb5fb07d982..38bd55573e59 100644 --- a/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ComponentManifest.java +++ b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ComponentManifest.java @@ -33,6 +33,7 @@ public class ComponentManifest implements Serializable { private List parameterProviders; private List flowRegistryClients; private List flowAnalysisRules; + private List connectors; @Schema(description = "Public interfaces defined in this bundle") public List getApis() { @@ -97,4 +98,13 @@ public void setFlowRegistryClients(List flowRegist this.flowRegistryClients = flowRegistryClients; } + @Schema(description = "Connectors provided in this bundle") + public List getConnectors() { + return (connectors != null ? Collections.unmodifiableList(connectors) : null); + } + + public void setConnectors(List connectors) { + this.connectors = connectors; + } + } diff --git a/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConfigurationStep.java b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConfigurationStep.java new file mode 100644 index 000000000000..88d76b69af74 --- /dev/null +++ b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConfigurationStep.java @@ -0,0 +1,81 @@ +/* + * 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.c2.protocol.component.api; + +import io.swagger.v3.oas.annotations.media.Schema; + +import java.io.Serializable; +import java.util.List; + +/** + * Represents a configuration step for a Connector. + */ +public class ConfigurationStep implements Serializable { + private static final long serialVersionUID = 1L; + + private String name; + private String description; + private boolean documented; + private List stepDependencies; + private List propertyGroups; + + @Schema(description = "The name of the configuration step") + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + @Schema(description = "The description of the configuration step") + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + @Schema(description = "Whether this configuration step has additional documentation") + public boolean isDocumented() { + return documented; + } + + public void setDocumented(boolean documented) { + this.documented = documented; + } + + @Schema(description = "The dependencies that this step has on other steps") + public List getStepDependencies() { + return stepDependencies; + } + + public void setStepDependencies(List stepDependencies) { + this.stepDependencies = stepDependencies; + } + + @Schema(description = "The property groups in this configuration step") + public List getPropertyGroups() { + return propertyGroups; + } + + public void setPropertyGroups(List propertyGroups) { + this.propertyGroups = propertyGroups; + } +} + diff --git a/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConfigurationStepDependency.java b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConfigurationStepDependency.java new file mode 100644 index 000000000000..1f1bc2a24d9e --- /dev/null +++ b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConfigurationStepDependency.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.c2.protocol.component.api; + +import io.swagger.v3.oas.annotations.media.Schema; + +import java.io.Serializable; +import java.util.List; + +/** + * Represents a dependency that a configuration step has on another step's property. + */ +public class ConfigurationStepDependency implements Serializable { + private static final long serialVersionUID = 1L; + + private String stepName; + private String propertyName; + private List dependentValues; + + @Schema(description = "The name of the step that this step depends on") + public String getStepName() { + return stepName; + } + + public void setStepName(String stepName) { + this.stepName = stepName; + } + + @Schema(description = "The name of the property within the step that this step depends on") + public String getPropertyName() { + return propertyName; + } + + public void setPropertyName(String propertyName) { + this.propertyName = propertyName; + } + + @Schema(description = "The values of the dependent property that enable this step") + public List getDependentValues() { + return dependentValues; + } + + public void setDependentValues(List dependentValues) { + this.dependentValues = dependentValues; + } +} + diff --git a/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorDefinition.java b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorDefinition.java new file mode 100644 index 000000000000..9ac1c85eea3f --- /dev/null +++ b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorDefinition.java @@ -0,0 +1,41 @@ +/* + * 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.c2.protocol.component.api; + +import io.swagger.v3.oas.annotations.media.Schema; + +import java.util.Collections; +import java.util.List; + +/** + * Definition of a Connector component. + */ +public class ConnectorDefinition extends ExtensionComponent { + private static final long serialVersionUID = 1L; + + private List configurationSteps; + + @Schema(description = "The configuration steps for this connector") + public List getConfigurationSteps() { + return (configurationSteps != null ? Collections.unmodifiableList(configurationSteps) : null); + } + + public void setConfigurationSteps(List configurationSteps) { + this.configurationSteps = configurationSteps; + } +} + diff --git a/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyDependency.java b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyDependency.java new file mode 100644 index 000000000000..b01b4143e0f6 --- /dev/null +++ b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyDependency.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.c2.protocol.component.api; + +import io.swagger.v3.oas.annotations.media.Schema; + +import java.io.Serializable; +import java.util.List; + +/** + * Represents a dependency that a connector property has on another property. + */ +public class ConnectorPropertyDependency implements Serializable { + private static final long serialVersionUID = 1L; + + private String propertyName; + private List dependentValues; + + @Schema(description = "The name of the property that this property depends on") + public String getPropertyName() { + return propertyName; + } + + public void setPropertyName(String propertyName) { + this.propertyName = propertyName; + } + + @Schema(description = "The values of the dependent property that enable this property") + public List getDependentValues() { + return dependentValues; + } + + public void setDependentValues(List dependentValues) { + this.dependentValues = dependentValues; + } +} + diff --git a/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyDescriptor.java b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyDescriptor.java new file mode 100644 index 000000000000..fc9b4519e4e7 --- /dev/null +++ b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyDescriptor.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.c2.protocol.component.api; + +import io.swagger.v3.oas.annotations.media.Schema; + +import java.io.Serializable; +import java.util.List; + +/** + * Represents a property descriptor for a Connector. + */ +public class ConnectorPropertyDescriptor implements Serializable { + private static final long serialVersionUID = 1L; + + private String name; + private String description; + private String defaultValue; + private boolean required; + private ConnectorPropertyType propertyType; + private boolean allowableValuesFetchable; + private List allowableValues; + private List dependencies; + + @Schema(description = "The name of the property") + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + @Schema(description = "The description of the property") + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + @Schema(description = "The default value of the property") + public String getDefaultValue() { + return defaultValue; + } + + public void setDefaultValue(String defaultValue) { + this.defaultValue = defaultValue; + } + + @Schema(description = "Whether or not the property is required") + public boolean isRequired() { + return required; + } + + public void setRequired(boolean required) { + this.required = required; + } + + @Schema(description = "The type of the property") + public ConnectorPropertyType getPropertyType() { + return propertyType; + } + + public void setPropertyType(ConnectorPropertyType propertyType) { + this.propertyType = propertyType; + } + + @Schema(description = "Whether or not the allowable values can be fetched dynamically") + public boolean isAllowableValuesFetchable() { + return allowableValuesFetchable; + } + + public void setAllowableValuesFetchable(boolean allowableValuesFetchable) { + this.allowableValuesFetchable = allowableValuesFetchable; + } + + @Schema(description = "The allowable values for this property") + public List getAllowableValues() { + return allowableValues; + } + + public void setAllowableValues(List allowableValues) { + this.allowableValues = allowableValues; + } + + @Schema(description = "The properties that this property depends on") + public List getDependencies() { + return dependencies; + } + + public void setDependencies(List dependencies) { + this.dependencies = dependencies; + } +} + diff --git a/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyGroup.java b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyGroup.java new file mode 100644 index 000000000000..8eaab4ec9745 --- /dev/null +++ b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyGroup.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.c2.protocol.component.api; + +import io.swagger.v3.oas.annotations.media.Schema; + +import java.io.Serializable; +import java.util.List; + +/** + * Represents a group of properties within a configuration step. + */ +public class ConnectorPropertyGroup implements Serializable { + private static final long serialVersionUID = 1L; + + private String name; + private String description; + private List properties; + + @Schema(description = "The name of the property group") + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + @Schema(description = "The description of the property group") + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + @Schema(description = "The properties in this group") + public List getProperties() { + return properties; + } + + public void setProperties(List properties) { + this.properties = properties; + } +} + diff --git a/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyType.java b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyType.java new file mode 100644 index 000000000000..906673ec9d1a --- /dev/null +++ b/c2/c2-protocol/c2-protocol-component-api/src/main/java/org/apache/nifi/c2/protocol/component/api/ConnectorPropertyType.java @@ -0,0 +1,43 @@ +/* + * 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.c2.protocol.component.api; + +/** + * Possible types for connector properties. + */ +public enum ConnectorPropertyType { + + STRING, + + INTEGER, + + BOOLEAN, + + FLOAT, + + DOUBLE, + + STRING_LIST, + + SECRET, + + ASSET, + + ASSET_LIST; + +} + diff --git a/checkstyle.xml b/checkstyle.xml index dc5e8a81d6a5..cd784213ed38 100644 --- a/checkstyle.xml +++ b/checkstyle.xml @@ -25,10 +25,6 @@ - - - - @@ -154,8 +150,5 @@ - - - diff --git a/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-framework-core/src/main/java/org/apache/nifi/minifi/c2/command/AgentPropertyValidationContext.java b/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-framework-core/src/main/java/org/apache/nifi/minifi/c2/command/AgentPropertyValidationContext.java index ed0434949cd3..7998a92065ab 100644 --- a/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-framework-core/src/main/java/org/apache/nifi/minifi/c2/command/AgentPropertyValidationContext.java +++ b/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-framework-core/src/main/java/org/apache/nifi/minifi/c2/command/AgentPropertyValidationContext.java @@ -109,4 +109,9 @@ public boolean isDependencySatisfied(PropertyDescriptor propertyDescriptor, Func public PropertyValue getProperty(PropertyDescriptor descriptor) { throw new UnsupportedOperationException(); } + + @Override + public String evaluateParameters(final String value) { + throw new UnsupportedOperationException(); + } } diff --git a/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-framework-core/src/test/java/org/apache/nifi/minifi/c2/command/DefaultFlowStateStrategyTest.java b/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-framework-core/src/test/java/org/apache/nifi/minifi/c2/command/DefaultFlowStateStrategyTest.java index ad23f6204afb..18252878cffa 100644 --- a/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-framework-core/src/test/java/org/apache/nifi/minifi/c2/command/DefaultFlowStateStrategyTest.java +++ b/minifi/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-framework-core/src/test/java/org/apache/nifi/minifi/c2/command/DefaultFlowStateStrategyTest.java @@ -31,7 +31,6 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Future; import static org.apache.nifi.c2.protocol.api.C2OperationState.OperationState.FULLY_APPLIED; import static org.apache.nifi.c2.protocol.api.C2OperationState.OperationState.NOT_APPLIED; @@ -111,13 +110,13 @@ public void testStopFullyApplied() { ProcessGroup rootProcessGroup = mock(ProcessGroup.class); RemoteProcessGroup remoteProcessGroup = mock(RemoteProcessGroup.class); Set remoteProcessGroups = Set.of(remoteProcessGroup); - CompletableFuture rootProcessGroupStopFuture = mock(CompletableFuture.class); - Future remoteProcessGroupStopFuture = mock(Future.class); + CompletableFuture rootProcessGroupStopFuture = CompletableFuture.completedFuture(null); + CompletableFuture remoteProcessGroupStopFuture = CompletableFuture.completedFuture(null); ProcessGroup nestedProcessGroup = mock(ProcessGroup.class); RemoteProcessGroup nestedRemoteProcessGroup = mock(RemoteProcessGroup.class); Set nestedRemoteProcessGroups = Set.of(nestedRemoteProcessGroup); - CompletableFuture nestedProcessGroupStopFuture = mock(CompletableFuture.class); - Future nestedRemoteProcessGroupStopFuture = mock(Future.class); + CompletableFuture nestedProcessGroupStopFuture = CompletableFuture.completedFuture(null); + CompletableFuture nestedRemoteProcessGroupStopFuture = CompletableFuture.completedFuture(null); when(flowManager.getRootGroup()).thenReturn(rootProcessGroup); when(rootProcessGroup.getRemoteProcessGroups()).thenReturn(remoteProcessGroups); diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml index 8308a723bdf6..def628fa54e5 100644 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -915,6 +915,12 @@ language governing permissions and limitations under the License. --> 2.9.0-SNAPSHOT nar + + org.apache.nifi + nifi-kafka-to-s3-nar + 2.9.0-SNAPSHOT + nar + org.aspectj diff --git a/nifi-commons/nifi-connector-utils/pom.xml b/nifi-commons/nifi-connector-utils/pom.xml new file mode 100644 index 000000000000..c44893483cdd --- /dev/null +++ b/nifi-commons/nifi-connector-utils/pom.xml @@ -0,0 +1,42 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-commons + 2.9.0-SNAPSHOT + + + nifi-connector-utils + nifi-connector-utils + + + + org.slf4j + slf4j-api + + + org.apache.nifi + nifi-api + + + com.fasterxml.jackson.core + jackson-databind + + + + diff --git a/nifi-commons/nifi-connector-utils/src/main/java/org/apache/nifi/components/connector/util/VersionedFlowUtils.java b/nifi-commons/nifi-connector-utils/src/main/java/org/apache/nifi/components/connector/util/VersionedFlowUtils.java new file mode 100644 index 000000000000..739280d16fa1 --- /dev/null +++ b/nifi-commons/nifi-connector-utils/src/main/java/org/apache/nifi/components/connector/util/VersionedFlowUtils.java @@ -0,0 +1,592 @@ +/* + * 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.components.connector.util; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.nifi.components.connector.ComponentBundleLookup; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.ComponentType; +import org.apache.nifi.flow.ConnectableComponent; +import org.apache.nifi.flow.ConnectableComponentType; +import org.apache.nifi.flow.PortType; +import org.apache.nifi.flow.Position; +import org.apache.nifi.flow.ScheduledState; +import org.apache.nifi.flow.VersionedConnection; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedParameter; +import org.apache.nifi.flow.VersionedParameterContext; +import org.apache.nifi.flow.VersionedPort; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; + +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.function.Predicate; + +public class VersionedFlowUtils { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + + public static VersionedExternalFlow loadFlowFromResource(final String resourceName) { + try (final InputStream in = Thread.currentThread().getContextClassLoader().getResourceAsStream(resourceName)) { + if (in == null) { + throw new IllegalArgumentException("Resource not found: " + resourceName); + } + + return OBJECT_MAPPER.readValue(in, VersionedExternalFlow.class); + } catch (final Exception e) { + throw new IllegalStateException("Unable to load resource: " + resourceName, e); + } + } + + public static Optional findProcessor(final VersionedProcessGroup group, final Predicate predicate) { + final List processors = findProcessors(group, predicate); + if (processors.size() == 1) { + return Optional.of(processors.getFirst()); + } + return Optional.empty(); + } + + public static List findProcessors(final VersionedProcessGroup group, final Predicate predicate) { + final List processors = new ArrayList<>(); + findProcessors(group, predicate, processors); + return processors; + } + + private static void findProcessors(final VersionedProcessGroup group, final Predicate predicate, final List processors) { + for (final VersionedProcessor processor : group.getProcessors()) { + if (predicate.test(processor)) { + processors.add(processor); + } + } + + for (final VersionedProcessGroup childGroup : group.getProcessGroups()) { + findProcessors(childGroup, predicate, processors); + } + } + + public static ConnectableComponent createConnectableComponent(final VersionedProcessor processor) { + final ConnectableComponent component = new ConnectableComponent(); + component.setId(processor.getIdentifier()); + component.setName(processor.getName()); + component.setType(ConnectableComponentType.PROCESSOR); + component.setGroupId(processor.getGroupIdentifier()); + return component; + } + + public static ConnectableComponent createConnectableComponent(final VersionedPort port) { + final ConnectableComponent component = new ConnectableComponent(); + component.setId(port.getIdentifier()); + component.setName(port.getName()); + component.setType(port.getComponentType() == ComponentType.INPUT_PORT ? ConnectableComponentType.INPUT_PORT : ConnectableComponentType.OUTPUT_PORT); + component.setGroupId(port.getGroupIdentifier()); + return component; + } + + public static VersionedConnection addConnection(final VersionedProcessGroup group, final ConnectableComponent source, final ConnectableComponent destination, + final Set relationships) { + final VersionedConnection connection = new VersionedConnection(); + connection.setSource(source); + connection.setDestination(destination); + connection.setSelectedRelationships(relationships); + connection.setBends(List.of()); + connection.setLabelIndex(0); + connection.setzIndex(0L); + connection.setGroupIdentifier(group.getIdentifier()); + connection.setLoadBalanceStrategy("DO_NOT_LOAD_BALANCE"); + connection.setBackPressureDataSizeThreshold("1 GB"); + connection.setBackPressureObjectThreshold(10000L); + connection.setFlowFileExpiration("0 sec"); + connection.setPrioritizers(new ArrayList<>()); + connection.setComponentType(ComponentType.CONNECTION); + + Set connections = group.getConnections(); + if (connections == null) { + connections = new HashSet<>(); + group.setConnections(connections); + } + connections.add(connection); + + final String uuid = generateDeterministicUuid(group, ComponentType.CONNECTION); + connection.setIdentifier(uuid); + return connection; + } + + public static List findOutboundConnections(final VersionedProcessGroup group, final VersionedProcessor processor) { + final VersionedProcessGroup processorGroup = findGroupForProcessor(group, processor); + if (processorGroup == null) { + return List.of(); + } + + final List outboundConnections = new ArrayList<>(); + final Set connections = processorGroup.getConnections(); + if (connections == null) { + return outboundConnections; + } + + for (final VersionedConnection connection : connections) { + final ConnectableComponent source = connection.getSource(); + if (Objects.equals(source.getId(), processor.getIdentifier()) && source.getType() == ConnectableComponentType.PROCESSOR) { + outboundConnections.add(connection); + } + } + + return outboundConnections; + } + + public static VersionedProcessGroup findGroupForProcessor(final VersionedProcessGroup rootGroup, final VersionedProcessor processor) { + if (rootGroup.getProcessors().contains(processor)) { + return rootGroup; + } + + for (final VersionedProcessGroup childGroup : rootGroup.getProcessGroups()) { + final VersionedProcessGroup foundGroup = findGroupForProcessor(childGroup, processor); + if (foundGroup != null) { + return foundGroup; + } + } + + return null; + } + + public static String generateDeterministicUuid(final VersionedProcessGroup group, final ComponentType componentType) { + final int componentCount = getComponentCount(group, componentType); + final String uuidSeed = "%s-%s-%d".formatted(group.getIdentifier(), componentType.name(), componentCount); + return UUID.nameUUIDFromBytes(uuidSeed.getBytes(StandardCharsets.UTF_8)).toString(); + } + + private static int getComponentCount(final VersionedProcessGroup group, final ComponentType componentType) { + final Collection components = switch (componentType) { + case PROCESSOR -> group.getProcessors(); + case INPUT_PORT -> group.getInputPorts(); + case OUTPUT_PORT -> group.getOutputPorts(); + case CONNECTION -> group.getConnections(); + case FUNNEL -> group.getFunnels(); + case LABEL -> group.getLabels(); + case PROCESS_GROUP -> group.getProcessGroups(); + case CONTROLLER_SERVICE -> group.getControllerServices(); + default -> List.of(); + }; + + return components == null ? 0 : components.size(); + } + + public static VersionedProcessor addProcessor(final VersionedProcessGroup group, final String processorType, final Bundle bundle, final String name, final Position position) { + final VersionedProcessor processor = new VersionedProcessor(); + + // Generate deterministic UUID based on group and component type + processor.setIdentifier(generateDeterministicUuid(group, ComponentType.PROCESSOR)); + + processor.setName(name); + processor.setType(processorType); + processor.setPosition(position); + processor.setBundle(bundle); + + // Set default processor configuration + processor.setProperties(new HashMap<>()); + processor.setPropertyDescriptors(new HashMap<>()); + processor.setStyle(new HashMap<>()); + processor.setSchedulingPeriod("0 sec"); + processor.setSchedulingStrategy("TIMER_DRIVEN"); + processor.setExecutionNode("ALL"); + processor.setPenaltyDuration("30 sec"); + processor.setYieldDuration("1 sec"); + processor.setBulletinLevel("WARN"); + processor.setRunDurationMillis(25L); + processor.setConcurrentlySchedulableTaskCount(1); + processor.setAutoTerminatedRelationships(new HashSet<>()); + processor.setScheduledState(ScheduledState.ENABLED); + processor.setRetryCount(10); + processor.setRetriedRelationships(new HashSet<>()); + processor.setBackoffMechanism("PENALIZE_FLOWFILE"); + processor.setMaxBackoffPeriod("10 mins"); + processor.setComponentType(ComponentType.PROCESSOR); + processor.setGroupIdentifier(group.getIdentifier()); + + group.getProcessors().add(processor); + return processor; + } + + public static VersionedControllerService addControllerService(final VersionedProcessGroup group, final String serviceType, final Bundle bundle, final String name) { + final VersionedControllerService controllerService = new VersionedControllerService(); + + // Generate deterministic UUID based on group and component type + controllerService.setIdentifier(generateDeterministicUuid(group, ComponentType.CONTROLLER_SERVICE)); + + controllerService.setName(name); + controllerService.setType(serviceType); + controllerService.setBundle(bundle); + controllerService.setComponentType(ComponentType.CONTROLLER_SERVICE); + + // Set default controller service configuration + controllerService.setProperties(new HashMap<>()); + controllerService.setPropertyDescriptors(new HashMap<>()); + controllerService.setControllerServiceApis(new ArrayList<>()); + controllerService.setAnnotationData(null); + controllerService.setScheduledState(ScheduledState.DISABLED); + controllerService.setBulletinLevel("WARN"); + controllerService.setComments(null); + controllerService.setGroupIdentifier(group.getIdentifier()); + + // Initialize controller services collection if it doesn't exist + Set controllerServices = group.getControllerServices(); + if (controllerServices == null) { + controllerServices = new HashSet<>(); + group.setControllerServices(controllerServices); + } + controllerServices.add(controllerService); + + return controllerService; + } + + public static VersionedProcessGroup createProcessGroup(final String identifier, final String name) { + final VersionedProcessGroup group = new VersionedProcessGroup(); + group.setIdentifier(identifier); + group.setName(name); + group.setProcessors(new HashSet<>()); + group.setProcessGroups(new HashSet<>()); + group.setConnections(new HashSet<>()); + group.setControllerServices(new HashSet<>()); + group.setInputPorts(new HashSet<>()); + group.setOutputPorts(new HashSet<>()); + group.setFunnels(new HashSet<>()); + group.setLabels(new HashSet<>()); + group.setComponentType(ComponentType.PROCESS_GROUP); + return group; + } + + public static VersionedPort addInputPort(final VersionedProcessGroup group, final String name, final Position position) { + return addPort(group, name, position, PortType.INPUT_PORT); + } + + public static VersionedPort addOutputPort(final VersionedProcessGroup group, final String name, final Position position) { + return addPort(group, name, position, PortType.OUTPUT_PORT); + } + + private static VersionedPort addPort(final VersionedProcessGroup group, final String name, final Position position, final PortType portType) { + final boolean isInput = portType == PortType.INPUT_PORT; + final ComponentType componentType = isInput ? ComponentType.INPUT_PORT : ComponentType.OUTPUT_PORT; + + final VersionedPort port = new VersionedPort(); + port.setIdentifier(generateDeterministicUuid(group, componentType)); + port.setName(name); + port.setPosition(position); + port.setType(portType); + port.setComponentType(componentType); + port.setScheduledState(ScheduledState.ENABLED); + port.setConcurrentlySchedulableTaskCount(1); + port.setAllowRemoteAccess(false); + port.setGroupIdentifier(group.getIdentifier()); + + if (isInput) { + Set inputPorts = group.getInputPorts(); + if (inputPorts == null) { + inputPorts = new HashSet<>(); + group.setInputPorts(inputPorts); + } + inputPorts.add(port); + } else { + Set outputPorts = group.getOutputPorts(); + if (outputPorts == null) { + outputPorts = new HashSet<>(); + group.setOutputPorts(outputPorts); + } + outputPorts.add(port); + } + + return port; + } + + public static Set getReferencedControllerServices(final VersionedProcessGroup group) { + final Set referencedServices = new HashSet<>(); + collectReferencedControllerServices(group, referencedServices); + return referencedServices; + } + + private static void collectReferencedControllerServices(final VersionedProcessGroup group, final Set referencedServices) { + final Map serviceMap = new HashMap<>(); + for (final VersionedControllerService service : group.getControllerServices()) { + serviceMap.put(service.getIdentifier(), service); + } + + for (final VersionedProcessor processor : group.getProcessors()) { + for (final String propertyValue : processor.getProperties().values()) { + final VersionedControllerService referencedService = serviceMap.get(propertyValue); + if (referencedService != null) { + referencedServices.add(referencedService); + } + } + } + + while (true) { + final Set newlyAddedServices = new HashSet<>(); + + for (final VersionedControllerService service : referencedServices) { + for (final String propertyValue : service.getProperties().values()) { + final VersionedControllerService referencedService = serviceMap.get(propertyValue); + if (referencedService != null && !referencedServices.contains(referencedService)) { + newlyAddedServices.add(referencedService); + } + } + } + + referencedServices.addAll(newlyAddedServices); + if (newlyAddedServices.isEmpty()) { + break; + } + } + + for (final VersionedProcessGroup childGroup : group.getProcessGroups()) { + collectReferencedControllerServices(childGroup, referencedServices); + } + } + + /** + * Returns the set of controller services that are transitively referenced by the given processor. + * This includes any services directly referenced by the processor's properties, as well as any services + * that those services reference, and so on. Only services that are accessible to the processor are considered, + * meaning services in the processor's own group and its ancestor groups. + * + * @param rootGroup the root process group to search for controller services + * @param processor the processor whose referenced services should be found + * @return the set of transitively referenced controller services + */ + public static Set getReferencedControllerServices(final VersionedProcessGroup rootGroup, final VersionedProcessor processor) { + return findTransitivelyReferencedServices(rootGroup, processor.getGroupIdentifier(), processor.getProperties()); + } + + /** + * Returns the set of controller services that are transitively referenced by the given controller service. + * This includes any services directly referenced by the service's properties, as well as any services + * that those services reference, and so on. Only services that are accessible to the given service are considered, + * meaning services in the service's own group and its ancestor groups. + * + * @param rootGroup the root process group to search for controller services + * @param controllerService the controller service whose referenced services should be found + * @return the set of transitively referenced controller services + */ + public static Set getReferencedControllerServices(final VersionedProcessGroup rootGroup, final VersionedControllerService controllerService) { + return findTransitivelyReferencedServices(rootGroup, controllerService.getGroupIdentifier(), controllerService.getProperties()); + } + + private static Set findTransitivelyReferencedServices(final VersionedProcessGroup rootGroup, final String componentGroupId, + final Map properties) { + final Map serviceMap = new HashMap<>(); + collectAccessibleControllerServices(rootGroup, componentGroupId, serviceMap); + + final Set referencedServices = new HashSet<>(); + for (final String propertyValue : properties.values()) { + final VersionedControllerService referencedService = serviceMap.get(propertyValue); + if (referencedService != null) { + referencedServices.add(referencedService); + } + } + + resolveTransitiveServiceReferences(referencedServices, serviceMap); + return referencedServices; + } + + /** + * Collects controller services that are accessible from the given target group. In NiFi, a component can reference + * controller services in its own group or any ancestor group. This method traverses from the root group down to the + * target group, collecting services from each group along the path. + * + * @param group the current group being examined + * @param targetGroupId the identifier of the group whose accessible services should be collected + * @param serviceMap the map to populate with accessible service identifiers and their corresponding services + * @return true if the target group was found at or beneath this group, false otherwise + */ + private static boolean collectAccessibleControllerServices(final VersionedProcessGroup group, final String targetGroupId, + final Map serviceMap) { + final boolean isTarget = group.getIdentifier().equals(targetGroupId); + + boolean foundInChild = false; + if (!isTarget) { + for (final VersionedProcessGroup childGroup : group.getProcessGroups()) { + if (collectAccessibleControllerServices(childGroup, targetGroupId, serviceMap)) { + foundInChild = true; + break; + } + } + } + + if (isTarget || foundInChild) { + for (final VersionedControllerService service : group.getControllerServices()) { + serviceMap.put(service.getIdentifier(), service); + } + return true; + } + + return false; + } + + private static void resolveTransitiveServiceReferences(final Set referencedServices, final Map serviceMap) { + while (true) { + final Set newlyAddedServices = new HashSet<>(); + + for (final VersionedControllerService service : referencedServices) { + for (final String propertyValue : service.getProperties().values()) { + final VersionedControllerService referencedService = serviceMap.get(propertyValue); + if (referencedService != null && !referencedServices.contains(referencedService)) { + newlyAddedServices.add(referencedService); + } + } + } + + referencedServices.addAll(newlyAddedServices); + if (newlyAddedServices.isEmpty()) { + break; + } + } + } + + public static void removeControllerServiceReferences(final VersionedProcessGroup processGroup, final String serviceIdentifier) { + for (final VersionedProcessor processor : processGroup.getProcessors()) { + removeValuesFromMap(processor.getProperties(), serviceIdentifier); + } + + for (final VersionedControllerService service : processGroup.getControllerServices()) { + removeValuesFromMap(service.getProperties(), serviceIdentifier); + } + + for (final VersionedProcessGroup childGroup : processGroup.getProcessGroups()) { + removeControllerServiceReferences(childGroup, serviceIdentifier); + } + } + + private static void removeValuesFromMap(final Map properties, final String valueToRemove) { + final List keysToRemove = new ArrayList<>(); + for (final Map.Entry entry : properties.entrySet()) { + if (Objects.equals(entry.getValue(), valueToRemove)) { + keysToRemove.add(entry.getKey()); + } + } + + keysToRemove.forEach(properties::remove); + } + + public static void setParameterValue(final VersionedExternalFlow externalFlow, final String parameterName, final String parameterValue) { + for (final VersionedParameterContext context : externalFlow.getParameterContexts().values()) { + setParameterValue(context, parameterName, parameterValue); + } + } + + public static void setParameterValue(final VersionedParameterContext parameterContext, final String parameterName, final String parameterValue) { + final Set parameters = parameterContext.getParameters(); + for (final VersionedParameter parameter : parameters) { + if (parameter.getName().equals(parameterName)) { + parameter.setValue(parameterValue); + } + } + } + + public static void setParameterValues(final VersionedExternalFlow externalFlow, final Map parameterValues) { + for (final Map.Entry entry : parameterValues.entrySet()) { + setParameterValue(externalFlow, entry.getKey(), entry.getValue()); + } + } + + public static void setParameterValues(final VersionedParameterContext parameterContext, final Map parameterValues) { + for (final Map.Entry entry : parameterValues.entrySet()) { + setParameterValue(parameterContext, entry.getKey(), entry.getValue()); + } + } + + public static void removeUnreferencedControllerServices(final VersionedProcessGroup processGroup) { + final Set referencedServices = getReferencedControllerServices(processGroup); + final Set referencedServiceIds = new HashSet<>(); + for (final VersionedControllerService service : referencedServices) { + referencedServiceIds.add(service.getIdentifier()); + } + + removeUnreferencedControllerServices(processGroup, referencedServiceIds); + } + + private static void removeUnreferencedControllerServices(final VersionedProcessGroup processGroup, final Set referencedServiceIds) { + processGroup.getControllerServices().removeIf(service -> !referencedServiceIds.contains(service.getIdentifier())); + + for (final VersionedProcessGroup childGroup : processGroup.getProcessGroups()) { + removeUnreferencedControllerServices(childGroup, referencedServiceIds); + } + } + + /** + * Updates all processors and controller services in the given process group (and its child groups) + * to use the latest available bundle version. See {@link ComponentBundleLookup#getLatestBundle(String)} for details on how + * version comparison is performed. + * + * @param processGroup the process group containing components to update + * @param componentBundleLookup the lookup used to find available bundles for each component type + */ + public static void updateToLatestBundles(final VersionedProcessGroup processGroup, final ComponentBundleLookup componentBundleLookup) { + for (final VersionedProcessor processor : processGroup.getProcessors()) { + updateToLatestBundle(processor, componentBundleLookup); + } + + for (final VersionedControllerService service : processGroup.getControllerServices()) { + updateToLatestBundle(service, componentBundleLookup); + } + + for (final VersionedProcessGroup childGroup : processGroup.getProcessGroups()) { + updateToLatestBundles(childGroup, componentBundleLookup); + } + } + + /** + * Updates the given processor to use the latest available bundle version. + * If no bundle is available, the processor's bundle is left unchanged. + * See {@link ComponentBundleLookup#getLatestBundle(String)} for details on how version comparison is performed. + * + * @param processor the processor to update + * @param componentBundleLookup the lookup used to find available bundles for the processor type + * @return true if the bundle was updated, false if no bundle was available + */ + public static boolean updateToLatestBundle(final VersionedProcessor processor, final ComponentBundleLookup componentBundleLookup) { + final Optional latestBundle = componentBundleLookup.getLatestBundle(processor.getType()); + latestBundle.ifPresent(processor::setBundle); + return latestBundle.isPresent(); + } + + /** + * Updates the given controller service to use the latest available bundle version. + * If no bundle is available, the service's bundle is left unchanged. + * See {@link ComponentBundleLookup#getLatestBundle(String)} for details on how version comparison is performed. + * + * @param service the controller service to update + * @param componentBundleLookup the lookup used to find available bundles for the service type + * @return true if the bundle was updated, false if no bundle was available + */ + public static boolean updateToLatestBundle(final VersionedControllerService service, final ComponentBundleLookup componentBundleLookup) { + final Optional latestBundle = componentBundleLookup.getLatestBundle(service.getType()); + latestBundle.ifPresent(service::setBundle); + return latestBundle.isPresent(); + } + +} diff --git a/nifi-commons/nifi-connector-utils/src/test/java/org/apache/nifi/components/connector/util/TestVersionedFlowUtils.java b/nifi-commons/nifi-connector-utils/src/test/java/org/apache/nifi/components/connector/util/TestVersionedFlowUtils.java new file mode 100644 index 000000000000..a7534d9088f5 --- /dev/null +++ b/nifi-commons/nifi-connector-utils/src/test/java/org/apache/nifi/components/connector/util/TestVersionedFlowUtils.java @@ -0,0 +1,285 @@ +/* + * 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.components.connector.util; + +import org.apache.nifi.components.connector.ComponentBundleLookup; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.Position; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; + +import java.util.Optional; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestVersionedFlowUtils { + + @Nested + class UpdateToLatestBundles { + private static final String PROCESSOR_TYPE = "org.apache.nifi.processors.TestProcessor"; + private static final String SERVICE_TYPE = "org.apache.nifi.services.TestService"; + + @Test + void testLookupReturnsNewerBundle() { + final VersionedProcessGroup group = createProcessGroup(); + final VersionedProcessor processor = VersionedFlowUtils.addProcessor(group, PROCESSOR_TYPE, new Bundle("group", "artifact", "2.0.0"), "Test Processor", new Position(0, 0)); + + final ComponentBundleLookup lookup = mock(ComponentBundleLookup.class); + when(lookup.getLatestBundle(PROCESSOR_TYPE)).thenReturn(Optional.of(new Bundle("group", "artifact", "3.0.0"))); + + VersionedFlowUtils.updateToLatestBundles(group, lookup); + + assertEquals("3.0.0", processor.getBundle().getVersion()); + } + + @Test + void testLookupReturnsLatestBundle() { + final VersionedProcessGroup group = createProcessGroup(); + final VersionedProcessor processor = VersionedFlowUtils.addProcessor(group, PROCESSOR_TYPE, new Bundle("group", "artifact", "2.0.0"), "Test Processor", new Position(0, 0)); + + final ComponentBundleLookup lookup = mock(ComponentBundleLookup.class); + when(lookup.getLatestBundle(PROCESSOR_TYPE)).thenReturn(Optional.of(new Bundle("group", "artifact", "4.0.0"))); + + VersionedFlowUtils.updateToLatestBundles(group, lookup); + + assertEquals("4.0.0", processor.getBundle().getVersion()); + } + + @Test + void testControllerServiceUpdatedToNewerBundle() { + final VersionedProcessGroup group = createProcessGroup(); + final VersionedControllerService service = VersionedFlowUtils.addControllerService(group, SERVICE_TYPE, new Bundle("group", "artifact", "1.5.0"), "Test Service"); + + final ComponentBundleLookup lookup = mock(ComponentBundleLookup.class); + when(lookup.getLatestBundle(SERVICE_TYPE)).thenReturn(Optional.of(new Bundle("group", "artifact", "2.5.0"))); + + VersionedFlowUtils.updateToLatestBundles(group, lookup); + + assertEquals("2.5.0", service.getBundle().getVersion()); + } + + @Test + void testNestedProcessorUpdatedToNewerBundle() { + final VersionedProcessGroup rootGroup = createProcessGroup(); + final VersionedProcessGroup childGroup = createChildProcessGroup(rootGroup, "child-group-id"); + final VersionedProcessor nestedProcessor = VersionedFlowUtils.addProcessor(childGroup, PROCESSOR_TYPE, new Bundle("group", "artifact", "1.0.0"), "Nested Processor", new Position(0, 0)); + + final ComponentBundleLookup lookup = mock(ComponentBundleLookup.class); + when(lookup.getLatestBundle(PROCESSOR_TYPE)).thenReturn(Optional.of(new Bundle("group", "artifact", "5.0.0"))); + + VersionedFlowUtils.updateToLatestBundles(rootGroup, lookup); + + assertEquals("5.0.0", nestedProcessor.getBundle().getVersion()); + } + + @Test + void testEmptyOptionalDoesNotChangeBundle() { + final VersionedProcessGroup group = createProcessGroup(); + final VersionedProcessor processor = VersionedFlowUtils.addProcessor(group, PROCESSOR_TYPE, new Bundle("group", "artifact", "2.0.0"), "Test Processor", new Position(0, 0)); + + final ComponentBundleLookup lookup = mock(ComponentBundleLookup.class); + when(lookup.getLatestBundle(PROCESSOR_TYPE)).thenReturn(Optional.empty()); + + VersionedFlowUtils.updateToLatestBundles(group, lookup); + + assertEquals("2.0.0", processor.getBundle().getVersion()); + } + + @Test + void testUpdateToLatestBundleReturnsTrueWhenUpdated() { + final VersionedProcessGroup group = createProcessGroup(); + final VersionedProcessor processor = VersionedFlowUtils.addProcessor(group, PROCESSOR_TYPE, new Bundle("group", "artifact", "2.0.0"), "Test Processor", new Position(0, 0)); + + final ComponentBundleLookup lookup = mock(ComponentBundleLookup.class); + when(lookup.getLatestBundle(PROCESSOR_TYPE)).thenReturn(Optional.of(new Bundle("group", "artifact", "3.0.0"))); + + final boolean updated = VersionedFlowUtils.updateToLatestBundle(processor, lookup); + + assertTrue(updated); + assertEquals("3.0.0", processor.getBundle().getVersion()); + } + + @Test + void testUpdateToLatestBundleReturnsFalseWhenNotUpdated() { + final VersionedProcessGroup group = createProcessGroup(); + final VersionedProcessor processor = VersionedFlowUtils.addProcessor(group, PROCESSOR_TYPE, new Bundle("group", "artifact", "2.0.0"), "Test Processor", new Position(0, 0)); + + final ComponentBundleLookup lookup = mock(ComponentBundleLookup.class); + when(lookup.getLatestBundle(PROCESSOR_TYPE)).thenReturn(Optional.empty()); + + final boolean updated = VersionedFlowUtils.updateToLatestBundle(processor, lookup); + + assertFalse(updated); + assertEquals("2.0.0", processor.getBundle().getVersion()); + } + + } + + @Nested + class GetReferencedControllerServices { + private static final String PROCESSOR_TYPE = "org.apache.nifi.processors.TestProcessor"; + private static final String SERVICE_TYPE = "org.apache.nifi.services.TestService"; + private static final Bundle TEST_BUNDLE = new Bundle("group", "artifact", "1.0.0"); + + @Test + void testProcessorWithNoReferences() { + final VersionedProcessGroup group = createProcessGroup(); + VersionedFlowUtils.addControllerService(group, SERVICE_TYPE, TEST_BUNDLE, "Unreferenced Service"); + final VersionedProcessor processor = VersionedFlowUtils.addProcessor(group, PROCESSOR_TYPE, TEST_BUNDLE, "Processor", new Position(0, 0)); + processor.getProperties().put("some-property", "not-a-service-id"); + + final Set result = VersionedFlowUtils.getReferencedControllerServices(group, processor); + + assertTrue(result.isEmpty()); + } + + @Test + void testProcessorWithDirectReferences() { + final VersionedProcessGroup group = createProcessGroup(); + final VersionedControllerService service1 = VersionedFlowUtils.addControllerService(group, SERVICE_TYPE, TEST_BUNDLE, "Service 1"); + final VersionedControllerService service2 = VersionedFlowUtils.addControllerService(group, SERVICE_TYPE, TEST_BUNDLE, "Service 2"); + final VersionedControllerService unreferencedService = VersionedFlowUtils.addControllerService(group, SERVICE_TYPE, TEST_BUNDLE, "Unreferenced Service"); + + final VersionedProcessor processor = VersionedFlowUtils.addProcessor(group, PROCESSOR_TYPE, TEST_BUNDLE, "Processor", new Position(0, 0)); + processor.getProperties().put("service-prop-1", service1.getIdentifier()); + processor.getProperties().put("service-prop-2", service2.getIdentifier()); + processor.getProperties().put("non-service-prop", "some-value"); + + final Set result = VersionedFlowUtils.getReferencedControllerServices(group, processor); + + assertEquals(2, result.size()); + assertTrue(result.contains(service1)); + assertTrue(result.contains(service2)); + assertFalse(result.contains(unreferencedService)); + } + + @Test + void testProcessorWithTransitiveReferences() { + final VersionedProcessGroup group = createProcessGroup(); + final VersionedControllerService serviceC = VersionedFlowUtils.addControllerService(group, SERVICE_TYPE, TEST_BUNDLE, "Service C"); + final VersionedControllerService serviceB = VersionedFlowUtils.addControllerService(group, SERVICE_TYPE, TEST_BUNDLE, "Service B"); + serviceB.getProperties().put("nested-service", serviceC.getIdentifier()); + final VersionedControllerService serviceA = VersionedFlowUtils.addControllerService(group, SERVICE_TYPE, TEST_BUNDLE, "Service A"); + serviceA.getProperties().put("nested-service", serviceB.getIdentifier()); + + final VersionedProcessor processor = VersionedFlowUtils.addProcessor(group, PROCESSOR_TYPE, TEST_BUNDLE, "Processor", new Position(0, 0)); + processor.getProperties().put("service-prop", serviceA.getIdentifier()); + + final Set result = VersionedFlowUtils.getReferencedControllerServices(group, processor); + + assertEquals(3, result.size()); + assertTrue(result.contains(serviceA)); + assertTrue(result.contains(serviceB)); + assertTrue(result.contains(serviceC)); + } + + @Test + void testProcessorReferencingServiceInAncestorGroup() { + final VersionedProcessGroup rootGroup = createProcessGroup(); + final VersionedControllerService parentService = VersionedFlowUtils.addControllerService(rootGroup, SERVICE_TYPE, TEST_BUNDLE, "Parent Service"); + final VersionedProcessGroup childGroup = createChildProcessGroup(rootGroup, "child-group-id"); + final VersionedProcessor processor = VersionedFlowUtils.addProcessor(childGroup, PROCESSOR_TYPE, TEST_BUNDLE, "Processor", new Position(0, 0)); + processor.getProperties().put("service-prop", parentService.getIdentifier()); + + final Set result = VersionedFlowUtils.getReferencedControllerServices(rootGroup, processor); + + assertEquals(1, result.size()); + assertTrue(result.contains(parentService)); + } + + @Test + void testProcessorDoesNotFindServiceInDescendantGroup() { + final VersionedProcessGroup rootGroup = createProcessGroup(); + final VersionedProcessGroup childGroup = createChildProcessGroup(rootGroup, "child-group-id"); + final VersionedControllerService childService = VersionedFlowUtils.addControllerService(childGroup, SERVICE_TYPE, TEST_BUNDLE, "Child Service"); + + final VersionedProcessor processor = VersionedFlowUtils.addProcessor(rootGroup, PROCESSOR_TYPE, TEST_BUNDLE, "Processor", new Position(0, 0)); + processor.getProperties().put("service-prop", childService.getIdentifier()); + + final Set result = VersionedFlowUtils.getReferencedControllerServices(rootGroup, processor); + + assertTrue(result.isEmpty()); + } + + @Test + void testProcessorWithCircularServiceReferences() { + final VersionedProcessGroup group = createProcessGroup(); + final VersionedControllerService serviceA = VersionedFlowUtils.addControllerService(group, SERVICE_TYPE, TEST_BUNDLE, "Service A"); + final VersionedControllerService serviceB = VersionedFlowUtils.addControllerService(group, SERVICE_TYPE, TEST_BUNDLE, "Service B"); + serviceA.getProperties().put("nested-service", serviceB.getIdentifier()); + serviceB.getProperties().put("nested-service", serviceA.getIdentifier()); + + final VersionedProcessor processor = VersionedFlowUtils.addProcessor(group, PROCESSOR_TYPE, TEST_BUNDLE, "Processor", new Position(0, 0)); + processor.getProperties().put("service-prop", serviceA.getIdentifier()); + + final Set result = VersionedFlowUtils.getReferencedControllerServices(group, processor); + + assertEquals(2, result.size()); + assertTrue(result.contains(serviceA)); + assertTrue(result.contains(serviceB)); + } + + @Test + void testControllerServiceWithNoReferences() { + final VersionedProcessGroup group = createProcessGroup(); + final VersionedControllerService service = VersionedFlowUtils.addControllerService(group, SERVICE_TYPE, TEST_BUNDLE, "Service"); + service.getProperties().put("some-property", "not-a-service-id"); + + final Set result = VersionedFlowUtils.getReferencedControllerServices(group, service); + + assertTrue(result.isEmpty()); + } + + @Test + void testControllerServiceWithDirectAndTransitiveReferences() { + final VersionedProcessGroup group = createProcessGroup(); + final VersionedControllerService serviceC = VersionedFlowUtils.addControllerService(group, SERVICE_TYPE, TEST_BUNDLE, "Service C"); + final VersionedControllerService serviceB = VersionedFlowUtils.addControllerService(group, SERVICE_TYPE, TEST_BUNDLE, "Service B"); + serviceB.getProperties().put("nested-service", serviceC.getIdentifier()); + final VersionedControllerService serviceA = VersionedFlowUtils.addControllerService(group, SERVICE_TYPE, TEST_BUNDLE, "Service A"); + serviceA.getProperties().put("nested-service", serviceB.getIdentifier()); + + final Set result = VersionedFlowUtils.getReferencedControllerServices(group, serviceA); + + assertEquals(2, result.size()); + assertTrue(result.contains(serviceB)); + assertTrue(result.contains(serviceC)); + assertFalse(result.contains(serviceA)); + } + } + + private static VersionedProcessGroup createProcessGroup() { + return VersionedFlowUtils.createProcessGroup("test-group-id", "Test Process Group"); + } + + private static VersionedProcessGroup createChildProcessGroup(final VersionedProcessGroup parent, final String identifier) { + final VersionedProcessGroup childGroup = VersionedFlowUtils.createProcessGroup(identifier, "Child Process Group"); + childGroup.setGroupIdentifier(parent.getIdentifier()); + parent.getProcessGroups().add(childGroup); + return childGroup; + } + +} diff --git a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java index 0d6a8412e658..eef06942f013 100644 --- a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java +++ b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java @@ -124,6 +124,9 @@ public class NiFiProperties extends ApplicationProperties { public static final String ASSET_MANAGER_IMPLEMENTATION = "nifi.asset.manager.implementation"; public static final String ASSET_MANAGER_PREFIX = "nifi.asset.manager.properties."; + public static final String CONNECTOR_ASSET_MANAGER_IMPLEMENTATION = "nifi.connector.asset.manager.implementation"; + public static final String CONNECTOR_ASSET_MANAGER_PREFIX = "nifi.connector.asset.manager.properties."; + // status repository properties public static final String COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION = "nifi.components.status.repository.implementation"; public static final String COMPONENT_STATUS_SNAPSHOT_FREQUENCY = "nifi.components.status.snapshot.frequency"; @@ -136,6 +139,16 @@ public class NiFiProperties extends ApplicationProperties { public static final String STATUS_REPOSITORY_QUESTDB_PERSIST_BATCH_SIZE = "nifi.status.repository.questdb.persist.batchsize"; public static final String STATUS_REPOSITORY_QUESTDB_PERSIST_FREQUENCY = "nifi.status.repository.questdb.persist.frequency"; + // Connector Repository properties + public static final String CONNECTOR_REPOSITORY_IMPLEMENTATION = "nifi.components.connectors.repository.implementation"; + + // Connector Configuration Provider properties + public static final String CONNECTOR_CONFIGURATION_PROVIDER_IMPLEMENTATION = "nifi.components.connectors.configuration.provider.implementation"; + public static final String CONNECTOR_CONFIGURATION_PROVIDER_PROPERTIES_PREFIX = "nifi.components.connectors.configuration.provider."; + + // Secrets Manager properties + public static final String SECRETS_MANAGER_IMPLEMENTATION = "nifi.secrets.manager.implementation"; + // security properties public static final String SECURITY_KEYSTORE = "nifi.security.keystore"; public static final String SECURITY_KEYSTORE_TYPE = "nifi.security.keystoreType"; @@ -1807,8 +1820,28 @@ public static NiFiProperties createBasicNiFiProperties(final String propertiesFi // The Properties(Properties) constructor does NOT inherit the provided values, just uses them as default values if (additionalProperties != null) { - additionalProperties.forEach(properties::put); + properties.putAll(additionalProperties); } + + return createNiFiProperties(properties); + } + + public static NiFiProperties createBasicNiFiProperties(final InputStream inputStream) { + return createBasicNiFiProperties(inputStream, null); + } + + public static NiFiProperties createBasicNiFiProperties(final InputStream inputStream, final Properties additionalProperties) { + final Properties properties = new Properties(); + readFromInputStream(inputStream, properties); + + if (additionalProperties != null) { + properties.putAll(additionalProperties); + } + + return createNiFiProperties(properties); + } + + private static NiFiProperties createNiFiProperties(final Properties properties) { return new NiFiProperties() { @Override public String getProperty(String key) { @@ -1828,9 +1861,7 @@ public int size() { } private static void readFromPropertiesFile(String propertiesFilePath, Properties properties) { - final String nfPropertiesFilePath = (propertiesFilePath == null) - ? System.getProperty(PROPERTIES_FILE_PATH) - : propertiesFilePath; + final String nfPropertiesFilePath = (propertiesFilePath == null) ? System.getProperty(PROPERTIES_FILE_PATH) : propertiesFilePath; if (nfPropertiesFilePath != null) { final File propertiesFile = new File(nfPropertiesFilePath.trim()); if (!propertiesFile.exists()) { @@ -1841,24 +1872,25 @@ private static void readFromPropertiesFile(String propertiesFilePath, Properties throw new RuntimeException("Properties file exists but cannot be read '" + propertiesFile.getAbsolutePath() + "'"); } - InputStream inStream = null; - try { - inStream = new BufferedInputStream(new FileInputStream(propertiesFile)); + + try (final InputStream fileIn = new FileInputStream(propertiesFile); + final InputStream inStream = new BufferedInputStream(fileIn)) { + properties.load(inStream); } catch (final Exception ex) { - throw new RuntimeException("Cannot load properties file due to " - + ex.getLocalizedMessage(), ex); - } finally { - if (null != inStream) { - try { - inStream.close(); - } catch (final Exception ignored) { - } - } + throw new RuntimeException("Cannot load properties file due to " + ex.getLocalizedMessage(), ex); } } } + private static void readFromInputStream(final InputStream inputStream, final Properties properties) { + try { + properties.load(inputStream); + } catch (final Exception ex) { + throw new RuntimeException("Cannot load properties file due to " + ex.getLocalizedMessage(), ex); + } + } + /** * This method is used to validate the NiFi properties when the file is loaded * for the first time. The objective is to stop NiFi startup in case a property diff --git a/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/HashMapSnapshot.java b/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/HashMapSnapshot.java index 4c768ed76c8d..b6d7682ae71a 100644 --- a/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/HashMapSnapshot.java +++ b/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/HashMapSnapshot.java @@ -175,14 +175,18 @@ public void update(final Collection records) { switch (updateType) { case DELETE: - recordMap.remove(recordId); + if (recordId != null) { + recordMap.remove(recordId); + } break; case SWAP_OUT: final String location = serdeFactory.getLocation(record); if (location == null) { logger.error(logMessage, recordId, UpdateType.SWAP_OUT, "Swapped Out to", "lost"); } else { - recordMap.remove(recordId); + if (recordId != null) { + recordMap.remove(recordId); + } this.swapLocations.add(location); } break; @@ -193,10 +197,30 @@ public void update(final Collection records) { } else { swapLocations.remove(swapLocation); } - recordMap.put(recordId, record); + if (recordId != null) { + recordMap.put(recordId, record); + } + break; + case SWAP_FILE_DELETED: + final String deletedSwapLocation = serdeFactory.getLocation(record); + if (deletedSwapLocation != null) { + swapLocations.remove(deletedSwapLocation); + } + break; + case SWAP_FILE_RENAMED: + final String originalLocation = serdeFactory.getOriginalLocation(record); + final String newLocation = serdeFactory.getLocation(record); + if (originalLocation != null) { + swapLocations.remove(originalLocation); + } + if (newLocation != null) { + swapLocations.add(newLocation); + } break; default: - recordMap.put(recordId, record); + if (recordId != null) { + recordMap.put(recordId, record); + } break; } } @@ -209,6 +233,9 @@ public int getRecordCount() { @Override public T lookup(final Object recordId) { + if (recordId == null) { + return null; + } return recordMap.get(recordId); } diff --git a/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/LengthDelimitedJournal.java b/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/LengthDelimitedJournal.java index 36350d0c8e14..9b9428539e4b 100644 --- a/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/LengthDelimitedJournal.java +++ b/nifi-commons/nifi-write-ahead-log/src/main/java/org/apache/nifi/wali/LengthDelimitedJournal.java @@ -475,8 +475,10 @@ public JournalRecovery recoverRecords(final Map recordMap, final Set< switch (updateType) { case DELETE: { - idsRemoved.add(recordId); - transactionRecordMap.remove(recordId); + if (recordId != null) { + idsRemoved.add(recordId); + transactionRecordMap.remove(recordId); + } break; } case SWAP_IN: { @@ -486,7 +488,9 @@ public JournalRecovery recoverRecords(final Map recordMap, final Set< } else { swapLocationsRemoved.add(location); swapLocationsAdded.remove(location); - transactionRecordMap.put(recordId, record); + if (recordId != null) { + transactionRecordMap.put(recordId, record); + } } break; } @@ -497,15 +501,40 @@ public JournalRecovery recoverRecords(final Map recordMap, final Set< } else { swapLocationsRemoved.remove(location); swapLocationsAdded.add(location); - idsRemoved.add(recordId); - transactionRecordMap.remove(recordId); + if (recordId != null) { + idsRemoved.add(recordId); + transactionRecordMap.remove(recordId); + } } break; } + case SWAP_FILE_DELETED: { + final String location = serde.getLocation(record); + if (location != null) { + swapLocationsRemoved.add(location); + swapLocationsAdded.remove(location); + } + break; + } + case SWAP_FILE_RENAMED: { + final String originalLocation = serde.getOriginalLocation(record); + final String newLocation = serde.getLocation(record); + if (originalLocation != null) { + swapLocationsRemoved.add(originalLocation); + swapLocationsAdded.remove(originalLocation); + } + if (newLocation != null) { + swapLocationsAdded.add(newLocation); + swapLocationsRemoved.remove(newLocation); + } + break; + } default: { - transactionRecordMap.put(recordId, record); - idsRemoved.remove(recordId); + if (recordId != null) { + transactionRecordMap.put(recordId, record); + idsRemoved.remove(recordId); + } break; } } diff --git a/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/SerDe.java b/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/SerDe.java index 356cf8457338..990484bdf967 100644 --- a/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/SerDe.java +++ b/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/SerDe.java @@ -136,6 +136,19 @@ default void readHeader(DataInputStream in) throws IOException { */ String getLocation(T record); + /** + * Returns the original external location of the given record; this is used when a + * swap file is renamed. For {@link UpdateType#SWAP_FILE_RENAMED} records, this + * returns the original location before the rename, while {@link #getLocation(Object)} + * returns the new location after the rename. + * + * @param record to get original location of + * @return original location, or null if not applicable + */ + default String getOriginalLocation(T record) { + return null; + } + /** * Returns the version that this SerDe will use when writing. This used used * when serializing/deserializing the edit logs so that if the version diff --git a/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/SerDeFactory.java b/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/SerDeFactory.java index 09e6f7b50c19..26a177ece88e 100644 --- a/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/SerDeFactory.java +++ b/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/SerDeFactory.java @@ -57,4 +57,17 @@ public interface SerDeFactory { * @return location */ String getLocation(T record); + + /** + * Returns the original external location of the given record; this is used when a + * swap file is renamed. For {@link UpdateType#SWAP_FILE_RENAMED} records, this + * returns the original location before the rename, while {@link #getLocation(Object)} + * returns the new location after the rename. + * + * @param record to get original location of + * @return original location, or null if not applicable + */ + default String getOriginalLocation(T record) { + return null; + } } diff --git a/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/UpdateType.java b/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/UpdateType.java index 1b039f82fcbb..0d383cbe3776 100644 --- a/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/UpdateType.java +++ b/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/UpdateType.java @@ -45,5 +45,13 @@ public enum UpdateType { * Used to indicate that a Record that was previously Swapped Out is now * being Swapped In */ - SWAP_IN; + SWAP_IN, + /** + * Used to indicate that a Swap File has been deleted + */ + SWAP_FILE_DELETED, + /** + * Used to indicate that a Swap File has been renamed + */ + SWAP_FILE_RENAMED; } diff --git a/nifi-commons/pom.xml b/nifi-commons/pom.xml index eba472df95d9..488b7ef209aa 100644 --- a/nifi-commons/pom.xml +++ b/nifi-commons/pom.xml @@ -68,6 +68,7 @@ nifi-write-ahead-log nifi-xml-processing nifi-calcite-utils + nifi-connector-utils 33.5.0-jre diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-api/pom.xml b/nifi-connector-mock-bundle/nifi-connector-mock-api/pom.xml new file mode 100644 index 000000000000..d43dffe3192f --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-api/pom.xml @@ -0,0 +1,35 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-connector-mock-bundle + 2.9.0-SNAPSHOT + + + nifi-connector-mock-api + jar + + + + org.apache.nifi + nifi-server-api + 2.9.0-SNAPSHOT + + + + \ No newline at end of file diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-api/src/main/java/org/apache/nifi/mock/connector/server/ConnectorConfigVerificationResult.java b/nifi-connector-mock-bundle/nifi-connector-mock-api/src/main/java/org/apache/nifi/mock/connector/server/ConnectorConfigVerificationResult.java new file mode 100644 index 000000000000..9ec3ac2878cd --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-api/src/main/java/org/apache/nifi/mock/connector/server/ConnectorConfigVerificationResult.java @@ -0,0 +1,31 @@ +/* + * 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.mock.connector.server; + +import org.apache.nifi.components.ConfigVerificationResult; + +import java.util.List; + +public interface ConnectorConfigVerificationResult { + + List getAllResults(); + + List getFailedResults(); + + void assertNoFailures(); +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-api/src/main/java/org/apache/nifi/mock/connector/server/ConnectorMockServer.java b/nifi-connector-mock-bundle/nifi-connector-mock-api/src/main/java/org/apache/nifi/mock/connector/server/ConnectorMockServer.java new file mode 100644 index 000000000000..ade061a9d479 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-api/src/main/java/org/apache/nifi/mock/connector/server/ConnectorMockServer.java @@ -0,0 +1,36 @@ +/* + * 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.mock.connector.server; + +import org.apache.nifi.NiFiServer; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.processor.Processor; + +import java.io.File; + +public interface ConnectorMockServer extends NiFiServer, ConnectorTestRunner { + + void instantiateConnector(String connectorClassName); + + void registerMockBundle(ClassLoader classLoader, File workingDirectory); + + void mockProcessor(String processorType, Class mockProcessorClass); + + void mockControllerService(String controllerServiceType, Class mockControllerServiceClass); + +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-api/src/main/java/org/apache/nifi/mock/connector/server/ConnectorTestRunner.java b/nifi-connector-mock-bundle/nifi-connector-mock-api/src/main/java/org/apache/nifi/mock/connector/server/ConnectorTestRunner.java new file mode 100644 index 000000000000..669f7be22d26 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-api/src/main/java/org/apache/nifi/mock/connector/server/ConnectorTestRunner.java @@ -0,0 +1,103 @@ +/* + * 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.mock.connector.server; + +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.AssetReference; +import org.apache.nifi.components.connector.ConnectorValueReference; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.SecretReference; +import org.apache.nifi.components.connector.StepConfiguration; +import org.apache.nifi.flow.VersionedExternalFlow; + +import java.io.Closeable; +import java.io.File; +import java.io.InputStream; +import java.time.Duration; +import java.util.List; +import java.util.Map; + +public interface ConnectorTestRunner extends Closeable { + String SECRET_PROVIDER_ID = "TestRunnerSecretsManager"; + String SECRET_PROVIDER_NAME = "TestRunnerSecretsManager"; + + void applyUpdate() throws FlowUpdateException; + + void configure(String stepName, StepConfiguration configuration) throws FlowUpdateException; + + void configure(String stepName, Map propertyValues) throws FlowUpdateException; + + void configure(String stepName, Map propertyValues, Map propertyReferences) throws FlowUpdateException; + + SecretReference createSecretReference(String secretName); + + ConnectorConfigVerificationResult verifyConfiguration(String stepName, Map propertyValueOverrides); + + ConnectorConfigVerificationResult verifyConfiguration(String stepName, Map propertyValueOverrides, Map referenceOverrides); + + ConnectorConfigVerificationResult verifyConfiguration(String stepName, StepConfiguration configurationOverrides); + + void addSecret(String name, String value); + + AssetReference addAsset(File file); + + AssetReference addAsset(String assetName, InputStream contents); + + void startConnector(); + + void stopConnector(); + + void waitForDataIngested(Duration maxWaitTime); + + void waitForIdle(Duration maxWaitTime); + + void waitForIdle(Duration minimumIdleTime, Duration maxWaitTime); + + List validate(); + + /** + * Returns the HTTP port on which the embedded Jetty server is listening, or -1 if no server is running. + * + * @return the HTTP port, or -1 if not applicable + */ + default int getHttpPort() { + return -1; + } + + List fetchAllowableValues(String stepName, String propertyName); + + /* + * Returns a {@link VersionedExternalFlow} representing the current state of the Active Flow Context. + * The Active Flow Context is the flow that is currently running (or most recently ran) in the Connector. + * This is useful for making assertions about how the flow is configured after updates have been applied. + * + * @return the VersionedExternalFlow for the Active Flow Context + */ + VersionedExternalFlow getActiveFlowSnapshot(); + + /** + * Returns a {@link VersionedExternalFlow} representing the current state of the Working Flow Context. + * The Working Flow Context is the flow that reflects configuration changes that have been made + * but not yet applied. This is useful for making assertions about how the flow will be configured + * once the update is applied. + * + * @return the VersionedExternalFlow for the Working Flow Context + */ + VersionedExternalFlow getWorkingFlowSnapshot(); +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server-nar/pom.xml b/nifi-connector-mock-bundle/nifi-connector-mock-server-nar/pom.xml new file mode 100644 index 000000000000..4b970dc4867a --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server-nar/pom.xml @@ -0,0 +1,55 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-connector-mock-bundle + 2.9.0-SNAPSHOT + + + nifi-connector-mock-server-nar + nar + + + + org.apache.nifi + nifi-connector-mock-server + 2.9.0-SNAPSHOT + + + org.apache.nifi + nifi-framework-nar + 2.9.0-SNAPSHOT + nar + + + + + + + + shibboleth + https://build.shibboleth.net/nexus/content/repositories/releases/ + + true + + + false + + + + \ No newline at end of file diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/pom.xml b/nifi-connector-mock-bundle/nifi-connector-mock-server/pom.xml new file mode 100644 index 000000000000..bcfb88f3e5fd --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/pom.xml @@ -0,0 +1,80 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-connector-mock-bundle + 2.9.0-SNAPSHOT + + + nifi-connector-mock-server + jar + + + + org.slf4j + slf4j-api + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-framework-core-api + 2.9.0-SNAPSHOT + provided + + + org.apache.nifi + nifi-framework-core + 2.9.0-SNAPSHOT + provided + + + org.apache.nifi + nifi-property-encryptor + 2.9.0-SNAPSHOT + + + org.apache.nifi + nifi-connector-mock-api + 2.9.0-SNAPSHOT + + + org.apache.nifi + nifi-server-api + 2.9.0-SNAPSHOT + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty.ee11 + jetty-ee11-webapp + + + + + org.junit.jupiter + junit-jupiter + test + + + + \ No newline at end of file diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockAuditService.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockAuditService.java new file mode 100644 index 000000000000..3aa7bbc543e3 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockAuditService.java @@ -0,0 +1,63 @@ +/* + * 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.mock.connector.server; + +import org.apache.nifi.action.Action; +import org.apache.nifi.admin.service.AuditService; +import org.apache.nifi.history.History; +import org.apache.nifi.history.HistoryQuery; +import org.apache.nifi.history.PreviousValue; + +import java.util.Collection; +import java.util.Date; +import java.util.List; +import java.util.Map; + +public class MockAuditService implements AuditService { + @Override + public void addActions(final Collection actions) { + } + + @Override + public Map> getPreviousValues(final String componentId) { + return Map.of(); + } + + @Override + public void deletePreviousValues(final String propertyName, final String componentId) { + } + + @Override + public History getActions(final HistoryQuery actionQuery) { + return null; + } + + @Override + public History getActions(final int firstActionId, final int maxActions) { + return null; + } + + @Override + public Action getAction(final Integer actionId) { + return null; + } + + @Override + public void purgeActions(final Date end, final Action purgeAction) { + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockConnectorAssetManager.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockConnectorAssetManager.java new file mode 100644 index 000000000000..92892f6275a9 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockConnectorAssetManager.java @@ -0,0 +1,179 @@ +/* + * 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.mock.connector.server; + +import org.apache.nifi.asset.Asset; +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.asset.AssetManagerInitializationContext; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardCopyOption; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; + +/** + * A Mock implementation of AssetManager for use in ConnectorTestRunner tests. + * This implementation stores assets in a temporary directory structure. + */ +public class MockConnectorAssetManager implements AssetManager { + + private static final String ASSET_STORAGE_LOCATION_PROPERTY = "directory"; + private static final String DEFAULT_ASSET_STORAGE_LOCATION = "target/mock_connector_assets"; + + private final Map assets = new ConcurrentHashMap<>(); + private volatile File assetStorageLocation; + + @Override + public void initialize(final AssetManagerInitializationContext context) { + final String storageLocation = context.getProperties().getOrDefault(ASSET_STORAGE_LOCATION_PROPERTY, DEFAULT_ASSET_STORAGE_LOCATION); + assetStorageLocation = new File(storageLocation); + + if (!assetStorageLocation.exists()) { + try { + Files.createDirectories(assetStorageLocation.toPath()); + } catch (final IOException e) { + throw new RuntimeException("Failed to create asset storage directory: " + storageLocation, e); + } + } + } + + @Override + public Asset createAsset(final String ownerId, final String assetName, final InputStream contents) throws IOException { + final String assetId = UUID.randomUUID().toString(); + return saveAsset(ownerId, assetId, assetName, contents); + } + + @Override + public Asset saveAsset(final String ownerId, final String assetId, final String assetName, final InputStream contents) throws IOException { + final File assetFile = getFile(ownerId, assetId, assetName); + final File parentDir = assetFile.getParentFile(); + + if (!parentDir.exists()) { + Files.createDirectories(parentDir.toPath()); + } + + Files.copy(contents, assetFile.toPath(), StandardCopyOption.REPLACE_EXISTING); + + final Asset asset = new MockAsset(assetId, ownerId, assetName, assetFile, null); + assets.put(assetId, asset); + return asset; + } + + @Override + public Optional getAsset(final String id) { + return Optional.ofNullable(assets.get(id)); + } + + @Override + public List getAssets(final String ownerId) { + final List ownerAssets = new ArrayList<>(); + for (final Asset asset : assets.values()) { + if (asset.getOwnerIdentifier().equals(ownerId)) { + ownerAssets.add(asset); + } + } + return ownerAssets; + } + + @Override + public Asset createMissingAsset(final String ownerId, final String assetName) { + final String assetId = UUID.randomUUID().toString(); + final File file = getFile(ownerId, assetId, assetName); + final Asset asset = new MockAsset(assetId, ownerId, assetName, file, null); + assets.put(assetId, asset); + return asset; + } + + @Override + public Optional deleteAsset(final String id) { + final Asset removed = assets.remove(id); + if (removed != null && removed.getFile().exists()) { + try { + Files.delete(removed.getFile().toPath()); + } catch (final IOException e) { + throw new RuntimeException("Failed to delete asset " + id + " from storage file " + removed.getFile().getAbsolutePath(), e); + } + } + + return Optional.ofNullable(removed); + } + + private File getFile(final String ownerId, final String assetId, final String assetName) { + final Path parentPath = assetStorageLocation.toPath().normalize(); + final Path assetPath = Path.of(ownerId, assetId, assetName).normalize(); + return parentPath.resolve(assetPath).toFile(); + } + + /** + * A simple Asset implementation for the mock. + */ + private static class MockAsset implements Asset { + private final String identifier; + private final String ownerIdentifier; + private final String name; + private final File file; + private final String digest; + + MockAsset(final String identifier, final String ownerIdentifier, final String name, final File file, final String digest) { + this.identifier = identifier; + this.ownerIdentifier = ownerIdentifier; + this.name = name; + this.file = file; + this.digest = digest; + } + + @Override + public String getIdentifier() { + return identifier; + } + + @Override + @Deprecated + public String getParameterContextIdentifier() { + return ownerIdentifier; + } + + @Override + public String getOwnerIdentifier() { + return ownerIdentifier; + } + + @Override + public String getName() { + return name; + } + + @Override + public File getFile() { + return file; + } + + @Override + public Optional getDigest() { + return Optional.ofNullable(digest); + } + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockConnectorInitializationContext.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockConnectorInitializationContext.java new file mode 100644 index 000000000000..02d276bc28dd --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockConnectorInitializationContext.java @@ -0,0 +1,78 @@ +/* + * 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.mock.connector.server; + +import org.apache.nifi.components.connector.BundleCompatibility; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.StandardConnectorInitializationContext; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; + +public class MockConnectorInitializationContext extends StandardConnectorInitializationContext { + + private final MockExtensionMapper mockExtensionMapper; + + protected MockConnectorInitializationContext(final Builder builder) { + super(builder); + this.mockExtensionMapper = builder.mockExtensionMapper; + } + + @Override + public void updateFlow(final FlowContext flowContext, final VersionedExternalFlow versionedExternalFlow, + final BundleCompatibility bundleCompatability) throws FlowUpdateException { + replaceMocks(versionedExternalFlow.getFlowContents()); + super.updateFlow(flowContext, versionedExternalFlow, bundleCompatability); + } + + private void replaceMocks(final VersionedProcessGroup group) { + if (group.getProcessors() != null) { + for (final VersionedProcessor processor : group.getProcessors()) { + mockExtensionMapper.mapProcessor(processor); + } + } + + if (group.getControllerServices() != null) { + for (final VersionedControllerService controllerService : group.getControllerServices()) { + mockExtensionMapper.mapControllerService(controllerService); + } + } + + if (group.getProcessGroups() != null) { + for (final VersionedProcessGroup childGroup : group.getProcessGroups()) { + replaceMocks(childGroup); + } + } + } + + + public static class Builder extends StandardConnectorInitializationContext.Builder { + private final MockExtensionMapper mockExtensionMapper; + + public Builder(final MockExtensionMapper mockExtensionMapper) { + this.mockExtensionMapper = mockExtensionMapper; + } + + @Override + public MockConnectorInitializationContext build() { + return new MockConnectorInitializationContext(this); + } + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockConnectorRepository.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockConnectorRepository.java new file mode 100644 index 000000000000..9f951116747d --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockConnectorRepository.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.mock.connector.server; + +import org.apache.nifi.components.connector.FrameworkConnectorInitializationContextBuilder; +import org.apache.nifi.components.connector.StandardConnectorRepository; + +public class MockConnectorRepository extends StandardConnectorRepository { + + private volatile MockExtensionMapper mockExtensionMapper; + + public void setMockExtensionMapper(final MockExtensionMapper mapper) { + mockExtensionMapper = mapper; + } + + @Override + public FrameworkConnectorInitializationContextBuilder createInitializationContextBuilder() { + return new MockConnectorInitializationContext.Builder(mockExtensionMapper); + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockExtensionDiscoveringManager.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockExtensionDiscoveringManager.java new file mode 100644 index 000000000000..22b2ca043ce6 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockExtensionDiscoveringManager.java @@ -0,0 +1,74 @@ +/* + * 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.mock.connector.server; + +import org.apache.nifi.bundle.Bundle; +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.bundle.BundleDetails; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.nar.InstanceClassLoader; +import org.apache.nifi.nar.StandardExtensionDiscoveringManager; +import org.apache.nifi.processor.Processor; + +import java.io.File; +import java.net.URL; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class MockExtensionDiscoveringManager extends StandardExtensionDiscoveringManager { + private final ConcurrentMap mockComponentClassLoaders = new ConcurrentHashMap<>(); + + public synchronized void addProcessor(final Class mockProcessorClass) { + final BundleDetails bundleDetails = new BundleDetails.Builder() + .workingDir(new File("target/work/extensions/mock-bundle")) + .coordinate(new BundleCoordinate("org.apache.nifi.mock", mockProcessorClass.getName(), "1.0.0")) + .build(); + + final Bundle mockBundle = new Bundle(bundleDetails, mockProcessorClass.getClassLoader()); + discoverExtensions(Set.of(mockBundle)); + + mockComponentClassLoaders.put(mockProcessorClass.getName(), mockProcessorClass.getClassLoader()); + registerExtensionClass(Processor.class, mockProcessorClass.getName(), mockBundle); + } + + public synchronized void addControllerService(final Class mockControllerServiceClass) { + final BundleDetails bundleDetails = new BundleDetails.Builder() + .workingDir(new File("target/work/extensions/mock-bundle")) + .coordinate(new BundleCoordinate("org.apache.nifi.mock", mockControllerServiceClass.getName(), "1.0.0")) + .build(); + + final Bundle mockBundle = new Bundle(bundleDetails, mockControllerServiceClass.getClassLoader()); + discoverExtensions(Set.of(mockBundle)); + + mockComponentClassLoaders.put(mockControllerServiceClass.getName(), mockControllerServiceClass.getClassLoader()); + registerExtensionClass(ControllerService.class, mockControllerServiceClass.getName(), mockBundle); + } + + @Override + public InstanceClassLoader createInstanceClassLoader(final String classType, final String instanceIdentifier, final Bundle bundle, final Set additionalUrls, + final boolean register, final String classloaderIsolationKey) { + + final ClassLoader classLoader = mockComponentClassLoaders.get(classType); + if (classLoader != null) { + return new InstanceClassLoader(instanceIdentifier, classType, additionalUrls, Set.of(), classLoader); + } + + return super.createInstanceClassLoader(classType, instanceIdentifier, bundle, additionalUrls, register, classloaderIsolationKey); + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockExtensionMapper.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockExtensionMapper.java new file mode 100644 index 000000000000..edf81cdeaac5 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockExtensionMapper.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.mock.connector.server; + +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedProcessor; + +import java.util.HashMap; +import java.util.Map; + +public class MockExtensionMapper { + + private final Map processorMocks = new HashMap<>(); + private final Map controllerServiceMocks = new HashMap<>(); + + public void mockProcessor(final String processorType, final String mockProcessorClassName) { + processorMocks.put(processorType, mockProcessorClassName); + } + + public void mockControllerService(final String controllerServiceType, final String mockControllerServiceClassName) { + controllerServiceMocks.put(controllerServiceType, mockControllerServiceClassName); + } + + public void mapProcessor(final VersionedProcessor processor) { + final String type = processor.getType(); + final String implementationClassName = processorMocks.get(type); + if (implementationClassName == null) { + return; + } + + processor.setType(implementationClassName); + processor.setBundle(new Bundle("org.apache.nifi.mock", implementationClassName, "1.0.0")); + } + + public void mapControllerService(final VersionedControllerService controllerService) { + final String type = controllerService.getType(); + final String implementationClassName = controllerServiceMocks.get(type); + if (implementationClassName == null) { + return; + } + + controllerService.setType(implementationClassName); + controllerService.setBundle(new Bundle("org.apache.nifi.mock", implementationClassName, "1.0.0")); + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockNiFiConnectorWebContext.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockNiFiConnectorWebContext.java new file mode 100644 index 000000000000..9e8891ee5270 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockNiFiConnectorWebContext.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.mock.connector.server; + +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorRepository; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.web.NiFiConnectorWebContext; + +/** + * Mock implementation of {@link NiFiConnectorWebContext} for the connector test runner. + * Provides direct access to the connector instance and its flow contexts without + * authorization proxying, since the mock server uses a permit-all authorizer. + */ +public class MockNiFiConnectorWebContext implements NiFiConnectorWebContext { + + private final ConnectorRepository connectorRepository; + + public MockNiFiConnectorWebContext(final ConnectorRepository connectorRepository) { + this.connectorRepository = connectorRepository; + } + + @Override + @SuppressWarnings("unchecked") + public ConnectorWebContext getConnectorWebContext(final String connectorId) throws IllegalArgumentException { + final ConnectorNode connectorNode = connectorRepository.getConnector(connectorId); + if (connectorNode == null) { + throw new IllegalArgumentException("Unable to find connector with id: " + connectorId); + } + + final T connector = (T) connectorNode.getConnector(); + final FlowContext workingFlowContext = connectorNode.getWorkingFlowContext(); + final FlowContext activeFlowContext = connectorNode.getActiveFlowContext(); + + return new ConnectorWebContext<>(connector, workingFlowContext, activeFlowContext); + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockRuleViolationsManager.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockRuleViolationsManager.java new file mode 100644 index 000000000000..bb0e2636ea5a --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockRuleViolationsManager.java @@ -0,0 +1,79 @@ +/* + * 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.mock.connector.server; + +import org.apache.nifi.flow.VersionedComponent; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.validation.RuleViolation; +import org.apache.nifi.validation.RuleViolationsManager; + +import java.util.Collection; +import java.util.List; +import java.util.Map; + +public class MockRuleViolationsManager implements RuleViolationsManager { + @Override + public void upsertComponentViolations(final String subjectId, final Collection violations) { + + } + + @Override + public void upsertGroupViolations(final VersionedProcessGroup processGroup, final Collection violations, + final Map> componentToRuleViolations) { + } + + @Override + public Collection getRuleViolationsForSubject(final String subjectId) { + return List.of(); + } + + @Override + public Collection getRuleViolationsForGroup(final String groupId) { + return List.of(); + } + + @Override + public Collection getRuleViolationsForGroups(final Collection groupIds) { + return List.of(); + } + + @Override + public Collection getAllRuleViolations() { + return List.of(); + } + + @Override + public void removeRuleViolationsForSubject(final String subjectId) { + + } + + @Override + public void removeRuleViolationsForRule(final String ruleId) { + + } + + @Override + public void cleanUp() { + + } + + @Override + public boolean isEmpty() { + return false; + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockServerConfigVerificationResult.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockServerConfigVerificationResult.java new file mode 100644 index 000000000000..fed82f929020 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockServerConfigVerificationResult.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.mock.connector.server; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.ConfigVerificationResult.Outcome; + +import java.util.Collections; +import java.util.List; + +public class MockServerConfigVerificationResult implements ConnectorConfigVerificationResult { + private final List results; + + public MockServerConfigVerificationResult(final List results) { + this.results = results; + } + + @Override + public List getAllResults() { + return Collections.unmodifiableList(results); + } + + @Override + public List getFailedResults() { + return results.stream() + .filter(result -> result.getOutcome() == Outcome.FAILED) + .toList(); + } + + @Override + public void assertNoFailures() { + final List failedResults = getFailedResults(); + if (!failedResults.isEmpty()) { + throw new AssertionError("Configuration verification failed: " + failedResults); + } + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockStateManager.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockStateManager.java new file mode 100644 index 000000000000..f22269ceabfa --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockStateManager.java @@ -0,0 +1,89 @@ +/* + * 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.mock.connector.server; + +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateManager; +import org.apache.nifi.components.state.StateMap; + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +public class MockStateManager implements StateManager { + private final AtomicLong versionCounter = new AtomicLong(0L); + private final boolean dropStateKeySupported; + + private volatile StateMap localStateMap = new MockStateMap(null, -1L); + private volatile StateMap clusterStateMap = new MockStateMap(null, -1L); + + public MockStateManager(final boolean dropStateKeySupported) { + this.dropStateKeySupported = dropStateKeySupported; + } + + @Override + public synchronized void setState(final Map state, final Scope scope) throws IOException { + final StateMap newStateMap = new MockStateMap(state, versionCounter.incrementAndGet()); + + if (scope == Scope.CLUSTER) { + clusterStateMap = newStateMap; + } else { + localStateMap = newStateMap; + } + } + + @Override + public synchronized StateMap getState(final Scope scope) throws IOException { + if (scope == Scope.CLUSTER) { + return clusterStateMap; + } else { + return localStateMap; + } + } + + @Override + public synchronized boolean replace(final StateMap oldValue, final Map newValue, final Scope scope) throws IOException { + final StateMap currentStateMap = scope == Scope.CLUSTER ? clusterStateMap : localStateMap; + + if (currentStateMap == oldValue) { + final StateMap newStateMap = new MockStateMap(newValue, versionCounter.incrementAndGet()); + + if (scope == Scope.CLUSTER) { + clusterStateMap = newStateMap; + } else { + localStateMap = newStateMap; + } + + return true; + } + + return false; + } + + @Override + public synchronized void clear(final Scope scope) throws IOException { + setState(Collections.emptyMap(), scope); + } + + @Override + public boolean isStateKeyDropSupported() { + return dropStateKeySupported; + } +} + diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockStateManagerProvider.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockStateManagerProvider.java new file mode 100644 index 000000000000..77484904f251 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockStateManagerProvider.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.mock.connector.server; + +import org.apache.nifi.components.state.StateManager; +import org.apache.nifi.components.state.StateManagerProvider; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class MockStateManagerProvider implements StateManagerProvider { + private final ConcurrentMap stateManagers = new ConcurrentHashMap<>(); + private volatile boolean clusterProviderEnabled = false; + + @Override + public StateManager getStateManager(final String componentId, final boolean dropStateKeySupported) { + return stateManagers.computeIfAbsent(componentId, id -> new MockStateManager(dropStateKeySupported)); + } + + @Override + public void onComponentRemoved(final String componentId) { + stateManagers.remove(componentId); + } + + @Override + public void shutdown() { + stateManagers.clear(); + } + + @Override + public void enableClusterProvider() { + clusterProviderEnabled = true; + } + + @Override + public void disableClusterProvider() { + clusterProviderEnabled = false; + } + + @Override + public boolean isClusterProviderEnabled() { + return clusterProviderEnabled; + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockStateMap.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockStateMap.java new file mode 100644 index 000000000000..54b0980d6162 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/MockStateMap.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.mock.connector.server; + +import org.apache.nifi.components.state.StateMap; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +public class MockStateMap implements StateMap { + private final Map stateValues; + private final long version; + + public MockStateMap(final Map stateValues, final long version) { + this.stateValues = stateValues == null ? Collections.emptyMap() : new HashMap<>(stateValues); + this.version = version; + } + + @Override + public Optional getStateVersion() { + return version == -1L ? Optional.empty() : Optional.of(Long.toString(version)); + } + + @Override + public String get(final String key) { + return stateValues.get(key); + } + + @Override + public Map toMap() { + return Collections.unmodifiableMap(stateValues); + } + + @Override + public String toString() { + return "MockStateMap[version=" + version + ", values=" + stateValues + "]"; + } +} + diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/NopPropertyEncryptor.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/NopPropertyEncryptor.java new file mode 100644 index 000000000000..56fbe816b7f3 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/NopPropertyEncryptor.java @@ -0,0 +1,32 @@ +/* + * 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.mock.connector.server; + +import org.apache.nifi.encrypt.PropertyEncryptor; + +public class NopPropertyEncryptor implements PropertyEncryptor { + @Override + public String encrypt(final String property) { + return property; + } + + @Override + public String decrypt(final String encryptedProperty) { + return encryptedProperty; + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/PermitAllAuthorizer.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/PermitAllAuthorizer.java new file mode 100644 index 000000000000..75b2577f25e5 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/PermitAllAuthorizer.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.mock.connector.server; + +import org.apache.nifi.authorization.AuthorizationRequest; +import org.apache.nifi.authorization.AuthorizationResult; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.AuthorizerConfigurationContext; +import org.apache.nifi.authorization.AuthorizerInitializationContext; +import org.apache.nifi.authorization.exception.AuthorizationAccessException; +import org.apache.nifi.authorization.exception.AuthorizerCreationException; +import org.apache.nifi.authorization.exception.AuthorizerDestructionException; + +public class PermitAllAuthorizer implements Authorizer { + @Override + public AuthorizationResult authorize(final AuthorizationRequest request) throws AuthorizationAccessException { + return AuthorizationResult.approved(); + } + + @Override + public void initialize(final AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException { + } + + @Override + public void onConfigured(final AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException { + } + + @Override + public void preDestruction() throws AuthorizerDestructionException { + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/StandardConnectorMockServer.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/StandardConnectorMockServer.java new file mode 100644 index 000000000000..d491d0e6c19e --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/StandardConnectorMockServer.java @@ -0,0 +1,629 @@ +/* + * 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.mock.connector.server; + +import jakarta.servlet.ServletContext; +import org.apache.nifi.admin.service.AuditService; +import org.apache.nifi.asset.Asset; +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.bundle.Bundle; +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.bundle.BundleDetails; +import org.apache.nifi.cluster.ClusterDetailsFactory; +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.AssetReference; +import org.apache.nifi.components.connector.Connector; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorRepository; +import org.apache.nifi.components.connector.ConnectorState; +import org.apache.nifi.components.connector.ConnectorValueReference; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.FrameworkFlowContext; +import org.apache.nifi.components.connector.GhostConnector; +import org.apache.nifi.components.connector.SecretReference; +import org.apache.nifi.components.connector.StandaloneConnectorRequestReplicator; +import org.apache.nifi.components.connector.StepConfiguration; +import org.apache.nifi.components.connector.StringLiteralValue; +import org.apache.nifi.components.connector.secrets.SecretsManager; +import org.apache.nifi.components.state.StateManagerProvider; +import org.apache.nifi.components.validation.DisabledServiceValidationResult; +import org.apache.nifi.components.validation.ValidationState; +import org.apache.nifi.connectable.FlowFileTransferCounts; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.DecommissionTask; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.metrics.DefaultComponentMetricReporter; +import org.apache.nifi.controller.repository.FlowFileEventRepository; +import org.apache.nifi.controller.repository.metrics.RingBufferEventRepository; +import org.apache.nifi.controller.status.history.StatusHistoryDumpFactory; +import org.apache.nifi.controller.status.history.StatusHistoryRepository; +import org.apache.nifi.controller.status.history.VolatileComponentStatusRepository; +import org.apache.nifi.diagnostics.DiagnosticsFactory; +import org.apache.nifi.encrypt.PropertyEncryptor; +import org.apache.nifi.engine.FlowEngine; +import org.apache.nifi.events.VolatileBulletinRepository; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedParameter; +import org.apache.nifi.flow.VersionedParameterContext; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.mock.connector.server.secrets.ConnectorTestRunnerSecretsManager; +import org.apache.nifi.nar.ExtensionMapping; +import org.apache.nifi.parameter.Parameter; +import org.apache.nifi.parameter.ParameterContext; +import org.apache.nifi.processor.Processor; +import org.apache.nifi.registry.flow.mapping.ComponentIdLookup; +import org.apache.nifi.registry.flow.mapping.FlowMappingOptions; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup; +import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.validation.RuleViolationsManager; +import org.apache.nifi.web.NiFiConnectorWebContext; +import org.eclipse.jetty.ee.webapp.WebAppClassLoader; +import org.eclipse.jetty.ee11.webapp.WebAppContext; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.handler.ContextHandlerCollection; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.jar.JarFile; +import java.util.stream.Stream; + +public class StandardConnectorMockServer implements ConnectorMockServer { + private static final Logger logger = LoggerFactory.getLogger(StandardConnectorMockServer.class); + + private static final String CONNECTOR_ID = "test-connector"; + private static final String NAR_DEPENDENCIES_PATH = "NAR-INF/bundled-dependencies"; + private static final String CONNECTOR_WAR_MANIFEST_PATH = "META-INF/nifi-connector"; + private static final String WAR_EXTENSION = ".war"; + + private Bundle systemBundle; + private Set bundles; + private NiFiProperties nifiProperties; + private FlowController flowController; + private MockExtensionDiscoveringManager extensionManager; + private ConnectorNode connectorNode; + private ConnectorRepository connectorRepository; + private FlowEngine flowEngine; + private MockExtensionMapper mockExtensionMapper; + private FlowFileTransferCounts initialFlowFileTransferCounts = new FlowFileTransferCounts(0L, 0L, 0L, 0L); + private Server jettyServer; + + @Override + public void start() { + extensionManager = new MockExtensionDiscoveringManager(); + extensionManager.discoverExtensions(systemBundle, bundles); + extensionManager.logClassLoaderMapping(); + + final FlowFileEventRepository flowFileEventRepository = new RingBufferEventRepository(5); + final Authorizer authorizer = new PermitAllAuthorizer(); + final AuditService auditService = new MockAuditService(); + final PropertyEncryptor propertyEncryptor = new NopPropertyEncryptor(); + final BulletinRepository bulletinRepository = new VolatileBulletinRepository(); + final StatusHistoryRepository statusHistoryRepository = new VolatileComponentStatusRepository(nifiProperties); + final RuleViolationsManager ruleViolationManager = new MockRuleViolationsManager(); + final StateManagerProvider stateManagerProvider = new MockStateManagerProvider(); + + flowController = FlowController.createStandaloneInstance( + flowFileEventRepository, + null, + nifiProperties, + authorizer, + auditService, + new DefaultComponentMetricReporter(), + propertyEncryptor, + bulletinRepository, + extensionManager, + statusHistoryRepository, + ruleViolationManager, + stateManagerProvider, + new StandaloneConnectorRequestReplicator()); + + try { + flowController.getRepositoryContextFactory().getFlowFileRepository().loadFlowFiles(Collections::emptyList); + } catch (final IOException e) { + throw new RuntimeException("Failed to initialize FlowFile Repository", e); + } + + connectorRepository = flowController.getConnectorRepository(); + if (!(connectorRepository instanceof MockConnectorRepository)) { + throw new IllegalStateException("Connector Repository is not an instance of MockConnectorRepository"); + } + + mockExtensionMapper = new MockExtensionMapper(); + ((MockConnectorRepository) connectorRepository).setMockExtensionMapper(mockExtensionMapper); + + flowEngine = new FlowEngine(4, "Connector Threads"); + + startJettyServer(); + } + + @Override + public void initialize(final NiFiProperties properties, final Bundle systemBundle, final Set bundles, final ExtensionMapping extensionMapping) { + this.systemBundle = systemBundle; + this.bundles = bundles; + this.nifiProperties = properties; + } + + @Override + public void stop() { + if (jettyServer != null) { + try { + jettyServer.stop(); + logger.info("Jetty server stopped"); + } catch (final Exception e) { + logger.warn("Failed to stop Jetty server", e); + } + } + if (flowEngine != null) { + flowEngine.shutdown(); + } + if (flowController != null) { + flowController.shutdown(false); + } + } + + @Override + public DiagnosticsFactory getDiagnosticsFactory() { + return null; + } + + @Override + public DiagnosticsFactory getThreadDumpFactory() { + return null; + } + + @Override + public DecommissionTask getDecommissionTask() { + return null; + } + + @Override + public ClusterDetailsFactory getClusterDetailsFactory() { + return null; + } + + @Override + public StatusHistoryDumpFactory getStatusHistoryDumpFactory() { + return null; + } + + @Override + public void instantiateConnector(final String connectorClassName) { + final List bundles = extensionManager.getBundles(connectorClassName); + if (bundles.isEmpty()) { + throw new IllegalStateException("No bundles found for connector class: " + connectorClassName + " - ensure that you have included all relevant NARs in the configured lib directory"); + } + if (bundles.size() > 1) { + throw new IllegalStateException("Multiple bundles found for connector class: " + connectorClassName + " - unable to determine which bundle to use. Ensure that only a single version of " + + "the Connector is included in the configured lib directory. Available bundles: " + bundles); + } + + final BundleCoordinate bundleCoordinate = bundles.getFirst().getBundleDetails().getCoordinate(); + connectorNode = flowController.getFlowManager().createConnector(connectorClassName, CONNECTOR_ID, bundleCoordinate, true, true); + + if (connectorNode.isExtensionMissing()) { + final Connector connector = connectorNode.getConnector(); + if (connector instanceof final GhostConnector ghostConnector) { + throw new IllegalStateException("Failed to create Connector of type " + connectorClassName, ghostConnector.getCauseOfGhost()); + } else { + throw new IllegalStateException("Failed to create Connector of type " + connectorClassName); + } + } + } + + @Override + public void registerMockBundle(final ClassLoader classLoader, final File workingDirectory) { + new BundleDetails.Builder() + .workingDir(workingDirectory) + .coordinate(new BundleCoordinate("org.apache.nifi.mock", "nifi-connector-mock-bundle", "1.0.0")) + .build(); + } + + @Override + public void applyUpdate() throws FlowUpdateException { + final ConnectorState initialDesiredState = connectorNode.getDesiredState(); + + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + connectorNode.applyUpdate(); + + if (initialDesiredState == ConnectorState.RUNNING) { + connectorNode.start(flowEngine); + } else { + connectorNode.stop(flowEngine); + } + } + + @Override + public void configure(final String stepName, final StepConfiguration configuration) throws FlowUpdateException { + connectorNode.setConfiguration(stepName, configuration); + } + + @Override + public void configure(final String stepName, final Map propertyValues) throws FlowUpdateException { + configure(stepName, propertyValues, Collections.emptyMap()); + } + + @Override + public void configure(final String stepName, final Map propertyValues, final Map propertyReferences) throws FlowUpdateException { + final StepConfiguration stepConfiguration = createStepConfiguration(propertyValues, propertyReferences); + configure(stepName, stepConfiguration); + } + + @Override + public ConnectorConfigVerificationResult verifyConfiguration(final String stepName, final Map propertyValueOverrides) { + return verifyConfiguration(stepName, propertyValueOverrides, Collections.emptyMap()); + } + + @Override + public ConnectorConfigVerificationResult verifyConfiguration(final String stepName, final Map propertyValueOverrides, + final Map referenceOverrides) { + + final StepConfiguration configuration = createStepConfiguration(propertyValueOverrides, referenceOverrides); + return verifyConfiguration(stepName, configuration); + } + + private StepConfiguration createStepConfiguration(final Map propertyValues, final Map propertyReferences) { + final Map references = new HashMap<>(); + propertyValues.forEach((key, value) -> references.put(key, value == null ? null : new StringLiteralValue(value))); + references.putAll(propertyReferences); + return new StepConfiguration(references); + } + + @Override + public ConnectorConfigVerificationResult verifyConfiguration(final String stepName, final StepConfiguration configurationOverrides) { + final List results = connectorNode.verifyConfigurationStep(stepName, configurationOverrides); + return new MockServerConfigVerificationResult(results); + } + + @Override + public void addSecret(final String name, final String value) { + final SecretsManager secretsManager = connectorRepository.getSecretsManager(); + if (!(secretsManager instanceof final ConnectorTestRunnerSecretsManager testRunnerSecretsManager)) { + throw new IllegalStateException("Secrets Manager is not an instance of ConnectorTestRunnerSecretsManager"); + } + + testRunnerSecretsManager.addSecret(name, value); + } + + @Override + public SecretReference createSecretReference(final String secretName) { + return new SecretReference(ConnectorTestRunner.SECRET_PROVIDER_ID, ConnectorTestRunner.SECRET_PROVIDER_NAME, secretName, secretName); + } + + @Override + public AssetReference addAsset(final File file) { + final AssetManager assetManager = flowController.getConnectorAssetManager(); + + try (final InputStream inputStream = new FileInputStream(file)) { + final Asset asset = assetManager.createAsset(CONNECTOR_ID, file.getName(), inputStream); + return new AssetReference(Set.of(asset.getIdentifier())); + } catch (final IOException e) { + throw new RuntimeException("Failed to add asset from file: " + file.getAbsolutePath(), e); + } + } + + @Override + public AssetReference addAsset(final String assetName, final InputStream contents) { + final AssetManager assetManager = flowController.getConnectorAssetManager(); + + try { + final Asset asset = assetManager.createAsset(CONNECTOR_ID, assetName, contents); + return new AssetReference(Set.of(asset.getIdentifier())); + } catch (final IOException e) { + throw new RuntimeException("Failed to add asset: " + assetName, e); + } + } + + @Override + public void startConnector() { + initialFlowFileTransferCounts = connectorNode.getFlowFileTransferCounts(); + + connectorNode.start(flowEngine); + } + + @Override + public void stopConnector() { + try { + connectorNode.stop(flowEngine).get(10, TimeUnit.SECONDS); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while waiting for connector to stop", e); + } catch (final Exception e) { + throw new RuntimeException("Failed to stop Connector", e); + } + } + + @Override + public void waitForDataIngested(final Duration maxWaitTime) { + final long startTime = System.currentTimeMillis(); + final long expirationTime = startTime + maxWaitTime.toMillis(); + + while (getTransferCountsSinceStart().getReceivedCount() == 0L) { + if (System.currentTimeMillis() > expirationTime) { + throw new RuntimeException("Timed out waiting for data to be ingested by the Connector"); + } + + try { + Thread.sleep(100L); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while waiting for data to be ingested by the Connector", e); + } + } + } + + private FlowFileTransferCounts getTransferCountsSinceStart() { + final FlowFileTransferCounts currentCounts = connectorNode.getFlowFileTransferCounts(); + return currentCounts.minus(initialFlowFileTransferCounts); + } + + @Override + public void waitForIdle(final Duration maxWaitTime) { + waitForIdle(Duration.ofMillis(0L), maxWaitTime); + } + + @Override + public void waitForIdle(final Duration minimumIdleTime, final Duration maxWaitTime) { + Optional idleTime = connectorNode.getIdleDuration(); + + // Wait until idleTime is not empty and is at least equal to minimumIdleTime + final long startTime = System.currentTimeMillis(); + final long expirationTime = startTime + maxWaitTime.toMillis(); + + while (idleTime.isEmpty() || idleTime.get().compareTo(minimumIdleTime) <= 0) { + if (System.currentTimeMillis() > expirationTime) { + throw new RuntimeException("Timed out waiting for Connector to be idle"); + } + + try { + Thread.sleep(100L); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while waiting for Connector to be idle", e); + } + + idleTime = connectorNode.getIdleDuration(); + } + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName) { + return connectorNode.fetchAllowableValues(stepName, propertyName); + } + + @Override + public List validate() { + final ValidationState validationState = connectorNode.performValidation(); + return validationState.getValidationErrors().stream() + .filter(result -> !result.isValid()) + .filter(result -> !DisabledServiceValidationResult.isMatch(result)) + .toList(); + } + + @Override + public VersionedExternalFlow getActiveFlowSnapshot() { + final FrameworkFlowContext activeFlowContext = connectorNode.getActiveFlowContext(); + if (activeFlowContext == null) { + throw new IllegalStateException("Active Flow Context is not available. The Connector may not have been initialized or may not have an initial flow."); + } + return createFlowSnapshot(activeFlowContext); + } + + @Override + public VersionedExternalFlow getWorkingFlowSnapshot() { + final FrameworkFlowContext workingFlowContext = connectorNode.getWorkingFlowContext(); + if (workingFlowContext == null) { + throw new IllegalStateException("Working Flow Context is not available. No configuration changes may have been made since the last update was applied."); + } + return createFlowSnapshot(workingFlowContext); + } + + private VersionedExternalFlow createFlowSnapshot(final FrameworkFlowContext flowContext) { + final ProcessGroup processGroup = flowContext.getManagedProcessGroup(); + + final FlowMappingOptions flowMappingOptions = new FlowMappingOptions.Builder() + .mapSensitiveConfiguration(false) + .mapPropertyDescriptors(true) + .stateLookup(VersionedComponentStateLookup.ENABLED_OR_DISABLED) + .sensitiveValueEncryptor(value -> value) + .componentIdLookup(ComponentIdLookup.VERSIONED_OR_GENERATE) + .mapInstanceIdentifiers(true) + .mapControllerServiceReferencesToVersionedId(true) + .mapFlowRegistryClientId(true) + .mapAssetReferences(true) + .build(); + + final VersionedComponentFlowMapper flowMapper = new VersionedComponentFlowMapper(extensionManager, flowMappingOptions); + final VersionedProcessGroup versionedGroup = flowMapper.mapProcessGroup( + processGroup, flowController.getControllerServiceProvider(), flowController.getFlowManager(), true); + + final VersionedExternalFlow externalFlow = new VersionedExternalFlow(); + externalFlow.setFlowContents(versionedGroup); + + final ParameterContext parameterContext = processGroup.getParameterContext(); + if (parameterContext != null) { + final Map parameterContexts = new HashMap<>(); + final VersionedParameterContext versionedParameterContext = createVersionedParameterContext(parameterContext); + parameterContexts.put(versionedParameterContext.getName(), versionedParameterContext); + externalFlow.setParameterContexts(parameterContexts); + } + + return externalFlow; + } + + private VersionedParameterContext createVersionedParameterContext(final ParameterContext parameterContext) { + final VersionedParameterContext versionedParameterContext = new VersionedParameterContext(); + versionedParameterContext.setName(parameterContext.getName()); + versionedParameterContext.setDescription(parameterContext.getDescription()); + versionedParameterContext.setIdentifier(parameterContext.getIdentifier()); + + final Set versionedParameters = new LinkedHashSet<>(); + for (final Parameter parameter : parameterContext.getParameters().values()) { + final VersionedParameter versionedParameter = new VersionedParameter(); + versionedParameter.setName(parameter.getDescriptor().getName()); + versionedParameter.setDescription(parameter.getDescriptor().getDescription()); + versionedParameter.setSensitive(parameter.getDescriptor().isSensitive()); + versionedParameter.setProvided(parameter.getDescriptor().isSensitive()); + if (!parameter.getDescriptor().isSensitive()) { + versionedParameter.setValue(parameter.getValue()); + } + versionedParameters.add(versionedParameter); + } + versionedParameterContext.setParameters(versionedParameters); + + return versionedParameterContext; + } + + @Override + public void mockProcessor(final String processorType, final Class mockProcessorClass) { + mockExtensionMapper.mockProcessor(processorType, mockProcessorClass.getName()); + extensionManager.addProcessor(mockProcessorClass); + } + + @Override + public void mockControllerService(final String controllerServiceType, final Class mockControllerServiceClass) { + mockExtensionMapper.mockControllerService(controllerServiceType, mockControllerServiceClass.getName()); + extensionManager.addControllerService(mockControllerServiceClass); + } + + @Override + public int getHttpPort() { + if (jettyServer == null) { + return -1; + } + + final ServerConnector connector = (ServerConnector) jettyServer.getConnectors()[0]; + return connector.getLocalPort(); + } + + @Override + public void close() { + stop(); + } + + private void startJettyServer() { + final String httpPortValue = nifiProperties.getProperty(NiFiProperties.WEB_HTTP_PORT); + if (httpPortValue == null || httpPortValue.isBlank()) { + logger.debug("No HTTP port configured; skipping Jetty server startup"); + return; + } + + final int httpPort = Integer.parseInt(httpPortValue); + final Map wars = findWars(bundles); + if (wars.isEmpty()) { + logger.debug("No WAR files found in NAR bundles; skipping Jetty server startup"); + return; + } + + jettyServer = new Server(); + + final ServerConnector serverConnector = new ServerConnector(jettyServer); + serverConnector.setPort(httpPort); + jettyServer.addConnector(serverConnector); + + final List webAppContexts = new ArrayList<>(); + final ContextHandlerCollection handlers = new ContextHandlerCollection(); + for (final Map.Entry entry : wars.entrySet()) { + final File warFile = entry.getKey(); + final Bundle bundle = entry.getValue(); + + final String warName = warFile.getName(); + final String contextPath = "/" + warName.substring(0, warName.length() - WAR_EXTENSION.length()); + + final WebAppContext webAppContext = new WebAppContext(warFile.getPath(), contextPath); + webAppContext.setClassLoader(new WebAppClassLoader(bundle.getClassLoader(), webAppContext)); + + handlers.addHandler(webAppContext); + webAppContexts.add(webAppContext); + logger.info("Deploying WAR [{}] at context path [{}]", warFile.getAbsolutePath(), contextPath); + } + + jettyServer.setHandler(handlers); + + try { + jettyServer.start(); + logger.info("Jetty server started on port [{}]", getHttpPort()); + } catch (final Exception e) { + throw new RuntimeException("Failed to start Jetty server", e); + } + + performInjectionForConnectorUis(webAppContexts); + } + + private void performInjectionForConnectorUis(final List webAppContexts) { + final NiFiConnectorWebContext connectorWebContext = new MockNiFiConnectorWebContext(connectorRepository); + for (final WebAppContext webAppContext : webAppContexts) { + final ServletContext servletContext = webAppContext.getServletHandler().getServletContext(); + servletContext.setAttribute("nifi-connector-web-context", connectorWebContext); + logger.info("Injected NiFiConnectorWebContext into WAR context [{}]", webAppContext.getContextPath()); + } + } + + public Map findWars(final Set bundles) { + final Map wars = new HashMap<>(); + + bundles.forEach(bundle -> { + final BundleDetails details = bundle.getBundleDetails(); + final Path bundledDependencies = new File(details.getWorkingDirectory(), NAR_DEPENDENCIES_PATH).toPath(); + if (Files.isDirectory(bundledDependencies)) { + try (final Stream dependencies = Files.list(bundledDependencies)) { + dependencies.filter(dependency -> dependency.getFileName().toString().endsWith(WAR_EXTENSION)) + .map(Path::toFile) + .filter(this::isConnectorWar) + .forEach(dependency -> wars.put(dependency, bundle)); + } catch (final IOException e) { + logger.warn("Failed to find WAR files in bundled-dependencies [{}]", bundledDependencies, e); + } + } + }); + + return wars; + } + + private boolean isConnectorWar(final File warFile) { + try (final JarFile jarFile = new JarFile(warFile)) { + return jarFile.getJarEntry(CONNECTOR_WAR_MANIFEST_PATH) != null; + } catch (final IOException e) { + logger.warn("Unable to inspect WAR file [{}] for connector manifest", warFile, e); + return false; + } + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/secrets/ConnectorTestRunnerAuthorizable.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/secrets/ConnectorTestRunnerAuthorizable.java new file mode 100644 index 000000000000..5600c34f143e --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/secrets/ConnectorTestRunnerAuthorizable.java @@ -0,0 +1,34 @@ +/* + * 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.mock.connector.server.secrets; + +import org.apache.nifi.authorization.Resource; +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.resource.ResourceFactory; + +public class ConnectorTestRunnerAuthorizable implements Authorizable { + @Override + public Authorizable getParentAuthorizable() { + return null; + } + + @Override + public Resource getResource() { + return ResourceFactory.getControllerResource(); + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/secrets/ConnectorTestRunnerSecretProvider.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/secrets/ConnectorTestRunnerSecretProvider.java new file mode 100644 index 000000000000..e12a3711bc1e --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/secrets/ConnectorTestRunnerSecretProvider.java @@ -0,0 +1,94 @@ +/* + * 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.mock.connector.server.secrets; + +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.components.connector.Secret; +import org.apache.nifi.components.connector.secrets.SecretProvider; +import org.apache.nifi.components.connector.secrets.StandardSecret; +import org.apache.nifi.mock.connector.server.ConnectorTestRunner; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ConnectorTestRunnerSecretProvider implements SecretProvider { + public static final String GROUP_NAME = "Default"; + private static final Authorizable AUTHORIZABLE = new ConnectorTestRunnerAuthorizable(); + + private final Map secrets = new HashMap<>(); + + public void addSecret(final String key, final String value) { + this.secrets.put(key, value); + } + + @Override + public String getProviderId() { + return ConnectorTestRunner.SECRET_PROVIDER_ID; + } + + @Override + public String getProviderName() { + return ConnectorTestRunner.SECRET_PROVIDER_ID; + } + + @Override + public List getAllSecrets() { + final List secrets = new ArrayList<>(); + for (final Map.Entry entry : this.secrets.entrySet()) { + final Secret secret = new StandardSecret.Builder() + .providerId(ConnectorTestRunner.SECRET_PROVIDER_ID) + .providerName(ConnectorTestRunner.SECRET_PROVIDER_NAME) + .groupName(GROUP_NAME) + .name(entry.getKey()) + .value(entry.getValue()) + .authorizable(AUTHORIZABLE) + .fullyQualifiedName(GROUP_NAME + "." + entry.getKey()) + .build(); + + secrets.add(secret); + } + + return secrets; + } + + @Override + public List getSecrets(final List fullyQualifiedSecretNames) { + final List matchingSecrets = new ArrayList<>(); + + for (final String secretName : fullyQualifiedSecretNames) { + final String value = secrets.get(secretName); + + if (value != null) { + final Secret secret = new StandardSecret.Builder() + .providerId(ConnectorTestRunner.SECRET_PROVIDER_ID) + .providerName(ConnectorTestRunner.SECRET_PROVIDER_NAME) + .groupName(GROUP_NAME) + .name(secretName) + .fullyQualifiedName(GROUP_NAME + "." + secretName) + .value(value) + .authorizable(AUTHORIZABLE) + .build(); + + matchingSecrets.add(secret); + } + } + return matchingSecrets; + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/secrets/ConnectorTestRunnerSecretsManager.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/secrets/ConnectorTestRunnerSecretsManager.java new file mode 100644 index 000000000000..e06f228454af --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/java/org/apache/nifi/mock/connector/server/secrets/ConnectorTestRunnerSecretsManager.java @@ -0,0 +1,99 @@ +/* + * 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.mock.connector.server.secrets; + +import org.apache.nifi.components.connector.Secret; +import org.apache.nifi.components.connector.SecretReference; +import org.apache.nifi.components.connector.secrets.SecretProvider; +import org.apache.nifi.components.connector.secrets.SecretsManager; +import org.apache.nifi.components.connector.secrets.SecretsManagerInitializationContext; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +public class ConnectorTestRunnerSecretsManager implements SecretsManager { + private final ConnectorTestRunnerSecretProvider secretProvider = new ConnectorTestRunnerSecretProvider(); + + @Override + public void initialize(final SecretsManagerInitializationContext initializationContext) { + } + + public void addSecret(final String name, final String value) { + secretProvider.addSecret(name, value); + } + + @Override + public List getAllSecrets() { + return secretProvider.getAllSecrets(); + } + + @Override + public Set getSecretProviders() { + return Set.of(secretProvider); + } + + @Override + public Optional getSecret(final SecretReference secretReference) { + // Check that appropriate provider given + final SecretProvider provider = getProvider(secretReference); + if (provider == null) { + return Optional.empty(); + } + + final List secrets = provider.getSecrets(List.of(secretReference.getFullyQualifiedName())); + return secrets.isEmpty() ? Optional.empty() : Optional.of(secrets.getFirst()); + } + + private SecretProvider getProvider(final SecretReference secretReference) { + final String providerId = secretReference.getProviderId(); + if (providerId != null) { + for (final SecretProvider provider : getSecretProviders()) { + if (provider.getProviderId().equals(providerId)) { + return provider; + } + } + + return null; + } + + final String providerName = secretReference.getProviderName(); + if (providerName != null) { + for (final SecretProvider provider : getSecretProviders()) { + if (provider.getProviderName().equals(providerName)) { + return provider; + } + } + } + + return null; + } + + @Override + public Map getSecrets(final Set secretReferences) { + final Map secrets = new HashMap<>(); + for (final SecretReference reference : secretReferences) { + final Secret secret = getSecret(reference).orElse(null); + secrets.put(reference, secret); + } + + return secrets; + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.NiFiServer b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.NiFiServer new file mode 100644 index 000000000000..70572a99eaec --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.NiFiServer @@ -0,0 +1,16 @@ +# 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. + +org.apache.nifi.mock.connector.server.StandardConnectorMockServer diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.asset.AssetManager b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.asset.AssetManager new file mode 100644 index 000000000000..c23d0b17077c --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.asset.AssetManager @@ -0,0 +1,16 @@ +# 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. + +org.apache.nifi.mock.connector.server.MockConnectorAssetManager diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.components.connector.ConnectorRepository b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.components.connector.ConnectorRepository new file mode 100644 index 000000000000..d9230afaec58 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.components.connector.ConnectorRepository @@ -0,0 +1,16 @@ +# 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. + +org.apache.nifi.mock.connector.server.MockConnectorRepository diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.components.connector.secrets.SecretsManager b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.components.connector.secrets.SecretsManager new file mode 100644 index 000000000000..a1c0de54d1f1 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/main/resources/META-INF/services/org.apache.nifi.components.connector.secrets.SecretsManager @@ -0,0 +1,16 @@ +# 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. + +org.apache.nifi.mock.connector.server.secrets.ConnectorTestRunnerSecretsManager diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/MockConnectorAssetManagerTest.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/MockConnectorAssetManagerTest.java new file mode 100644 index 000000000000..29521787641b --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/MockConnectorAssetManagerTest.java @@ -0,0 +1,245 @@ +/* + * 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.mock.connector.server; + +import org.apache.nifi.asset.Asset; +import org.apache.nifi.asset.AssetManagerInitializationContext; +import org.apache.nifi.asset.AssetReferenceLookup; +import org.apache.nifi.controller.NodeTypeProvider; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class MockConnectorAssetManagerTest { + + private static final String CONNECTOR_ID = "test-connector"; + private static final String ASSET_NAME = "test-asset.txt"; + private static final String ASSET_CONTENTS = "test-contents"; + + @TempDir + private Path tempDir; + + private MockConnectorAssetManager assetManager; + + @BeforeEach + void setUp() { + assetManager = new MockConnectorAssetManager(); + final AssetManagerInitializationContext context = createInitializationContext(tempDir); + assetManager.initialize(context); + } + + @Test + void testCreateAsset() throws IOException { + final InputStream contents = new ByteArrayInputStream(ASSET_CONTENTS.getBytes(StandardCharsets.UTF_8)); + + final Asset asset = assetManager.createAsset(CONNECTOR_ID, ASSET_NAME, contents); + + assertNotNull(asset); + assertNotNull(asset.getIdentifier()); + assertEquals(CONNECTOR_ID, asset.getOwnerIdentifier()); + assertEquals(ASSET_NAME, asset.getName()); + assertTrue(asset.getFile().exists()); + assertEquals(ASSET_CONTENTS, Files.readString(asset.getFile().toPath(), StandardCharsets.UTF_8)); + } + + @Test + void testCreateAssetGeneratesUniqueIdentifiers() throws IOException { + final InputStream contents1 = new ByteArrayInputStream(ASSET_CONTENTS.getBytes(StandardCharsets.UTF_8)); + final InputStream contents2 = new ByteArrayInputStream(ASSET_CONTENTS.getBytes(StandardCharsets.UTF_8)); + + final Asset asset1 = assetManager.createAsset(CONNECTOR_ID, ASSET_NAME, contents1); + final Asset asset2 = assetManager.createAsset(CONNECTOR_ID, ASSET_NAME, contents2); + + assertNotEquals(asset1.getIdentifier(), asset2.getIdentifier()); + } + + @Test + void testSaveAsset() throws IOException { + final String assetId = "specific-asset-id"; + final InputStream contents = new ByteArrayInputStream(ASSET_CONTENTS.getBytes(StandardCharsets.UTF_8)); + + final Asset asset = assetManager.saveAsset(CONNECTOR_ID, assetId, ASSET_NAME, contents); + + assertNotNull(asset); + assertEquals(assetId, asset.getIdentifier()); + assertEquals(CONNECTOR_ID, asset.getOwnerIdentifier()); + assertEquals(ASSET_NAME, asset.getName()); + assertTrue(asset.getFile().exists()); + assertEquals(ASSET_CONTENTS, Files.readString(asset.getFile().toPath(), StandardCharsets.UTF_8)); + } + + @Test + void testSaveAssetOverwritesExisting() throws IOException { + final String assetId = "overwrite-asset-id"; + final String originalContents = "original"; + final String updatedContents = "updated"; + + final InputStream originalStream = new ByteArrayInputStream(originalContents.getBytes(StandardCharsets.UTF_8)); + final Asset originalAsset = assetManager.saveAsset(CONNECTOR_ID, assetId, ASSET_NAME, originalStream); + assertEquals(originalContents, Files.readString(originalAsset.getFile().toPath(), StandardCharsets.UTF_8)); + + final InputStream updatedStream = new ByteArrayInputStream(updatedContents.getBytes(StandardCharsets.UTF_8)); + final Asset updatedAsset = assetManager.saveAsset(CONNECTOR_ID, assetId, ASSET_NAME, updatedStream); + assertEquals(updatedContents, Files.readString(updatedAsset.getFile().toPath(), StandardCharsets.UTF_8)); + } + + @Test + void testGetAsset() throws IOException { + final InputStream contents = new ByteArrayInputStream(ASSET_CONTENTS.getBytes(StandardCharsets.UTF_8)); + final Asset createdAsset = assetManager.createAsset(CONNECTOR_ID, ASSET_NAME, contents); + + final Optional retrieved = assetManager.getAsset(createdAsset.getIdentifier()); + + assertTrue(retrieved.isPresent()); + assertEquals(createdAsset.getIdentifier(), retrieved.get().getIdentifier()); + assertEquals(createdAsset.getName(), retrieved.get().getName()); + assertEquals(createdAsset.getOwnerIdentifier(), retrieved.get().getOwnerIdentifier()); + } + + @Test + void testGetAssetReturnsEmptyForNonExistent() { + final Optional retrieved = assetManager.getAsset("non-existent-id"); + + assertTrue(retrieved.isEmpty()); + } + + @Test + void testGetAssets() throws IOException { + final InputStream contents1 = new ByteArrayInputStream("content1".getBytes(StandardCharsets.UTF_8)); + final InputStream contents2 = new ByteArrayInputStream("content2".getBytes(StandardCharsets.UTF_8)); + final InputStream contents3 = new ByteArrayInputStream("content3".getBytes(StandardCharsets.UTF_8)); + + assetManager.createAsset(CONNECTOR_ID, "asset1.txt", contents1); + assetManager.createAsset(CONNECTOR_ID, "asset2.txt", contents2); + assetManager.createAsset("other-connector", "asset3.txt", contents3); + + final List connectorAssets = assetManager.getAssets(CONNECTOR_ID); + assertEquals(2, connectorAssets.size()); + assertTrue(connectorAssets.stream().allMatch(a -> a.getOwnerIdentifier().equals(CONNECTOR_ID))); + + final List otherAssets = assetManager.getAssets("other-connector"); + assertEquals(1, otherAssets.size()); + assertEquals("other-connector", otherAssets.get(0).getOwnerIdentifier()); + } + + @Test + void testGetAssetsReturnsEmptyListForUnknownOwner() { + final List assets = assetManager.getAssets("unknown-owner"); + + assertNotNull(assets); + assertTrue(assets.isEmpty()); + } + + @Test + void testCreateMissingAsset() { + final Asset missingAsset = assetManager.createMissingAsset(CONNECTOR_ID, ASSET_NAME); + + assertNotNull(missingAsset); + assertNotNull(missingAsset.getIdentifier()); + assertEquals(CONNECTOR_ID, missingAsset.getOwnerIdentifier()); + assertEquals(ASSET_NAME, missingAsset.getName()); + assertFalse(missingAsset.getFile().exists()); + assertTrue(missingAsset.getDigest().isEmpty()); + } + + @Test + void testCreateMissingAssetCanBeRetrieved() { + final Asset missingAsset = assetManager.createMissingAsset(CONNECTOR_ID, ASSET_NAME); + + final Optional retrieved = assetManager.getAsset(missingAsset.getIdentifier()); + assertTrue(retrieved.isPresent()); + assertEquals(missingAsset.getIdentifier(), retrieved.get().getIdentifier()); + + final List ownerAssets = assetManager.getAssets(CONNECTOR_ID); + assertEquals(1, ownerAssets.size()); + } + + @Test + void testDeleteAsset() throws IOException { + final InputStream contents = new ByteArrayInputStream(ASSET_CONTENTS.getBytes(StandardCharsets.UTF_8)); + final Asset createdAsset = assetManager.createAsset(CONNECTOR_ID, ASSET_NAME, contents); + final Path assetFilePath = createdAsset.getFile().toPath(); + assertTrue(Files.exists(assetFilePath)); + + final Optional deleted = assetManager.deleteAsset(createdAsset.getIdentifier()); + + assertTrue(deleted.isPresent()); + assertEquals(createdAsset.getIdentifier(), deleted.get().getIdentifier()); + assertFalse(Files.exists(assetFilePath)); + assertTrue(assetManager.getAsset(createdAsset.getIdentifier()).isEmpty()); + } + + @Test + void testDeleteAssetReturnsEmptyForNonExistent() { + final Optional deleted = assetManager.deleteAsset("non-existent-id"); + + assertTrue(deleted.isEmpty()); + } + + @Test + void testInitializeCreatesStorageDirectory() { + final Path newStorageDir = tempDir.resolve("new-storage"); + assertFalse(Files.exists(newStorageDir)); + + final MockConnectorAssetManager newManager = new MockConnectorAssetManager(); + final AssetManagerInitializationContext context = createInitializationContext(newStorageDir); + newManager.initialize(context); + + assertTrue(Files.exists(newStorageDir)); + assertTrue(Files.isDirectory(newStorageDir)); + } + + private AssetManagerInitializationContext createInitializationContext(final Path storageDirectory) { + final Map properties = new HashMap<>(); + properties.put("directory", storageDirectory.toAbsolutePath().toString()); + + return new AssetManagerInitializationContext() { + @Override + public AssetReferenceLookup getAssetReferenceLookup() { + return null; + } + + @Override + public Map getProperties() { + return properties; + } + + @Override + public NodeTypeProvider getNodeTypeProvider() { + return null; + } + }; + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/MockNiFiConnectorWebContextTest.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/MockNiFiConnectorWebContextTest.java new file mode 100644 index 000000000000..659fe2e8fb73 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/MockNiFiConnectorWebContextTest.java @@ -0,0 +1,80 @@ +/* + * 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.mock.connector.server; + +import org.apache.nifi.components.connector.Connector; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorRepository; +import org.apache.nifi.components.connector.FrameworkFlowContext; +import org.apache.nifi.web.NiFiConnectorWebContext; +import org.apache.nifi.web.NiFiConnectorWebContext.ConnectorWebContext; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class MockNiFiConnectorWebContextTest { + + private static final String CONNECTOR_ID = "test-connector"; + + @Mock + private ConnectorRepository connectorRepository; + + @Mock + private ConnectorNode connectorNode; + + @Mock + private Connector connector; + + @Mock + private FrameworkFlowContext workingFlowContext; + + @Mock + private FrameworkFlowContext activeFlowContext; + + @Test + void testGetConnectorWebContextReturnsConnectorAndFlowContexts() { + when(connectorRepository.getConnector(CONNECTOR_ID)).thenReturn(connectorNode); + when(connectorNode.getConnector()).thenReturn(connector); + when(connectorNode.getWorkingFlowContext()).thenReturn(workingFlowContext); + when(connectorNode.getActiveFlowContext()).thenReturn(activeFlowContext); + + final NiFiConnectorWebContext context = new MockNiFiConnectorWebContext(connectorRepository); + final ConnectorWebContext result = context.getConnectorWebContext(CONNECTOR_ID); + + assertNotNull(result); + assertEquals(connector, result.connector()); + assertEquals(workingFlowContext, result.workingFlowContext()); + assertEquals(activeFlowContext, result.activeFlowContext()); + } + + @Test + void testGetConnectorWebContextThrowsForUnknownConnector() { + when(connectorRepository.getConnector("unknown-id")).thenReturn(null); + + final NiFiConnectorWebContext context = new MockNiFiConnectorWebContext(connectorRepository); + + assertThrows(IllegalArgumentException.class, () -> context.getConnectorWebContext("unknown-id")); + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/MockStateManagerProviderTest.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/MockStateManagerProviderTest.java new file mode 100644 index 000000000000..d8ecf8e8729d --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/MockStateManagerProviderTest.java @@ -0,0 +1,267 @@ +/* + * 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.mock.connector.server; + +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateManager; +import org.apache.nifi.components.state.StateMap; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class MockStateManagerProviderTest { + + @Test + void testGetStateManagerReturnsSameInstance() { + final MockStateManagerProvider provider = new MockStateManagerProvider(); + final StateManager stateManager1 = provider.getStateManager("component1", false); + final StateManager stateManager2 = provider.getStateManager("component1", false); + + assertNotNull(stateManager1); + assertSame(stateManager1, stateManager2); + } + + @Test + void testGetStateManagerReturnsDifferentInstancesForDifferentComponents() { + final MockStateManagerProvider provider = new MockStateManagerProvider(); + final StateManager stateManager1 = provider.getStateManager("component1", false); + final StateManager stateManager2 = provider.getStateManager("component2", false); + + assertNotNull(stateManager1); + assertNotNull(stateManager2); + assertNotSame(stateManager1, stateManager2); + } + + @Test + void testOnComponentRemoved() { + final MockStateManagerProvider provider = new MockStateManagerProvider(); + final StateManager stateManager1 = provider.getStateManager("component1", false); + assertNotNull(stateManager1); + + provider.onComponentRemoved("component1"); + + final StateManager stateManager2 = provider.getStateManager("component1", false); + assertNotNull(stateManager2); + assertNotSame(stateManager1, stateManager2); + } + + @Test + void testShutdown() { + final MockStateManagerProvider provider = new MockStateManagerProvider(); + provider.getStateManager("component1", false); + provider.getStateManager("component2", false); + + provider.shutdown(); + + final StateManager stateManager = provider.getStateManager("component1", false); + assertNotNull(stateManager); + } + + @Test + void testClusterProviderEnabled() { + final MockStateManagerProvider provider = new MockStateManagerProvider(); + assertFalse(provider.isClusterProviderEnabled()); + + provider.enableClusterProvider(); + assertTrue(provider.isClusterProviderEnabled()); + + provider.disableClusterProvider(); + assertFalse(provider.isClusterProviderEnabled()); + } + + @Test + void testStateManagerSetAndGetLocalState() throws IOException { + final MockStateManagerProvider provider = new MockStateManagerProvider(); + final StateManager stateManager = provider.getStateManager("component1", false); + + final Map state = new HashMap<>(); + state.put("key1", "value1"); + state.put("key2", "value2"); + + stateManager.setState(state, Scope.LOCAL); + + final StateMap stateMap = stateManager.getState(Scope.LOCAL); + assertNotNull(stateMap); + assertTrue(stateMap.getStateVersion().isPresent()); + assertEquals("value1", stateMap.get("key1")); + assertEquals("value2", stateMap.get("key2")); + assertEquals(2, stateMap.toMap().size()); + } + + @Test + void testStateManagerSetAndGetClusterState() throws IOException { + final MockStateManagerProvider provider = new MockStateManagerProvider(); + final StateManager stateManager = provider.getStateManager("component1", false); + + final Map state = new HashMap<>(); + state.put("key1", "value1"); + state.put("key2", "value2"); + + stateManager.setState(state, Scope.CLUSTER); + + final StateMap stateMap = stateManager.getState(Scope.CLUSTER); + assertNotNull(stateMap); + assertTrue(stateMap.getStateVersion().isPresent()); + assertEquals("value1", stateMap.get("key1")); + assertEquals("value2", stateMap.get("key2")); + } + + @Test + void testStateManagerReplace() throws IOException { + final MockStateManagerProvider provider = new MockStateManagerProvider(); + final StateManager stateManager = provider.getStateManager("component1", false); + + final Map initialState = new HashMap<>(); + initialState.put("key1", "value1"); + stateManager.setState(initialState, Scope.LOCAL); + + final StateMap oldStateMap = stateManager.getState(Scope.LOCAL); + + final Map newState = new HashMap<>(); + newState.put("key1", "value2"); + + final boolean replaced = stateManager.replace(oldStateMap, newState, Scope.LOCAL); + assertTrue(replaced); + + final StateMap updatedStateMap = stateManager.getState(Scope.LOCAL); + assertEquals("value2", updatedStateMap.get("key1")); + + final boolean replacedAgain = stateManager.replace(oldStateMap, initialState, Scope.LOCAL); + assertFalse(replacedAgain); + } + + @Test + void testStateManagerClear() throws IOException { + final MockStateManagerProvider provider = new MockStateManagerProvider(); + final StateManager stateManager = provider.getStateManager("component1", false); + + final Map state = new HashMap<>(); + state.put("key1", "value1"); + stateManager.setState(state, Scope.LOCAL); + + stateManager.clear(Scope.LOCAL); + + final StateMap stateMap = stateManager.getState(Scope.LOCAL); + assertNotNull(stateMap); + assertTrue(stateMap.toMap().isEmpty()); + } + + @Test + void testStateKeyDropSupported() { + final MockStateManagerProvider provider = new MockStateManagerProvider(); + final StateManager stateManager1 = provider.getStateManager("component1", false); + assertFalse(stateManager1.isStateKeyDropSupported()); + + final StateManager stateManager2 = provider.getStateManager("component2", true); + assertTrue(stateManager2.isStateKeyDropSupported()); + } + + @Test + void testThreadSafety() throws InterruptedException, IOException { + final MockStateManagerProvider provider = new MockStateManagerProvider(); + final StateManager stateManager = provider.getStateManager("component1", false); + final int threadCount = 10; + final int operationsPerThread = 100; + final ExecutorService executorService = Executors.newFixedThreadPool(threadCount); + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch completionLatch = new CountDownLatch(threadCount); + final AtomicInteger successCount = new AtomicInteger(0); + + for (int i = 0; i < threadCount; i++) { + final int threadId = i; + executorService.submit(() -> { + try { + startLatch.await(); + for (int j = 0; j < operationsPerThread; j++) { + final Map state = new HashMap<>(); + state.put("thread", String.valueOf(threadId)); + state.put("operation", String.valueOf(j)); + stateManager.setState(state, Scope.LOCAL); + + final StateMap stateMap = stateManager.getState(Scope.LOCAL); + if (stateMap != null && stateMap.toMap().size() == 2) { + successCount.incrementAndGet(); + } + } + } catch (final Exception e) { + e.printStackTrace(); + } finally { + completionLatch.countDown(); + } + }); + } + + startLatch.countDown(); + completionLatch.await(30, TimeUnit.SECONDS); + executorService.shutdown(); + + assertEquals(threadCount * operationsPerThread, successCount.get()); + final StateMap finalState = stateManager.getState(Scope.LOCAL); + assertNotNull(finalState); + assertEquals(2, finalState.toMap().size()); + } + + @Test + void testEmptyStateMap() throws IOException { + final MockStateManagerProvider provider = new MockStateManagerProvider(); + final StateManager stateManager = provider.getStateManager("component1", false); + + final StateMap stateMap = stateManager.getState(Scope.LOCAL); + assertNotNull(stateMap); + assertFalse(stateMap.getStateVersion().isPresent()); + assertTrue(stateMap.toMap().isEmpty()); + } + + @Test + void testVersionIncrementing() throws IOException { + final MockStateManagerProvider provider = new MockStateManagerProvider(); + final StateManager stateManager = provider.getStateManager("component1", false); + + final Map state1 = new HashMap<>(); + state1.put("key", "value1"); + stateManager.setState(state1, Scope.LOCAL); + final StateMap stateMap1 = stateManager.getState(Scope.LOCAL); + + final Map state2 = new HashMap<>(); + state2.put("key", "value2"); + stateManager.setState(state2, Scope.LOCAL); + final StateMap stateMap2 = stateManager.getState(Scope.LOCAL); + + assertTrue(stateMap1.getStateVersion().isPresent()); + assertTrue(stateMap2.getStateVersion().isPresent()); + + final long version1 = Long.parseLong(stateMap1.getStateVersion().get()); + final long version2 = Long.parseLong(stateMap2.getStateVersion().get()); + assertTrue(version2 > version1); + } +} + diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/StandardConnectorMockServerJettyTest.java b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/StandardConnectorMockServerJettyTest.java new file mode 100644 index 000000000000..980d8341c975 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-server/src/test/java/org/apache/nifi/mock/connector/server/StandardConnectorMockServerJettyTest.java @@ -0,0 +1,186 @@ +/* + * 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.mock.connector.server; + +import org.apache.nifi.bundle.Bundle; +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.bundle.BundleDetails; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Map; +import java.util.Set; +import java.util.jar.JarEntry; +import java.util.jar.JarOutputStream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class StandardConnectorMockServerJettyTest { + + private static final String NAR_DEPENDENCIES_PATH = "NAR-INF/bundled-dependencies"; + + @TempDir + private Path tempDir; + + @Test + void testFindWarsDiscoversSingleWarFile() throws Exception { + final Path bundleWorkingDir = tempDir.resolve("test-bundle"); + final Path depsDir = bundleWorkingDir.resolve(NAR_DEPENDENCIES_PATH); + Files.createDirectories(depsDir); + + final Path warFile = depsDir.resolve("my-app.war"); + createConnectorWar(warFile); + + final Bundle bundle = createBundle(bundleWorkingDir); + final StandardConnectorMockServer server = new StandardConnectorMockServer(); + final Map wars = server.findWars(Set.of(bundle)); + + assertEquals(1, wars.size()); + assertTrue(wars.containsKey(warFile.toFile())); + assertEquals(bundle, wars.get(warFile.toFile())); + } + + @Test + void testFindWarsIgnoresNonWarFiles() throws Exception { + final Path bundleWorkingDir = tempDir.resolve("test-bundle"); + final Path depsDir = bundleWorkingDir.resolve(NAR_DEPENDENCIES_PATH); + Files.createDirectories(depsDir); + + Files.createFile(depsDir.resolve("some-lib.jar")); + Files.createFile(depsDir.resolve("config.xml")); + + final Bundle bundle = createBundle(bundleWorkingDir); + final StandardConnectorMockServer server = new StandardConnectorMockServer(); + final Map wars = server.findWars(Set.of(bundle)); + + assertTrue(wars.isEmpty()); + } + + @Test + void testFindWarsIgnoresWarWithoutConnectorManifest() throws Exception { + final Path bundleWorkingDir = tempDir.resolve("test-bundle"); + final Path depsDir = bundleWorkingDir.resolve(NAR_DEPENDENCIES_PATH); + Files.createDirectories(depsDir); + + final Path warFile = depsDir.resolve("non-connector.war"); + createWarWithoutConnectorManifest(warFile); + + final Bundle bundle = createBundle(bundleWorkingDir); + final StandardConnectorMockServer server = new StandardConnectorMockServer(); + final Map wars = server.findWars(Set.of(bundle)); + + assertTrue(wars.isEmpty()); + } + + @Test + void testFindWarsHandlesMissingDependenciesDirectory() throws Exception { + final Path bundleWorkingDir = tempDir.resolve("empty-bundle"); + Files.createDirectories(bundleWorkingDir); + + final Bundle bundle = createBundle(bundleWorkingDir); + final StandardConnectorMockServer server = new StandardConnectorMockServer(); + final Map wars = server.findWars(Set.of(bundle)); + + assertTrue(wars.isEmpty()); + } + + @Test + void testFindWarsDiscoversMultipleWarFiles() throws Exception { + final Path bundleWorkingDir = tempDir.resolve("multi-war-bundle"); + final Path depsDir = bundleWorkingDir.resolve(NAR_DEPENDENCIES_PATH); + Files.createDirectories(depsDir); + + createConnectorWar(depsDir.resolve("app-one.war")); + createConnectorWar(depsDir.resolve("app-two.war")); + Files.createFile(depsDir.resolve("some-lib.jar")); + + final Bundle bundle = createBundle(bundleWorkingDir); + final StandardConnectorMockServer server = new StandardConnectorMockServer(); + final Map wars = server.findWars(Set.of(bundle)); + + assertEquals(2, wars.size()); + } + + @Test + void testFindWarsFromMultipleBundles() throws Exception { + final Path bundleDir1 = tempDir.resolve("bundle-1"); + final Path depsDir1 = bundleDir1.resolve(NAR_DEPENDENCIES_PATH); + Files.createDirectories(depsDir1); + createConnectorWar(depsDir1.resolve("first-app.war")); + + final Path bundleDir2 = tempDir.resolve("bundle-2"); + final Path depsDir2 = bundleDir2.resolve(NAR_DEPENDENCIES_PATH); + Files.createDirectories(depsDir2); + createConnectorWar(depsDir2.resolve("second-app.war")); + + final Bundle bundle1 = createBundle(bundleDir1, "test-bundle-1"); + final Bundle bundle2 = createBundle(bundleDir2, "test-bundle-2"); + + final StandardConnectorMockServer server = new StandardConnectorMockServer(); + final Map wars = server.findWars(Set.of(bundle1, bundle2)); + + assertEquals(2, wars.size()); + } + + @Test + void testGetHttpPortReturnsNegativeOneWhenNoServer() { + final StandardConnectorMockServer server = new StandardConnectorMockServer(); + assertEquals(-1, server.getHttpPort()); + } + + @Test + void testFindWarsReturnsEmptyForEmptyBundleSet() throws Exception { + final StandardConnectorMockServer server = new StandardConnectorMockServer(); + final Map wars = server.findWars(Set.of()); + assertTrue(wars.isEmpty()); + } + + private static void createConnectorWar(final Path warPath) throws IOException { + try (final JarOutputStream jarOut = new JarOutputStream(new FileOutputStream(warPath.toFile()))) { + jarOut.putNextEntry(new JarEntry("META-INF/nifi-connector")); + jarOut.closeEntry(); + } + } + + private static void createWarWithoutConnectorManifest(final Path warPath) throws IOException { + try (final JarOutputStream jarOut = new JarOutputStream(new FileOutputStream(warPath.toFile()))) { + jarOut.putNextEntry(new JarEntry("WEB-INF/web.xml")); + jarOut.closeEntry(); + } + } + + private Bundle createBundle(final Path workingDir) { + return createBundle(workingDir, "test-bundle"); + } + + private Bundle createBundle(final Path workingDir, final String artifactId) { + final BundleDetails details = new BundleDetails.Builder() + .workingDir(workingDir.toFile()) + .coordinate(new BundleCoordinate("org.test", artifactId, "1.0.0")) + .build(); + + return new Bundle(details, ClassLoader.getSystemClassLoader()); + } + +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/pom.xml b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/pom.xml new file mode 100644 index 000000000000..88ab60703157 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/pom.xml @@ -0,0 +1,130 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-connector-mock-test-bundle + 2.9.0-SNAPSHOT + + + nifi-connector-mock-integration-tests + jar + + + + org.slf4j + slf4j-api + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-connector-mock + 2.9.0-SNAPSHOT + + + org.apache.nifi + nifi-connector-mock-api + 2.9.0-SNAPSHOT + + + org.apache.nifi + nifi-lookup-service-api + 2.9.0-SNAPSHOT + provided + + + + + org.junit.jupiter + junit-jupiter + test + + + + + org.apache.nifi + nifi-connector-mock-server-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-connector-mock-test-connectors-nar + 2.9.0-SNAPSHOT + nar + + + + + org.apache.nifi + nifi-standard-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-standard-services-api-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-standard-shared-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-update-attribute-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-lookup-services-nar + 2.9.0-SNAPSHOT + nar + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-nar + generate-resources + + copy-dependencies + + + ${project.build.directory}/libDir + jar,nar + compile + + + + + + + + diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/AllowableValuesIT.java b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/AllowableValuesIT.java new file mode 100644 index 000000000000..a350730b722a --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/AllowableValuesIT.java @@ -0,0 +1,87 @@ +/* + * 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.mock.connectors.tests; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.mock.connector.StandardConnectorTestRunner; +import org.apache.nifi.mock.connector.server.ConnectorConfigVerificationResult; +import org.apache.nifi.mock.connector.server.ConnectorTestRunner; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Integration tests that verify fetchable allowable values are properly validated + * during configuration verification. These tests exercise the fix for NIFI-15258, + * where {@code StandardConnectorNode.fetchAllowableValues} incorrectly used the + * active flow context instead of the working flow context passed as a parameter. + * + *

The {@code AllowableValuesConnector} returns allowable values only when the + * flow context type is WORKING. If the framework incorrectly passes the ACTIVE + * context, the connector returns an empty list, causing all values to be accepted.

+ */ +public class AllowableValuesIT { + + private static final String CONNECTOR_CLASS = "org.apache.nifi.mock.connectors.AllowableValuesConnector"; + + @Test + public void testVerifyConfigurationRejectsInvalidAllowableValue() throws IOException { + try (final ConnectorTestRunner runner = createRunner()) { + final ConnectorConfigVerificationResult result = runner.verifyConfiguration("Selection", Map.of("Color", "purple")); + + final List failedResults = result.getFailedResults(); + assertFalse(failedResults.isEmpty(), "Expected at least one failed result for invalid allowable value 'purple'"); + assertEquals(1, failedResults.size()); + + final String explanation = failedResults.getFirst().getExplanation(); + assertTrue(explanation.contains("allowable values"), "Expected explanation to mention allowable values but was: " + explanation); + } + } + + @Test + public void testVerifyConfigurationAcceptsValidAllowableValue() throws IOException { + try (final ConnectorTestRunner runner = createRunner()) { + final ConnectorConfigVerificationResult result = runner.verifyConfiguration("Selection", Map.of("Color", "red")); + result.assertNoFailures(); + } + } + + @Test + public void testVerifyConfigurationAcceptsAllValidAllowableValues() throws IOException { + try (final ConnectorTestRunner runner = createRunner()) { + for (final String validColor : List.of("red", "green", "blue")) { + final ConnectorConfigVerificationResult result = runner.verifyConfiguration("Selection", Map.of("Color", validColor)); + result.assertNoFailures(); + } + } + } + + private ConnectorTestRunner createRunner() { + return new StandardConnectorTestRunner.Builder() + .connectorClassName(CONNECTOR_CLASS) + .narLibraryDirectory(new File("target/libDir")) + .build(); + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/CreateConnectorIT.java b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/CreateConnectorIT.java new file mode 100644 index 000000000000..18884a8d8309 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/CreateConnectorIT.java @@ -0,0 +1,92 @@ +/* + * 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.mock.connectors.tests; + +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; +import org.apache.nifi.mock.connector.StandardConnectorTestRunner; +import org.apache.nifi.mock.connector.server.ConnectorTestRunner; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class CreateConnectorIT { + + @Test + public void testCreateStartAndStopGenerateAndUpdateConnector() throws IOException { + try (final ConnectorTestRunner testRunner = new StandardConnectorTestRunner.Builder() + .connectorClassName("org.apache.nifi.mock.connectors.GenerateAndLog") + .narLibraryDirectory(new File("target/libDir")) + .build()) { + + // Verify the active flow snapshot reflects the initial flow loaded from Generate_and_Update.json + final VersionedExternalFlow activeFlow = testRunner.getActiveFlowSnapshot(); + final VersionedProcessGroup rootGroup = activeFlow.getFlowContents(); + + final Set processors = rootGroup.getProcessors(); + assertEquals(3, processors.size()); + assertTrue(findProcessorByType(processors, "org.apache.nifi.processors.standard.GenerateFlowFile").isPresent()); + assertTrue(findProcessorByType(processors, "org.apache.nifi.processors.standard.LookupAttribute").isPresent()); + assertTrue(findProcessorByType(processors, "org.apache.nifi.processors.attributes.UpdateAttribute").isPresent()); + + assertEquals(2, rootGroup.getConnections().size()); + + final Set controllerServices = rootGroup.getControllerServices(); + assertEquals(1, controllerServices.size()); + assertEquals("org.apache.nifi.lookup.SimpleKeyValueLookupService", controllerServices.iterator().next().getType()); + + testRunner.startConnector(); + testRunner.stopConnector(); + } + } + + @Test + public void testConnectorWithMissingBundleFailsValidate() throws IOException { + + try (final ConnectorTestRunner testRunner = new StandardConnectorTestRunner.Builder() + .connectorClassName("org.apache.nifi.mock.connectors.MissingBundleConnector") + .narLibraryDirectory(new File("target/libDir")) + .build()) { + + final List results = testRunner.validate(); + assertEquals(results.size(), 1); + final String message = results.getFirst().getExplanation(); + assertTrue(message.contains("com.example.nonexistent:missing-nar:1.0.0"), "Expected exception message to contain missing bundle coordinates but was: " + message); + assertTrue(message.contains("com.example.nonexistent.MissingProcessor"), "Expected exception message to contain missing processor type but was: " + message); + } + } + + private Optional findProcessorByType(final Set processors, final String type) { + for (final VersionedProcessor processor : processors) { + if (type.equals(processor.getType())) { + return Optional.of(processor); + } + } + return Optional.empty(); + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/CronScheduleIT.java b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/CronScheduleIT.java new file mode 100644 index 000000000000..10dea97a4327 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/CronScheduleIT.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.mock.connectors.tests; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.mock.connector.StandardConnectorTestRunner; +import org.apache.nifi.mock.connector.server.ConnectorConfigVerificationResult; +import org.apache.nifi.mock.connector.server.ConnectorTestRunner; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertFalse; + +/** + * Integration tests that verify CRON expression validation during configuration verification. + * The {@code CronScheduleConnector} maps the "Trigger Schedule" connector property to a + * CRON-driven GenerateFlowFile processor's scheduling period parameter, and verification + * delegates to the processor's validation which checks the CRON expression. + */ +public class CronScheduleIT { + + private static final String CONNECTOR_CLASS = "org.apache.nifi.mock.connectors.CronScheduleConnector"; + + @Test + public void testValidCronExpression() throws IOException { + try (final ConnectorTestRunner runner = createRunner()) { + final ConnectorConfigVerificationResult result = runner.verifyConfiguration("Schedule", + Map.of("Trigger Schedule", "0 0 * * * *")); + result.assertNoFailures(); + } + } + + @Test + public void testInvalidCronExpression() throws IOException { + try (final ConnectorTestRunner runner = createRunner()) { + final ConnectorConfigVerificationResult result = runner.verifyConfiguration("Schedule", + Map.of("Trigger Schedule", "invalid-cron")); + final List failedResults = result.getFailedResults(); + assertFalse(failedResults.isEmpty()); + } + } + + private ConnectorTestRunner createRunner() { + return new StandardConnectorTestRunner.Builder() + .connectorClassName(CONNECTOR_CLASS) + .narLibraryDirectory(new File("target/libDir")) + .build(); + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/MockControllerServiceIT.java b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/MockControllerServiceIT.java new file mode 100644 index 000000000000..1f044ccf21c6 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/MockControllerServiceIT.java @@ -0,0 +1,87 @@ +/* + * 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.mock.connectors.tests; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.Validator; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.lookup.LookupFailureException; +import org.apache.nifi.lookup.StringLookupService; +import org.apache.nifi.mock.connector.StandardConnectorTestRunner; +import org.apache.nifi.mock.connector.server.ConnectorTestRunner; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.File; +import java.io.IOException; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class MockControllerServiceIT { + + @Test + @Timeout(10) + public void testMockControllerService() throws IOException { + try (final ConnectorTestRunner runner = new StandardConnectorTestRunner.Builder() + .narLibraryDirectory(new File("target/libDir")) + .connectorClassName("org.apache.nifi.mock.connectors.GenerateAndLog") + .mockControllerService("org.apache.nifi.lookup.SimpleKeyValueLookupService", MockStringLookupService.class) + .build()) { + + runner.startConnector(); + + // Wait until MockStringLookupService.lookup is invoked at least once. + // We use @Timeout on the test to avoid hanging indefinitely in case of failure. + while (MockStringLookupService.lookupCounter.get() < 1) { + Thread.yield(); + } + + assertTrue(MockStringLookupService.lookupCounter.get() >= 1); + + runner.stopConnector(); + } + } + + public static class MockStringLookupService extends AbstractControllerService implements StringLookupService { + private static final AtomicLong lookupCounter = new AtomicLong(0L); + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .name(propertyDescriptorName) + .addValidator(Validator.VALID) + .dynamic(true) + .build(); + } + + @Override + public Optional lookup(final Map coordinates) throws LookupFailureException { + lookupCounter.incrementAndGet(); + return Optional.of("mock-value"); + } + + @Override + public Set getRequiredKeys() { + return Set.of("key"); + } + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/MockProcessorIT.java b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/MockProcessorIT.java new file mode 100644 index 000000000000..465d0340002b --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/src/test/java/org/apache/nifi/mock/connectors/tests/MockProcessorIT.java @@ -0,0 +1,91 @@ +/* + * 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.mock.connectors.tests; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.Validator; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.mock.connector.StandardConnectorTestRunner; +import org.apache.nifi.mock.connector.server.ConnectorTestRunner; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.File; +import java.io.IOException; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; + +public class MockProcessorIT { + + @Test + @Timeout(10) + public void testMockProcessor() throws IOException { + try (final ConnectorTestRunner runner = new StandardConnectorTestRunner.Builder() + .narLibraryDirectory(new File("target/libDir")) + .connectorClassName("org.apache.nifi.mock.connectors.GenerateAndLog") + .mockProcessor("org.apache.nifi.processors.attributes.UpdateAttribute", MockProcessor.class) + .build()) { + + runner.startConnector(); + + // Wait until MockProcessor is triggered at least once. We use @Timeout on the test to avoid + // hanging indefinitely in case of failure. + while (MockProcessor.invocationCounter.get() < 1) { + Thread.yield(); + } + runner.stopConnector(); + } + } + + public static class MockProcessor extends AbstractProcessor { + private static final AtomicLong invocationCounter = new AtomicLong(0L); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .build(); + + @Override + public Set getRelationships() { + return Set.of(REL_SUCCESS); + } + + // Support any properties so that properties of UpdateAttribute can be set without the Processor becoming invalid + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .name(propertyDescriptorName) + .addValidator(Validator.VALID) + .build(); + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + invocationCounter.incrementAndGet(); + + final FlowFile flowFile = session.get(); + if (flowFile != null) { + session.transfer(flowFile, REL_SUCCESS); + } + } + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors-nar/pom.xml b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors-nar/pom.xml new file mode 100644 index 000000000000..c2d17f8b288a --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors-nar/pom.xml @@ -0,0 +1,35 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-connector-mock-test-bundle + 2.9.0-SNAPSHOT + + + nifi-connector-mock-test-connectors-nar + nar + + + + org.apache.nifi + nifi-connector-mock-test-connectors + 2.9.0-SNAPSHOT + + + + diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/pom.xml b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/pom.xml new file mode 100644 index 000000000000..479fa9db0784 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/pom.xml @@ -0,0 +1,48 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-connector-mock-test-bundle + 2.9.0-SNAPSHOT + + + nifi-connector-mock-test-connectors + jar + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-connector-utils + 2.9.0-SNAPSHOT + + + + + + org.junit.jupiter + junit-jupiter + test + + + + + diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/AllowableValuesConnector.java b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/AllowableValuesConnector.java new file mode 100644 index 000000000000..a263482eda28 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/AllowableValuesConnector.java @@ -0,0 +1,106 @@ +/* + * 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.mock.connectors; + +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.ConnectorPropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorPropertyGroup; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.components.FlowContextType; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.VersionedExternalFlow; + +import java.util.List; +import java.util.Map; + +/** + * A test connector with a configuration step that has fetchable allowable values. + * The {@link #fetchAllowableValues} method returns allowable values only when invoked + * with a WORKING {@link FlowContextType}. If invoked with an ACTIVE context, it returns + * an empty list. This design verifies that the framework passes the correct (working) + * context during configuration verification, reproducing the scenario fixed in NIFI-15258. + */ +public class AllowableValuesConnector extends AbstractConnector { + + static final ConnectorPropertyDescriptor COLOR = new ConnectorPropertyDescriptor.Builder() + .name("Color") + .description("The color to select") + .allowableValuesFetchable(true) + .required(true) + .build(); + + static final ConnectorPropertyGroup SELECTION_GROUP = new ConnectorPropertyGroup.Builder() + .name("Color Selection") + .addProperty(COLOR) + .build(); + + static final ConfigurationStep SELECTION_STEP = new ConfigurationStep.Builder() + .name("Selection") + .propertyGroups(List.of(SELECTION_GROUP)) + .build(); + + private static final List CONFIGURATION_STEPS = List.of(SELECTION_STEP); + + @Override + public VersionedExternalFlow getInitialFlow() { + return VersionedFlowUtils.loadFlowFromResource("flows/Generate_and_Update.json"); + } + + @Override + public List getConfigurationSteps() { + return CONFIGURATION_STEPS; + } + + @Override + protected void onStepConfigured(final String stepName, final FlowContext flowContext) { + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext workingContext) { + return List.of(); + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName, final FlowContext flowContext) { + if ("Selection".equals(stepName) && "Color".equals(propertyName)) { + if (flowContext.getType() == FlowContextType.WORKING) { + return List.of( + new AllowableValue("red", "Red"), + new AllowableValue("green", "Green"), + new AllowableValue("blue", "Blue") + ); + } + + // Return empty when the context is ACTIVE, simulating the pre-fix behavior + // where using the wrong context caused allowable values to be empty and + // any value to be incorrectly accepted. + return List.of(); + } + + return super.fetchAllowableValues(stepName, propertyName, flowContext); + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/CronScheduleConnector.java b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/CronScheduleConnector.java new file mode 100644 index 000000000000..1e257c245c6d --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/CronScheduleConnector.java @@ -0,0 +1,106 @@ +/* + * 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.mock.connectors; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.ConnectorConfigurationContext; +import org.apache.nifi.components.connector.ConnectorPropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorPropertyGroup; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.components.ProcessorFacade; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.VersionedExternalFlow; + +import java.util.List; +import java.util.Map; + +/** + * A test connector that validates CRON scheduling expressions via the "Trigger Schedule" parameter. + * The underlying flow ({@code Cron_Schedule_Connector.json}) contains a CRON-driven + * GenerateFlowFile processor whose scheduling period references the {@code #{Trigger Schedule}} parameter. + * Verification delegates to the processor's validate/verify cycle, which checks the CRON expression. + */ +public class CronScheduleConnector extends AbstractConnector { + + static final String SCHEDULE_STEP_NAME = "Schedule"; + static final String TRIGGER_SCHEDULE_PARAM = "Trigger Schedule"; + + static final ConnectorPropertyDescriptor TRIGGER_SCHEDULE = new ConnectorPropertyDescriptor.Builder() + .name(TRIGGER_SCHEDULE_PARAM) + .description("CRON expression for the GenerateFlowFile trigger schedule") + .required(true) + .build(); + + static final ConnectorPropertyGroup SCHEDULE_GROUP = new ConnectorPropertyGroup.Builder() + .name("Schedule Settings") + .addProperty(TRIGGER_SCHEDULE) + .build(); + + static final ConfigurationStep SCHEDULE_STEP = new ConfigurationStep.Builder() + .name(SCHEDULE_STEP_NAME) + .propertyGroups(List.of(SCHEDULE_GROUP)) + .build(); + + @Override + public VersionedExternalFlow getInitialFlow() { + return VersionedFlowUtils.loadFlowFromResource("flows/Cron_Schedule_Connector.json"); + } + + @Override + public List getConfigurationSteps() { + return List.of(SCHEDULE_STEP); + } + + @Override + protected void onStepConfigured(final String stepName, final FlowContext flowContext) throws FlowUpdateException { + if (SCHEDULE_STEP_NAME.equals(stepName)) { + final String triggerSchedule = flowContext.getConfigurationContext() + .getProperty(SCHEDULE_STEP_NAME, TRIGGER_SCHEDULE_PARAM).getValue(); + final VersionedExternalFlow flow = getInitialFlow(); + VersionedFlowUtils.setParameterValue(flow, TRIGGER_SCHEDULE_PARAM, triggerSchedule); + getInitializationContext().updateFlow(flowContext, flow); + } + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext flowContext) { + if (SCHEDULE_STEP_NAME.equals(stepName)) { + final ConnectorConfigurationContext configContext = flowContext.getConfigurationContext().createWithOverrides(stepName, overrides); + final String triggerSchedule = configContext.getProperty(SCHEDULE_STEP_NAME, TRIGGER_SCHEDULE_PARAM).getValue(); + + final VersionedExternalFlow flow = getInitialFlow(); + VersionedFlowUtils.setParameterValue(flow, TRIGGER_SCHEDULE_PARAM, triggerSchedule); + + final ProcessorFacade generateFlowFile = flowContext.getRootGroup().getProcessors().stream() + .filter(p -> p.getDefinition().getType().endsWith("GenerateFlowFile")) + .findFirst() + .orElseThrow(); + + return generateFlowFile.verify(flow, Map.of()); + } + + return List.of(); + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/GenerateAndLog.java b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/GenerateAndLog.java new file mode 100644 index 000000000000..96b9fa63c38e --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/GenerateAndLog.java @@ -0,0 +1,55 @@ +/* + * 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.mock.connectors; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.VersionedExternalFlow; + +import java.util.List; +import java.util.Map; + +public class GenerateAndLog extends AbstractConnector { + + @Override + public VersionedExternalFlow getInitialFlow() { + return VersionedFlowUtils.loadFlowFromResource("flows/Generate_and_Update.json"); + } + + @Override + protected void onStepConfigured(final String stepName, final FlowContext flowContext) { + } + + @Override + public List getConfigurationSteps() { + return List.of(); + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext workingContext) { + return List.of(); + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/MissingBundleConnector.java b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/MissingBundleConnector.java new file mode 100644 index 000000000000..d49674ffd038 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/java/org/apache/nifi/mock/connectors/MissingBundleConnector.java @@ -0,0 +1,108 @@ +/* + * 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.mock.connectors; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.Position; +import org.apache.nifi.flow.ScheduledState; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +/** + * A test connector that returns an initial flow containing a processor with a bundle that does not exist. + * This is used to test the behavior when a connector's initial flow references unavailable components. + */ +public class MissingBundleConnector extends AbstractConnector { + + @Override + public VersionedExternalFlow getInitialFlow() { + final VersionedProcessGroup rootGroup = new VersionedProcessGroup(); + rootGroup.setIdentifier(UUID.randomUUID().toString()); + rootGroup.setInstanceIdentifier(UUID.randomUUID().toString()); + rootGroup.setName("Missing Bundle Connector Flow"); + rootGroup.setPosition(new Position(0.0, 0.0)); + rootGroup.setProcessGroups(new HashSet<>()); + rootGroup.setConnections(new HashSet<>()); + rootGroup.setInputPorts(new HashSet<>()); + rootGroup.setOutputPorts(new HashSet<>()); + rootGroup.setControllerServices(new HashSet<>()); + rootGroup.setFunnels(new HashSet<>()); + rootGroup.setLabels(new HashSet<>()); + + final VersionedProcessor missingProcessor = new VersionedProcessor(); + missingProcessor.setIdentifier(UUID.randomUUID().toString()); + missingProcessor.setInstanceIdentifier(UUID.randomUUID().toString()); + missingProcessor.setName("Missing Processor"); + missingProcessor.setType("com.example.nonexistent.MissingProcessor"); + missingProcessor.setPosition(new Position(100.0, 100.0)); + missingProcessor.setScheduledState(ScheduledState.ENABLED); + missingProcessor.setSchedulingPeriod("0 sec"); + missingProcessor.setSchedulingStrategy("TIMER_DRIVEN"); + missingProcessor.setExecutionNode("ALL"); + missingProcessor.setPenaltyDuration("30 sec"); + missingProcessor.setYieldDuration("1 sec"); + missingProcessor.setBulletinLevel("WARN"); + missingProcessor.setRunDurationMillis(0L); + missingProcessor.setConcurrentlySchedulableTaskCount(1); + missingProcessor.setAutoTerminatedRelationships(new HashSet<>()); + missingProcessor.setProperties(Map.of()); + missingProcessor.setPropertyDescriptors(Map.of()); + missingProcessor.setGroupIdentifier(rootGroup.getIdentifier()); + + final Bundle missingBundle = new Bundle(); + missingBundle.setGroup("com.example.nonexistent"); + missingBundle.setArtifact("missing-nar"); + missingBundle.setVersion("1.0.0"); + missingProcessor.setBundle(missingBundle); + + rootGroup.setProcessors(new HashSet<>(List.of(missingProcessor))); + + final VersionedExternalFlow externalFlow = new VersionedExternalFlow(); + externalFlow.setFlowContents(rootGroup); + return externalFlow; + } + + @Override + protected void onStepConfigured(final String stepName, final FlowContext flowContext) { + } + + @Override + public List getConfigurationSteps() { + return List.of(); + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext workingContext) { + return List.of(); + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/resources/META-INF/services/org.apache.nifi.components.connector.Connector b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/resources/META-INF/services/org.apache.nifi.components.connector.Connector new file mode 100644 index 000000000000..379d9026452a --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/resources/META-INF/services/org.apache.nifi.components.connector.Connector @@ -0,0 +1,19 @@ +# 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. + +org.apache.nifi.mock.connectors.AllowableValuesConnector +org.apache.nifi.mock.connectors.GenerateAndLog +org.apache.nifi.mock.connectors.MissingBundleConnector +org.apache.nifi.mock.connectors.CronScheduleConnector \ No newline at end of file diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/resources/flows/Cron_Schedule_Connector.json b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/resources/flows/Cron_Schedule_Connector.json new file mode 100644 index 000000000000..d078d4d6fa83 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/resources/flows/Cron_Schedule_Connector.json @@ -0,0 +1,410 @@ +{ + "flowContents": { + "identifier": "1800c04e-f9b9-3293-bfc7-b35f43e0706c", + "instanceIdentifier": "4f8ca484-019c-1000-955f-68c5defeb22b", + "name": "Cron_Schedule_Connector", + "comments": "", + "position": { + "x": -1254.0, + "y": -437.70139741897583 + }, + "processGroups": [], + "remoteProcessGroups": [], + "processors": [ + { + "identifier": "4d4160b1-736c-3be4-bc3a-84fc6eb4ad2a", + "instanceIdentifier": "499f9781-71c5-367f-aa17-5441bff29de9", + "name": "UpdateAttribute", + "comments": "", + "position": { + "x": -360.0, + "y": 24.0 + }, + "type": "org.apache.nifi.processors.attributes.UpdateAttribute", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-update-attribute-nar", + "version": "2026.1.20.21-SNAPSHOT" + }, + "properties": { + "Delete Attributes Expression": null, + "Store State": "Do not store state", + "Cache Value Lookup Cache Size": "100", + "Stateful Variables Initial Value": null + }, + "propertyDescriptors": { + "Delete Attributes Expression": { + "name": "Delete Attributes Expression", + "displayName": "Delete Attributes Expression", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Store State": { + "name": "Store State", + "displayName": "Store State", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Cache Value Lookup Cache Size": { + "name": "Cache Value Lookup Cache Size", + "displayName": "Cache Value Lookup Cache Size", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Stateful Variables Initial Value": { + "name": "Stateful Variables Initial Value", + "displayName": "Stateful Variables Initial Value", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + } + }, + "style": {}, + "schedulingPeriod": "0 sec", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 25, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [ + "success" + ], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "1800c04e-f9b9-3293-bfc7-b35f43e0706c" + }, + { + "identifier": "54ab8572-0e10-39a5-b553-931f9c253023", + "instanceIdentifier": "6d6b9cd3-6d31-330a-40f9-185959ad1c78", + "name": "GenerateFlowFile", + "comments": "", + "position": { + "x": -360.0, + "y": -371.5 + }, + "type": "org.apache.nifi.processors.standard.GenerateFlowFile", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-nar", + "version": "2026.1.20.21-SNAPSHOT" + }, + "properties": { + "File Size": "0B", + "Batch Size": "1", + "Unique FlowFiles": "false", + "Mime Type": null, + "Custom Text": null, + "Character Set": "UTF-8", + "Data Format": "Text", + "key": "test.key" + }, + "propertyDescriptors": { + "File Size": { + "name": "File Size", + "displayName": "File Size", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Batch Size": { + "name": "Batch Size", + "displayName": "Batch Size", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Unique FlowFiles": { + "name": "Unique FlowFiles", + "displayName": "Unique FlowFiles", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Mime Type": { + "name": "Mime Type", + "displayName": "Mime Type", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Custom Text": { + "name": "Custom Text", + "displayName": "Custom Text", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Character Set": { + "name": "Character Set", + "displayName": "Character Set", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Data Format": { + "name": "Data Format", + "displayName": "Data Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "key": { + "name": "key", + "displayName": "key", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": true + } + }, + "style": {}, + "schedulingPeriod": "#{Trigger Schedule}", + "schedulingStrategy": "CRON_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 0, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "1800c04e-f9b9-3293-bfc7-b35f43e0706c" + }, + { + "identifier": "34c1ac1b-1f21-3fd6-a734-726d5b142b7a", + "instanceIdentifier": "4f8d0670-019c-1000-1ac6-c81ef75a70d0", + "name": "LookupAttribute", + "comments": "", + "position": { + "x": -360.0, + "y": -168.0 + }, + "type": "org.apache.nifi.processors.standard.LookupAttribute", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-nar", + "version": "2026.1.20.21-SNAPSHOT" + }, + "properties": { + "Lookup Service": "b013f870-aee8-3cc4-b022-ac385ded928d", + "test.attribute": "${key}", + "Include Empty Values": "true" + }, + "propertyDescriptors": { + "Lookup Service": { + "name": "Lookup Service", + "displayName": "Lookup Service", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "test.attribute": { + "name": "test.attribute", + "displayName": "test.attribute", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": true + }, + "Include Empty Values": { + "name": "Include Empty Values", + "displayName": "Include Empty Values", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + } + }, + "style": {}, + "schedulingPeriod": "0 sec", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 0, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [ + "failure" + ], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "1800c04e-f9b9-3293-bfc7-b35f43e0706c" + } + ], + "inputPorts": [], + "outputPorts": [], + "connections": [ + { + "identifier": "893ad2c9-4a07-3447-b772-6b7149cfd6c1", + "instanceIdentifier": "5b9c96a4-7982-3746-2937-45511fb20c96", + "name": "", + "source": { + "id": "54ab8572-0e10-39a5-b553-931f9c253023", + "type": "PROCESSOR", + "groupId": "1800c04e-f9b9-3293-bfc7-b35f43e0706c", + "name": "GenerateFlowFile", + "comments": "", + "instanceIdentifier": "6d6b9cd3-6d31-330a-40f9-185959ad1c78" + }, + "destination": { + "id": "34c1ac1b-1f21-3fd6-a734-726d5b142b7a", + "type": "PROCESSOR", + "groupId": "1800c04e-f9b9-3293-bfc7-b35f43e0706c", + "name": "LookupAttribute", + "comments": "", + "instanceIdentifier": "4f8d0670-019c-1000-1ac6-c81ef75a70d0" + }, + "labelIndex": 0, + "zIndex": 1, + "selectedRelationships": [ + "success" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "1800c04e-f9b9-3293-bfc7-b35f43e0706c" + }, + { + "identifier": "f63459de-cf1f-3773-8f93-405518e085e4", + "instanceIdentifier": "4f93c5e2-019c-1000-2bea-06d8d611d5f6", + "name": "", + "source": { + "id": "34c1ac1b-1f21-3fd6-a734-726d5b142b7a", + "type": "PROCESSOR", + "groupId": "1800c04e-f9b9-3293-bfc7-b35f43e0706c", + "name": "LookupAttribute", + "comments": "", + "instanceIdentifier": "4f8d0670-019c-1000-1ac6-c81ef75a70d0" + }, + "destination": { + "id": "4d4160b1-736c-3be4-bc3a-84fc6eb4ad2a", + "type": "PROCESSOR", + "groupId": "1800c04e-f9b9-3293-bfc7-b35f43e0706c", + "name": "UpdateAttribute", + "comments": "", + "instanceIdentifier": "499f9781-71c5-367f-aa17-5441bff29de9" + }, + "labelIndex": 0, + "zIndex": 2, + "selectedRelationships": [ + "matched", + "unmatched" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "1800c04e-f9b9-3293-bfc7-b35f43e0706c" + } + ], + "labels": [], + "funnels": [], + "controllerServices": [ + { + "identifier": "b013f870-aee8-3cc4-b022-ac385ded928d", + "instanceIdentifier": "4f8d53d4-019c-1000-d376-abfaa091dc37", + "name": "SimpleKeyValueLookupService", + "comments": "", + "type": "org.apache.nifi.lookup.SimpleKeyValueLookupService", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-lookup-services-nar", + "version": "2026.1.20.21-SNAPSHOT" + }, + "properties": { + "test.key": "Test Value" + }, + "propertyDescriptors": { + "test.key": { + "name": "test.key", + "displayName": "test.key", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": true + } + }, + "controllerServiceApis": [ + { + "type": "org.apache.nifi.lookup.StringLookupService", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-services-api-nar", + "version": "2026.1.20.21-SNAPSHOT" + } + }, + { + "type": "org.apache.nifi.lookup.LookupService", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-services-api-nar", + "version": "2026.1.20.21-SNAPSHOT" + } + } + ], + "scheduledState": "DISABLED", + "bulletinLevel": "WARN", + "componentType": "CONTROLLER_SERVICE", + "groupIdentifier": "1800c04e-f9b9-3293-bfc7-b35f43e0706c" + } + ], + "parameterContextName": "Generate Parameters", + "defaultFlowFileExpiration": "0 sec", + "defaultBackPressureObjectThreshold": 10000, + "defaultBackPressureDataSizeThreshold": "1 GB", + "scheduledState": "ENABLED", + "executionEngine": "INHERITED", + "maxConcurrentTasks": 1, + "statelessFlowTimeout": "1 min", + "flowFileConcurrency": "UNBOUNDED", + "flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE", + "componentType": "PROCESS_GROUP" + }, + "externalControllerServices": {}, + "parameterContexts": { + "Generate Parameters": { + "name": "Generate Parameters", + "parameters": [ + { + "name": "Trigger Schedule", + "description": "", + "sensitive": false, + "provided": false, + "value": "" + } + ], + "inheritedParameterContexts": [], + "description": "", + "componentType": "PARAMETER_CONTEXT" + } + }, + "flowEncodingVersion": "1.0", + "parameterProviders": {}, + "latest": false +} \ No newline at end of file diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/resources/flows/Generate_and_Update.json b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/resources/flows/Generate_and_Update.json new file mode 100644 index 000000000000..d4f5fc798ede --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-test-connectors/src/main/resources/flows/Generate_and_Update.json @@ -0,0 +1,393 @@ +{ + "flowContents": { + "identifier": "1800c04e-f9b9-3293-bfc7-b35f43e0706c", + "instanceIdentifier": "4f8ca484-019c-1000-955f-68c5defeb22b", + "name": "Generate_and_Update", + "comments": "", + "position": { + "x": -1254.0, + "y": -437.70139741897583 + }, + "processGroups": [], + "remoteProcessGroups": [], + "processors": [ + { + "identifier": "4d4160b1-736c-3be4-bc3a-84fc6eb4ad2a", + "instanceIdentifier": "499f9781-71c5-367f-aa17-5441bff29de9", + "name": "UpdateAttribute", + "comments": "", + "position": { + "x": -360.0, + "y": 24.0 + }, + "type": "org.apache.nifi.processors.attributes.UpdateAttribute", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-update-attribute-nar", + "version": "2026.1.20.21-SNAPSHOT" + }, + "properties": { + "Delete Attributes Expression": null, + "Store State": "Do not store state", + "Cache Value Lookup Cache Size": "100", + "Stateful Variables Initial Value": null + }, + "propertyDescriptors": { + "Delete Attributes Expression": { + "name": "Delete Attributes Expression", + "displayName": "Delete Attributes Expression", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Store State": { + "name": "Store State", + "displayName": "Store State", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Cache Value Lookup Cache Size": { + "name": "Cache Value Lookup Cache Size", + "displayName": "Cache Value Lookup Cache Size", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Stateful Variables Initial Value": { + "name": "Stateful Variables Initial Value", + "displayName": "Stateful Variables Initial Value", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + } + }, + "style": {}, + "schedulingPeriod": "0 sec", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 25, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [ + "success" + ], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "1800c04e-f9b9-3293-bfc7-b35f43e0706c" + }, + { + "identifier": "54ab8572-0e10-39a5-b553-931f9c253023", + "instanceIdentifier": "6d6b9cd3-6d31-330a-40f9-185959ad1c78", + "name": "GenerateFlowFile", + "comments": "", + "position": { + "x": -360.0, + "y": -371.5 + }, + "type": "org.apache.nifi.processors.standard.GenerateFlowFile", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-nar", + "version": "2026.1.20.21-SNAPSHOT" + }, + "properties": { + "File Size": "0B", + "Batch Size": "1", + "Unique FlowFiles": "false", + "Mime Type": null, + "Custom Text": null, + "Character Set": "UTF-8", + "Data Format": "Text", + "key": "test.key" + }, + "propertyDescriptors": { + "File Size": { + "name": "File Size", + "displayName": "File Size", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Batch Size": { + "name": "Batch Size", + "displayName": "Batch Size", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Unique FlowFiles": { + "name": "Unique FlowFiles", + "displayName": "Unique FlowFiles", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Mime Type": { + "name": "Mime Type", + "displayName": "Mime Type", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Custom Text": { + "name": "Custom Text", + "displayName": "Custom Text", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Character Set": { + "name": "Character Set", + "displayName": "Character Set", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Data Format": { + "name": "Data Format", + "displayName": "Data Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "key": { + "name": "key", + "displayName": "key", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": true + } + }, + "style": {}, + "schedulingPeriod": "0 sec", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 0, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "1800c04e-f9b9-3293-bfc7-b35f43e0706c" + }, + { + "identifier": "34c1ac1b-1f21-3fd6-a734-726d5b142b7a", + "instanceIdentifier": "4f8d0670-019c-1000-1ac6-c81ef75a70d0", + "name": "LookupAttribute", + "comments": "", + "position": { + "x": -360.0, + "y": -168.0 + }, + "type": "org.apache.nifi.processors.standard.LookupAttribute", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-nar", + "version": "2026.1.20.21-SNAPSHOT" + }, + "properties": { + "Lookup Service": "b013f870-aee8-3cc4-b022-ac385ded928d", + "test.attribute": "${key}", + "Include Empty Values": "true" + }, + "propertyDescriptors": { + "Lookup Service": { + "name": "Lookup Service", + "displayName": "Lookup Service", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "test.attribute": { + "name": "test.attribute", + "displayName": "test.attribute", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": true + }, + "Include Empty Values": { + "name": "Include Empty Values", + "displayName": "Include Empty Values", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + } + }, + "style": {}, + "schedulingPeriod": "0 sec", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 0, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [ + "failure" + ], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "1800c04e-f9b9-3293-bfc7-b35f43e0706c" + } + ], + "inputPorts": [], + "outputPorts": [], + "connections": [ + { + "identifier": "893ad2c9-4a07-3447-b772-6b7149cfd6c1", + "instanceIdentifier": "5b9c96a4-7982-3746-2937-45511fb20c96", + "name": "", + "source": { + "id": "54ab8572-0e10-39a5-b553-931f9c253023", + "type": "PROCESSOR", + "groupId": "1800c04e-f9b9-3293-bfc7-b35f43e0706c", + "name": "GenerateFlowFile", + "comments": "", + "instanceIdentifier": "6d6b9cd3-6d31-330a-40f9-185959ad1c78" + }, + "destination": { + "id": "34c1ac1b-1f21-3fd6-a734-726d5b142b7a", + "type": "PROCESSOR", + "groupId": "1800c04e-f9b9-3293-bfc7-b35f43e0706c", + "name": "LookupAttribute", + "comments": "", + "instanceIdentifier": "4f8d0670-019c-1000-1ac6-c81ef75a70d0" + }, + "labelIndex": 0, + "zIndex": 1, + "selectedRelationships": [ + "success" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "1800c04e-f9b9-3293-bfc7-b35f43e0706c" + }, + { + "identifier": "f63459de-cf1f-3773-8f93-405518e085e4", + "instanceIdentifier": "4f93c5e2-019c-1000-2bea-06d8d611d5f6", + "name": "", + "source": { + "id": "34c1ac1b-1f21-3fd6-a734-726d5b142b7a", + "type": "PROCESSOR", + "groupId": "1800c04e-f9b9-3293-bfc7-b35f43e0706c", + "name": "LookupAttribute", + "comments": "", + "instanceIdentifier": "4f8d0670-019c-1000-1ac6-c81ef75a70d0" + }, + "destination": { + "id": "4d4160b1-736c-3be4-bc3a-84fc6eb4ad2a", + "type": "PROCESSOR", + "groupId": "1800c04e-f9b9-3293-bfc7-b35f43e0706c", + "name": "UpdateAttribute", + "comments": "", + "instanceIdentifier": "499f9781-71c5-367f-aa17-5441bff29de9" + }, + "labelIndex": 0, + "zIndex": 2, + "selectedRelationships": [ + "matched", + "unmatched" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "1800c04e-f9b9-3293-bfc7-b35f43e0706c" + } + ], + "labels": [], + "funnels": [], + "controllerServices": [ + { + "identifier": "b013f870-aee8-3cc4-b022-ac385ded928d", + "instanceIdentifier": "4f8d53d4-019c-1000-d376-abfaa091dc37", + "name": "SimpleKeyValueLookupService", + "comments": "", + "type": "org.apache.nifi.lookup.SimpleKeyValueLookupService", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-lookup-services-nar", + "version": "2026.1.20.21-SNAPSHOT" + }, + "properties": { + "test.key": "Test Value" + }, + "propertyDescriptors": { + "test.key": { + "name": "test.key", + "displayName": "test.key", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": true + } + }, + "controllerServiceApis": [ + { + "type": "org.apache.nifi.lookup.LookupService", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-services-api-nar", + "version": "2026.1.20.21-SNAPSHOT" + } + }, + { + "type": "org.apache.nifi.lookup.StringLookupService", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-services-api-nar", + "version": "2026.1.20.21-SNAPSHOT" + } + } + ], + "scheduledState": "DISABLED", + "bulletinLevel": "WARN", + "componentType": "CONTROLLER_SERVICE", + "groupIdentifier": "1800c04e-f9b9-3293-bfc7-b35f43e0706c" + } + ], + "defaultFlowFileExpiration": "0 sec", + "defaultBackPressureObjectThreshold": 10000, + "defaultBackPressureDataSizeThreshold": "1 GB", + "scheduledState": "ENABLED", + "executionEngine": "INHERITED", + "maxConcurrentTasks": 1, + "statelessFlowTimeout": "1 min", + "flowFileConcurrency": "UNBOUNDED", + "flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE", + "componentType": "PROCESS_GROUP" + }, + "externalControllerServices": {}, + "parameterContexts": {}, + "flowEncodingVersion": "1.0", + "parameterProviders": {}, + "latest": false +} \ No newline at end of file diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/pom.xml b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/pom.xml new file mode 100644 index 000000000000..bc7d2ebc1a8a --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/pom.xml @@ -0,0 +1,33 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-connector-mock-bundle + 2.9.0-SNAPSHOT + + + nifi-connector-mock-test-bundle + pom + + + nifi-connector-mock-test-connectors + nifi-connector-mock-test-connectors-nar + nifi-connector-mock-integration-tests + + + diff --git a/nifi-connector-mock-bundle/nifi-connector-mock/pom.xml b/nifi-connector-mock-bundle/nifi-connector-mock/pom.xml new file mode 100644 index 000000000000..e32af9beacc9 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock/pom.xml @@ -0,0 +1,147 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-connector-mock-bundle + 2.9.0-SNAPSHOT + + + nifi-connector-mock + jar + + + + org.slf4j + slf4j-api + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-nar-utils + 2.9.0-SNAPSHOT + + + org.apache.nifi + nifi-properties + 2.9.0-SNAPSHOT + provided + + + org.apache.nifi + nifi-connector-mock-api + 2.9.0-SNAPSHOT + + + + + org.apache.nifi + nifi-framework-nar-utils + 2.9.0-SNAPSHOT + + + + + + org.apache.nifi + nifi-framework-api + 2.9.0-SNAPSHOT + + + org.apache.nifi + nifi-property-utils + 2.9.0-SNAPSHOT + + + org.apache.nifi + nifi-python-framework-api + 2.9.0-SNAPSHOT + + + org.apache.nifi + nifi-stateless-api + 2.9.0-SNAPSHOT + + + + + org.slf4j + log4j-over-slf4j + + + org.slf4j + jul-to-slf4j + + + org.slf4j + jcl-over-slf4j + + + + + org.junit.jupiter + junit-jupiter + test + + + + + org.apache.nifi + nifi-framework-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-jetty-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-provenance-repository-nar + 2.9.0-SNAPSHOT + nar + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-nar + generate-resources + + copy-dependencies + + + ${project.build.directory}/libDir + jar,nar + compile + + + + + + + \ No newline at end of file diff --git a/nifi-connector-mock-bundle/nifi-connector-mock/src/main/java/org/apache/nifi/mock/connector/StandardConnectorTestRunner.java b/nifi-connector-mock-bundle/nifi-connector-mock/src/main/java/org/apache/nifi/mock/connector/StandardConnectorTestRunner.java new file mode 100644 index 000000000000..7153f8d868b3 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock/src/main/java/org/apache/nifi/mock/connector/StandardConnectorTestRunner.java @@ -0,0 +1,282 @@ +/* + * 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.mock.connector; + +import org.apache.nifi.NiFiServer; +import org.apache.nifi.bundle.Bundle; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.AssetReference; +import org.apache.nifi.components.connector.ConnectorValueReference; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.SecretReference; +import org.apache.nifi.components.connector.StepConfiguration; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.mock.connector.server.ConnectorConfigVerificationResult; +import org.apache.nifi.mock.connector.server.ConnectorMockServer; +import org.apache.nifi.mock.connector.server.ConnectorTestRunner; +import org.apache.nifi.nar.ExtensionMapping; +import org.apache.nifi.nar.NarClassLoaders; +import org.apache.nifi.nar.NarUnpackMode; +import org.apache.nifi.nar.NarUnpacker; +import org.apache.nifi.nar.SystemBundle; +import org.apache.nifi.processor.Processor; +import org.apache.nifi.util.NiFiProperties; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Path; +import java.time.Duration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +public class StandardConnectorTestRunner implements ConnectorTestRunner, Closeable { + private final File narLibraryDirectory; + private final int httpPort; + + private ConnectorMockServer mockServer; + + private StandardConnectorTestRunner(final Builder builder) { + this.narLibraryDirectory = builder.narLibraryDirectory; + this.httpPort = builder.httpPort; + + try { + bootstrapInstance(); + } catch (final Exception e) { + throw new RuntimeException("Failed to bootstrap ConnectorTestRunner", e); + } + + // It is important that we register the processor and controller service mocks before instantiating the connector. + // Otherwise, the call to instantiateConnector will initialize the Connector, which may update the flow. + // If the flow is updated before the mocks are registered, the components will be created without + // using the mocks. Subsequent updates to the flow will not replace the components already created because + // these are not recognized as updates to the flow, since the framework assumes that the type of a component + // with a given ID does not change. + builder.processorMocks.forEach(mockServer::mockProcessor); + builder.controllerServiceMocks.forEach(mockServer::mockControllerService); + + mockServer.instantiateConnector(builder.connectorClassName); + } + + private void bootstrapInstance() throws IOException, ClassNotFoundException { + final List libDirectoryPaths = List.of(narLibraryDirectory.toPath()); + final File extensionsWorkingDir = new File("target/work/extensions"); + final File frameworkWorkingDir = new File("target/work/framework"); + + final Bundle systemBundle = SystemBundle.create(narLibraryDirectory.getAbsolutePath(), ClassLoader.getSystemClassLoader()); + + final ExtensionMapping extensionMapping = NarUnpacker.unpackNars(systemBundle, frameworkWorkingDir, extensionsWorkingDir, libDirectoryPaths, true, + NarClassLoaders.FRAMEWORK_NAR_ID, true, false, NarUnpackMode.UNPACK_INDIVIDUAL_JARS, bundleCoordinate -> true); + + final NarClassLoaders narClassLoaders = new NarClassLoaders(); + narClassLoaders.init(frameworkWorkingDir, extensionsWorkingDir); + + final NiFiServer nifiServer = narClassLoaders.getServer(); + if (nifiServer == null) { + throw new RuntimeException("Could not find NiFiServer instance"); + } + if (!(nifiServer instanceof ConnectorMockServer)) { + throw new RuntimeException("Test ClassPath does not contain ConnectorMockServer. " + + "Ensure that the appropriate module is packaged in the NAR library directory: " + narLibraryDirectory.getAbsolutePath()); + } + + // Set Application Server Class Loader for subsequent operations + final ClassLoader applicationServerClassLoader = narClassLoaders.getServer().getClass().getClassLoader(); + Thread.currentThread().setContextClassLoader(applicationServerClassLoader); + + final Set narBundles = narClassLoaders.getBundles(); + + final Properties additionalProperties = new Properties(); + if (httpPort >= 0) { + additionalProperties.setProperty(NiFiProperties.WEB_HTTP_PORT, String.valueOf(httpPort)); + } + + final NiFiProperties properties; + try (final InputStream propertiesIn = getClass().getClassLoader().getResourceAsStream("nifi.properties")) { + properties = NiFiProperties.createBasicNiFiProperties(propertiesIn, additionalProperties); + } + + nifiServer.initialize(properties, systemBundle, narBundles, extensionMapping); + nifiServer.start(); + + mockServer = (ConnectorMockServer) nifiServer; + mockServer.registerMockBundle(getClass().getClassLoader(), new File(extensionsWorkingDir, "mock-implementations-bundle")); + } + + @Override + public void close() { + if (mockServer != null) { + mockServer.stop(); + } + } + + @Override + public void applyUpdate() throws FlowUpdateException { + mockServer.applyUpdate(); + } + + @Override + public void configure(final String stepName, final StepConfiguration configuration) throws FlowUpdateException { + mockServer.configure(stepName, configuration); + } + + @Override + public void configure(final String stepName, final Map propertyValues) throws FlowUpdateException { + mockServer.configure(stepName, propertyValues); + } + + @Override + public void configure(final String stepName, final Map propertyValues, final Map propertyReferences) throws FlowUpdateException { + mockServer.configure(stepName, propertyValues, propertyReferences); + } + + @Override + public SecretReference createSecretReference(final String secretName) { + return mockServer.createSecretReference(secretName); + } + + @Override + public ConnectorConfigVerificationResult verifyConfiguration(final String stepName, final Map propertyValueOverrides) { + return mockServer.verifyConfiguration(stepName, propertyValueOverrides); + } + + @Override + public ConnectorConfigVerificationResult verifyConfiguration(final String stepName, final Map propertyValueOverrides, + final Map referenceOverrides) { + + return mockServer.verifyConfiguration(stepName, propertyValueOverrides, referenceOverrides); + } + + @Override + public ConnectorConfigVerificationResult verifyConfiguration(final String stepName, final StepConfiguration configurationOverrides) { + return mockServer.verifyConfiguration(stepName, configurationOverrides); + } + + @Override + public void addSecret(final String name, final String value) { + mockServer.addSecret(name, value); + } + + @Override + public AssetReference addAsset(final File file) { + return mockServer.addAsset(file); + } + + @Override + public AssetReference addAsset(final String assetName, final InputStream contents) { + return mockServer.addAsset(assetName, contents); + } + + @Override + public void startConnector() { + mockServer.startConnector(); + } + + @Override + public void stopConnector() { + mockServer.stopConnector(); + } + + @Override + public void waitForDataIngested(final Duration maxWaitTime) { + mockServer.waitForDataIngested(maxWaitTime); + } + + @Override + public void waitForIdle(final Duration maxWaitTime) { + mockServer.waitForIdle(maxWaitTime); + } + + @Override + public void waitForIdle(final Duration minIdleTime, final Duration maxWaitTime) { + mockServer.waitForIdle(minIdleTime, maxWaitTime); + } + + @Override + public List validate() { + return mockServer.validate(); + } + + @Override + public int getHttpPort() { + return mockServer.getHttpPort(); + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName) { + return mockServer.fetchAllowableValues(stepName, propertyName); + } + + @Override + public VersionedExternalFlow getActiveFlowSnapshot() { + return mockServer.getActiveFlowSnapshot(); + } + + @Override + public VersionedExternalFlow getWorkingFlowSnapshot() { + return mockServer.getWorkingFlowSnapshot(); + } + + + public static class Builder { + private String connectorClassName; + private File narLibraryDirectory; + private int httpPort = -1; + private final Map> processorMocks = new HashMap<>(); + private final Map> controllerServiceMocks = new HashMap<>(); + + public Builder connectorClassName(final String connectorClassName) { + this.connectorClassName = connectorClassName; + return this; + } + + public Builder narLibraryDirectory(final File libDirectory) { + this.narLibraryDirectory = libDirectory; + return this; + } + + public Builder httpPort(final int httpPort) { + this.httpPort = httpPort; + return this; + } + + public Builder mockProcessor(final String processorType, final Class mockProcessorClass) { + processorMocks.put(processorType, mockProcessorClass); + return this; + } + + public Builder mockControllerService(final String controllerServiceType, final Class mockControllerServiceClass) { + controllerServiceMocks.put(controllerServiceType, mockControllerServiceClass); + return this; + } + + public StandardConnectorTestRunner build() { + if (!narLibraryDirectory.exists() || !narLibraryDirectory.isDirectory()) { + throw new IllegalArgumentException("NAR file does not exist or is not a directory: " + narLibraryDirectory.getAbsolutePath()); + } + + return new StandardConnectorTestRunner(this); + } + } +} diff --git a/nifi-connector-mock-bundle/nifi-connector-mock/src/main/resources/nifi.properties b/nifi-connector-mock-bundle/nifi-connector-mock/src/main/resources/nifi.properties new file mode 100644 index 000000000000..44deabc71ad3 --- /dev/null +++ b/nifi-connector-mock-bundle/nifi-connector-mock/src/main/resources/nifi.properties @@ -0,0 +1,165 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Core Properties # +nifi.flow.configuration.file=target/nifi-storage/conf/flow.json.gz +nifi.flow.configuration.archive.enabled=true +nifi.flow.configuration.archive.dir=target/nifi-storage/conf/archive/ +nifi.flow.configuration.archive.max.time=30 days +nifi.flow.configuration.archive.max.storage=500 MB +nifi.flow.configuration.archive.max.count= +nifi.flowcontroller.autoResumeState=true +nifi.flowcontroller.graceful.shutdown.period=10 sec +nifi.flowservice.writedelay.interval=500 ms +nifi.administrative.yield.duration=3 sec +# If a component has no work to do (is "bored"), how long should we wait before checking again for work? +nifi.bored.yield.duration=10 millis +nifi.queue.backpressure.count=10000 +nifi.queue.backpressure.size=1 GB + +nifi.nar.library.directory=./lib +nifi.nar.library.autoload.directory=./extensions +nifi.nar.working.directory=./work/nar/ + +##################### +# Python Extensions # +##################### +# Uncomment in order to enable Python Extensions. +#nifi.python.command=python +nifi.python.framework.source.directory=./python/framework +nifi.python.extensions.source.directory.default=./python/extensions +nifi.python.working.directory=./work/python +nifi.python.max.processes=100 +nifi.python.max.processes.per.extension.type=10 + +#################### +# State Management # +#################### +nifi.state.management.configuration.file=target/nifi-storage/state-management.xml +# The ID of the local state provider +nifi.state.management.provider.local=local-provider + +# Database Settings +nifi.database.directory=target/nifi-storage/database_repository + +# FlowFile Repository +nifi.flowfile.repository.implementation=org.apache.nifi.controller.repository.WriteAheadFlowFileRepository +nifi.flowfile.repository.wal.implementation=org.apache.nifi.wali.SequentialAccessWriteAheadLog +nifi.flowfile.repository.directory=target/nifi-storage/flowfile_repository +nifi.flowfile.repository.checkpoint.interval=1 mins +nifi.flowfile.repository.always.sync=false + +nifi.swap.manager.implementation=org.apache.nifi.controller.FileSystemSwapManager +nifi.queue.swap.threshold=20000 +nifi.swap.in.period=5 sec +nifi.swap.in.threads=1 +nifi.swap.out.period=5 sec +nifi.swap.out.threads=4 + +# Content Repository +nifi.content.repository.implementation=org.apache.nifi.controller.repository.FileSystemRepository +nifi.content.claim.max.appendable.size=50 KB +nifi.content.repository.directory.default=target/nifi-storage/content_repository +nifi.content.repository.archive.max.retention.period=12 hours +nifi.content.repository.archive.max.usage.percentage=90% +nifi.content.repository.archive.enabled=false +nifi.content.repository.always.sync=false + +# Provenance Repository Properties +nifi.provenance.repository.implementation=org.apache.nifi.provenance.VolatileProvenanceRepository + +# Volatile Provenance Respository Properties +nifi.provenance.repository.buffer.size=1000 + +# Component Status Repository +nifi.components.status.repository.implementation=org.apache.nifi.controller.status.history.VolatileComponentStatusRepository +nifi.components.status.repository.buffer.size=1440 +nifi.components.status.snapshot.frequency=1 min + +# NAR Persistence Provider Properties +nifi.nar.persistence.provider.properties.directory=target/nifi-storage/nar_repository + +# Asset Management +nifi.asset.manager.properties.directory=target/nifi-storage/assets + +# Connector Asset Manager +nifi.connector.asset.manager.implementation=org.apache.nifi.mock.connector.server.MockConnectorAssetManager + +# Secrets Manager +nifi.secrets.manager.implementation=org.apache.nifi.mock.connector.server.secrets.ConnectorTestRunnerSecretsManager + +# Site to Site properties +nifi.remote.input.host=localhost +nifi.remote.input.secure=true +nifi.remote.input.socket.port=7780 +nifi.remote.input.http.enabled=true +nifi.remote.input.http.transaction.ttl=30 sec +nifi.remote.contents.cache.expiration=30 secs + +# web properties # +nifi.web.war.directory=./lib +nifi.web.http.host= +nifi.web.http.port= +nifi.web.http.network.interface.default= +nifi.web.https.host=localhost +nifi.web.https.port=0 +nifi.web.https.network.interface.default= +nifi.web.jetty.working.directory=target/work/jetty +nifi.web.jetty.threads=200 +nifi.web.max.header.size=16 KB +nifi.web.proxy.context.path= +nifi.web.proxy.host= + +# security properties # +nifi.sensitive.props.key=nifi-connector-mock-tests +nifi.sensitive.props.algorithm=NIFI_PBKDF2_AES_GCM_256 + +nifi.security.keystore= +nifi.security.keystoreType=PKCS12 +nifi.security.keystorePasswd= +nifi.security.keyPasswd= +nifi.security.truststore= +nifi.security.truststoreType=PKCS12 +nifi.security.truststorePasswd= +nifi.security.user.authorizer=system-test-authorizer +nifi.security.user.login.identity.provider= + +# cluster common properties (all nodes must have same values) # +nifi.cluster.protocol.heartbeat.interval=5 sec +nifi.cluster.protocol.is.secure=true + +# cluster node properties (only configure for cluster nodes) # +nifi.cluster.is.node=false +nifi.cluster.node.address= +nifi.cluster.node.protocol.port= +nifi.cluster.node.protocol.threads=10 +nifi.cluster.node.protocol.max.threads=50 +nifi.cluster.node.event.history.size=25 +nifi.cluster.node.connection.timeout=5 sec +nifi.cluster.node.read.timeout=5 sec +nifi.cluster.node.max.concurrent.requests=100 +nifi.cluster.firewall.file= +nifi.cluster.flow.election.max.wait.time=5 mins +nifi.cluster.flow.election.max.candidates= + +# cluster load balancing properties # +nifi.cluster.load.balance.host= +nifi.cluster.load.balance.port=6342 +nifi.cluster.load.balance.connections.per.node=4 +nifi.cluster.load.balance.max.thread.count=8 +nifi.cluster.load.balance.comms.timeout=30 sec + +# Connector Repository +nifi.components.connectors.repository.implementation=org.apache.nifi.mock.connector.server.MockConnectorRepository diff --git a/nifi-connector-mock-bundle/pom.xml b/nifi-connector-mock-bundle/pom.xml new file mode 100644 index 000000000000..64fd76bf604e --- /dev/null +++ b/nifi-connector-mock-bundle/pom.xml @@ -0,0 +1,34 @@ + + + + 4.0.0 + + org.apache.nifi + nifi + 2.9.0-SNAPSHOT + + + nifi-connector-mock-bundle + pom + + + nifi-connector-mock + nifi-connector-mock-api + nifi-connector-mock-server + nifi-connector-mock-server-nar + nifi-connector-mock-test-bundle + + \ No newline at end of file diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/pom.xml b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/pom.xml new file mode 100644 index 000000000000..a58046afaf9a --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/pom.xml @@ -0,0 +1,40 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-kafka-to-s3-bundle + 2.9.0-SNAPSHOT + + + nifi-kafka-to-s3-connector + + + + org.slf4j + slf4j-api + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-connector-utils + + + diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaConnectionStep.java b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaConnectionStep.java new file mode 100644 index 000000000000..793b1e9be0de --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaConnectionStep.java @@ -0,0 +1,135 @@ +/* + * 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.connectors.kafkas3; + +import org.apache.nifi.components.Validator; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.ConnectorPropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorPropertyGroup; +import org.apache.nifi.components.connector.PropertyType; +import org.apache.nifi.processor.util.StandardValidators; + +import java.util.List; + +public class KafkaConnectionStep { + public static final String STEP_NAME = "Kafka Connection"; + + public static final ConnectorPropertyDescriptor KAFKA_BROKERS = new ConnectorPropertyDescriptor.Builder() + .name("Kafka Brokers") + .description("A comma-separated list of Kafka brokers to connect to.") + .required(true) + .validators(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR) + .type(PropertyType.STRING_LIST) + .build(); + + public static final ConnectorPropertyDescriptor SECURITY_PROTOCOL = new ConnectorPropertyDescriptor.Builder() + .name("Security Protocol") + .description("The security protocol to use when connecting to Kafka brokers.") + .required(true) + .type(PropertyType.STRING) + .defaultValue("SASL_PLAINTEXT") + .allowableValues("PLAINTEXT", "SSL", "SASL_PLAINTEXT", "SASL_SSL") + .build(); + + public static final ConnectorPropertyDescriptor SASL_MECHANISM = new ConnectorPropertyDescriptor.Builder() + .name("SASL Mechanism") + .description("The SASL mechanism to use for authentication.") + .required(true) + .type(PropertyType.STRING) + .allowableValues("PLAIN", "SCRAM-SHA-256", "SCRAM-SHA-512", "GSSAPI", "OAUTHBEARER") + .dependsOn(SECURITY_PROTOCOL, "SASL_PLAINTEXT", "SASL_SSL") + .defaultValue("SCRAM-SHA-512") + .build(); + + public static final ConnectorPropertyDescriptor USERNAME = new ConnectorPropertyDescriptor.Builder() + .name("Username") + .description("The username for SASL authentication.") + .required(true) + .type(PropertyType.STRING) + .validators(StandardValidators.NON_EMPTY_VALIDATOR) + .dependsOn(SECURITY_PROTOCOL, "SASL_PLAINTEXT", "SASL_SSL") + .build(); + + public static final ConnectorPropertyDescriptor PASSWORD = new ConnectorPropertyDescriptor.Builder() + .name("Password") + .description("The password for SASL authentication.") + .required(true) + .type(PropertyType.SECRET) + .validators(StandardValidators.NON_EMPTY_VALIDATOR) + .dependsOn(SECURITY_PROTOCOL, "SASL_PLAINTEXT", "SASL_SSL") + .build(); + + public static final ConnectorPropertyGroup KAFKA_SERVER_GROUP = new ConnectorPropertyGroup.Builder() + .name("Kafka Server Settings") + .description("Settings for connecting to the Kafka server") + .properties(List.of( + KAFKA_BROKERS, + SECURITY_PROTOCOL, + SASL_MECHANISM, + USERNAME, + PASSWORD + )) + .build(); + + public static final ConnectorPropertyDescriptor SCHEMA_REGISTRY_URL = new ConnectorPropertyDescriptor.Builder() + .name("Schema Registry URL") + .description("The URL of the Schema Registry.") + .required(false) + .type(PropertyType.STRING) + .validators(StandardValidators.URL_VALIDATOR) + .validators(Validator.VALID) + .build(); + + public static final ConnectorPropertyDescriptor SCHEMA_REGISTRY_USERNAME = new ConnectorPropertyDescriptor.Builder() + .name("Schema Registry Username") + .description("The username for Schema Registry authentication.") + .required(false) + .type(PropertyType.STRING) + .validators(StandardValidators.NON_EMPTY_VALIDATOR) + .dependsOn(SCHEMA_REGISTRY_URL) + .build(); + + public static final ConnectorPropertyDescriptor SCHEMA_REGISTRY_PASSWORD = new ConnectorPropertyDescriptor.Builder() + .name("Schema Registry Password") + .description("The password for Schema Registry authentication.") + .required(false) + .type(PropertyType.SECRET) + .validators(StandardValidators.NON_EMPTY_VALIDATOR) + .dependsOn(SCHEMA_REGISTRY_URL) + .build(); + + + public static final ConnectorPropertyGroup SCHEMA_REGISTRY_GROUP = new ConnectorPropertyGroup.Builder() + .name("Schema Registry Settings") + .description("Settings for connecting to the Schema Registry") + .properties(List.of( + SCHEMA_REGISTRY_URL, + SCHEMA_REGISTRY_USERNAME, + SCHEMA_REGISTRY_PASSWORD + )) + .build(); + + public static final ConfigurationStep KAFKA_CONNECTION_STEP = new ConfigurationStep.Builder() + .name(STEP_NAME) + .description("Configure Kafka connection settings") + .propertyGroups(List.of( + KAFKA_SERVER_GROUP, + SCHEMA_REGISTRY_GROUP + )) + .build(); +} diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaToS3.java b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaToS3.java new file mode 100644 index 000000000000..9c4bc1b7d386 --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaToS3.java @@ -0,0 +1,297 @@ +/* + * 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.connectors.kafkas3; + +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.ConfigVerificationResult.Outcome; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.ConnectorConfigurationContext; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.InvocationFailedException; +import org.apache.nifi.components.connector.components.ControllerServiceFacade; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.components.ProcessGroupFacade; +import org.apache.nifi.components.connector.components.ProcessorFacade; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedExternalFlow; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +@CapabilityDescription("Provides the ability to ingest data from Apache Kafka topics, merge it together into an object of reasonable " + + "size, and write that data to Amazon S3.") +@Tags({"kafka", "s3"}) +public class KafkaToS3 extends AbstractConnector { + + private static final List configurationSteps = List.of( + KafkaConnectionStep.KAFKA_CONNECTION_STEP, + KafkaTopicsStep.KAFKA_TOPICS_STEP, + S3Step.S3_STEP + ); + + private volatile CompletableFuture drainFlowFileFuture = null; + + + @Override + public List getConfigurationSteps() { + return configurationSteps; + } + + @Override + public void prepareForUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + final String activeS3DataFormat = activeContext.getConfigurationContext().getProperty( + S3Step.S3_STEP_NAME, S3Step.S3_DATA_FORMAT.getName()).getValue(); + final String workingS3DataFormat = workingContext.getConfigurationContext().getProperty( + S3Step.S3_STEP_NAME, S3Step.S3_DATA_FORMAT.getName()).getValue(); + + if (!activeS3DataFormat.equals(workingS3DataFormat)) { + getLogger().info("S3 Data Format changed from {} to {}; draining flow before updating it", activeS3DataFormat, workingS3DataFormat); + + drainFlowFileFuture = drainFlowFiles(activeContext); + try { + drainFlowFileFuture.get(5, TimeUnit.MINUTES); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + throw new FlowUpdateException("Interrupted while waiting for all FlowFiles to drain from the flow", e); + } catch (final TimeoutException e) { + throw new FlowUpdateException("Timed out waiting for all FlowFiles to drain from the flow", e); + } catch (final ExecutionException e) { + throw new FlowUpdateException("Failed to drain FlowFiles from flow", e.getCause()); + } + + getLogger().info("All FlowFiles drained from the flow; proceeding with flow update"); + } + } + + @Override + public void abortUpdate(final FlowContext workingContext, final Throwable throwable) { + if (drainFlowFileFuture != null) { + drainFlowFileFuture.cancel(true); + drainFlowFileFuture = null; + } + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + final VersionedExternalFlow flow = buildFlow(workingContext.getConfigurationContext()); + getInitializationContext().updateFlow(activeContext, flow); + } + + @Override + public VersionedExternalFlow getInitialFlow() { + return KafkaToS3FlowBuilder.loadInitialFlow(); + } + + @Override + public void onStepConfigured(final String stepName, final FlowContext workingContext) throws FlowUpdateException { + final VersionedExternalFlow flow = buildFlow(workingContext.getConfigurationContext()); + getInitializationContext().updateFlow(workingContext, flow); + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map propertyValueOverrides, final FlowContext workingFlowContext) { + // Get the current ConfigurationContext and then create a new one that contains the provided property values + final ConnectorConfigurationContext configurationContext = workingFlowContext.getConfigurationContext().createWithOverrides(stepName, propertyValueOverrides); + final VersionedExternalFlow flow = buildFlow(configurationContext); + + // Validate Connectivity + if (stepName.equals(KafkaConnectionStep.STEP_NAME)) { + return verifyKafkaConnectivity(workingFlowContext, flow); + } + if (stepName.equals(KafkaTopicsStep.STEP_NAME)) { + final List results = new ArrayList<>(); + results.addAll(verifyTopicsExists(workingFlowContext)); + results.addAll(verifyKafkaParsability(workingFlowContext, flow)); + return results; + } + + return Collections.emptyList(); + } + + private List verifyKafkaParsability(final FlowContext workingFlowContext, final VersionedExternalFlow flow) { + // Enable Controller Services necessary for parsing records. + // We determine which Controller Services are referenced by the flow and enable them, but we do not use + // getRootGroup().getLifecycle().enableReferencedControllerServices(ControllerServiceReferenceScope.INCLUDE_REFERENCED_SERVICES_ONLY) + // because that would include the Controller Services that are referenced by the currently configured flow, and it's possible that the + // what is being verified uses a different set of Controller Services (e.g., the verified flow may use a JSON Reader while the current + // flow uses an Avro Reader). + final ProcessGroupFacade rootGroup = workingFlowContext.getRootGroup(); + final Set referencedServices = VersionedFlowUtils.getReferencedControllerServices(flow.getFlowContents()); + final Set serviceIds = referencedServices.stream() + .map(VersionedControllerService::getIdentifier) + .collect(Collectors.toSet()); + + try { + rootGroup.getLifecycle().enableControllerServices(serviceIds).get(10, TimeUnit.SECONDS); + } catch (final Exception e) { + return List.of(new ConfigVerificationResult.Builder() + .verificationStepName("Record Parsing") + .outcome(Outcome.FAILED) + .explanation("Failed to enable Controller Services due to " + e) + .build()); + } + + try { + final ProcessorFacade consumeKafkaFacade = findProcessors(rootGroup, + processor -> processor.getDefinition().getType().endsWith("ConsumeKafka")).getFirst(); + + final List configVerificationResults = consumeKafkaFacade.verify(flow, Map.of()); + for (final ConfigVerificationResult result : configVerificationResults) { + if (result.getOutcome() == Outcome.FAILED) { + return List.of(result); + } + } + + return Collections.emptyList(); + } finally { + rootGroup.getLifecycle().disableControllerServices(serviceIds); + } + } + + + private List verifyTopicsExists(final FlowContext workingFlowContext) { + final List topicsAvailable; + try { + topicsAvailable = getAvailableTopics(workingFlowContext); + } catch (final Exception e) { + return List.of(new ConfigVerificationResult.Builder() + .verificationStepName("Verify Kafka topics exist") + .outcome(Outcome.SKIPPED) + .explanation("Unable to validate that topics exist due to " + e) + .build()); + } + + final Set topicNames = new HashSet<>(topicsAvailable); + final List specifiedTopics = workingFlowContext.getConfigurationContext().getProperty(KafkaTopicsStep.STEP_NAME, + KafkaTopicsStep.TOPIC_NAMES.getName()).asList(); + final String missingTopics = specifiedTopics.stream() + .filter(topic -> !topicNames.contains(topic)) + .collect(Collectors.joining(", ")); + + if (!missingTopics.isEmpty()) { + return List.of(new ConfigVerificationResult.Builder() + .verificationStepName("Verify Kafka topics exist") + .outcome(Outcome.FAILED) + .explanation("The following topics do not exist in the Kafka cluster: " + missingTopics) + .build()); + } else { + return List.of(new ConfigVerificationResult.Builder() + .verificationStepName("Verify Kafka topics exist") + .outcome(Outcome.SUCCESSFUL) + .explanation("All specified topics exist in the Kafka cluster") + .build()); + } + } + + private List verifyKafkaConnectivity(final FlowContext workingFlowContext, final VersionedExternalFlow flow) { + // Build a new version of the flow so that we can get the relevant properties of the Kafka Connection Service + final ControllerServiceFacade connectionService = getKafkaConnectionService(workingFlowContext); + final List configVerificationResults = connectionService.verify(flow, Map.of()); + + for (final ConfigVerificationResult result : configVerificationResults) { + if (result.getOutcome() == Outcome.FAILED) { + return List.of(new ConfigVerificationResult.Builder() + .verificationStepName("Verify Kafka connectivity") + .outcome(Outcome.FAILED) + .explanation(result.getExplanation()) + .build()); + } + } + + return Collections.emptyList(); + } + + private VersionedExternalFlow buildFlow(final ConnectorConfigurationContext configurationContext) { + final KafkaToS3FlowBuilder flowBuilder = new KafkaToS3FlowBuilder(configurationContext); + return flowBuilder.buildFlow(); + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName, final FlowContext flowContext) { + if (stepName.equals(KafkaTopicsStep.STEP_NAME) && propertyName.equals(KafkaTopicsStep.TOPIC_NAMES.getName())) { + return createAllowableValues(getAvailableTopics(flowContext)); + } else if (stepName.equals(S3Step.S3_STEP_NAME) && propertyName.equals(S3Step.S3_REGION.getName())) { + return createAllowableValues(getPossibleS3Regions(flowContext)); + } + + return super.fetchAllowableValues(stepName, propertyName, flowContext); + } + + private List createAllowableValues(final List values) { + return values.stream().map(this::createAllowableValue).collect(Collectors.toList()); + } + + private DescribedValue createAllowableValue(final String value) { + return new AllowableValue(value, value, value); + } + + @SuppressWarnings("unchecked") + private List getAvailableTopics(final FlowContext flowContext) { + // If Kafka Brokers not yet set, return empty list + final ConnectorConfigurationContext config = flowContext.getConfigurationContext(); + if (!config.getProperty(KafkaConnectionStep.KAFKA_CONNECTION_STEP, KafkaConnectionStep.KAFKA_BROKERS).isSet()) { + return List.of(); + } + + final ControllerServiceFacade kafkaConnectionService = getKafkaConnectionService(flowContext); + + try { + return (List) kafkaConnectionService.invokeConnectorMethod("listTopicNames", Map.of()); + } catch (final Exception e) { + getLogger().warn("Failed to retrieve available Kafka topics", e); + return List.of(); + } + } + + private ControllerServiceFacade getKafkaConnectionService(final FlowContext flowContext) { + return flowContext.getRootGroup().getControllerServices().stream() + .filter(service -> service.getDefinition().getType().endsWith("Kafka3ConnectionService")) + .findFirst() + .orElseThrow(); + } + + @SuppressWarnings("unchecked") + private List getPossibleS3Regions(final FlowContext flowContext) { + final ProcessorFacade processorFacade = flowContext.getRootGroup().getProcessors().stream() + .filter(proc -> proc.getDefinition().getType().endsWith("PutS3Object")) + .findFirst() + .orElseThrow(); + + try { + return (List) processorFacade.invokeConnectorMethod("getAvailableRegions", Map.of()); + } catch (final InvocationFailedException e) { + getLogger().error("Failed to obtain list of available S3 regions", e); + return Collections.emptyList(); + } + } +} diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaToS3FlowBuilder.java b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaToS3FlowBuilder.java new file mode 100644 index 000000000000..e7d5a75c3338 --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaToS3FlowBuilder.java @@ -0,0 +1,213 @@ +/* + * 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.connectors.kafkas3; + +import org.apache.nifi.components.connector.ConnectorConfigurationContext; +import org.apache.nifi.components.connector.StepConfigurationContext; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.processor.DataUnit; + +import java.util.HashMap; +import java.util.Map; + +public class KafkaToS3FlowBuilder { + private static final String FLOW_JSON_PATH = "flows/Kafka_to_S3.json"; + + private final ConnectorConfigurationContext configContext; + + public KafkaToS3FlowBuilder(final ConnectorConfigurationContext configurationContext) { + this.configContext = configurationContext; + } + + public static VersionedExternalFlow loadInitialFlow() { + return VersionedFlowUtils.loadFlowFromResource(FLOW_JSON_PATH); + } + + public VersionedExternalFlow buildFlow() { + final VersionedExternalFlow externalFlow = VersionedFlowUtils.loadFlowFromResource(FLOW_JSON_PATH); + configureSchemaRegistry(externalFlow); + + updateKafkaConnectionParameters(externalFlow); + updateSchemaRegistryParameters(externalFlow); + updateReaderWriter(externalFlow); + updateKafkaTopicsParameters(externalFlow); + updateS3Config(externalFlow); + + return externalFlow; + } + + private void configureSchemaRegistry(final VersionedExternalFlow externalFlow) { + final String schemaRegistryUrl = configContext.getProperty(KafkaConnectionStep.KAFKA_CONNECTION_STEP, KafkaConnectionStep.SCHEMA_REGISTRY_URL).getValue(); + + if (schemaRegistryUrl == null) { + final VersionedProcessGroup processGroup = externalFlow.getFlowContents(); + + // Remove any references to the Schema Registry service. + final VersionedControllerService schemaRegistryService = processGroup.getControllerServices().stream() + .filter(service -> service.getType().endsWith("ConfluentSchemaRegistry")) + .findFirst() + .orElseThrow(); + VersionedFlowUtils.removeControllerServiceReferences(processGroup, schemaRegistryService.getIdentifier()); + + final VersionedControllerService schemaReferenceReader = processGroup.getControllerServices().stream() + .filter(service -> service.getType().endsWith("ConfluentEncodedSchemaReferenceReader")) + .findFirst() + .orElseThrow(); + VersionedFlowUtils.removeControllerServiceReferences(processGroup, schemaReferenceReader.getIdentifier()); + + processGroup.getControllerServices().stream() + .filter(service -> service.getType().endsWith("JsonTreeReader")) + .forEach(service -> service.getProperties().put("Schema Access Strategy", "infer-schema")); + + processGroup.getControllerServices().stream() + .filter(service -> service.getType().endsWith("JsonRecordSetWriter")) + .forEach(service -> service.getProperties().put("Schema Write Strategy", "no-schema")); + } + } + + private void updateSchemaRegistryParameters(final VersionedExternalFlow externalFlow) { + final StepConfigurationContext stepContext = configContext.scopedToStep(KafkaConnectionStep.KAFKA_CONNECTION_STEP); + + final String schemaRegistryUrl = stepContext.getProperty(KafkaConnectionStep.SCHEMA_REGISTRY_URL).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "Schema Registry URLs", schemaRegistryUrl); + + if (schemaRegistryUrl == null) { + final Map properties = new HashMap<>(); + properties.put("schema-access-strategy", "infer-schema"); + properties.put("schema-registry", null); + properties.put("schema-reference-reader", null); + + externalFlow.getFlowContents().getControllerServices().stream() + .filter(service -> service.getType().endsWith("JsonTreeReader")) + .findFirst() + .ifPresent(service -> service.setProperties(properties)); + } else { + final String username = stepContext.getProperty(KafkaConnectionStep.SCHEMA_REGISTRY_USERNAME).getValue(); + final String password = stepContext.getProperty(KafkaConnectionStep.SCHEMA_REGISTRY_PASSWORD).getValue(); + + VersionedFlowUtils.setParameterValue(externalFlow, "Schema Registry Username", username); + VersionedFlowUtils.setParameterValue(externalFlow, "Schema Registry Password", password); + + final String authenticationType = (username == null || username.isEmpty()) ? "NONE" : "BASIC"; + VersionedFlowUtils.setParameterValue(externalFlow, "Schema Registry Authentication Type", authenticationType); + } + } + + private void updateKafkaConnectionParameters(final VersionedExternalFlow externalFlow) { + final StepConfigurationContext stepContext = configContext.scopedToStep(KafkaConnectionStep.KAFKA_CONNECTION_STEP); + + final String kafkaBrokers = stepContext.getProperty(KafkaConnectionStep.KAFKA_BROKERS).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "Kafka Bootstrap Servers", kafkaBrokers); + + final String securityProtocol = stepContext.getProperty(KafkaConnectionStep.SECURITY_PROTOCOL).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "Kafka Security Protocol", securityProtocol); + + if (securityProtocol.contains("SASL")) { + final String saslMechanism = stepContext.getProperty(KafkaConnectionStep.SASL_MECHANISM).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "Kafka SASL Mechanism", saslMechanism); + + final String username = stepContext.getProperty(KafkaConnectionStep.USERNAME).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "Kafka SASL Username", username); + + final String password = stepContext.getProperty(KafkaConnectionStep.PASSWORD).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "Kafka SASL Password", password); + } + } + + private void updateReaderWriter(final VersionedExternalFlow externalFlow) { + final VersionedProcessGroup rootGroup = externalFlow.getFlowContents(); + + final VersionedControllerService avroService = rootGroup.getControllerServices().stream() + .filter(service -> service.getType().endsWith("AvroReader")) + .findFirst() + .orElseThrow(); + + final String kafkaDataFormat = configContext.getProperty(KafkaTopicsStep.STEP_NAME, KafkaTopicsStep.KAFKA_DATA_FORMAT.getName()).getValue(); + if (!kafkaDataFormat.equalsIgnoreCase("JSON")) { + // Update ConsumeKafka processor to use Avro Reader + rootGroup.getProcessors().stream() + .filter(versionedProcessor -> versionedProcessor.getType().endsWith("ConsumeKafka")) + .findFirst() + .ifPresent(processor -> processor.getProperties().put("Record Reader", avroService.getIdentifier())); + } + + VersionedFlowUtils.setParameterValue(externalFlow, "Kafka Data Format", kafkaDataFormat); + + final String s3DataFormat = configContext.getProperty(S3Step.S3_STEP, S3Step.S3_DATA_FORMAT).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "S3 Data Format", s3DataFormat); + } + + private void updateKafkaTopicsParameters(final VersionedExternalFlow externalFlow) { + final StepConfigurationContext stepContext = configContext.scopedToStep(KafkaTopicsStep.STEP_NAME); + + final String topics = stepContext.getProperty(KafkaTopicsStep.TOPIC_NAMES.getName()).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "Topic Names", topics); + + final String groupId = stepContext.getProperty(KafkaTopicsStep.CONSUMER_GROUP_ID.getName()).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "Consumer Group ID", groupId); + + final String offsetReset = stepContext.getProperty(KafkaTopicsStep.OFFSET_RESET.getName()).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "Kafka Auto Offset Reset", offsetReset); + } + + private void updateS3Config(final VersionedExternalFlow externalFlow) { + final StepConfigurationContext stepContext = configContext.scopedToStep(S3Step.S3_STEP); + + final String region = stepContext.getProperty(S3Step.S3_REGION).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "S3 Region", region); + + final String bucket = stepContext.getProperty(S3Step.S3_BUCKET).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "S3 Bucket", bucket); + + final String prefix = stepContext.getProperty(S3Step.S3_PREFIX).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "S3 Prefix", prefix); + + final String endpointOverrideUrl = stepContext.getProperty(S3Step.S3_ENDPOINT_OVERRIDE_URL).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "S3 Endpoint Override URL", endpointOverrideUrl); + + final String authStrategy = stepContext.getProperty(S3Step.S3_AUTHENTICATION_STRATEGY).getValue(); + if (authStrategy.equals(S3Step.DEFAULT_CREDENTIALS)) { + final VersionedControllerService credentialsService = externalFlow.getFlowContents().getControllerServices().stream() + .filter(service -> service.getType().endsWith("AWSCredentialsProviderControllerService")) + .findFirst() + .orElseThrow(); + + credentialsService.setProperties(Map.of("default-credentials", "true")); + } else { + final String accessKey = stepContext.getProperty(S3Step.S3_ACCESS_KEY_ID).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "S3 Access Key ID", accessKey); + + final String secretKey = stepContext.getProperty(S3Step.S3_SECRET_ACCESS_KEY).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "S3 Secret Access Key", secretKey); + } + + final long mergeBytes = stepContext.getProperty(S3Step.TARGET_OBJECT_SIZE).asDataSize(DataUnit.B).longValue(); + final String mergeSize = stepContext.getProperty(S3Step.TARGET_OBJECT_SIZE).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "Target Object Size", mergeSize); + + // Max Bin size will be either 10% more than target size or target size + 100MB, whichever is smaller + final long maxBinSize = (long) Math.min(mergeBytes + 100_000_000, mergeBytes * 1.1D); + VersionedFlowUtils.setParameterValue(externalFlow, "Maximum Object Size", maxBinSize + " B"); + + final String mergeLatency = stepContext.getProperty(S3Step.MERGE_LATENCY).getValue(); + VersionedFlowUtils.setParameterValue(externalFlow, "Merge Latency", mergeLatency); + } +} diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaTopicsStep.java b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaTopicsStep.java new file mode 100644 index 000000000000..754a708c3955 --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/KafkaTopicsStep.java @@ -0,0 +1,85 @@ +/* + * 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.connectors.kafkas3; + +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.ConnectorPropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorPropertyGroup; +import org.apache.nifi.components.connector.PropertyType; +import org.apache.nifi.processor.util.StandardValidators; + +import java.util.List; + +public class KafkaTopicsStep { + public static final String STEP_NAME = "Kafka Topics"; + + public static final ConnectorPropertyDescriptor TOPIC_NAMES = new ConnectorPropertyDescriptor.Builder() + .name("Topic Names") + .description("A comma-separated list of Kafka topics to consume from.") + .required(true) + .type(PropertyType.STRING_LIST) + .allowableValuesFetchable(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final ConnectorPropertyDescriptor CONSUMER_GROUP_ID = new ConnectorPropertyDescriptor.Builder() + .name("Consumer Group ID") + .description("The consumer group ID to use when connecting to Kafka.") + .required(true) + .type(PropertyType.STRING) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final ConnectorPropertyDescriptor OFFSET_RESET = new ConnectorPropertyDescriptor.Builder() + .name("Offset Reset") + .description("What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server.") + .required(true) + .type(PropertyType.STRING) + .defaultValue("latest") + .allowableValues("earliest", "latest", "none") + .build(); + + public static final ConnectorPropertyDescriptor KAFKA_DATA_FORMAT = new ConnectorPropertyDescriptor.Builder() + .name("Kafka Data Format") + .description("The format of the data in Kafka topics.") + .required(true) + .type(PropertyType.STRING) + .defaultValue("Avro") + .allowableValues("Avro", "JSON") + .build(); + + public static final ConnectorPropertyGroup KAFKA_TOPICS_GROUP = new ConnectorPropertyGroup.Builder() + .name("Kafka Topics Configuration") + .description("Properties for configuring Kafka topics consumption.") + .properties(List.of( + TOPIC_NAMES, + CONSUMER_GROUP_ID, + OFFSET_RESET, + KAFKA_DATA_FORMAT + )) + .build(); + + public static final ConfigurationStep KAFKA_TOPICS_STEP = new ConfigurationStep.Builder() + .name(STEP_NAME) + .description("Kafka topics to consume from.") + .propertyGroups(List.of( + KAFKA_TOPICS_GROUP + )) + .build(); + +} diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/S3Step.java b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/S3Step.java new file mode 100644 index 000000000000..ad29115c4ad6 --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/java/org/apache/nifi/connectors/kafkas3/S3Step.java @@ -0,0 +1,156 @@ +/* + * 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.connectors.kafkas3; + +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.ConnectorPropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorPropertyGroup; +import org.apache.nifi.components.connector.PropertyType; +import org.apache.nifi.processor.util.StandardValidators; + +import java.util.List; + +public class S3Step { + public static final String S3_STEP_NAME = "S3 Configuration"; + public static final String ACCESS_KEY_ID_SECRET_KEY = "Access Key ID and Secret Key"; + public static final String DEFAULT_CREDENTIALS = "Default AWS Credentials"; + + + public static final ConnectorPropertyDescriptor S3_BUCKET = new ConnectorPropertyDescriptor.Builder() + .name("S3 Bucket") + .description("The name of the S3 bucket to write data to.") + .required(true) + .build(); + + public static final ConnectorPropertyDescriptor S3_REGION = new ConnectorPropertyDescriptor.Builder() + .name("S3 Region") + .description("The AWS region where the S3 bucket is located.") + .allowableValuesFetchable(true) + .required(true) + .build(); + + public static final ConnectorPropertyDescriptor S3_DATA_FORMAT = new ConnectorPropertyDescriptor.Builder() + .name("S3 Data Format") + .description("The format to use when writing data to S3.") + .required(true) + .defaultValue("JSON") + .allowableValues("Avro", "JSON") + .build(); + + public static final ConnectorPropertyDescriptor S3_PREFIX = new ConnectorPropertyDescriptor.Builder() + .name("S3 Prefix") + .description("An optional prefix to prepend to all object keys written to the S3 bucket.") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final ConnectorPropertyDescriptor S3_ENDPOINT_OVERRIDE_URL = new ConnectorPropertyDescriptor.Builder() + .name("S3 Endpoint Override URL") + .description("An optional endpoint URL to use instead of the default AWS S3 endpoint. " + + "This can be used to connect to S3-compatible storage systems but should be left unset for connecting to S3.") + .required(false) + .addValidator(StandardValidators.URL_VALIDATOR) + .build(); + + public static final ConnectorPropertyDescriptor TARGET_OBJECT_SIZE = new ConnectorPropertyDescriptor.Builder() + .name("Target Object Size") + .description("The target size for each object written to S3. The connector will attempt to " + + "combine messages until this size is reached before writing an object to S3. Note that this size is approximate " + + "and may vary from object to object.") + .required(true) + .defaultValue("256 MB") + .addValidator(StandardValidators.DATA_SIZE_VALIDATOR) + .build(); + + public static final ConnectorPropertyDescriptor MERGE_LATENCY = new ConnectorPropertyDescriptor.Builder() + .name("Merge Latency") + .description("The maximum amount of time to wait while merging messages before writing an object to S3. " + + "If this time is reached before the target object size is met, the current set of merged messages " + + "will be written to S3.") + .required(true) + .defaultValue("5 min") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .build(); + + public static final ConnectorPropertyDescriptor S3_AUTHENTICATION_STRATEGY = new ConnectorPropertyDescriptor.Builder() + .name("S3 Authentication Strategy") + .description("The authentication strategy to use when connecting to S3.") + .required(true) + .defaultValue(ACCESS_KEY_ID_SECRET_KEY) + .allowableValues(ACCESS_KEY_ID_SECRET_KEY, DEFAULT_CREDENTIALS) + .build(); + + public static final ConnectorPropertyDescriptor S3_ACCESS_KEY_ID = new ConnectorPropertyDescriptor.Builder() + .name("S3 Access Key ID") + .description("The AWS Access Key ID used to authenticate to S3.") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .dependsOn(S3_AUTHENTICATION_STRATEGY, ACCESS_KEY_ID_SECRET_KEY) + .build(); + + public static final ConnectorPropertyDescriptor S3_SECRET_ACCESS_KEY = new ConnectorPropertyDescriptor.Builder() + .name("S3 Secret Access Key") + .description("The AWS Secret Access Key used to authenticate to S3.") + .required(true) + .type(PropertyType.SECRET) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .dependsOn(S3_AUTHENTICATION_STRATEGY, ACCESS_KEY_ID_SECRET_KEY) + .build(); + + + public static final ConnectorPropertyGroup S3_DESTINATION_GROUP = new ConnectorPropertyGroup.Builder() + .name("S3 Destination Configuration") + .description("Properties required to connect to S3 and specify the target bucket.") + .properties(List.of( + S3_BUCKET, + S3_PREFIX, + S3_REGION, + S3_DATA_FORMAT, + S3_ENDPOINT_OVERRIDE_URL + )) + .build(); + + public static final ConnectorPropertyGroup S3_CREDENTIALS_GROUP = new ConnectorPropertyGroup.Builder() + .name("S3 Credentials") + .description("Properties required to authenticate to S3.") + .properties(List.of( + S3_AUTHENTICATION_STRATEGY, + S3_ACCESS_KEY_ID, + S3_SECRET_ACCESS_KEY + )) + .build(); + + public static final ConnectorPropertyGroup MERGE_GROUP = new ConnectorPropertyGroup.Builder() + .name("Merge Configuration") + .description("Configuration for how data should be merged together before being persisted to S3.") + .properties(List.of( + TARGET_OBJECT_SIZE, + MERGE_LATENCY + )) + .build(); + + public static final ConfigurationStep S3_STEP = new ConfigurationStep.Builder() + .name(S3_STEP_NAME) + .description("Configure connection to S3 and target bucket details.") + .propertyGroups(List.of( + S3_DESTINATION_GROUP, + MERGE_GROUP, + S3_CREDENTIALS_GROUP + )) + .build(); +} diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/META-INF/services/org.apache.nifi.components.connector.Connector b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/META-INF/services/org.apache.nifi.components.connector.Connector new file mode 100644 index 000000000000..d5b0fe6db690 --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/META-INF/services/org.apache.nifi.components.connector.Connector @@ -0,0 +1,17 @@ +# 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. + +org.apache.nifi.connectors.kafkas3.KafkaToS3 + diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Connection.md b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Connection.md new file mode 100644 index 000000000000..1621b370616a --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Connection.md @@ -0,0 +1,27 @@ +# Kafka Connection Configuration + +This step configures the connection to your Apache Kafka cluster. + +## Kafka Server Settings + +Enter the bootstrap servers for your Kafka cluster. You can specify multiple brokers +as a comma-separated list (e.g., `broker1:9092,broker2:9092,broker3:9092`). + +### Security Configuration + +Select the appropriate security protocol based on your Kafka cluster configuration: + +| Protocol | Description | +|----------|-------------| +| PLAINTEXT | No encryption or authentication | +| SSL | TLS encryption without SASL authentication | +| SASL_PLAINTEXT | SASL authentication without encryption | +| SASL_SSL | Both SASL authentication and TLS encryption (recommended) | + +If using SASL authentication, provide your username and password credentials. + +## Schema Registry (Optional) + +If your Kafka topics use Avro, Protobuf, or JSON Schema, configure the Schema Registry +URL to enable schema-based serialization and deserialization. + diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Topics.md b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Topics.md new file mode 100644 index 000000000000..2e2739676c13 --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Topics.md @@ -0,0 +1,34 @@ +# Kafka Topics Configuration + +This step configures which Kafka topics to consume from and how to consume them. + +## Topic Names + +Select one or more Kafka topics to consume from. The connector will automatically +fetch the list of available topics from your Kafka cluster. + +## Consumer Group ID + +Specify a unique consumer group ID for this connector. Kafka uses consumer groups +to track message offsets and ensure each message is processed only once within a group. + +**Best Practice:** Use a descriptive name that identifies the purpose of this connector, +such as `kafka-to-s3-production` or `analytics-pipeline-consumer`. + +## Offset Reset + +Controls the behavior when no prior offset exists or the current offset is invalid: + +| Value | Description | +|-------|-------------| +| earliest | Start reading from the oldest available message | +| latest | Start reading from the newest messages only | +| none | Fail if no prior offset exists | + +## Kafka Data Format + +Specify the format of messages in your Kafka topics: + +- **Avro**: Messages are encoded using Apache Avro (requires Schema Registry) +- **JSON**: Messages are plain JSON objects + diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/S3_Configuration.md b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/S3_Configuration.md new file mode 100644 index 000000000000..fcd6092d4344 --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/S3_Configuration.md @@ -0,0 +1,65 @@ +# S3 Configuration + +This step configures the connection to Amazon S3 or an S3-compatible storage system. + +## S3 Destination Configuration + +### S3 Bucket + +The name of the S3 bucket where data will be written. Ensure the bucket exists +and your credentials have write permissions. + +### S3 Prefix + +An optional prefix (folder path) to prepend to all object keys. For example, +setting this to `kafka-data/` will result in objects like: +`kafka-data/2024/01/15/data-001.json` + +### S3 Region + +The AWS region where your S3 bucket is located (e.g., `us-east-1`, `eu-west-1`). + +### S3 Data Format + +The format to use when writing objects to S3: + +- **JSON**: Write data as JSON objects (human-readable) +- **Avro**: Write data in Apache Avro format (compact, schema-embedded) + +### S3 Endpoint Override URL + +Leave this blank for standard AWS S3. Use this field only when connecting to +S3-compatible storage systems like MinIO, LocalStack, or Ceph. + +## Merge Configuration + +### Target Object Size + +The connector merges multiple Kafka messages together before writing to S3 to +reduce the number of objects created. Specify the target size for merged objects +(e.g., `256 MB`, `1 GB`). + +### Merge Latency + +The maximum time to wait while collecting messages before writing to S3. Even if +the target size hasn't been reached, data will be written after this duration to +ensure timely data availability. + +## S3 Credentials + +### Authentication Strategy + +Choose how to authenticate with AWS: + +| Strategy | Description | +|----------|-------------| +| Access Key ID and Secret Key | Use explicit AWS access credentials | +| Default AWS Credentials | Use the default credential chain (environment variables, IAM roles, etc.) | + +### S3 Access Key ID / Secret Access Key + +When using explicit credentials, provide your AWS access key ID and secret access key. + +**Security Note:** Store sensitive credentials using NiFi's sensitive property +protection or parameter contexts with secret providers. + diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/flows/Kafka_to_S3.json b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/flows/Kafka_to_S3.json new file mode 100644 index 000000000000..fabee83a2abf --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/flows/Kafka_to_S3.json @@ -0,0 +1,2105 @@ +{ + "flowContents": { + "identifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b", + "instanceIdentifier": "2c825ae7-019a-1000-f3b7-8275175b7b5b", + "name": "Kafka to S3", + "comments": "", + "position": { + "x": -355.0, + "y": -238.5 + }, + "processGroups": [], + "remoteProcessGroups": [], + "processors": [ + { + "identifier": "4ce9587d-a171-3dff-bea8-bc67f4b356cd", + "instanceIdentifier": "40e02539-8a55-3a44-53d9-e9d41f7db23a", + "name": "MergeRecord", + "comments": "", + "position": { + "x": -250.0, + "y": -55.5 + }, + "type": "org.apache.nifi.processors.standard.MergeRecord", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-nar", + "version": "2.9.0-SNAPSHOT" + }, + "properties": { + "max-bin-age": "#{Merge Latency}", + "correlation-attribute-name": "kafka.topic", + "Attribute Strategy": "Keep Only Common Attributes", + "min-records": "1", + "max-bin-size": "#{Maximum Object size}", + "record-writer": "2384a9ae-311e-3d5f-983b-13f82174244c", + "merge-strategy": "Bin-Packing Algorithm", + "record-reader": "cd697ee5-9621-3680-a6b8-af3237472b5b", + "max-records": "1000000000", + "min-bin-size": "#{Target Object Size}", + "max.bin.count": "10" + }, + "propertyDescriptors": { + "max-bin-age": { + "name": "max-bin-age", + "displayName": "Max Bin Age", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "correlation-attribute-name": { + "name": "correlation-attribute-name", + "displayName": "Correlation Attribute Name", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Attribute Strategy": { + "name": "Attribute Strategy", + "displayName": "Attribute Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "min-records": { + "name": "min-records", + "displayName": "Minimum Number of Records", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "max-bin-size": { + "name": "max-bin-size", + "displayName": "Maximum Bin Size", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "record-writer": { + "name": "record-writer", + "displayName": "Record Writer", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "merge-strategy": { + "name": "merge-strategy", + "displayName": "Merge Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "record-reader": { + "name": "record-reader", + "displayName": "Record Reader", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "max-records": { + "name": "max-records", + "displayName": "Maximum Number of Records", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "min-bin-size": { + "name": "min-bin-size", + "displayName": "Minimum Bin Size", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "max.bin.count": { + "name": "max.bin.count", + "displayName": "Maximum Number of Bins", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + } + }, + "style": {}, + "schedulingPeriod": "0 sec", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 0, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [ + "original", + "failure" + ], + "scheduledState": "ENABLED", + "retryCount": 10000, + "retriedRelationships": [ + "failure" + ], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b" + }, + { + "identifier": "e427a2b2-355c-3ede-9749-b3e4a8561d22", + "instanceIdentifier": "e048d449-7da8-30f3-8ff4-71c87c1d73db", + "name": "ConsumeKafka", + "comments": "", + "position": { + "x": -250.0, + "y": -268.5 + }, + "type": "org.apache.nifi.kafka.processors.ConsumeKafka", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-kafka-nar", + "version": "2.9.0-SNAPSHOT" + }, + "properties": { + "Topics": "#{Topic Names}", + "Commit Offsets": "true", + "Header Name Pattern": null, + "Key Format": "byte-array", + "Key Record Reader": null, + "Message Demarcator": null, + "Record Reader": "919b2a35-4bc7-380c-b592-6ad2ffffbed5", + "Key Attribute Encoding": "utf-8", + "Max Uncommitted Size": null, + "Topic Format": "names", + "Header Encoding": "UTF-8", + "Max Uncommitted Time": "5 secs", + "Kafka Connection Service": "ccf048dc-004a-35be-8e4d-c7a763a30b02", + "Separate By Key": "false", + "Processing Strategy": "RECORD", + "Record Writer": "2384a9ae-311e-3d5f-983b-13f82174244c", + "Group ID": "#{Consumer Group ID}", + "auto.offset.reset": "#{Kafka Auto Offset Reset}", + "Output Strategy": "USE_VALUE" + }, + "propertyDescriptors": { + "Topics": { + "name": "Topics", + "displayName": "Topics", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Commit Offsets": { + "name": "Commit Offsets", + "displayName": "Commit Offsets", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Header Name Pattern": { + "name": "Header Name Pattern", + "displayName": "Header Name Pattern", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Key Format": { + "name": "Key Format", + "displayName": "Key Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Key Record Reader": { + "name": "Key Record Reader", + "displayName": "Key Record Reader", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Message Demarcator": { + "name": "Message Demarcator", + "displayName": "Message Demarcator", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Record Reader": { + "name": "Record Reader", + "displayName": "Record Reader", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Key Attribute Encoding": { + "name": "Key Attribute Encoding", + "displayName": "Key Attribute Encoding", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Max Uncommitted Size": { + "name": "Max Uncommitted Size", + "displayName": "Max Uncommitted Size", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Topic Format": { + "name": "Topic Format", + "displayName": "Topic Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Header Encoding": { + "name": "Header Encoding", + "displayName": "Header Encoding", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Max Uncommitted Time": { + "name": "Max Uncommitted Time", + "displayName": "Max Uncommitted Time", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Kafka Connection Service": { + "name": "Kafka Connection Service", + "displayName": "Kafka Connection Service", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Separate By Key": { + "name": "Separate By Key", + "displayName": "Separate By Key", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Processing Strategy": { + "name": "Processing Strategy", + "displayName": "Processing Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Record Writer": { + "name": "Record Writer", + "displayName": "Record Writer", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Group ID": { + "name": "Group ID", + "displayName": "Group ID", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "auto.offset.reset": { + "name": "auto.offset.reset", + "displayName": "Auto Offset Reset", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Output Strategy": { + "name": "Output Strategy", + "displayName": "Output Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + } + }, + "style": {}, + "schedulingPeriod": "0 sec", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 0, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [ + "parse failure" + ], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b" + }, + { + "identifier": "501f25bb-cef8-328f-bfc2-fe58ad12e7b6", + "instanceIdentifier": "3de30158-1d3e-38d9-6e68-9578471aa26d", + "name": "PutS3Object", + "comments": "", + "position": { + "x": -250.0, + "y": 169.5 + }, + "type": "org.apache.nifi.processors.aws.s3.PutS3Object", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-aws-nar", + "version": "2.9.0-SNAPSHOT" + }, + "properties": { + "FullControl User List": "${s3.permissions.full.users}", + "Owner": "${s3.owner}", + "proxy-configuration-service": null, + "Resource Transfer Source": "FLOWFILE_CONTENT", + "Cache Control": null, + "Endpoint Override URL": "#{S3 Endpoint Override URL}", + "File Resource Service": null, + "Multipart Upload Max Age Threshold": "7 days", + "Object Tags Prefix": null, + "Signer Override": "Default Signature", + "Content Type": "${mime.type}", + "Canned ACL": "${s3.permissions.cannedacl}", + "Use Path Style Access": "false", + "Encryption Service": null, + "Write ACL User List": "${s3.permissions.writeacl.users}", + "Use Chunked Encoding": "true", + "Read ACL User List": "${s3.permissions.readacl.users}", + "Content Disposition": null, + "Storage Class": "Standard", + "Multipart Part Size": "5 GB", + "Remove Tag Prefix": "false", + "Object Key": "${#{S3 Prefix}:append(\"/\"):replaceAll('^/$', '')}${filename}", + "AWS Credentials Provider service": "3f5da96c-9c54-39aa-8c84-e8098a44dfab", + "Multipart Threshold": "5 GB", + "Temporary Directory Multipart State": "${java.io.tmpdir}", + "Custom Signer Module Location": null, + "SSL Context Service": null, + "Server Side Encryption": "None", + "Bucket": "#{S3 Bucket}", + "Multipart Upload AgeOff Interval": "60 min", + "Write Permission User List": "${s3.permissions.write.users}", + "Communications Timeout": "30 secs", + "Region": "#{S3 Region}", + "Read Permission User List": "${s3.permissions.read.users}", + "Custom Signer Class Name": null, + "Expiration Time Rule": null + }, + "propertyDescriptors": { + "FullControl User List": { + "name": "FullControl User List", + "displayName": "FullControl User List", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Owner": { + "name": "Owner", + "displayName": "Owner", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "proxy-configuration-service": { + "name": "proxy-configuration-service", + "displayName": "Proxy Configuration Service", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Resource Transfer Source": { + "name": "Resource Transfer Source", + "displayName": "Resource Transfer Source", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Cache Control": { + "name": "Cache Control", + "displayName": "Cache Control", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Endpoint Override URL": { + "name": "Endpoint Override URL", + "displayName": "Endpoint Override URL", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "File Resource Service": { + "name": "File Resource Service", + "displayName": "File Resource Service", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Multipart Upload Max Age Threshold": { + "name": "Multipart Upload Max Age Threshold", + "displayName": "Multipart Upload Max Age Threshold", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Object Tags Prefix": { + "name": "Object Tags Prefix", + "displayName": "Object Tags Prefix", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Signer Override": { + "name": "Signer Override", + "displayName": "Signer Override", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Content Type": { + "name": "Content Type", + "displayName": "Content Type", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Canned ACL": { + "name": "Canned ACL", + "displayName": "Canned ACL", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Use Path Style Access": { + "name": "Use Path Style Access", + "displayName": "Use Path Style Access", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Encryption Service": { + "name": "Encryption Service", + "displayName": "Encryption Service", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Write ACL User List": { + "name": "Write ACL User List", + "displayName": "Write ACL User List", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Use Chunked Encoding": { + "name": "Use Chunked Encoding", + "displayName": "Use Chunked Encoding", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Read ACL User List": { + "name": "Read ACL User List", + "displayName": "Read ACL User List", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Content Disposition": { + "name": "Content Disposition", + "displayName": "Content Disposition", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Storage Class": { + "name": "Storage Class", + "displayName": "Storage Class", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Multipart Part Size": { + "name": "Multipart Part Size", + "displayName": "Multipart Part Size", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Remove Tag Prefix": { + "name": "Remove Tag Prefix", + "displayName": "Remove Tag Prefix", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Object Key": { + "name": "Object Key", + "displayName": "Object Key", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "AWS Credentials Provider service": { + "name": "AWS Credentials Provider service", + "displayName": "AWS Credentials Provider service", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Multipart Threshold": { + "name": "Multipart Threshold", + "displayName": "Multipart Threshold", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Temporary Directory Multipart State": { + "name": "Temporary Directory Multipart State", + "displayName": "Temporary Directory Multipart State", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Custom Signer Module Location": { + "name": "Custom Signer Module Location", + "displayName": "Custom Signer Module Location", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false, + "resourceDefinition": { + "cardinality": "MULTIPLE", + "resourceTypes": [ + "FILE", + "DIRECTORY" + ] + } + }, + "SSL Context Service": { + "name": "SSL Context Service", + "displayName": "SSL Context Service", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Server Side Encryption": { + "name": "Server Side Encryption", + "displayName": "Server Side Encryption", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Bucket": { + "name": "Bucket", + "displayName": "Bucket", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Multipart Upload AgeOff Interval": { + "name": "Multipart Upload AgeOff Interval", + "displayName": "Multipart Upload AgeOff Interval", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Write Permission User List": { + "name": "Write Permission User List", + "displayName": "Write Permission User List", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Communications Timeout": { + "name": "Communications Timeout", + "displayName": "Communications Timeout", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Region": { + "name": "Region", + "displayName": "Region", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Read Permission User List": { + "name": "Read Permission User List", + "displayName": "Read Permission User List", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Custom Signer Class Name": { + "name": "Custom Signer Class Name", + "displayName": "Custom Signer Class Name", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Expiration Time Rule": { + "name": "Expiration Time Rule", + "displayName": "Expiration Time Rule", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + } + }, + "style": {}, + "schedulingPeriod": "0 sec", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 0, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [ + "success", + "failure" + ], + "scheduledState": "ENABLED", + "retryCount": 100000, + "retriedRelationships": [ + "failure" + ], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b" + } + ], + "inputPorts": [], + "outputPorts": [], + "connections": [ + { + "identifier": "4bd001fc-7fa6-3c34-b297-bae4987889ad", + "instanceIdentifier": "d6a7b460-571b-39b3-b55f-7d82f62b397a", + "name": "", + "source": { + "id": "e427a2b2-355c-3ede-9749-b3e4a8561d22", + "type": "PROCESSOR", + "groupId": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b", + "name": "ConsumeKafka", + "comments": "", + "instanceIdentifier": "e048d449-7da8-30f3-8ff4-71c87c1d73db" + }, + "destination": { + "id": "4ce9587d-a171-3dff-bea8-bc67f4b356cd", + "type": "PROCESSOR", + "groupId": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b", + "name": "MergeRecord", + "comments": "", + "instanceIdentifier": "40e02539-8a55-3a44-53d9-e9d41f7db23a" + }, + "labelIndex": 0, + "zIndex": 1, + "selectedRelationships": [ + "success" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b" + }, + { + "identifier": "08803783-a109-3362-9177-0f2016086a2a", + "instanceIdentifier": "45276df0-1076-39a8-7d84-097b7f593486", + "name": "", + "source": { + "id": "4ce9587d-a171-3dff-bea8-bc67f4b356cd", + "type": "PROCESSOR", + "groupId": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b", + "name": "MergeRecord", + "comments": "", + "instanceIdentifier": "40e02539-8a55-3a44-53d9-e9d41f7db23a" + }, + "destination": { + "id": "501f25bb-cef8-328f-bfc2-fe58ad12e7b6", + "type": "PROCESSOR", + "groupId": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b", + "name": "PutS3Object", + "comments": "", + "instanceIdentifier": "3de30158-1d3e-38d9-6e68-9578471aa26d" + }, + "labelIndex": 0, + "zIndex": 2, + "selectedRelationships": [ + "merged" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b" + } + ], + "labels": [], + "funnels": [], + "controllerServices": [ + { + "identifier": "cd697ee5-9621-3680-a6b8-af3237472b5b", + "instanceIdentifier": "2cdfec74-019a-1000-2786-edf80b919dfc", + "name": "Merge JSON Reader", + "comments": "", + "type": "org.apache.nifi.json.JsonTreeReader", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-record-serialization-services-nar", + "version": "2.9.0-SNAPSHOT" + }, + "properties": { + "Schema Branch": null, + "Schema Reference Reader": null, + "Schema Text": "${avro.schema}", + "Max String Length": "20 MB", + "schema-application-strategy": "SELECTED_PART", + "Schema Registry": "34d29160-0744-373c-a1c7-39c548e7e6b1", + "Schema Access Strategy": "schema-name", + "Schema Name": "${schema.name}", + "Timestamp Format": null, + "schema-inference-cache": null, + "Date Format": null, + "starting-field-strategy": "ROOT_NODE", + "Schema Version": null, + "starting-field-name": null, + "Time Format": null, + "Allow Comments": "false" + }, + "propertyDescriptors": { + "Schema Branch": { + "name": "Schema Branch", + "displayName": "Schema Branch", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Reference Reader": { + "name": "Schema Reference Reader", + "displayName": "Schema Reference Reader", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Schema Text": { + "name": "Schema Text", + "displayName": "Schema Text", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Max String Length": { + "name": "Max String Length", + "displayName": "Max String Length", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "schema-application-strategy": { + "name": "schema-application-strategy", + "displayName": "Schema Application Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Registry": { + "name": "Schema Registry", + "displayName": "Schema Registry", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Schema Access Strategy": { + "name": "Schema Access Strategy", + "displayName": "Schema Access Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Name": { + "name": "Schema Name", + "displayName": "Schema Name", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Timestamp Format": { + "name": "Timestamp Format", + "displayName": "Timestamp Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "schema-inference-cache": { + "name": "schema-inference-cache", + "displayName": "Schema Inference Cache", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Date Format": { + "name": "Date Format", + "displayName": "Date Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "starting-field-strategy": { + "name": "starting-field-strategy", + "displayName": "Starting Field Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Version": { + "name": "Schema Version", + "displayName": "Schema Version", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "starting-field-name": { + "name": "starting-field-name", + "displayName": "Starting Field Name", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Time Format": { + "name": "Time Format", + "displayName": "Time Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Allow Comments": { + "name": "Allow Comments", + "displayName": "Allow Comments", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + } + }, + "controllerServiceApis": [ + { + "type": "org.apache.nifi.serialization.RecordReaderFactory", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-services-api-nar", + "version": "2.9.0-SNAPSHOT" + } + } + ], + "scheduledState": "DISABLED", + "bulletinLevel": "WARN", + "componentType": "CONTROLLER_SERVICE", + "groupIdentifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b" + }, + { + "identifier": "b01590c6-5149-3c50-9d16-3a8e7b1bcdf6", + "instanceIdentifier": "2f0a61a3-b66d-3969-09d7-df35dd0b2740", + "name": "AvroRecordSetWriter", + "comments": "", + "type": "org.apache.nifi.avro.AvroRecordSetWriter", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-record-serialization-services-nar", + "version": "2.9.0-SNAPSHOT" + }, + "properties": { + "Schema Branch": null, + "Schema Reference Reader": null, + "Schema Text": "${avro.schema}", + "Schema Registry": null, + "Schema Access Strategy": "inherit-record-schema", + "Schema Name": "${schema.name}", + "compression-format": "NONE", + "Schema Write Strategy": "avro-embedded", + "cache-size": "1000", + "Schema Cache": null, + "Schema Version": null, + "encoder-pool-size": "32", + "Schema Reference Writer": null + }, + "propertyDescriptors": { + "Schema Branch": { + "name": "Schema Branch", + "displayName": "Schema Branch", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Reference Reader": { + "name": "Schema Reference Reader", + "displayName": "Schema Reference Reader", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Schema Text": { + "name": "Schema Text", + "displayName": "Schema Text", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Registry": { + "name": "Schema Registry", + "displayName": "Schema Registry", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Schema Access Strategy": { + "name": "Schema Access Strategy", + "displayName": "Schema Access Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Name": { + "name": "Schema Name", + "displayName": "Schema Name", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "compression-format": { + "name": "compression-format", + "displayName": "Compression Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Write Strategy": { + "name": "Schema Write Strategy", + "displayName": "Schema Write Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "cache-size": { + "name": "cache-size", + "displayName": "Cache Size", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Cache": { + "name": "Schema Cache", + "displayName": "Schema Cache", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Schema Version": { + "name": "Schema Version", + "displayName": "Schema Version", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "encoder-pool-size": { + "name": "encoder-pool-size", + "displayName": "Encoder Pool Size", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Reference Writer": { + "name": "Schema Reference Writer", + "displayName": "Schema Reference Writer", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + } + }, + "controllerServiceApis": [ + { + "type": "org.apache.nifi.serialization.RecordSetWriterFactory", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-services-api-nar", + "version": "2.9.0-SNAPSHOT" + } + } + ], + "scheduledState": "DISABLED", + "bulletinLevel": "WARN", + "componentType": "CONTROLLER_SERVICE", + "groupIdentifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b" + }, + { + "identifier": "919b2a35-4bc7-380c-b592-6ad2ffffbed5", + "instanceIdentifier": "c169a376-4b21-3062-e8fb-1d243dd0e5df", + "name": "Kafka JSON Reader", + "comments": "", + "type": "org.apache.nifi.json.JsonTreeReader", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-record-serialization-services-nar", + "version": "2.9.0-SNAPSHOT" + }, + "properties": { + "Schema Branch": null, + "Schema Reference Reader": "ecba6fe5-6beb-3b68-bad9-163a3f14c4c8", + "Schema Text": "${avro.schema}", + "Max String Length": "20 MB", + "schema-application-strategy": "SELECTED_PART", + "Schema Registry": "34d29160-0744-373c-a1c7-39c548e7e6b1", + "Schema Access Strategy": "schema-reference-reader", + "Schema Name": "${schema.name}", + "Timestamp Format": null, + "schema-inference-cache": null, + "Date Format": null, + "starting-field-strategy": "ROOT_NODE", + "Schema Version": null, + "starting-field-name": null, + "Time Format": null, + "Allow Comments": "false" + }, + "propertyDescriptors": { + "Schema Branch": { + "name": "Schema Branch", + "displayName": "Schema Branch", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Reference Reader": { + "name": "Schema Reference Reader", + "displayName": "Schema Reference Reader", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Schema Text": { + "name": "Schema Text", + "displayName": "Schema Text", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Max String Length": { + "name": "Max String Length", + "displayName": "Max String Length", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "schema-application-strategy": { + "name": "schema-application-strategy", + "displayName": "Schema Application Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Registry": { + "name": "Schema Registry", + "displayName": "Schema Registry", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Schema Access Strategy": { + "name": "Schema Access Strategy", + "displayName": "Schema Access Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Name": { + "name": "Schema Name", + "displayName": "Schema Name", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Timestamp Format": { + "name": "Timestamp Format", + "displayName": "Timestamp Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "schema-inference-cache": { + "name": "schema-inference-cache", + "displayName": "Schema Inference Cache", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Date Format": { + "name": "Date Format", + "displayName": "Date Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "starting-field-strategy": { + "name": "starting-field-strategy", + "displayName": "Starting Field Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Version": { + "name": "Schema Version", + "displayName": "Schema Version", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "starting-field-name": { + "name": "starting-field-name", + "displayName": "Starting Field Name", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Time Format": { + "name": "Time Format", + "displayName": "Time Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Allow Comments": { + "name": "Allow Comments", + "displayName": "Allow Comments", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + } + }, + "controllerServiceApis": [ + { + "type": "org.apache.nifi.serialization.RecordReaderFactory", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-services-api-nar", + "version": "2.9.0-SNAPSHOT" + } + } + ], + "scheduledState": "DISABLED", + "bulletinLevel": "WARN", + "componentType": "CONTROLLER_SERVICE", + "groupIdentifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b" + }, + { + "identifier": "174d0040-161d-39dc-8da9-0f127fc46373", + "instanceIdentifier": "bf48cf99-1cae-375b-bfe1-bd9aa7368984", + "name": "AvroReader", + "comments": "", + "type": "org.apache.nifi.avro.AvroReader", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-record-serialization-services-nar", + "version": "2.9.0-SNAPSHOT" + }, + "properties": { + "Schema Branch": null, + "Schema Reference Reader": "ecba6fe5-6beb-3b68-bad9-163a3f14c4c8", + "Schema Text": "${avro.schema}", + "cache-size": "1000", + "Schema Version": null, + "Schema Registry": "34d29160-0744-373c-a1c7-39c548e7e6b1", + "Schema Access Strategy": "schema-reference-reader", + "Schema Name": "${schema.name}" + }, + "propertyDescriptors": { + "Schema Branch": { + "name": "Schema Branch", + "displayName": "Schema Branch", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Reference Reader": { + "name": "Schema Reference Reader", + "displayName": "Schema Reference Reader", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Schema Text": { + "name": "Schema Text", + "displayName": "Schema Text", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "cache-size": { + "name": "cache-size", + "displayName": "Cache Size", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Version": { + "name": "Schema Version", + "displayName": "Schema Version", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Registry": { + "name": "Schema Registry", + "displayName": "Schema Registry", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Schema Access Strategy": { + "name": "Schema Access Strategy", + "displayName": "Schema Access Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Name": { + "name": "Schema Name", + "displayName": "Schema Name", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + } + }, + "controllerServiceApis": [ + { + "type": "org.apache.nifi.serialization.RecordReaderFactory", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-services-api-nar", + "version": "2.9.0-SNAPSHOT" + } + } + ], + "scheduledState": "DISABLED", + "bulletinLevel": "WARN", + "componentType": "CONTROLLER_SERVICE", + "groupIdentifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b" + }, + { + "identifier": "3f5da96c-9c54-39aa-8c84-e8098a44dfab", + "instanceIdentifier": "7ea5a25b-89be-3652-9778-e30a69f84806", + "name": "AWSCredentialsProviderControllerService", + "comments": "", + "type": "org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderControllerService", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-aws-nar", + "version": "2.9.0-SNAPSHOT" + }, + "properties": { + "Assume Role STS Region": "us-west-2", + "Assume Role Proxy Configuration Service": null, + "Access Key ID": "#{S3 Access Key ID}", + "Assume Role ARN": null, + "Assume Role Session Time": "3600", + "Assume Role Session Name": null, + "Custom Signer Module Location": null, + "Use Default Credentials": "false", + "Assume Role External ID": null, + "Assume Role STS Endpoint Override": null, + "Profile Name": null, + "Credentials File": null, + "Custom Signer Class Name": null, + "Assume Role STS Signer Override": "Default Signature", + "Use Anonymous Credentials": "false", + "Secret Access Key": "#{S3 Secret Access Key}", + "Assume Role SSL Context Service": null + }, + "propertyDescriptors": { + "Assume Role STS Region": { + "name": "Assume Role STS Region", + "displayName": "Assume Role STS Region", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Assume Role Proxy Configuration Service": { + "name": "Assume Role Proxy Configuration Service", + "displayName": "Assume Role Proxy Configuration Service", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Access Key ID": { + "name": "Access Key ID", + "displayName": "Access Key ID", + "identifiesControllerService": false, + "sensitive": true, + "dynamic": false + }, + "Assume Role ARN": { + "name": "Assume Role ARN", + "displayName": "Assume Role ARN", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Assume Role Session Time": { + "name": "Assume Role Session Time", + "displayName": "Assume Role Session Time", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Assume Role Session Name": { + "name": "Assume Role Session Name", + "displayName": "Assume Role Session Name", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Custom Signer Module Location": { + "name": "Custom Signer Module Location", + "displayName": "Custom Signer Module Location", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false, + "resourceDefinition": { + "cardinality": "MULTIPLE", + "resourceTypes": [ + "FILE", + "DIRECTORY" + ] + } + }, + "Use Default Credentials": { + "name": "Use Default Credentials", + "displayName": "Use Default Credentials", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Assume Role External ID": { + "name": "Assume Role External ID", + "displayName": "Assume Role External ID", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Assume Role STS Endpoint Override": { + "name": "Assume Role STS Endpoint Override", + "displayName": "Assume Role STS Endpoint Override", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Profile Name": { + "name": "Profile Name", + "displayName": "Profile Name", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Credentials File": { + "name": "Credentials File", + "displayName": "Credentials File", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false, + "resourceDefinition": { + "cardinality": "SINGLE", + "resourceTypes": [ + "FILE" + ] + } + }, + "Custom Signer Class Name": { + "name": "Custom Signer Class Name", + "displayName": "Custom Signer Class Name", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Assume Role STS Signer Override": { + "name": "Assume Role STS Signer Override", + "displayName": "Assume Role STS Signer Override", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Use Anonymous Credentials": { + "name": "Use Anonymous Credentials", + "displayName": "Use Anonymous Credentials", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Secret Access Key": { + "name": "Secret Access Key", + "displayName": "Secret Access Key", + "identifiesControllerService": false, + "sensitive": true, + "dynamic": false + }, + "Assume Role SSL Context Service": { + "name": "Assume Role SSL Context Service", + "displayName": "Assume Role SSL Context Service", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + } + }, + "controllerServiceApis": [ + { + "type": "org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-aws-service-api-nar", + "version": "2.9.0-SNAPSHOT" + } + }, + { + "type": "org.apache.nifi.processors.aws.credentials.provider.AwsCredentialsProviderService", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-aws-service-api-nar", + "version": "2.9.0-SNAPSHOT" + } + } + ], + "scheduledState": "DISABLED", + "bulletinLevel": "WARN", + "componentType": "CONTROLLER_SERVICE", + "groupIdentifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b" + }, + { + "identifier": "2384a9ae-311e-3d5f-983b-13f82174244c", + "instanceIdentifier": "d02eb508-0c95-31df-b86b-e1adadd77845", + "name": "JsonRecordSetWriter", + "comments": "", + "type": "org.apache.nifi.json.JsonRecordSetWriter", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-record-serialization-services-nar", + "version": "2.9.0-SNAPSHOT" + }, + "properties": { + "Schema Branch": null, + "Schema Reference Reader": null, + "Schema Text": "${avro.schema}", + "Allow Scientific Notation": "false", + "compression-level": "1", + "Schema Registry": null, + "Schema Access Strategy": "inherit-record-schema", + "Schema Name": "${schema.name}", + "Timestamp Format": null, + "Date Format": null, + "Pretty Print JSON": "false", + "compression-format": "none", + "Schema Write Strategy": "schema-name", + "suppress-nulls": "never-suppress", + "output-grouping": "output-oneline", + "Schema Cache": null, + "Schema Version": null, + "Time Format": null, + "Schema Reference Writer": null + }, + "propertyDescriptors": { + "Schema Branch": { + "name": "Schema Branch", + "displayName": "Schema Branch", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Reference Reader": { + "name": "Schema Reference Reader", + "displayName": "Schema Reference Reader", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Schema Text": { + "name": "Schema Text", + "displayName": "Schema Text", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Allow Scientific Notation": { + "name": "Allow Scientific Notation", + "displayName": "Allow Scientific Notation", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "compression-level": { + "name": "compression-level", + "displayName": "Compression Level", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Registry": { + "name": "Schema Registry", + "displayName": "Schema Registry", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Schema Access Strategy": { + "name": "Schema Access Strategy", + "displayName": "Schema Access Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Name": { + "name": "Schema Name", + "displayName": "Schema Name", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Timestamp Format": { + "name": "Timestamp Format", + "displayName": "Timestamp Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Date Format": { + "name": "Date Format", + "displayName": "Date Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Pretty Print JSON": { + "name": "Pretty Print JSON", + "displayName": "Pretty Print JSON", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "compression-format": { + "name": "compression-format", + "displayName": "Compression Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Write Strategy": { + "name": "Schema Write Strategy", + "displayName": "Schema Write Strategy", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "suppress-nulls": { + "name": "suppress-nulls", + "displayName": "Suppress Null Values", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "output-grouping": { + "name": "output-grouping", + "displayName": "Output Grouping", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Cache": { + "name": "Schema Cache", + "displayName": "Schema Cache", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Schema Version": { + "name": "Schema Version", + "displayName": "Schema Version", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Time Format": { + "name": "Time Format", + "displayName": "Time Format", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Reference Writer": { + "name": "Schema Reference Writer", + "displayName": "Schema Reference Writer", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + } + }, + "controllerServiceApis": [ + { + "type": "org.apache.nifi.serialization.RecordSetWriterFactory", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-services-api-nar", + "version": "2.9.0-SNAPSHOT" + } + } + ], + "scheduledState": "DISABLED", + "bulletinLevel": "WARN", + "componentType": "CONTROLLER_SERVICE", + "groupIdentifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b" + }, + { + "identifier": "34d29160-0744-373c-a1c7-39c548e7e6b1", + "instanceIdentifier": "e0f7c153-e070-369b-451d-745b53ebe26a", + "name": "ConfluentSchemaRegistry", + "comments": "", + "type": "org.apache.nifi.confluent.schemaregistry.ConfluentSchemaRegistry", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-confluent-platform-nar", + "version": "2.9.0-SNAPSHOT" + }, + "properties": { + "SSL Context Service": null, + "Username": "#{Schema Registry Username}", + "Schema Registry URLs": "#{Schema Registry URLs}", + "Communications Timeout": "30 secs", + "Cache Size": "1000", + "Authentication Type": "#{Schema Registry Authentication Type}", + "Cache Expiration": "1 hour", + "Password": "#{Schema Registry Password}" + }, + "propertyDescriptors": { + "SSL Context Service": { + "name": "SSL Context Service", + "displayName": "SSL Context Service", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "Username": { + "name": "Username", + "displayName": "Username", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Schema Registry URLs": { + "name": "Schema Registry URLs", + "displayName": "Schema Registry URLs", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Communications Timeout": { + "name": "Communications Timeout", + "displayName": "Communications Timeout", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Cache Size": { + "name": "Cache Size", + "displayName": "Cache Size", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Authentication Type": { + "name": "Authentication Type", + "displayName": "Authentication Type", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Cache Expiration": { + "name": "Cache Expiration", + "displayName": "Cache Expiration", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "Password": { + "name": "Password", + "displayName": "Password", + "identifiesControllerService": false, + "sensitive": true, + "dynamic": false + } + }, + "controllerServiceApis": [ + { + "type": "org.apache.nifi.schemaregistry.services.SchemaRegistry", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-services-api-nar", + "version": "2.9.0-SNAPSHOT" + } + } + ], + "scheduledState": "DISABLED", + "bulletinLevel": "WARN", + "componentType": "CONTROLLER_SERVICE", + "groupIdentifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b" + }, + { + "identifier": "ecba6fe5-6beb-3b68-bad9-163a3f14c4c8", + "instanceIdentifier": "da19419b-4821-315c-74b2-03d97cf411dc", + "name": "ConfluentEncodedSchemaReferenceReader", + "comments": "", + "type": "org.apache.nifi.confluent.schemaregistry.ConfluentEncodedSchemaReferenceReader", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-confluent-platform-nar", + "version": "2.9.0-SNAPSHOT" + }, + "properties": {}, + "propertyDescriptors": {}, + "controllerServiceApis": [ + { + "type": "org.apache.nifi.schemaregistry.services.SchemaReferenceReader", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-standard-services-api-nar", + "version": "2.9.0-SNAPSHOT" + } + } + ], + "scheduledState": "DISABLED", + "bulletinLevel": "WARN", + "componentType": "CONTROLLER_SERVICE", + "groupIdentifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b" + }, + { + "identifier": "ccf048dc-004a-35be-8e4d-c7a763a30b02", + "instanceIdentifier": "5549d8f6-48eb-3677-821e-292b9b7e73d2", + "name": "Kafka3ConnectionService", + "comments": "", + "type": "org.apache.nifi.kafka.service.Kafka3ConnectionService", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-kafka-3-service-nar", + "version": "2.9.0-SNAPSHOT" + }, + "properties": { + "ack.wait.time": "10 sec", + "sasl.username": "#{Kafka SASL Username}", + "max.poll.records": "20000", + "bootstrap.servers": "#{Kafka Bootstrap Servers}", + "sasl.kerberos.service.name": null, + "security.protocol": "#{Kafka Security Protocol}", + "SSL Context Service": null, + "sasl.mechanism": "#{Kafka SASL Mechanism}", + "isolation.level": "read_committed", + "oauth2-access-token-provider-service": null, + "max.block.ms": "10 sec", + "kerberos-user-service": null, + "sasl.password": "#{Kafka SASL Password}", + "default.api.timeout.ms": "60 sec" + }, + "propertyDescriptors": { + "ack.wait.time": { + "name": "ack.wait.time", + "displayName": "Acknowledgment Wait Time", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "sasl.username": { + "name": "sasl.username", + "displayName": "SASL Username", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "max.poll.records": { + "name": "max.poll.records", + "displayName": "Max Poll Records", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "bootstrap.servers": { + "name": "bootstrap.servers", + "displayName": "Bootstrap Servers", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "sasl.kerberos.service.name": { + "name": "sasl.kerberos.service.name", + "displayName": "Kerberos Service Name", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "security.protocol": { + "name": "security.protocol", + "displayName": "Security Protocol", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "SSL Context Service": { + "name": "SSL Context Service", + "displayName": "SSL Context Service", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "sasl.mechanism": { + "name": "sasl.mechanism", + "displayName": "SASL Mechanism", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "isolation.level": { + "name": "isolation.level", + "displayName": "Transaction Isolation Level", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "oauth2-access-token-provider-service": { + "name": "oauth2-access-token-provider-service", + "displayName": "OAuth2 Access Token Provider Service", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "max.block.ms": { + "name": "max.block.ms", + "displayName": "Max Metadata Wait Time", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + }, + "kerberos-user-service": { + "name": "kerberos-user-service", + "displayName": "Kerberos User Service", + "identifiesControllerService": true, + "sensitive": false, + "dynamic": false + }, + "sasl.password": { + "name": "sasl.password", + "displayName": "SASL Password", + "identifiesControllerService": false, + "sensitive": true, + "dynamic": false + }, + "default.api.timeout.ms": { + "name": "default.api.timeout.ms", + "displayName": "Client Timeout", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + } + }, + "controllerServiceApis": [ + { + "type": "org.apache.nifi.kafka.service.api.KafkaConnectionService", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-kafka-service-api-nar", + "version": "2.9.0-SNAPSHOT" + } + } + ], + "scheduledState": "DISABLED", + "bulletinLevel": "WARN", + "componentType": "CONTROLLER_SERVICE", + "groupIdentifier": "b3a5c0de-2130-3ba9-942b-9bae5735bd4b" + } + ], + "parameterContextName": "Kafka to S3", + "defaultFlowFileExpiration": "0 sec", + "defaultBackPressureObjectThreshold": 10000, + "defaultBackPressureDataSizeThreshold": "1 GB", + "scheduledState": "ENABLED", + "executionEngine": "INHERITED", + "maxConcurrentTasks": 1, + "statelessFlowTimeout": "1 min", + "componentType": "PROCESS_GROUP", + "flowFileConcurrency": "UNBOUNDED", + "flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE" + }, + "externalControllerServices": {}, + "parameterContexts": { + "Kafka to S3": { + "name": "Kafka to S3", + "parameters": [ + { + "name": "S3 Secret Access Key", + "description": "", + "sensitive": true, + "provided": false + }, + { + "name": "Schema Registry Password", + "description": "", + "sensitive": true, + "provided": false + }, + { + "name": "Consumer Group ID", + "description": "", + "sensitive": false, + "provided": false + }, + { + "name": "Kafka SASL Mechanism", + "description": "", + "sensitive": false, + "provided": false, + "value": "SCRAM-SHA-512" + }, + { + "name": "Kafka Data Format", + "description": "", + "sensitive": false, + "provided": false, + "value": "JSON" + }, + { + "name": "Schema Registry Username", + "description": "", + "sensitive": false, + "provided": false + }, + { + "name": "S3 Data Format", + "description": "", + "sensitive": false, + "provided": false, + "value": "JSON" + }, + { + "name": "Schema Registry URLs", + "description": "", + "sensitive": false, + "provided": false, + "value": "http://localhost:8081" + }, + { + "name": "Merge Latency", + "description": "", + "sensitive": false, + "provided": false, + "value": "10 mins" + }, + { + "name": "S3 Endpoint Override URL", + "description": "", + "sensitive": false, + "provided": false + }, + { + "name": "Schema Registry Authentication Type", + "description": "", + "sensitive": false, + "provided": false, + "value": "BASIC" + }, + { + "name": "Kafka Security Protocol", + "description": "", + "sensitive": false, + "provided": false, + "value": "SASL_SSL" + }, + { + "name": "Maximum Object size", + "description": "", + "sensitive": false, + "provided": false, + "value": "300 MB" + }, + { + "name": "S3 Region", + "description": "", + "sensitive": false, + "provided": false, + "value": "us-west-2" + }, + { + "name": "Kafka SASL Password", + "description": "", + "sensitive": true, + "provided": false + }, + { + "name": "Kafka Bootstrap Servers", + "description": "", + "sensitive": false, + "provided": false + }, + { + "name": "Kafka Auto Offset Reset", + "description": "", + "sensitive": false, + "provided": false, + "value": "latest" + }, + { + "name": "S3 Bucket", + "description": "", + "sensitive": false, + "provided": false + }, + { + "name": "Kafka SASL Username", + "description": "", + "sensitive": false, + "provided": false + }, + { + "name": "S3 Access Key ID", + "description": "", + "sensitive": true, + "provided": false + }, + { + "name": "Target Object Size", + "description": "", + "sensitive": false, + "provided": false, + "value": "256 MB" + }, + { + "name": "Topic Names", + "description": "", + "sensitive": false, + "provided": false + }, + { + "name": "S3 Prefix", + "description": "", + "sensitive": false, + "provided": false + } + ], + "inheritedParameterContexts": [], + "description": "", + "componentType": "PARAMETER_CONTEXT" + } + }, + "flowEncodingVersion": "1.0", + "parameterProviders": {}, + "latest": false +} \ No newline at end of file diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-integration-tests/pom.xml b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-integration-tests/pom.xml new file mode 100644 index 000000000000..628f7b5fed63 --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-integration-tests/pom.xml @@ -0,0 +1,238 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-kafka-to-s3-bundle + 2.9.0-SNAPSHOT + + + nifi-kafka-to-s3-integration-tests + + + + confluent + https://packages.confluent.io/maven/ + + + + + + org.slf4j + slf4j-api + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-connector-mock-api + 2.9.0-SNAPSHOT + + + org.apache.nifi + nifi-connector-mock + 2.9.0-SNAPSHOT + + + + + org.testcontainers + testcontainers-kafka + 2.0.1 + test + + + org.testcontainers + localstack + 1.20.4 + test + + + + + + org.apache.kafka + kafka-clients + 4.1.1 + + + + org.lz4 + lz4-java + + + + + at.yawk.lz4 + lz4-java + 1.10.1 + + + + + io.confluent + kafka-avro-serializer + 7.8.0 + test + + + io.confluent + kafka-schema-registry-client + 7.8.0 + test + + + org.apache.avro + avro + 1.11.3 + test + + + + + software.amazon.awssdk + s3 + 2.36.0 + test + + + software.amazon.awssdk + url-connection-client + 2.36.0 + test + + + + + org.apache.nifi + nifi-kafka-to-s3-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-kafka-3-service-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-kafka-service-api-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-kafka-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-aws-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-aws-service-api-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-standard-services-api-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-standard-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-standard-shared-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-record-serialization-services-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-confluent-platform-nar + 2.9.0-SNAPSHOT + nar + + + + + org.apache.nifi + nifi-framework-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-jetty-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-provenance-repository-nar + 2.9.0-SNAPSHOT + nar + + + org.apache.nifi + nifi-connector-mock-server-nar + 2.9.0-SNAPSHOT + nar + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-nar + generate-resources + + copy-dependencies + + + ${project.build.directory}/libDir + nar + + + + + + + diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-integration-tests/src/test/java/org/apache/nifi/connectors/kafkas3/KafkaToS3IT.java b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-integration-tests/src/test/java/org/apache/nifi/connectors/kafkas3/KafkaToS3IT.java new file mode 100644 index 000000000000..f715ea2e4e7c --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-integration-tests/src/test/java/org/apache/nifi/connectors/kafkas3/KafkaToS3IT.java @@ -0,0 +1,758 @@ +/* + * 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.connectors.kafkas3; + +import io.confluent.kafka.serializers.KafkaAvroSerializer; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.ConnectorValueReference; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.mock.connector.StandardConnectorTestRunner; +import org.apache.nifi.mock.connector.server.ConnectorConfigVerificationResult; +import org.apache.nifi.mock.connector.server.ConnectorTestRunner; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.Network; +import org.testcontainers.containers.localstack.LocalStackContainer; +import org.testcontainers.containers.wait.strategy.Wait; +import org.testcontainers.kafka.ConfluentKafkaContainer; +import org.testcontainers.utility.DockerImageName; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.CreateBucketRequest; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; +import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; +import software.amazon.awssdk.services.s3.model.S3Object; + +import java.io.File; +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.ExecutionException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class KafkaToS3IT { + + private static ConnectorTestRunner runner; + private static Network network; + private static ConfluentKafkaContainer kafkaContainer; + private static GenericContainer schemaRegistryContainer; + private static LocalStackContainer localStackContainer; + private static S3Client s3Client; + + private static final String SCRAM_USERNAME = "testuser"; + private static final String SCRAM_PASSWORD = "testpassword"; + + private static final String S3_REGION = "us-west-2"; + + // JAAS configuration for Kafka broker SASL/PLAIN authentication. + // The 'username' and 'password' fields are credentials the broker uses for inter-broker communication. + // The 'user_=""' entries define client users that can authenticate to this broker. + // In this setup: + // - Broker uses 'admin' / 'admin-secret' for inter-broker communication (though we use PLAINTEXT for that) + // - Clients can authenticate using 'testuser' / 'testpassword' on the SASL listener with PLAIN mechanism + private static final String JAAS_CONFIG_CONTENT = """ + KafkaServer { + org.apache.kafka.common.security.plain.PlainLoginModule required + username="admin" + password="admin-secret" + user_%s="%s"; + }; + """.formatted(SCRAM_USERNAME, SCRAM_PASSWORD); + + + @BeforeAll + public static void setupTestContainers() { + network = Network.newNetwork(); + + kafkaContainer = new ConfluentKafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:7.8.0")) + .withNetwork(network) + .withNetworkAliases("kafka") + .withStartupTimeout(Duration.ofSeconds(10)) + .withEnv("KAFKA_LISTENER_SECURITY_PROTOCOL_MAP", "CONTROLLER:PLAINTEXT,BROKER:PLAINTEXT,PLAINTEXT:PLAINTEXT,SASL:SASL_PLAINTEXT") + .withEnv("KAFKA_LISTENERS", "CONTROLLER://0.0.0.0:9094,BROKER://0.0.0.0:9092,PLAINTEXT://0.0.0.0:19092,SASL://0.0.0.0:9093") + .withEnv("KAFKA_ADVERTISED_LISTENERS", "BROKER://kafka:9092,PLAINTEXT://kafka:19092,SASL://localhost:9093") + .withEnv("KAFKA_CONTROLLER_LISTENER_NAMES", "CONTROLLER") + .withEnv("KAFKA_INTER_BROKER_LISTENER_NAME", "BROKER") + .withEnv("KAFKA_SASL_ENABLED_MECHANISMS", "PLAIN") + .withEnv("KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS", "0") + .withEnv("KAFKA_GROUP_MIN_SESSION_TIMEOUT_MS", "1000") + .withEnv("KAFKA_GROUP_MAX_SESSION_TIMEOUT_MS", "60000") + .withEnv("KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR", "1") + .withEnv("KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR", "1") + .withEnv("KAFKA_TRANSACTION_STATE_LOG_MIN_ISR", "1") + .withEnv("KAFKA_OPTS", "-Djava.security.auth.login.config=/tmp/kafka_jaas.conf") + .withCommand( + "sh", "-c", + "echo '" + JAAS_CONFIG_CONTENT + "' > /tmp/kafka_jaas.conf && " + + "/etc/confluent/docker/run" + ); + + kafkaContainer.setPortBindings(List.of("9093:9093")); + kafkaContainer.start(); + + schemaRegistryContainer = new GenericContainer<>(DockerImageName.parse("confluentinc/cp-schema-registry:7.8.0")) + .withNetwork(network) + .withExposedPorts(8081) + .withEnv("SCHEMA_REGISTRY_HOST_NAME", "schema-registry") + .withEnv("SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS", "PLAINTEXT://kafka:19092") + .withEnv("SCHEMA_REGISTRY_LISTENERS", "http://0.0.0.0:8081") + .withStartupTimeout(Duration.ofSeconds(60)) + .waitingFor(Wait.forHttp("/subjects").forStatusCode(200)) + .dependsOn(kafkaContainer); + + schemaRegistryContainer.start(); + + localStackContainer = new LocalStackContainer(DockerImageName.parse("localstack/localstack:4.1.0")) + .withServices(LocalStackContainer.Service.S3) + .withStartupTimeout(Duration.ofSeconds(30)); + + localStackContainer.start(); + + s3Client = S3Client.builder() + .endpointOverride(localStackContainer.getEndpoint()) + .credentialsProvider(StaticCredentialsProvider.create(AwsBasicCredentials.create(localStackContainer.getAccessKey(), localStackContainer.getSecretKey()))) + .region(Region.of(S3_REGION)) + .httpClient(UrlConnectionHttpClient.builder().build()) + .forcePathStyle(true) + .build(); + } + + @BeforeEach + public void setupRunner() { + runner = new StandardConnectorTestRunner.Builder() + .connectorClassName("org.apache.nifi.connectors.kafkas3.KafkaToS3") + .narLibraryDirectory(new File("target/libDir")) + .build(); + assertNotNull(runner); + } + + @AfterAll + public static void cleanupTestContainers() { + if (s3Client != null) { + s3Client.close(); + } + + if (localStackContainer != null) { + localStackContainer.stop(); + } + + if (schemaRegistryContainer != null) { + schemaRegistryContainer.stop(); + } + + if (kafkaContainer != null) { + kafkaContainer.stop(); + } + + if (network != null) { + network.close(); + } + } + + @AfterEach + public void cleanupRunner() throws IOException { + if (runner != null) { + runner.close(); + } + } + + + private void createS3Bucket(final String bucketName) { + s3Client.createBucket(CreateBucketRequest.builder().bucket(bucketName).build()); + } + + private void createKafkaTopics(final String... topicNames) throws ExecutionException, InterruptedException { + final Properties adminProps = new Properties(); + adminProps.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9093"); + adminProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SASL_PLAINTEXT"); + adminProps.put(SaslConfigs.SASL_MECHANISM, "PLAIN"); + adminProps.put(SaslConfigs.SASL_JAAS_CONFIG, String.format( + "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"%s\" password=\"%s\";", + SCRAM_USERNAME, SCRAM_PASSWORD + )); + + try (final AdminClient adminClient = AdminClient.create(adminProps)) { + final List topics = new ArrayList<>(); + for (final String topicName : topicNames) { + topics.add(new NewTopic(topicName, 1, (short) 1)); + } + + adminClient.createTopics(topics).all().get(); + } + } + + private void produceRecordsToTopic(final String topicName, final String... records) throws ExecutionException, InterruptedException { + final Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9093"); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SASL_PLAINTEXT"); + producerProps.put(SaslConfigs.SASL_MECHANISM, "PLAIN"); + producerProps.put(SaslConfigs.SASL_JAAS_CONFIG, String.format( + "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"%s\" password=\"%s\";", + SCRAM_USERNAME, SCRAM_PASSWORD + )); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + for (final String record : records) { + final ProducerRecord producerRecord = new ProducerRecord<>(topicName, record); + producer.send(producerRecord).get(); + } + + producer.flush(); + } + } + + private String getSchemaRegistryUrl() { + return String.format("http://%s:%d", schemaRegistryContainer.getHost(), schemaRegistryContainer.getMappedPort(8081)); + } + + private void produceAvroRecordsToTopic(final String topicName, final Schema schema, final GenericRecord... records) throws ExecutionException, InterruptedException { + final Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9093"); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, KafkaAvroSerializer.class.getName()); + producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SASL_PLAINTEXT"); + producerProps.put(SaslConfigs.SASL_MECHANISM, "PLAIN"); + producerProps.put(SaslConfigs.SASL_JAAS_CONFIG, String.format( + "org.apache.kafka.common.security.plain.PlainLoginModule required username=\"%s\" password=\"%s\";", + SCRAM_USERNAME, SCRAM_PASSWORD + )); + producerProps.put("schema.registry.url", getSchemaRegistryUrl()); + + try (final KafkaProducer producer = new KafkaProducer<>(producerProps)) { + for (final GenericRecord record : records) { + final ProducerRecord producerRecord = new ProducerRecord<>(topicName, record); + producer.send(producerRecord).get(); + } + + producer.flush(); + } + } + + + @Test + public void testVerification() throws ExecutionException, InterruptedException, FlowUpdateException { + createKafkaTopics("topic-1", "topic-2", "topic-3", "topic-4", "topic-5", "Z-topic", "an-important-topic"); + + produceRecordsToTopic("topic-1", + """ + {"id": 1, "name": "Alice", "age": 30}""", + """ + {"id": 2, "name": "Bob", "age": 25}""", + """ + {"id": 3, "name": "Charlie", "age": 35}""" + ); + + produceRecordsToTopic("an-important-topic", + "This is a plaintext message", + "Another important message", + "Final plaintext record" + ); + + final Map kafkaServerConfig = Map.of( + "Kafka Brokers", "localhost:9093", + "Security Protocol", "SASL_PLAINTEXT", + "SASL Mechanism", "PLAIN", + "Username", SCRAM_USERNAME + ); + runner.addSecret("kafka-password", SCRAM_PASSWORD); + final Map kafkaServerSecretRefs = Map.of( + "Password", runner.createSecretReference("kafka-password") + ); + + // Perform verification to ensure that valid server configuration passes + final ConnectorConfigVerificationResult connectionVerificationResults = runner.verifyConfiguration("Kafka Connection", kafkaServerConfig, kafkaServerSecretRefs); + connectionVerificationResults.assertNoFailures(); + + // Apply the configuration that we've now validated + runner.configure("Kafka Connection", kafkaServerConfig, kafkaServerSecretRefs); + + // Perform verification to ensure that valid topic configuration passes + final Map topic1Config = Map.of( + "Topic Names", "topic-1", + "Consumer Group ID", "nifi-kafka-to-s3-testSuccessfulFlow", + "Offset Reset", "earliest", + "Kafka Data Format", "JSON" + ); + final ConnectorConfigVerificationResult topic1VerificationResults = runner.verifyConfiguration("Kafka Topics", topic1Config); + topic1VerificationResults.assertNoFailures(); + + // Perform verification against a topic with invalid data for the selected data format + final Map importantTopicConfig = Map.of( + "Topic Names", "an-important-topic", + "Consumer Group ID", "nifi-kafka-to-s3-testSuccessfulFlow", + "Offset Reset", "earliest", + "Kafka Data Format", "JSON" + ); + + final ConnectorConfigVerificationResult importantTopicVerificationResults = runner.verifyConfiguration("Kafka Topics", importantTopicConfig); + final List invalidImportantTopicResults = importantTopicVerificationResults.getFailedResults(); + assertEquals(1, invalidImportantTopicResults.size()); + final ConfigVerificationResult invalidResult = invalidImportantTopicResults.getFirst(); + assertTrue(invalidResult.getExplanation().contains("parse"), "Unexpected validation reason: " + invalidResult.getExplanation()); + + runner.applyUpdate(); + } + + @Test + public void testJsonFlow() throws IOException, ExecutionException, InterruptedException, FlowUpdateException { + final String bucketName = "test-json-flow"; + createS3Bucket(bucketName); + createKafkaTopics("story"); + + produceRecordsToTopic("story", + """ + {"page": 1, "words": "Once upon a time, there was a NiFi developer." }""", + """ + {"page": 2, "words": "The developer wanted to build a connector to move data from Kafka to S3." }""", + """ + {"page": 3, "words": "After much effort, the connector was complete and worked flawlessly!" }""", + """ + {"page": 4, "words": "The end." }""" + ); + + final Map kafkaServerConfig = Map.of( + "Kafka Brokers", "localhost:9093", + "Security Protocol", "SASL_PLAINTEXT", + "SASL Mechanism", "PLAIN", + "Username", SCRAM_USERNAME + ); + runner.addSecret("kafka-password", SCRAM_PASSWORD); + final Map kafkaSecretRefs = Map.of( + "Password", runner.createSecretReference("kafka-password") + ); + + final Map kafkaTopicConfig = Map.of( + "Topic Names", "story", + "Consumer Group ID", "nifi-kafka-to-s3-testSuccessfulFlow", + "Offset Reset", "earliest", + "Kafka Data Format", "JSON" + ); + + final Map s3Config = Map.ofEntries( + Map.entry("S3 Region", S3_REGION), + Map.entry("S3 Data Format", "Avro"), + Map.entry("S3 Bucket", bucketName), + Map.entry("S3 Endpoint Override URL", localStackContainer.getEndpoint().toString()), + Map.entry("S3 Authentication Strategy", "Access Key ID and Secret Key"), + Map.entry("S3 Access Key ID", localStackContainer.getAccessKey()), + Map.entry("Target Object Size", "1 MB"), + Map.entry("Merge Latency", "1 sec") + ); + runner.addSecret("s3-secret-key", localStackContainer.getSecretKey()); + final Map s3SecretRefs = Map.of( + "S3 Secret Access Key", runner.createSecretReference("s3-secret-key") + ); + + runner.configure("Kafka Connection", kafkaServerConfig, kafkaSecretRefs); + runner.configure("Kafka Topics", kafkaTopicConfig); + runner.configure("S3 Configuration", s3Config, s3SecretRefs); + runner.applyUpdate(); + + final List validationResults = runner.validate(); + assertEquals(Collections.emptyList(), validationResults); + + runner.startConnector(); + try { + runner.waitForDataIngested(Duration.ofSeconds(10)); + runner.waitForIdle(Duration.ofSeconds(30)); + } finally { + runner.stopConnector(); + } + + verifyS3ObjectsCreated(bucketName); + } + + private void verifyS3ObjectsCreated(final String bucketName) throws IOException { + final ListObjectsV2Response listResponse = s3Client.listObjectsV2(ListObjectsV2Request.builder().bucket(bucketName).build()); + final List objects = listResponse.contents(); + + assertFalse(objects.isEmpty(), "Expected at least one object in S3 bucket"); + + for (final S3Object s3Object : objects) { + final GetObjectRequest getObjectRequest = GetObjectRequest.builder().bucket(bucketName).key(s3Object.key()).build(); + try (final ResponseInputStream objectContent = s3Client.getObject(getObjectRequest)) { + final long objectSize = objectContent.response().contentLength(); + assertTrue(objectSize > 0, "Expected S3 object " + s3Object.key() + " to have content"); + } + } + } + + @Test + public void testSchemaRegistryVerification() throws ExecutionException, InterruptedException, FlowUpdateException { + createKafkaTopics("avro-topic"); + + final String schemaString = """ + { + "type": "record", + "name": "TestRecord", + "namespace": "org.apache.nifi.test", + "fields": [ + {"name": "id", "type": "int"}, + {"name": "message", "type": "string"} + ] + }"""; + + final Schema schema = new Schema.Parser().parse(schemaString); + + final GenericRecord record1 = new GenericData.Record(schema); + record1.put("id", 100); + record1.put("message", "Test message 1"); + + final GenericRecord record2 = new GenericData.Record(schema); + record2.put("id", 200); + record2.put("message", "Test message 2"); + + produceAvroRecordsToTopic("avro-topic", schema, record1, record2); + + final Map kafkaConnectionConfig = Map.of( + "Kafka Brokers", "localhost:9093", + "Security Protocol", "SASL_PLAINTEXT", + "SASL Mechanism", "PLAIN", + "Username", SCRAM_USERNAME, + "Schema Registry URL", getSchemaRegistryUrl() + ); + runner.addSecret("kafka-password", SCRAM_PASSWORD); + final Map kafkaSecretRefs = Map.of( + "Password", runner.createSecretReference("kafka-password") + ); + + final ConnectorConfigVerificationResult connectionVerificationResults = runner.verifyConfiguration("Kafka Connection", kafkaConnectionConfig, kafkaSecretRefs); + connectionVerificationResults.assertNoFailures(); + + runner.configure("Kafka Connection", kafkaConnectionConfig, kafkaSecretRefs); + + final Map avroTopicConfig = Map.of( + "Topic Names", "avro-topic", + "Consumer Group ID", "nifi-kafka-to-s3-testSchemaRegistryVerification", + "Offset Reset", "earliest", + "Kafka Data Format", "Avro" + ); + + final ConnectorConfigVerificationResult avroTopicVerificationResults = runner.verifyConfiguration("Kafka Topics", avroTopicConfig); + avroTopicVerificationResults.assertNoFailures(); + + runner.applyUpdate(); + } + + @Test + public void testWithSchemaRegistry() throws IOException, ExecutionException, InterruptedException, FlowUpdateException { + final String bucketName = "test-schema-registry"; + createS3Bucket(bucketName); + createKafkaTopics("user-events"); + + final String schemaString = """ + { + "type": "record", + "name": "UserEvent", + "namespace": "org.apache.nifi.test", + "fields": [ + {"name": "userId", "type": "int"}, + {"name": "userName", "type": "string"}, + {"name": "eventType", "type": "string"}, + {"name": "timestamp", "type": "long"} + ] + }"""; + + final Schema schema = new Schema.Parser().parse(schemaString); + + final GenericRecord record1 = new GenericData.Record(schema); + record1.put("userId", 1001); + record1.put("userName", "alice"); + record1.put("eventType", "login"); + record1.put("timestamp", System.currentTimeMillis()); + + final GenericRecord record2 = new GenericData.Record(schema); + record2.put("userId", 1002); + record2.put("userName", "bob"); + record2.put("eventType", "purchase"); + record2.put("timestamp", System.currentTimeMillis()); + + final GenericRecord record3 = new GenericData.Record(schema); + record3.put("userId", 1003); + record3.put("userName", "charlie"); + record3.put("eventType", "logout"); + record3.put("timestamp", System.currentTimeMillis()); + + produceAvroRecordsToTopic("user-events", schema, record1, record2, record3); + + final Map kafkaConnectionConfig = Map.of( + "Kafka Brokers", "localhost:9093", + "Security Protocol", "SASL_PLAINTEXT", + "SASL Mechanism", "PLAIN", + "Username", SCRAM_USERNAME, + "Schema Registry URL", getSchemaRegistryUrl() + ); + runner.addSecret("kafka-password", SCRAM_PASSWORD); + final Map kafkaSecretRefs = Map.of( + "Password", runner.createSecretReference("kafka-password") + ); + + final Map kafkaTopicConfig = Map.of( + "Topic Names", "user-events", + "Consumer Group ID", "nifi-kafka-to-s3-testSchemaRegistry", + "Offset Reset", "earliest", + "Kafka Data Format", "Avro" + ); + + final Map s3Config = Map.ofEntries( + Map.entry("S3 Region", S3_REGION), + Map.entry("S3 Data Format", "Avro"), + Map.entry("S3 Bucket", bucketName), + Map.entry("S3 Endpoint Override URL", localStackContainer.getEndpoint().toString()), + Map.entry("S3 Authentication Strategy", "Access Key ID and Secret Key"), + Map.entry("S3 Access Key ID", localStackContainer.getAccessKey()), + Map.entry("Target Object Size", "1 MB"), + Map.entry("Merge Latency", "1 sec") + ); + runner.addSecret("s3-secret-key", localStackContainer.getSecretKey()); + final Map s3SecretRefs = Map.of( + "S3 Secret Access Key", runner.createSecretReference("s3-secret-key") + ); + + runner.configure("Kafka Connection", kafkaConnectionConfig, kafkaSecretRefs); + runner.configure("Kafka Topics", kafkaTopicConfig); + runner.configure("S3 Configuration", s3Config, s3SecretRefs); + runner.applyUpdate(); + + final List validationResults = runner.validate(); + assertEquals(Collections.emptyList(), validationResults); + + runner.startConnector(); + try { + runner.waitForDataIngested(Duration.ofSeconds(10)); + runner.waitForIdle(Duration.ofSeconds(30)); + } finally { + runner.stopConnector(); + } + + verifyS3ObjectsCreated(bucketName); + } + + @Test + public void testSwitchFromJsonToAvroS3Format() throws IOException, ExecutionException, InterruptedException, FlowUpdateException { + final String bucketName = "test-switch-formats"; + createS3Bucket(bucketName); + + // Add data to json and avro topics + createKafkaTopics("json", "avro"); + + produceRecordsToTopic("json", + """ + {"id": 1, "type": "json", "data": "First JSON record"}""", + """ + {"id": 2, "type": "json", "data": "Second JSON record"}""", + """ + {"id": 3, "type": "json", "data": "Third JSON record"}""" + ); + + final String schemaString = """ + { + "type": "record", + "name": "AvroRecord", + "namespace": "org.apache.nifi.test", + "fields": [ + {"name": "id", "type": "int"}, + {"name": "type", "type": "string"}, + {"name": "data", "type": "string"} + ] + }"""; + + final Schema schema = new Schema.Parser().parse(schemaString); + + final GenericRecord avroRecord1 = new GenericData.Record(schema); + avroRecord1.put("id", 10); + avroRecord1.put("type", "avro"); + avroRecord1.put("data", "First Avro record"); + + final GenericRecord avroRecord2 = new GenericData.Record(schema); + avroRecord2.put("id", 20); + avroRecord2.put("type", "avro"); + avroRecord2.put("data", "Second Avro record"); + + final GenericRecord avroRecord3 = new GenericData.Record(schema); + avroRecord3.put("id", 30); + avroRecord3.put("type", "avro"); + avroRecord3.put("data", "Third Avro record"); + + produceAvroRecordsToTopic("avro", schema, avroRecord1, avroRecord2, avroRecord3); + + // Configure Connector to consume from JSON Kafka topic and write to S3 in JSON format, but with an invalid S3 endpoint. + // This will cause the data to remain queued, since PutS3Object will fail to write the data. + final Map kafkaServerConfig = Map.of( + "Kafka Brokers", "localhost:9093", + "Security Protocol", "SASL_PLAINTEXT", + "SASL Mechanism", "PLAIN", + "Username", SCRAM_USERNAME + ); + runner.addSecret("kafka-password", SCRAM_PASSWORD); + final Map kafkaSecretRefs = Map.of( + "Password", runner.createSecretReference("kafka-password") + ); + + final Map jsonTopicConfig = Map.of( + "Topic Names", "json", + "Consumer Group ID", "nifi-kafka-to-s3-testReconfiguration", + "Offset Reset", "earliest", + "Kafka Data Format", "JSON" + ); + + final Map s3InvalidConfig = Map.ofEntries( + Map.entry("S3 Region", S3_REGION), + Map.entry("S3 Data Format", "JSON"), + Map.entry("S3 Bucket", bucketName), + Map.entry("S3 Endpoint Override URL", "http://invalid-s3-endpoint:9999"), + Map.entry("S3 Authentication Strategy", "Access Key ID and Secret Key"), + Map.entry("S3 Access Key ID", localStackContainer.getAccessKey()), + Map.entry("Target Object Size", "1 MB"), + Map.entry("Merge Latency", "1 sec") + ); + runner.addSecret("s3-secret-key", localStackContainer.getSecretKey()); + final Map s3SecretRefs = Map.of( + "S3 Secret Access Key", runner.createSecretReference("s3-secret-key") + ); + + runner.configure("Kafka Connection", kafkaServerConfig, kafkaSecretRefs); + runner.configure("Kafka Topics", jsonTopicConfig); + runner.configure("S3 Configuration", s3InvalidConfig, s3SecretRefs); + runner.applyUpdate(); + + // Run the Connector with the invalid S3 endpoint to queue the JSON data. Wait for data to be queued up. + runner.startConnector(); + try { + runner.waitForDataIngested(Duration.ofSeconds(30)); + } finally { + runner.stopConnector(); + } + + // Apply configuration to specify the correct S3 endpoint. Keep S3 Data Format as JSON for now. + // This will allow the Connector to write data to S3 and properly drain the data when we switch the S3 format from JSON to Avro. + final Map s3ValidJsonConfig = Map.ofEntries( + Map.entry("S3 Region", S3_REGION), + Map.entry("S3 Data Format", "JSON"), + Map.entry("S3 Bucket", bucketName), + Map.entry("S3 Endpoint Override URL", localStackContainer.getEndpoint().toString()), + Map.entry("S3 Authentication Strategy", "Access Key ID and Secret Key"), + Map.entry("S3 Access Key ID", localStackContainer.getAccessKey()), + Map.entry("Target Object Size", "1 MB"), + Map.entry("Merge Latency", "1 sec") + ); + + runner.configure("S3 Configuration", s3ValidJsonConfig, s3SecretRefs); + runner.applyUpdate(); + + // Make sure there is no data in S3 yet. + final ListObjectsV2Response initialListingResponse = s3Client.listObjectsV2(ListObjectsV2Request.builder().bucket(bucketName).build()); + final List initialS3Objects = initialListingResponse.contents(); + assertEquals(List.of(), initialS3Objects); + + // Now change the S3 Data Format from JSON to Avro. This should trigger draining of the queued JSON data. + final Map s3ValidAvroConfig = Map.ofEntries( + Map.entry("S3 Region", S3_REGION), + Map.entry("S3 Data Format", "Avro"), + Map.entry("S3 Bucket", bucketName), + Map.entry("S3 Endpoint Override URL", localStackContainer.getEndpoint().toString()), + Map.entry("S3 Authentication Strategy", "Access Key ID and Secret Key"), + Map.entry("S3 Access Key ID", localStackContainer.getAccessKey()), + Map.entry("Target Object Size", "1 MB"), + Map.entry("Merge Latency", "1 sec") + ); + + // Configure to consume from the Avro topic and change S3 format to Avro + final Map avroTopicConfig = Map.of( + "Topic Names", "avro", + "Consumer Group ID", "nifi-kafka-to-s3-testReconfiguration", + "Offset Reset", "earliest", + "Kafka Data Format", "Avro" + ); + + final Map kafkaConnectionWithSchemaRegistry = Map.of( + "Kafka Brokers", "localhost:9093", + "Security Protocol", "SASL_PLAINTEXT", + "SASL Mechanism", "PLAIN", + "Username", SCRAM_USERNAME, + "Schema Registry URL", getSchemaRegistryUrl() + ); + + runner.configure("Kafka Connection", kafkaConnectionWithSchemaRegistry, kafkaSecretRefs); + runner.configure("Kafka Topics", avroTopicConfig); + runner.configure("S3 Configuration", s3ValidAvroConfig, s3SecretRefs); + runner.applyUpdate(); + + // After draining, there should be one JSON file in S3. + final ListObjectsV2Response jsonOnlyListing = s3Client.listObjectsV2(ListObjectsV2Request.builder().bucket(bucketName).build()); + final List jsonObjects = jsonOnlyListing.contents(); + assertEquals(1, jsonObjects.size()); + + // Start connector, wait for more data to be ingested, and then wait for the Connector to be idle. + runner.startConnector(); + try { + runner.waitForDataIngested(Duration.ofSeconds(30)); + runner.waitForIdle(Duration.ofSeconds(30)); + } finally { + runner.stopConnector(); + } + + // Verify that there are two objects in S3: one in JSON format and one in Avro format. + final ListObjectsV2Response listResponse = s3Client.listObjectsV2(ListObjectsV2Request.builder().bucket(bucketName).build()); + final List objects = listResponse.contents(); + + assertEquals(2, objects.size(), "Expected exactly 2 objects in S3 bucket: one for JSON data and one for Avro data"); + + for (final S3Object s3Object : objects) { + final GetObjectRequest getObjectRequest = GetObjectRequest.builder().bucket(bucketName).key(s3Object.key()).build(); + try (final ResponseInputStream objectContent = s3Client.getObject(getObjectRequest)) { + final long objectSize = objectContent.response().contentLength(); + assertTrue(objectSize > 0, "Expected S3 object " + s3Object.key() + " to have content"); + } + } + } + +} diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-nar/pom.xml b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-nar/pom.xml new file mode 100644 index 000000000000..b167c8459532 --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-nar/pom.xml @@ -0,0 +1,36 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-kafka-to-s3-bundle + 2.9.0-SNAPSHOT + + + nifi-kafka-to-s3-nar + nar + + + + org.apache.nifi + nifi-kafka-to-s3-connector + 2.9.0-SNAPSHOT + + + \ No newline at end of file diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/pom.xml b/nifi-connectors/nifi-kafka-to-s3-bundle/pom.xml new file mode 100644 index 000000000000..95960d7a0ea4 --- /dev/null +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/pom.xml @@ -0,0 +1,39 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-connectors + 2.9.0-SNAPSHOT + + + nifi-kafka-to-s3-bundle + pom + + + nifi-kafka-to-s3-connector + nifi-kafka-to-s3-nar + nifi-kafka-to-s3-integration-tests + + + + + + + \ No newline at end of file diff --git a/nifi-connectors/pom.xml b/nifi-connectors/pom.xml new file mode 100644 index 000000000000..8e1b10a447d5 --- /dev/null +++ b/nifi-connectors/pom.xml @@ -0,0 +1,42 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi + 2.9.0-SNAPSHOT + + + nifi-connectors + pom + + + nifi-kafka-to-s3-bundle + + + + + + org.apache.nifi + nifi-connector-utils + 2.9.0-SNAPSHOT + + + + diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc index afa152201513..4c4e89b0e3b0 100644 --- a/nifi-docs/src/main/asciidoc/administration-guide.adoc +++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc @@ -1510,6 +1510,11 @@ Global access policies govern the following system level authorizations: |Parameter Contexts |`/parameter-contexts` +|access connectors +|Allows users to view/modify Connectors +|N/A +|`/connectors` + |query provenance |Allows users to submit a Provenance Search and request Event Lineage |Data Provenance diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java index 6ef907fdec22..b4e9beac7001 100644 --- a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java +++ b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-abstract-processors/src/main/java/org/apache/nifi/processors/aws/s3/AbstractS3Processor.java @@ -17,7 +17,9 @@ package org.apache.nifi.processors.aws.s3; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.connector.components.ConnectorMethod; import org.apache.nifi.context.PropertyContext; import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.flowfile.FlowFile; @@ -26,6 +28,7 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processors.aws.AbstractAwsSyncProcessor; +import org.apache.nifi.processors.aws.region.RegionUtil; import software.amazon.awssdk.awscore.defaultsmode.DefaultsMode; import software.amazon.awssdk.awscore.exception.AwsErrorDetails; import software.amazon.awssdk.awscore.exception.AwsServiceException; @@ -37,6 +40,7 @@ import software.amazon.awssdk.services.s3.model.ServerSideEncryption; import java.util.Arrays; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; @@ -342,4 +346,11 @@ protected void setEncryptionAttributes(final Map attributes, fin attributes.put(S3_ENCRYPTION_STRATEGY, encryptionService.getStrategyName()); } } + + @ConnectorMethod(name = "getAvailableRegions", description = "Returns the list of available AWS regions") + public List getAvailableRegions() { + return RegionUtil.getAwsRegionAllowableValues().stream() + .map(AllowableValue::getValue) + .toList(); + } } diff --git a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-regions/src/main/java/org/apache/nifi/processors/aws/region/RegionUtil.java b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-regions/src/main/java/org/apache/nifi/processors/aws/region/RegionUtil.java index b5e35d8b3e3d..2626c41c8a10 100644 --- a/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-regions/src/main/java/org/apache/nifi/processors/aws/region/RegionUtil.java +++ b/nifi-extension-bundles/nifi-aws-bundle/nifi-aws-regions/src/main/java/org/apache/nifi/processors/aws/region/RegionUtil.java @@ -100,7 +100,7 @@ private static AllowableValue[] getRegionAllowableValues() { * * @return All available AWS regions as AllowableValues. */ - private static List getAwsRegionAllowableValues() { + public static List getAwsRegionAllowableValues() { final List values = new ArrayList<>(); for (final Region region : Region.regions()) { values.add(createAwsRegionAllowableValue(region)); diff --git a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/kafka/processors/ConsumeKafka.java b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/kafka/processors/ConsumeKafka.java index b92bd862a36a..bd79089542c9 100644 --- a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/kafka/processors/ConsumeKafka.java +++ b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/kafka/processors/ConsumeKafka.java @@ -26,9 +26,11 @@ import org.apache.nifi.annotation.lifecycle.OnStopped; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.ConfigVerificationResult.Outcome; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; import org.apache.nifi.components.Validator; +import org.apache.nifi.components.connector.components.ConnectorMethod; import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.kafka.processors.common.KafkaUtils; import org.apache.nifi.kafka.processors.consumer.OffsetTracker; @@ -58,11 +60,14 @@ import org.apache.nifi.processor.VerifiableProcessor; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.serialization.RecordReader; import org.apache.nifi.serialization.RecordReaderFactory; import org.apache.nifi.serialization.RecordSetWriterFactory; import org.apache.nifi.util.StringUtils; +import java.io.ByteArrayInputStream; import java.io.IOException; +import java.io.InputStream; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.time.Duration; @@ -562,26 +567,110 @@ public List verify(final ProcessContext context, final final List verificationResults = new ArrayList<>(); final KafkaConnectionService connectionService = context.getProperty(CONNECTION_SERVICE).asControllerService(KafkaConnectionService.class); - final PollingContext pollingContext = createPollingContext(context); - final KafkaConsumerService consumerService = connectionService.getConsumerService(pollingContext); + final PollingContext pollingContext = createPollingContext(context, null, AutoOffsetReset.EARLIEST); + try (final KafkaConsumerService consumerService = connectionService.getConsumerService(pollingContext)) { + final ConfigVerificationResult partitionVerification = verifyPartitions(consumerService, pollingContext); + verificationResults.add(partitionVerification); + + final ConfigVerificationResult parsingResult = verifyCanParse(context, consumerService, verificationLogger); + verificationResults.add(parsingResult); + } catch (final IOException e) { + verificationResults.add(new ConfigVerificationResult.Builder() + .verificationStepName("Communicate with Kafka Broker") + .outcome(Outcome.FAILED) + .explanation("There was an I/O failure when communicating with Kafka: " + e) + .build()); + } + + return verificationResults; + } - final ConfigVerificationResult.Builder verificationPartitions = new ConfigVerificationResult.Builder() - .verificationStepName("Verify Topic Partitions"); + private ConfigVerificationResult verifyPartitions(final KafkaConsumerService consumerService, final PollingContext pollingContext) { + final ConfigVerificationResult.Builder partitionVerification = new ConfigVerificationResult.Builder() + .verificationStepName("Verify Topic Partitions"); try { final List partitionStates = consumerService.getPartitionStates(); - verificationPartitions - .outcome(ConfigVerificationResult.Outcome.SUCCESSFUL) - .explanation(String.format("Partitions [%d] found for Topics %s", partitionStates.size(), pollingContext.getTopics())); + partitionVerification + .outcome(ConfigVerificationResult.Outcome.SUCCESSFUL) + .explanation(String.format("Found [%d] partitions for Topics %s", partitionStates.size(), pollingContext.getTopics())); } catch (final Exception e) { getLogger().error("Topics {} Partition verification failed", pollingContext.getTopics(), e); - verificationPartitions - .outcome(ConfigVerificationResult.Outcome.FAILED) - .explanation(String.format("Topics %s Partition access failed: %s", pollingContext.getTopics(), e)); + partitionVerification + .outcome(ConfigVerificationResult.Outcome.FAILED) + .explanation(String.format("Topics %s Partition access failed: %s", pollingContext.getTopics(), e)); } - verificationResults.add(verificationPartitions.build()); - return verificationResults; + return partitionVerification.build(); + } + + private ConfigVerificationResult verifyCanParse(final ProcessContext context, final KafkaConsumerService consumerService, final ComponentLog verificationLogger) { + final Iterable records = consumerService.poll(Duration.ofSeconds(60)); + final ProcessingStrategy processingStrategy = context.getProperty(PROCESSING_STRATEGY).asAllowableValue(ProcessingStrategy.class); + if (processingStrategy != ProcessingStrategy.RECORD) { + return new ConfigVerificationResult.Builder() + .verificationStepName("Parse Records") + .outcome(Outcome.SKIPPED) + .explanation("Processing Strategy is set to " + processingStrategy.getValue() + " so skipping record parsing verification") + .build(); + } + + final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class); + int recordIndex = 0; + for (final ByteRecord byteRecord : records) { + recordIndex++; + final Map recordAttributes = KafkaUtils.toAttributes( + byteRecord, keyEncoding, headerNamePattern, headerEncoding, commitOffsets); + + try (final InputStream inputStream = new ByteArrayInputStream(byteRecord.getValue()); + final RecordReader reader = readerFactory.createRecordReader(recordAttributes, inputStream, byteRecord.getValue().length, verificationLogger)) { + + while (reader.nextRecord() != null) { + } + } catch (final Exception e) { + return new ConfigVerificationResult.Builder() + .verificationStepName("Parse Records") + .outcome(Outcome.FAILED) + .explanation("Failed to parse Record number " + recordIndex + ": " + e) + .build(); + } + } + + if (recordIndex == 0) { + return new ConfigVerificationResult.Builder() + .verificationStepName("Parse Records") + .outcome(Outcome.SKIPPED) + .explanation("No records were received to parse") + .build(); + } + + return new ConfigVerificationResult.Builder() + .verificationStepName("Parse Records") + .outcome(Outcome.SUCCESSFUL) + .explanation("Successfully parsed " + recordIndex + " records") + .build(); + } + + + @ConnectorMethod( + name = "sampleTopics", + description = "Returns a list of sample data from the topics that would be consumed by this processor." + ) + public List sampleTopics(final ProcessContext context) throws IOException { + final KafkaConnectionService connectionService = context.getProperty(CONNECTION_SERVICE).asControllerService(KafkaConnectionService.class); + final PollingContext pollingContext = createPollingContext(context, "nifi-validation-" + System.currentTimeMillis(), AutoOffsetReset.EARLIEST); + try (final KafkaConsumerService consumerService = connectionService.getConsumerService(pollingContext)) { + final Iterable records = consumerService.poll(Duration.ofSeconds(60)); + final List samples = new ArrayList<>(); + for (final ByteRecord record : records) { + samples.add(record.getValue()); + if (samples.size() >= 10) { + break; + } + } + + return samples; + } } private KafkaConsumerService getConsumerService(final ProcessContext context) { @@ -670,6 +759,10 @@ private PollingContext createPollingContext(final ProcessContext context) { final String groupId = context.getProperty(GROUP_ID).getValue(); final String offsetReset = context.getProperty(AUTO_OFFSET_RESET).getValue(); final AutoOffsetReset autoOffsetReset = AutoOffsetReset.valueOf(offsetReset.toUpperCase()); + return createPollingContext(context, groupId, autoOffsetReset); + } + + private PollingContext createPollingContext(final ProcessContext context, final String groupId, final AutoOffsetReset autoOffsetReset) { final String topics = context.getProperty(TOPICS).evaluateAttributeExpressions().getValue(); final String topicFormat = context.getProperty(TOPIC_FORMAT).getValue(); @@ -686,4 +779,5 @@ private PollingContext createPollingContext(final ProcessContext context) { return pollingContext; } + } diff --git a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/kafka/processors/ConsumeKafkaTest.java b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/kafka/processors/ConsumeKafkaTest.java index 531e5fff0744..14c50d45babe 100644 --- a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/kafka/processors/ConsumeKafkaTest.java +++ b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/test/java/org/apache/nifi/kafka/processors/ConsumeKafkaTest.java @@ -17,6 +17,7 @@ package org.apache.nifi.kafka.processors; import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.ConfigVerificationResult.Outcome; import org.apache.nifi.kafka.service.api.KafkaConnectionService; import org.apache.nifi.kafka.service.api.common.PartitionState; import org.apache.nifi.kafka.service.api.consumer.KafkaConsumerService; @@ -36,6 +37,7 @@ import static org.apache.nifi.kafka.processors.ConsumeKafka.GROUP_ID; import static org.apache.nifi.kafka.processors.ConsumeKafka.TOPICS; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.when; @@ -96,10 +98,15 @@ public void testVerifySuccessful() throws InitializationException { runner.setProperty(GROUP_ID, CONSUMER_GROUP_ID); final List results = processor.verify(runner.getProcessContext(), runner.getLogger(), Collections.emptyMap()); - assertEquals(1, results.size()); + final List successResults = results.stream() + .filter(result -> result.getOutcome() == ConfigVerificationResult.Outcome.SUCCESSFUL) + .toList(); + assertEquals(1, successResults.size()); - final ConfigVerificationResult firstResult = results.iterator().next(); - assertEquals(ConfigVerificationResult.Outcome.SUCCESSFUL, firstResult.getOutcome()); + final boolean anyFailures = results.stream().anyMatch(result -> result.getOutcome() == Outcome.FAILED); + assertFalse(anyFailures, "At least one verification result was a failure: " + results); + + final ConfigVerificationResult firstResult = successResults.getFirst(); assertNotNull(firstResult.getExplanation()); } @@ -113,9 +120,14 @@ public void testVerifyFailed() throws InitializationException { runner.setProperty(GROUP_ID, CONSUMER_GROUP_ID); final List results = processor.verify(runner.getProcessContext(), runner.getLogger(), Collections.emptyMap()); - assertEquals(1, results.size()); + assertEquals(2, results.size()); + + final List failedResults = results.stream() + .filter(result -> result.getOutcome() == ConfigVerificationResult.Outcome.FAILED) + .toList(); + assertEquals(1, failedResults.size()); - final ConfigVerificationResult firstResult = results.iterator().next(); + final ConfigVerificationResult firstResult = failedResults.getFirst(); assertEquals(ConfigVerificationResult.Outcome.FAILED, firstResult.getOutcome()); assertNotNull(firstResult.getExplanation()); } diff --git a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-api/src/main/java/org/apache/nifi/kafka/service/api/consumer/PollingContext.java b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-api/src/main/java/org/apache/nifi/kafka/service/api/consumer/PollingContext.java index 27b07b41e3bc..171ff02fdf9b 100644 --- a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-api/src/main/java/org/apache/nifi/kafka/service/api/consumer/PollingContext.java +++ b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-api/src/main/java/org/apache/nifi/kafka/service/api/consumer/PollingContext.java @@ -34,7 +34,7 @@ public class PollingContext { public PollingContext(final String groupId, final Collection topics, final AutoOffsetReset autoOffsetReset) { - this.groupId = Objects.requireNonNull(groupId, "Group ID required"); + this.groupId = groupId; this.topics = Collections.unmodifiableCollection(Objects.requireNonNull(topics, "Topics required")); this.topicPattern = null; this.autoOffsetReset = Objects.requireNonNull(autoOffsetReset, "Auto Offset Reset required"); @@ -42,7 +42,7 @@ public PollingContext(final String groupId, final Collection topics, public PollingContext(final String groupId, final Pattern topicPattern, final AutoOffsetReset autoOffsetReset) { - this.groupId = Objects.requireNonNull(groupId, "Group ID required"); + this.groupId = groupId; this.topics = Collections.emptyList(); this.topicPattern = Objects.requireNonNull(topicPattern, "Topic Patten required"); this.autoOffsetReset = Objects.requireNonNull(autoOffsetReset, "Auto Offset Reset required"); diff --git a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-shared/src/main/java/org/apache/nifi/kafka/service/Kafka3ConnectionService.java b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-shared/src/main/java/org/apache/nifi/kafka/service/Kafka3ConnectionService.java index a03c0ecac359..c7cc4782ff00 100644 --- a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-shared/src/main/java/org/apache/nifi/kafka/service/Kafka3ConnectionService.java +++ b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-shared/src/main/java/org/apache/nifi/kafka/service/Kafka3ConnectionService.java @@ -16,7 +16,9 @@ */ package org.apache.nifi.kafka.service; +import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.ListTopicsResult; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -30,6 +32,8 @@ import org.apache.nifi.components.ConfigVerificationResult; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.connector.components.ConnectorMethod; +import org.apache.nifi.components.connector.components.MethodArgument; import org.apache.nifi.context.PropertyContext; import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.controller.ConfigurationContext; @@ -42,6 +46,7 @@ import org.apache.nifi.kafka.service.api.consumer.PollingContext; import org.apache.nifi.kafka.service.api.producer.KafkaProducerService; import org.apache.nifi.kafka.service.api.producer.ProducerConfiguration; +import org.apache.nifi.kafka.service.consumer.Kafka3AssignmentService; import org.apache.nifi.kafka.service.consumer.Kafka3ConsumerService; import org.apache.nifi.kafka.service.consumer.Subscription; import org.apache.nifi.kafka.service.producer.Kafka3ProducerService; @@ -59,11 +64,14 @@ import org.apache.nifi.ssl.SSLContextService; import java.time.Duration; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; @@ -235,18 +243,25 @@ protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String public KafkaConsumerService getConsumerService(final PollingContext pollingContext) { Objects.requireNonNull(pollingContext, "Polling Context required"); - final Subscription subscription = createSubscription(pollingContext); + final String groupId = pollingContext.getGroupId(); final Properties properties = new Properties(); properties.putAll(consumerProperties); - properties.put(ConsumerConfig.GROUP_ID_CONFIG, subscription.getGroupId()); - properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, subscription.getAutoOffsetReset().getValue()); + if (groupId != null) { + properties.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + } + properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, pollingContext.getAutoOffsetReset().getValue()); properties.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); final ByteArrayDeserializer deserializer = new ByteArrayDeserializer(); final Consumer consumer = new KafkaConsumer<>(properties, deserializer, deserializer); - return new Kafka3ConsumerService(getLogger(), consumer, subscription); + if (groupId == null) { + return new Kafka3AssignmentService(consumer, pollingContext.getTopics()); + } else { + final Subscription subscription = createSubscription(pollingContext); + return new Kafka3ConsumerService(getLogger(), consumer, subscription); + } } private Subscription createSubscription(final PollingContext pollingContext) { @@ -291,6 +306,26 @@ public String getBrokerUri() { return uri; } + @ConnectorMethod( + name = "listTopicNames", + description = "Returns a list of topic names available in the Kafka cluster", + arguments = { + @MethodArgument(name = "context", type = ConfigurationContext.class, description = "The configuration context that specifies connectivity details") + }) + public List listTopicNames(final ConfigurationContext context) throws ExecutionException, InterruptedException { + final Properties clientProperties = getClientProperties(context); + final Properties consumerProperties = getConsumerProperties(context, clientProperties); + + try (final Admin admin = Admin.create(consumerProperties)) { + final ListTopicsResult result = admin.listTopics(); + final Set topicNames = result.names().get(); + final List sortedTopicNames = new ArrayList<>(topicNames); + sortedTopicNames.sort(String.CASE_INSENSITIVE_ORDER); + return sortedTopicNames; + } + } + + @Override public List verify(final ConfigurationContext configurationContext, final ComponentLog verificationLogger, final Map variables) { // Build Client Properties based on configured values and defaults from Consumer Properties diff --git a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-shared/src/main/java/org/apache/nifi/kafka/service/consumer/Kafka3AssignmentService.java b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-shared/src/main/java/org/apache/nifi/kafka/service/consumer/Kafka3AssignmentService.java new file mode 100644 index 000000000000..1fa421d3a2a2 --- /dev/null +++ b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-shared/src/main/java/org/apache/nifi/kafka/service/consumer/Kafka3AssignmentService.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.kafka.service.consumer; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.nifi.kafka.service.api.common.PartitionState; +import org.apache.nifi.kafka.service.api.common.TopicPartitionSummary; +import org.apache.nifi.kafka.service.api.consumer.KafkaConsumerService; +import org.apache.nifi.kafka.service.api.consumer.PollingSummary; +import org.apache.nifi.kafka.service.api.record.ByteRecord; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.OptionalLong; +import java.util.Set; + +public class Kafka3AssignmentService implements KafkaConsumerService { + private final Consumer consumer; + private volatile boolean closed = false; + + public Kafka3AssignmentService(final Consumer consumer, final Collection topicNames) { + this.consumer = consumer; + + final List topicPartitions = new ArrayList<>(); + for (final String topicName : topicNames) { + final List partitionInfos = consumer.partitionsFor(topicName); + partitionInfos.forEach(info -> topicPartitions.add(new TopicPartition(info.topic(), info.partition()))); + } + + consumer.assign(topicPartitions); + } + + @Override + public void commit(final PollingSummary pollingSummary) { + throw new UnsupportedOperationException("Commit not supported for assignment-based consumer"); + } + + @Override + public void rollback() { + // no-op + } + + @Override + public boolean isClosed() { + return false; + } + + @Override + public Iterable poll(final Duration maxWaitDuration) { + final ConsumerRecords consumerRecords = consumer.poll(maxWaitDuration); + if (consumerRecords.isEmpty()) { + return List.of(); + } + + return new RecordIterable(consumerRecords); + } + + @Override + public List getPartitionStates() { + final Set topicPartitions = consumer.assignment(); + return topicPartitions.stream() + .map(tp -> new PartitionState(tp.topic(), tp.partition())) + .toList(); + } + + @Override + public OptionalLong currentLag(final TopicPartitionSummary topicPartitionSummary) { + return OptionalLong.empty(); + } + + @Override + public void close() { + if (closed) { + return; + } + + closed = true; + consumer.close(); + } +} diff --git a/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-shared/src/main/java/org/apache/nifi/kafka/service/consumer/RecordIterable.java b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-shared/src/main/java/org/apache/nifi/kafka/service/consumer/RecordIterable.java new file mode 100644 index 000000000000..6f8229eb2f81 --- /dev/null +++ b/nifi-extension-bundles/nifi-kafka-bundle/nifi-kafka-service-shared/src/main/java/org/apache/nifi/kafka/service/consumer/RecordIterable.java @@ -0,0 +1,79 @@ +/* + * 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.kafka.service.consumer; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.nifi.kafka.service.api.header.RecordHeader; +import org.apache.nifi.kafka.service.api.record.ByteRecord; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +public class RecordIterable implements Iterable { + private final Iterator records; + + public RecordIterable(final Iterable> consumerRecords) { + this.records = new RecordIterator(consumerRecords); + } + + @Override + public Iterator iterator() { + return records; + } + + private static class RecordIterator implements Iterator { + private final Iterator> consumerRecords; + + private RecordIterator(final Iterable> records) { + this.consumerRecords = records.iterator(); + } + + @Override + public boolean hasNext() { + return consumerRecords.hasNext(); + } + + @Override + public ByteRecord next() { + final ConsumerRecord consumerRecord = consumerRecords.next(); + final List recordHeaders = new ArrayList<>(); + consumerRecord.headers().forEach(header -> { + final RecordHeader recordHeader = new RecordHeader(header.key(), header.value()); + recordHeaders.add(recordHeader); + }); + + // Support Kafka tombstones + byte[] value = consumerRecord.value(); + if (value == null) { + value = new byte[0]; + } + + return new ByteRecord( + consumerRecord.topic(), + consumerRecord.partition(), + consumerRecord.offset(), + consumerRecord.timestamp(), + recordHeaders, + consumerRecord.key(), + value, + 1 + ); + } + } +} diff --git a/nifi-extension-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/impl/ValidationContextAdapter.java b/nifi-extension-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/impl/ValidationContextAdapter.java index d78d01347820..8951412fc005 100644 --- a/nifi-extension-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/impl/ValidationContextAdapter.java +++ b/nifi-extension-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/impl/ValidationContextAdapter.java @@ -115,4 +115,9 @@ public Collection getReferencedParameters(final String propertyName) { public boolean isDependencySatisfied(final PropertyDescriptor propertyDescriptor, final Function propertyDescriptorLookup) { return innerValidationContext.isDependencySatisfied(propertyDescriptor, propertyDescriptorLookup); } + + @Override + public String evaluateParameters(final String value) { + return innerValidationContext.evaluateParameters(value); + } } diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DuplicateFlowFile.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DuplicateFlowFile.java index 3297dc621744..94f8b0ac1d46 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DuplicateFlowFile.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DuplicateFlowFile.java @@ -43,7 +43,7 @@ + "generated copies are sent to the 'success' relationship. In addition, each FlowFile gets an attribute 'copy.index' set to the copy number, where the original FlowFile gets " + "a value of zero, and all copies receive incremented integer values.") @WritesAttributes({ - @WritesAttribute(attribute = "copy.index", description = "A zero-based incrementing integer value based on which copy the FlowFile is.") + @WritesAttribute(attribute = "copy.index", description = "A zero-based incrementing integer value based on which copy the FlowFile is.") }) public class DuplicateFlowFile extends AbstractProcessor { diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/asset/AssetManager.java b/nifi-framework-api/src/main/java/org/apache/nifi/asset/AssetManager.java index d1501c2f9335..321d0afe7b9f 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/asset/AssetManager.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/asset/AssetManager.java @@ -31,13 +31,25 @@ public interface AssetManager { /** * Creates a new Asset with the given name and contents. - * @param parameterContextId the id of the parameter context + * @param ownerId the id of the resource that this asset belongs to * @param assetName the name of the asset * @param contents the contents of the asset * @return the created asset * @throws IOException if there is an error creating the asset */ - Asset createAsset(String parameterContextId, String assetName, InputStream contents) throws IOException; + Asset createAsset(String ownerId, String assetName, InputStream contents) throws IOException; + + /** + * Saves the given asset. If an asset already exists with the given identifier, the content will be replaced. + * + * @param ownerId the id of the resource that this asset belongs to + * @param assetId the identifier of the asset + * @param assetName the name of the asset + * @param contents the new contents of the asset + * @return the created or updated Asset + * @throws IOException if there is an error writing the asset + */ + Asset saveAsset(String ownerId, String assetId, String assetName, InputStream contents) throws IOException; /** * Retrieves the Asset with the given id, if it exists. @@ -47,22 +59,22 @@ public interface AssetManager { Optional getAsset(String id); /** - * Retrieves the Assets that belong to the given parameter context. - * @param parameterContextId the id of the parameter context - * @return the list of assets for the given context + * Retrieves the Assets that belong to the given owner. + * @param ownerId the id of the owner resource + * @return the list of assets for the given owner */ - List getAssets(String parameterContextId); + List getAssets(String ownerId); /** - * Creates an Asset with the given name and associates it with the given parameter context. If the asset already exists, it is returned. Otherwise, an asset is created + * Creates an Asset with the given name and associates it with the given owner. If the asset already exists, it is returned. Otherwise, an asset is created * but the underlying file is not created. This allows the asset to be referenced but any component that attempts to use the asset will still see a File that does not exist, which * will typically lead to an invalid component. * - * @param parameterContextId the id of the parameter context + * @param ownerId the id of the owner * @param assetName the name of the asset * @return the created asset */ - Asset createMissingAsset(String parameterContextId, String assetName); + Asset createMissingAsset(String ownerId, String assetName); /** * Deletes the Asset with the given id, if it exists. diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfigurationProvider.java b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfigurationProvider.java new file mode 100644 index 000000000000..4d2ff3ecc88c --- /dev/null +++ b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfigurationProvider.java @@ -0,0 +1,161 @@ +/* + * 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.components.connector; + +import java.io.InputStream; +import java.util.Optional; + +/** + * Extension point interface for external management of Connector working configuration and assets. + * Implementations allow a Connector's name, working flow configuration, and binary assets to be + * persisted in an external store and to be externally modified. + * + *

When a ConnectorConfigurationProvider is configured, the framework will: + *

    + *
  • On reads (e.g., getConnector): load configuration from the provider and override the in-memory working configuration
  • + *
  • On writes (e.g., configureConnector): save configuration to the provider before modifying the in-memory state
  • + *
  • On discard: notify the provider that the working configuration has been discarded
  • + *
  • On delete: notify the provider that the connector has been removed
  • + *
+ * + *

Asset management: the provider owns the mapping between NiFi asset UUIDs and external asset + * identifiers. The {@link #load} method must return {@link ConnectorWorkingConfiguration} with + * NiFi UUIDs in {@code assetIds} fields. The {@link #save} method receives NiFi UUIDs and the + * provider translates to external identifiers when persisting. The local state of assets + * (NiFi UUID mapping and digest tracking) is managed by the provider using the + * {@link org.apache.nifi.asset.AssetManager} provided via the initialization context.

+ */ +public interface ConnectorConfigurationProvider { + + /** + * Initializes the ConnectorConfigurationProvider with the given context. + * + * @param context the initialization context providing configuration properties + */ + void initialize(ConnectorConfigurationProviderInitializationContext context); + + /** + * Loads the externally managed working configuration for the connector with the given identifier. + * + * @param connectorId the identifier of the connector + * @return an Optional containing the working configuration if one exists in the external store, + * or an empty Optional if no external configuration exists (in which case the in-memory configuration is used) + */ + Optional load(String connectorId); + + /** + * Saves the working configuration for the connector with the given identifier to the external store. + * This is called when working configuration properties are modified, such as during configureConnector + * or when connector metadata (e.g., name) is updated. + * + * @param connectorId the identifier of the connector + * @param configuration the working configuration to save + */ + void save(String connectorId, ConnectorWorkingConfiguration configuration); + + /** + * Notifies the provider that the working configuration for the given connector has been discarded + * (i.e., reset to match the active configuration). This is semantically distinct from {@link #save} + * because the external store may need to handle this differently, such as deleting the working copy + * rather than overwriting it. + * + * @param connectorId the identifier of the connector whose working configuration was discarded + */ + void discard(String connectorId); + + /** + * Notifies the provider that the connector with the given identifier has been removed entirely. + * The provider should clean up any stored configuration for this connector. + * + * @param connectorId the identifier of the connector that was removed + */ + void delete(String connectorId); + + /** + * Verifies that the provider can support creating a connector with the given identifier. + * This is called before the connector is actually created, giving the provider an opportunity + * to reject the operation (for example, if it has reached a capacity limit or the connector + * already exists in the external store in an incompatible state). + * + *

If the provider cannot support the create operation, it should throw a + * {@link ConnectorConfigurationProviderException}.

+ * + * @param connectorId the identifier of the connector to be created + */ + void verifyCreate(String connectorId); + + /** + * Stores an asset to the local {@link org.apache.nifi.asset.AssetManager} and to the external + * store. The provider records the NiFi UUID to external identifier mapping in its local state. + * If the external store upload fails, the provider must roll back the local asset and throw. + * + * @param connectorId the identifier of the connector that owns the asset + * @param nifiUuid the NiFi-assigned UUID for this asset + * @param assetName the filename of the asset (e.g., "postgresql-42.6.0.jar") + * @param content the binary content of the asset + * @throws java.io.IOException if the asset cannot be stored + */ + void storeAsset(String connectorId, String nifiUuid, String assetName, InputStream content) throws java.io.IOException; + + /** + * Deletes an asset from the local {@link org.apache.nifi.asset.AssetManager} and from the + * external store. The provider uses the NiFi UUID to look up the external identifier from its + * local state, then cleans up both stores and removes the mapping entry. + * + * @param connectorId the identifier of the connector that owns the asset + * @param nifiUuid the NiFi-assigned UUID of the asset to delete + */ + void deleteAsset(String connectorId, String nifiUuid); + + /** + * Called when a connector update is requested (e.g., applying a committed configuration change) + * to determine whether the framework should proceed with the standard internal update process + * (stopping, re-configuring, and restarting the connector). + * + *

Returning {@code true} (the default) indicates the framework should proceed normally.

+ * + *

Returning {@code false} indicates the framework should skip the update and return + * immediately -- this is not a failure; the provider may have handled the update externally + * by doing some bookkeeping logic and the provider may re-trigger the update process by starting + * a new request to the nifi framework once it is ready to proceed. If the provider wants to fail + * the request, it should throw a runtime exception instead.

+ * + * @param connectorId the identifier of the connector to update + * @return {@code true} if the framework should proceed with the standard update process, + * {@code false} if the framework should skip the update (no-op) + */ + default boolean shouldApplyUpdate(final String connectorId) { + return true; + } + + /** + * Ensures that local asset binaries are up to date with the external store. For each asset + * tracked in the provider's local state, this method compares the external store's current + * content digest to the last-known digest. If changed or missing locally, the binary is + * downloaded via the {@link org.apache.nifi.asset.AssetManager} using a new UUID so that + * any existing local file for a running connector is not overwritten. The local state file + * is updated with the new UUID and digest. This method does not modify the external store's + * configuration. + * + *

After this method returns, callers should invoke {@link #load} to obtain the updated + * working configuration reflecting any new NiFi UUIDs assigned during sync.

+ * + * @param connectorId the identifier of the connector whose assets should be synced + */ + void syncAssets(String connectorId); +} diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfigurationProviderException.java b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfigurationProviderException.java new file mode 100644 index 000000000000..de772d0a17e7 --- /dev/null +++ b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfigurationProviderException.java @@ -0,0 +1,37 @@ +/* + * 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.components.connector; + +/** + * Runtime exception thrown by {@link ConnectorConfigurationProvider} implementations + * to indicate a failure in an external configuration operation such as load, save, discard, or delete. + * + *

This exception type allows provider implementations to signal failures using a well-defined + * exception rather than generic exceptions like IOException or database-specific exceptions. + * The framework will propagate these exceptions to callers rather than handling them silently.

+ */ +public class ConnectorConfigurationProviderException extends RuntimeException { + + public ConnectorConfigurationProviderException(final String message) { + super(message); + } + + public ConnectorConfigurationProviderException(final String message, final Throwable cause) { + super(message, cause); + } +} diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfigurationProviderInitializationContext.java b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfigurationProviderInitializationContext.java new file mode 100644 index 000000000000..bcf7dce26c4c --- /dev/null +++ b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfigurationProviderInitializationContext.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.components.connector; + +import org.apache.nifi.asset.AssetManager; + +import java.util.Map; + +/** + * Initialization context for a {@link ConnectorConfigurationProvider}. + * Provides the configuration properties necessary for the provider to initialize itself, + * such as database connection strings or other external store configuration. + * + *

Properties are extracted from NiFi properties with the prefix + * {@code nifi.components.connectors.configuration.provider.} stripped. For example, + * a NiFi property {@code nifi.components.connectors.configuration.provider.db.url=jdbc:...} + * becomes {@code db.url=jdbc:...} in the returned map.

+ */ +public interface ConnectorConfigurationProviderInitializationContext { + + /** + * Returns the configuration properties for this provider. + * + * @return a map of property names to values + */ + Map getProperties(); + + /** + * Returns the {@link AssetManager} that the provider should use for local asset storage. + * The provider uses this to create, save, and delete asset files on the local NiFi filesystem. + * + * @return the AssetManager for connector assets + */ + AssetManager getAssetManager(); +} diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorRequestContext.java b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorRequestContext.java new file mode 100644 index 000000000000..a722abac257a --- /dev/null +++ b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorRequestContext.java @@ -0,0 +1,88 @@ +/* + * 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.components.connector; + +import org.apache.nifi.authorization.user.NiFiUser; + +import java.util.List; +import java.util.Map; + +/** + * Provides request-scoped context from an HTTP request to a {@link ConnectorConfigurationProvider}. + * + *

When a connector operation is triggered by an HTTP request (e.g., from the Runtime UI), + * the framework populates this context with the authenticated NiFi user and all HTTP headers + * from the original request. Provider implementations can use this to make decisions based on + * who is making the request and to extract forwarded credentials (e.g., OAuth tokens relayed + * by a gateway).

+ * + *

This context is available via {@link ConnectorRequestContextHolder#getContext()} on the + * request thread. When no HTTP request is in scope (e.g., background operations), the holder + * returns {@code null}.

+ */ +public interface ConnectorRequestContext { + + /** + * Returns the authenticated NiFi user who initiated the request. For proxied requests + * (e.g., through a gateway), this is the end-user with the proxy chain accessible + * via {@link NiFiUser#getChain()}. + * + * @return the authenticated NiFi user, or {@code null} if not available + */ + NiFiUser getAuthenticatedUser(); + + /** + * Returns all HTTP headers from the original request as an immutable, case-insensitive + * multi-valued map. Each header name maps to an unmodifiable list of its values. + * + *

Provider implementations should read the specific headers they need and ignore + * the rest. This allows new headers to be forwarded by a gateway without requiring + * changes to the NiFi framework.

+ * + * @return an immutable map of header names to their values + */ + Map> getRequestHeaders(); + + /** + * Returns whether the request contains a header with the given name. + * Header name matching is case-insensitive per the HTTP specification. + * + * @param headerName the header name to check + * @return {@code true} if the header is present, {@code false} otherwise + */ + boolean hasRequestHeader(String headerName); + + /** + * Returns all values for the given header name, or an empty list if the header is not present. + * Header name matching is case-insensitive per the HTTP specification. + * + * @param headerName the header name to look up + * @return an unmodifiable list of header values, or an empty list if not present + */ + List getRequestHeaderValues(String headerName); + + /** + * Returns the first value for the given header name, or {@code null} if the header is + * not present or has no values. Header name matching is case-insensitive per the HTTP + * specification. + * + * @param headerName the header name to look up + * @return the first header value, or {@code null} if not present + */ + String getFirstRequestHeaderValue(String headerName); +} diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorRequestContextHolder.java b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorRequestContextHolder.java new file mode 100644 index 000000000000..db46ec53a303 --- /dev/null +++ b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorRequestContextHolder.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.components.connector; + +/** + * Thread-local holder for {@link ConnectorRequestContext}. This class resides in + * {@code nifi-framework-api} so that it is on the shared classloader, accessible + * from both the NiFi web layer (which sets the context) and NAR-loaded + * {@link ConnectorConfigurationProvider} implementations (which read it). + * + *

The context is set by the framework's request filter before connector operations + * and cleared after the request completes. Provider implementations should access the + * context via {@link #getContext()} and must not assume it is always present -- it will + * be {@code null} for operations not triggered by an HTTP request (e.g., background tasks).

+ */ +public final class ConnectorRequestContextHolder { + + private static final ThreadLocal CONTEXT = new ThreadLocal<>(); + + private ConnectorRequestContextHolder() { + } + + /** + * Returns the {@link ConnectorRequestContext} for the current thread, or {@code null} + * if no context has been set (e.g., for background or non-HTTP-request operations). + * + * @return the current request context, or {@code null} + */ + public static ConnectorRequestContext getContext() { + return CONTEXT.get(); + } + + /** + * Sets the {@link ConnectorRequestContext} for the current thread. + * + * @param context the request context to set + */ + public static void setContext(final ConnectorRequestContext context) { + CONTEXT.set(context); + } + + /** + * Clears the {@link ConnectorRequestContext} from the current thread. + * This must be called after request processing completes to prevent memory leaks. + */ + public static void clearContext() { + CONTEXT.remove(); + } +} diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorWorkingConfiguration.java b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorWorkingConfiguration.java new file mode 100644 index 000000000000..3d38d74c90ec --- /dev/null +++ b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/ConnectorWorkingConfiguration.java @@ -0,0 +1,55 @@ +/* + * 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.components.connector; + +import org.apache.nifi.flow.VersionedConfigurationStep; + +import java.util.List; + +/** + * Represents the externally managed working configuration of a Connector, + * including its name and working flow configuration steps. + * + *

This is a mutable POJO used as the input/output for {@link ConnectorConfigurationProvider} + * operations. The style follows the same pattern as {@link VersionedConfigurationStep} and other + * versioned types in the NiFi API.

+ * + *

Asset references in {@code workingFlowConfiguration} use NiFi UUIDs in the {@code assetIds} + * fields. The {@link ConnectorConfigurationProvider} is responsible for translating between NiFi + * UUIDs and any external asset identifiers used by the external store.

+ */ +public class ConnectorWorkingConfiguration { + private String name; + private List workingFlowConfiguration; + + public String getName() { + return name; + } + + public void setName(final String name) { + this.name = name; + } + + public List getWorkingFlowConfiguration() { + return workingFlowConfiguration; + } + + public void setWorkingFlowConfiguration(final List workingFlowConfiguration) { + this.workingFlowConfiguration = workingFlowConfiguration; + } +} diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/StandardConnectorRequestContext.java b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/StandardConnectorRequestContext.java new file mode 100644 index 000000000000..97a7e006019f --- /dev/null +++ b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/StandardConnectorRequestContext.java @@ -0,0 +1,84 @@ +/* + * 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.components.connector; + +import org.apache.nifi.authorization.user.NiFiUser; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +/** + * Standard implementation of {@link ConnectorRequestContext} that stores an authenticated + * {@link NiFiUser} and HTTP headers in a case-insensitive map. + * + *

Header name lookups via {@link #hasRequestHeader}, {@link #getRequestHeaderValues}, and + * {@link #getFirstRequestHeaderValue} are case-insensitive per the HTTP specification. + * The backing map uses {@link String#CASE_INSENSITIVE_ORDER} to guarantee this.

+ */ +public class StandardConnectorRequestContext implements ConnectorRequestContext { + + private final NiFiUser niFiUser; + private final Map> requestHeaders; + + /** + * Creates a new context with the given user and headers. The provided headers map is + * copied into a case-insensitive map; the original map is not retained. + * + * @param niFiUser the authenticated NiFi user, or {@code null} if not available + * @param requestHeaders the HTTP headers from the request; may be {@code null} or empty + */ + public StandardConnectorRequestContext(final NiFiUser niFiUser, final Map> requestHeaders) { + this.niFiUser = niFiUser; + if (requestHeaders == null || requestHeaders.isEmpty()) { + this.requestHeaders = Map.of(); + } else { + final Map> caseInsensitive = new TreeMap<>(String.CASE_INSENSITIVE_ORDER); + caseInsensitive.putAll(requestHeaders); + this.requestHeaders = Collections.unmodifiableMap(caseInsensitive); + } + } + + @Override + public NiFiUser getAuthenticatedUser() { + return niFiUser; + } + + @Override + public Map> getRequestHeaders() { + return requestHeaders; + } + + @Override + public boolean hasRequestHeader(final String headerName) { + return requestHeaders.containsKey(headerName); + } + + @Override + public List getRequestHeaderValues(final String headerName) { + final List values = requestHeaders.get(headerName); + return values != null ? values : List.of(); + } + + @Override + public String getFirstRequestHeaderValue(final String headerName) { + final List values = getRequestHeaderValues(headerName); + return values.isEmpty() ? null : values.getFirst(); + } +} diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/secrets/AuthorizableSecret.java b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/secrets/AuthorizableSecret.java new file mode 100644 index 000000000000..3bcf6b099cd1 --- /dev/null +++ b/nifi-framework-api/src/main/java/org/apache/nifi/components/connector/secrets/AuthorizableSecret.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.components.connector.secrets; + +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.components.connector.Secret; + +public interface AuthorizableSecret extends Secret, Authorizable { + +} diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/DropFlowFileStatus.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/DropFlowFileStatus.java index 737fbe3884f5..13d25bb2ed1a 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/DropFlowFileStatus.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/DropFlowFileStatus.java @@ -17,6 +17,8 @@ package org.apache.nifi.controller.queue; +import java.util.concurrent.CompletableFuture; + /** * Represents the status of a Drop FlowFile Request that has been issued to * a {@link FlowFileQueue}. When a queue is requested to drop its FlowFiles, @@ -75,4 +77,9 @@ public interface DropFlowFileStatus { * @return the reason that the state is set to a Failure state, or null if the state is not {@link DropFlowFileState#FAILURE}. */ String getFailureReason(); + + /** + * @return a Future that can be used to determine when the drop operation has completed + */ + CompletableFuture getCompletionFuture(); } 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 6911fd7853d8..2a0f4d6895cb 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 @@ -16,9 +16,11 @@ */ package org.apache.nifi.controller.queue; +import org.apache.nifi.components.connector.DropFlowFileSummary; import org.apache.nifi.controller.repository.FlowFileRecord; import org.apache.nifi.controller.repository.SwapSummary; import org.apache.nifi.controller.status.FlowFileAvailability; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.processor.FlowFileFilter; @@ -27,6 +29,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.function.Predicate; public interface FlowFileQueue { @@ -217,6 +220,18 @@ public interface FlowFileQueue { */ DropFlowFileStatus cancelDropFlowFileRequest(String requestIdentifier); + /** + * Synchronously drops all FlowFiles in this queue that match the given predicate. This method filters + * FlowFiles in the active queue, swap queue, and any swapped-out swap files. The FlowFile Repository + * and Provenance Repository are updated atomically after all matching FlowFiles have been identified. + * + * @param predicate the predicate used to determine which FlowFiles should be dropped; FlowFiles for which + * the predicate returns true will be dropped + * @return a summary of the FlowFiles that were dropped, including the count and total size in bytes + * @throws IOException if an error occurs while reading or writing swap files + */ + DropFlowFileSummary dropFlowFiles(Predicate predicate) throws IOException; + /** *

* Initiates a request to obtain a listing of FlowFiles in this queue. This method returns a diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/LocalQueuePartitionDiagnostics.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/LocalQueuePartitionDiagnostics.java index cc097b1a52e9..87ae47a4ee6e 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/LocalQueuePartitionDiagnostics.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/queue/LocalQueuePartitionDiagnostics.java @@ -29,4 +29,14 @@ public interface LocalQueuePartitionDiagnostics { boolean isAnyActiveFlowFilePenalized(); boolean isAllActiveFlowFilesPenalized(); + + /** + * @return the QueueSize representing the penalized FlowFiles (count and total content size) + */ + QueueSize getPenalizedQueueSize(); + + /** + * @return the total QueueSize across all swap files (count and total content size) + */ + QueueSize getTotalSwapFileQueueSize(); } diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/FlowFileSwapManager.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/FlowFileSwapManager.java index 018f27425a58..b29bb8b38024 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/FlowFileSwapManager.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/FlowFileSwapManager.java @@ -126,6 +126,16 @@ public interface FlowFileSwapManager { */ void purge(); + /** + * Deletes the swap file at the given location without updating the FlowFile Repository. + * This method is intended for use after the FlowFile Repository has already been updated + * to reflect that the FlowFiles are no longer in this swap file. + * + * @param swapLocation the location of the swap file to delete + * @throws IOException if unable to delete the swap file + */ + void deleteSwapFile(String swapLocation) throws IOException; + /** * Returns the ID of the queue that the given swap file belongs to * @param swapLocation the swap location diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecord.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecord.java index 7a56f2bfce3b..62b5495e3c0a 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecord.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecord.java @@ -82,6 +82,14 @@ public interface RepositoryRecord { */ String getSwapLocation(); + /** + * @return For SWAP_FILE_RENAMED records, provides the original swap location before the rename. + * For other record types, returns null. + */ + default String getOriginalSwapLocation() { + return null; + } + /** * @return a List of Content Claims that are "transient," meaning that they existed only for the * life of the Process Session in which they were created and should not be persisted. diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordType.java b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordType.java index 50221bba6a8a..af419fa40772 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordType.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordType.java @@ -21,5 +21,5 @@ */ public enum RepositoryRecordType { - UPDATE, CREATE, DELETE, CONTENTMISSING, SWAP_IN, SWAP_OUT, CLEANUP_TRANSIENT_CLAIMS; + UPDATE, CREATE, DELETE, CONTENTMISSING, SWAP_IN, SWAP_OUT, CLEANUP_TRANSIENT_CLAIMS, SWAP_FILE_DELETED, SWAP_FILE_RENAMED; } diff --git a/nifi-framework-api/src/main/java/org/apache/nifi/web/UiExtensionType.java b/nifi-framework-api/src/main/java/org/apache/nifi/web/UiExtensionType.java index 17c2d91e2df5..3cde479f0fc7 100644 --- a/nifi-framework-api/src/main/java/org/apache/nifi/web/UiExtensionType.java +++ b/nifi-framework-api/src/main/java/org/apache/nifi/web/UiExtensionType.java @@ -31,4 +31,5 @@ public enum UiExtensionType { ReportingTaskConfiguration, ParameterProviderConfiguration, FlowRegistryClientConfiguration, + Connector, } diff --git a/nifi-framework-api/src/test/java/org/apache/nifi/components/connector/ConnectorRequestContextHolderTest.java b/nifi-framework-api/src/test/java/org/apache/nifi/components/connector/ConnectorRequestContextHolderTest.java new file mode 100644 index 000000000000..ce9cba3eb672 --- /dev/null +++ b/nifi-framework-api/src/test/java/org/apache/nifi/components/connector/ConnectorRequestContextHolderTest.java @@ -0,0 +1,69 @@ +/* + * 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.components.connector; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + +class ConnectorRequestContextHolderTest { + + @AfterEach + void clearContext() { + ConnectorRequestContextHolder.clearContext(); + } + + @Test + void testGetContextReturnsNullByDefault() { + assertNull(ConnectorRequestContextHolder.getContext()); + } + + @Test + void testSetAndGetContext() { + final ConnectorRequestContext context = new StandardConnectorRequestContext(null, Map.of()); + ConnectorRequestContextHolder.setContext(context); + + assertEquals(context, ConnectorRequestContextHolder.getContext()); + } + + @Test + void testClearContextRemovesContext() { + final ConnectorRequestContext context = new StandardConnectorRequestContext(null, Map.of()); + ConnectorRequestContextHolder.setContext(context); + ConnectorRequestContextHolder.clearContext(); + + assertNull(ConnectorRequestContextHolder.getContext()); + } + + @Test + void testContextIsThreadLocal() throws Exception { + final ConnectorRequestContext context = new StandardConnectorRequestContext(null, Map.of()); + ConnectorRequestContextHolder.setContext(context); + + final ConnectorRequestContext[] otherThreadContext = new ConnectorRequestContext[1]; + final Thread thread = new Thread(() -> otherThreadContext[0] = ConnectorRequestContextHolder.getContext()); + thread.start(); + thread.join(); + + assertNull(otherThreadContext[0]); + assertEquals(context, ConnectorRequestContextHolder.getContext()); + } +} diff --git a/nifi-framework-bundle/nifi-framework-extensions/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java b/nifi-framework-bundle/nifi-framework-extensions/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java index 5680d3c3306e..e02b8fba8553 100644 --- a/nifi-framework-bundle/nifi-framework-extensions/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java +++ b/nifi-framework-bundle/nifi-framework-extensions/nifi-provenance-repository-bundle/nifi-volatile-provenance-repository/src/test/java/org/apache/nifi/provenance/TestVolatileProvenanceRepository.java @@ -48,7 +48,7 @@ public static void setup() { @Test public void testAddAndGet() throws IOException { - repo = new VolatileProvenanceRepository(NiFiProperties.createBasicNiFiProperties(null)); + repo = new VolatileProvenanceRepository(NiFiProperties.createBasicNiFiProperties((String) null)); final Map attributes = new HashMap<>(); attributes.put("abc", "xyz"); @@ -81,7 +81,7 @@ public void testAddAndGet() throws IOException { @Test public void testIndexAndCompressOnRolloverAndSubsequentSearchAsync() throws InterruptedException { - repo = new VolatileProvenanceRepository(NiFiProperties.createBasicNiFiProperties(null)); + repo = new VolatileProvenanceRepository(NiFiProperties.createBasicNiFiProperties((String) null)); final String uuid = "00000000-0000-0000-0000-000000000000"; final Map attributes = new HashMap<>(); @@ -120,7 +120,7 @@ public void testIndexAndCompressOnRolloverAndSubsequentSearchAsync() throws Inte @Test public void testSearchForInverseValue() throws InterruptedException { - repo = new VolatileProvenanceRepository(NiFiProperties.createBasicNiFiProperties(null)); + repo = new VolatileProvenanceRepository(NiFiProperties.createBasicNiFiProperties((String) null)); final Map attributes = new HashMap<>(); attributes.put("abc", "xyz"); diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConfigVerificationResultDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConfigVerificationResultDTO.java index 13841ca76f61..8134edc5036f 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConfigVerificationResultDTO.java +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConfigVerificationResultDTO.java @@ -22,6 +22,7 @@ public class ConfigVerificationResultDTO { private String outcome; private String verificationStepName; + private String subject; private String explanation; @Schema(description = "The outcome of the verification", allowableValues = {"SUCCESSFUL", "FAILED", "SKIPPED"}) @@ -42,6 +43,15 @@ public void setVerificationStepName(final String verificationStepName) { this.verificationStepName = verificationStepName; } + @Schema(description = "The subject of the verification step") + public String getSubject() { + return subject; + } + + public void setSubject(final String subject) { + this.subject = subject; + } + @Schema(description = "An explanation of why the step was or was not successful") public String getExplanation() { return explanation; diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConfigurationStepConfigurationDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConfigurationStepConfigurationDTO.java new file mode 100644 index 000000000000..402ecb60a00c --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConfigurationStepConfigurationDTO.java @@ -0,0 +1,96 @@ +/* + * 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; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlType; + +import java.util.List; +import java.util.Set; + +/** + * The configuration for a configuration step. + */ +@XmlType(name = "configurationStepConfiguration") +public class ConfigurationStepConfigurationDTO { + + private String configurationStepName; + private String configurationStepDescription; + private boolean documented; + private List propertyGroupConfigurations; + private Set dependencies; + + /** + * @return the configuration step name + */ + @Schema(description = "The name of the configuration step.") + public String getConfigurationStepName() { + return configurationStepName; + } + + public void setConfigurationStepName(final String configurationStepName) { + this.configurationStepName = configurationStepName; + } + + /** + * @return the configuration step description + */ + @Schema(description = "The description of the configuration step.") + public String getConfigurationStepDescription() { + return configurationStepDescription; + } + + public void setConfigurationStepDescription(final String configurationStepDescription) { + this.configurationStepDescription = configurationStepDescription; + } + + /** + * @return whether this step has extended documentation available + */ + @Schema(description = "Whether extended documentation is available for this configuration step.") + public boolean isDocumented() { + return documented; + } + + public void setDocumented(final boolean documented) { + this.documented = documented; + } + + /** + * @return the property group configurations + */ + @Schema(description = "The list of property group configurations for this configuration step.") + public List getPropertyGroupConfigurations() { + return propertyGroupConfigurations; + } + + public void setPropertyGroupConfigurations(final List propertyGroupConfigurations) { + this.propertyGroupConfigurations = propertyGroupConfigurations; + } + + /** + * @return the dependencies that this configuration step has on other configuration steps' properties + */ + @Schema(description = "The dependencies that this configuration step has on other configuration steps' properties.") + public Set getDependencies() { + return dependencies; + } + + public void setDependencies(final Set dependencies) { + this.dependencies = dependencies; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConfigurationStepDependencyDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConfigurationStepDependencyDTO.java new file mode 100644 index 000000000000..ee59271cfe95 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConfigurationStepDependencyDTO.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.api.dto; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlType; + +import java.util.Set; + +/** + * A dependency that a configuration step has on another configuration step's property. + */ +@XmlType(name = "configurationStepDependency") +public class ConfigurationStepDependencyDTO { + + private String stepName; + private String propertyName; + private Set dependentValues; + + /** + * @return the name of the configuration step that this step depends on + */ + @Schema(description = "The name of the configuration step that this step depends on.") + public String getStepName() { + return stepName; + } + + public void setStepName(final String stepName) { + this.stepName = stepName; + } + + /** + * @return the name of the property within the dependent step that must have a value + */ + @Schema(description = "The name of the property within the dependent step that must have a value.") + public String getPropertyName() { + return propertyName; + } + + public void setPropertyName(final String propertyName) { + this.propertyName = propertyName; + } + + /** + * @return the values of the dependent property that satisfy this dependency, or null if any non-null value satisfies the dependency + */ + @Schema(description = "The values of the dependent property that satisfy this dependency. If null, any non-null value satisfies the dependency.") + public Set getDependentValues() { + return dependentValues; + } + + public void setDependentValues(final Set dependentValues) { + this.dependentValues = dependentValues; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorActionDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorActionDTO.java new file mode 100644 index 000000000000..97e27fbfcb7e --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorActionDTO.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.api.dto; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlType; + +/** + * Represents an action that can be performed on a Connector. + */ +@XmlType(name = "connectorAction") +public class ConnectorActionDTO { + + private String name; + private String description; + private Boolean allowed; + private String reasonNotAllowed; + + @Schema(description = "The name of the action.") + public String getName() { + return name; + } + + public void setName(final String name) { + this.name = name; + } + + @Schema(description = "A description of what this action does.") + public String getDescription() { + return description; + } + + public void setDescription(final String description) { + this.description = description; + } + + @Schema(description = "Whether this action is currently allowed to be performed, based on the state of the Connector. " + + "Note that a value of 'true' does not imply that the user has permission to perform the action.") + public Boolean getAllowed() { + return allowed; + } + + public void setAllowed(final Boolean allowed) { + this.allowed = allowed; + } + + @Schema(description = "The reason why this action is not allowed, or null if the action is allowed.") + public String getReasonNotAllowed() { + return reasonNotAllowed; + } + + public void setReasonNotAllowed(final String reasonNotAllowed) { + this.reasonNotAllowed = reasonNotAllowed; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorConfigurationDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorConfigurationDTO.java new file mode 100644 index 000000000000..6ba11d535c01 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorConfigurationDTO.java @@ -0,0 +1,43 @@ +/* + * 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; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlType; + +import java.util.List; + +/** + * The configuration for a Connector. + */ +@XmlType(name = "connectorConfiguration") +public class ConnectorConfigurationDTO { + + private List configurationStepConfigurations; + + /** + * @return the configuration step configurations + */ + @Schema(description = "The list of configuration step configurations for this Connector.") + public List getConfigurationStepConfigurations() { + return configurationStepConfigurations; + } + + public void setConfigurationStepConfigurations(final List configurationStepConfigurations) { + this.configurationStepConfigurations = configurationStepConfigurations; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorDTO.java new file mode 100644 index 000000000000..ee891b10affe --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorDTO.java @@ -0,0 +1,172 @@ +/* + * 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; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlType; + +import java.util.Collection; +import java.util.List; + +/** + * Component representing a Connector instance. + */ +@XmlType(name = "connector") +public class ConnectorDTO extends ComponentDTO { + private String name; + private String type; + private BundleDTO bundle; + private String state; // RUNNING, STOPPED + private String managedProcessGroupId; + private ConnectorConfigurationDTO activeConfiguration; + private ConnectorConfigurationDTO workingConfiguration; + + private Collection validationErrors; + private String validationStatus; + private Boolean multipleVersionsAvailable; + private Boolean extensionMissing; + + private String configurationUrl; + private String detailsUrl; + private List availableActions; + + @Schema(description = "The name of the Connector.") + public String getName() { + return name; + } + + public void setName(final String name) { + this.name = name; + } + + @Schema(description = "The fully qualified type of the Connector.") + public String getType() { + return type; + } + + public void setType(final String type) { + this.type = type; + } + + @Schema(description = "The details of the artifact that bundled this Connector type.") + public BundleDTO getBundle() { + return bundle; + } + + public void setBundle(final BundleDTO bundle) { + this.bundle = bundle; + } + + @Schema(description = "The state of the Connector.", allowableValues = {"RUNNING", "STOPPED"}) + public String getState() { + return state; + } + + public void setState(final String state) { + this.state = state; + } + + @Schema(description = "The identifier of the root Process Group managed by this Connector.") + public String getManagedProcessGroupId() { + return managedProcessGroupId; + } + + public void setManagedProcessGroupId(final String managedProcessGroupId) { + this.managedProcessGroupId = managedProcessGroupId; + } + + @Schema(description = "The active configuration of the Connector.") + public ConnectorConfigurationDTO getActiveConfiguration() { + return activeConfiguration; + } + + public void setActiveConfiguration(final ConnectorConfigurationDTO activeConfiguration) { + this.activeConfiguration = activeConfiguration; + } + + @Schema(description = "The working configuration of the Connector.") + public ConnectorConfigurationDTO getWorkingConfiguration() { + return workingConfiguration; + } + + public void setWorkingConfiguration(final ConnectorConfigurationDTO workingConfiguration) { + this.workingConfiguration = workingConfiguration; + } + + @Schema(description = "The validation errors for the connector.") + public Collection getValidationErrors() { + return validationErrors; + } + + public void setValidationErrors(final Collection validationErrors) { + this.validationErrors = validationErrors; + } + + @Schema(description = "Indicates whether the Connector is valid, invalid, or still in the process of validating") + public String getValidationStatus() { + return validationStatus; + } + + public void setValidationStatus(final String validationStatus) { + this.validationStatus = validationStatus; + } + + @Schema(description = "Whether multiple versions of this connector are available.") + public Boolean getMultipleVersionsAvailable() { + return multipleVersionsAvailable; + } + + public void setMultipleVersionsAvailable(final Boolean multipleVersionsAvailable) { + this.multipleVersionsAvailable = multipleVersionsAvailable; + } + + @Schema(description = "The URL for this connector's configuration/wizard custom UI, if applicable.") + public String getConfigurationUrl() { + return configurationUrl; + } + + public void setConfigurationUrl(final String configurationUrl) { + this.configurationUrl = configurationUrl; + } + + @Schema(description = "The URL for this connector's details custom UI, if applicable.") + public String getDetailsUrl() { + return detailsUrl; + } + + public void setDetailsUrl(final String detailsUrl) { + this.detailsUrl = detailsUrl; + } + + @Schema(description = "Whether the extension for this connector is missing.") + public Boolean getExtensionMissing() { + return extensionMissing; + } + + public void setExtensionMissing(final Boolean extensionMissing) { + this.extensionMissing = extensionMissing; + } + + @Schema(description = "The available actions that can be performed on this Connector.") + public List getAvailableActions() { + return availableActions; + } + + public void setAvailableActions(final List availableActions) { + this.availableActions = availableActions; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorPropertyDependencyDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorPropertyDependencyDTO.java new file mode 100644 index 000000000000..2604448a7a7c --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorPropertyDependencyDTO.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.api.dto; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlType; + +import java.util.Set; + +/** + * A dependency that a connector property has on another property. + */ +@XmlType(name = "connectorPropertyDependency") +public class ConnectorPropertyDependencyDTO { + + private String propertyName; + private Set dependentValues; + + /** + * @return the name of the property that this property depends on + */ + @Schema(description = "The name of the property that this property depends on.") + public String getPropertyName() { + return propertyName; + } + + public void setPropertyName(final String propertyName) { + this.propertyName = propertyName; + } + + /** + * @return the values of the dependent property that must be set for this property to be applicable + */ + @Schema(description = "The values of the dependent property that must be set for this property to be applicable.") + public Set getDependentValues() { + return dependentValues; + } + + public void setDependentValues(final Set dependentValues) { + this.dependentValues = dependentValues; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorPropertyDescriptorDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorPropertyDescriptorDTO.java new file mode 100644 index 000000000000..edc1e9c780ae --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorPropertyDescriptorDTO.java @@ -0,0 +1,137 @@ +/* + * 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; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlType; +import org.apache.nifi.web.api.entity.AllowableValueEntity; + +import java.util.List; +import java.util.Set; + +/** + * A description of a connector property. + */ +@XmlType(name = "connectorPropertyDescriptor") +public class ConnectorPropertyDescriptorDTO { + + private String name; + private String description; + private String defaultValue; + private Boolean required; + private String type; + private List allowableValues; + private Boolean allowableValuesFetchable; + private Set dependencies; + + /** + * @return property name + */ + @Schema(description = "The name of the property.") + public String getName() { + return name; + } + + public void setName(final String name) { + this.name = name; + } + + /** + * @return An explanation of the meaning of the given property + */ + @Schema(description = "The description of the property.") + public String getDescription() { + return description; + } + + public void setDescription(final String description) { + this.description = description; + } + + /** + * @return default value for this property + */ + @Schema(description = "The default value for the property.") + public String getDefaultValue() { + return defaultValue; + } + + public void setDefaultValue(final String defaultValue) { + this.defaultValue = defaultValue; + } + + /** + * @return whether this property is required + */ + @Schema(description = "Whether the property is required.") + public Boolean getRequired() { + return required; + } + + public void setRequired(final Boolean required) { + this.required = required; + } + + /** + * @return the property type (STRING, INTEGER, BOOLEAN, DOUBLE, STRING_LIST, etc.) + */ + @Schema(description = "The type of the property (STRING, INTEGER, BOOLEAN, DOUBLE, STRING_LIST).") + public String getType() { + return type; + } + + public void setType(final String type) { + this.type = type; + } + + /** + * @return set of allowable values for this property. If empty then the allowable values are not constrained + */ + @Schema(description = "Allowable values for the property. If empty then the allowed values are not constrained.") + public List getAllowableValues() { + return allowableValues; + } + + public void setAllowableValues(final List allowableValues) { + this.allowableValues = allowableValues; + } + + /** + * @return whether the allowable values can be fetched dynamically + */ + @Schema(description = "Whether the allowable values are dynamically fetchable based on other property values.") + public Boolean getAllowableValuesFetchable() { + return allowableValuesFetchable; + } + + public void setAllowableValuesFetchable(final Boolean allowableValuesFetchable) { + this.allowableValuesFetchable = allowableValuesFetchable; + } + + /** + * @return the dependencies this property has on other properties + */ + @Schema(description = "The dependencies this property has on other properties.") + public Set getDependencies() { + return dependencies; + } + + public void setDependencies(final Set dependencies) { + this.dependencies = dependencies; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorValueReferenceDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorValueReferenceDTO.java new file mode 100644 index 000000000000..ed79bdb3ac00 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectorValueReferenceDTO.java @@ -0,0 +1,120 @@ +/* + * 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; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlType; + +import java.util.List; + +/** + * A reference to a connector property value, which includes the value and its type. + * The value can be a string literal, a reference to an asset, or a reference to a secret. + */ +@XmlType(name = "connectorValueReference") +public class ConnectorValueReferenceDTO { + + private String valueType; + private String value; + private String secretProviderId; + private String secretProviderName; + private String secretName; + private String fullyQualifiedSecretName; + private List assetReferences; + + /** + * @return the type of value (STRING_LITERAL, ASSET_REFERENCE, or SECRET_REFERENCE) + */ + @Schema(description = "The type of value (STRING_LITERAL, ASSET_REFERENCE, or SECRET_REFERENCE).") + public String getValueType() { + return valueType; + } + + public void setValueType(final String valueType) { + this.valueType = valueType; + } + + /** + * @return the string literal value when valueType is STRING_LITERAL + */ + @Schema(description = "The string literal value. Applicable when valueType is STRING_LITERAL.") + public String getValue() { + return value; + } + + public void setValue(final String value) { + this.value = value; + } + + /** + * @return the secret provider identifier when valueType is SECRET_REFERENCE + */ + @Schema(description = "The secret provider identifier. Applicable when valueType is SECRET_REFERENCE.") + public String getSecretProviderId() { + return secretProviderId; + } + + public void setSecretProviderId(final String secretProviderId) { + this.secretProviderId = secretProviderId; + } + + /** + * @return the secret provider name when valueType is SECRET_REFERENCE + */ + @Schema(description = "The secret provider name. Applicable when valueType is SECRET_REFERENCE.") + public String getSecretProviderName() { + return secretProviderName; + } + + public void setSecretProviderName(final String secretProviderName) { + this.secretProviderName = secretProviderName; + } + + /** + * @return the secret name when valueType is SECRET_REFERENCE + */ + @Schema(description = "The secret name. Applicable when valueType is SECRET_REFERENCE.") + public String getSecretName() { + return secretName; + } + + public void setSecretName(final String secretName) { + this.secretName = secretName; + } + + public void setFullyQualifiedSecretName(final String fullyQualifiedSecretName) { + this.fullyQualifiedSecretName = fullyQualifiedSecretName; + } + + @Schema(description = "The fully qualified secret name. Applicable when valueType is SECRET_REFERENCE.") + public String getFullyQualifiedSecretName() { + return fullyQualifiedSecretName; + } + + /** + * @return the asset identifiers when valueType is ASSET_REFERENCE + */ + @Schema(description = "The asset references. Applicable when valueType is ASSET_REFERENCE.") + public List getAssetReferences() { + return assetReferences; + } + + public void setAssetReferences(final List assetReferences) { + this.assetReferences = assetReferences; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyGroupConfigurationDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyGroupConfigurationDTO.java new file mode 100644 index 000000000000..a78f946e41fc --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyGroupConfigurationDTO.java @@ -0,0 +1,82 @@ +/* + * 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; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlType; + +import java.util.Map; + +/** + * The configuration for a property group. + */ +@XmlType(name = "propertyGroupConfiguration") +public class PropertyGroupConfigurationDTO { + + private String propertyGroupName; + private String propertyGroupDescription; + private Map propertyDescriptors; + private Map propertyValues; + + /** + * @return the property group name + */ + @Schema(description = "The name of the property group.") + public String getPropertyGroupName() { + return propertyGroupName; + } + + public void setPropertyGroupName(final String propertyGroupName) { + this.propertyGroupName = propertyGroupName; + } + + /** + * @return the property group description + */ + @Schema(description = "The description of the property group.") + public String getPropertyGroupDescription() { + return propertyGroupDescription; + } + + public void setPropertyGroupDescription(final String propertyGroupDescription) { + this.propertyGroupDescription = propertyGroupDescription; + } + + /** + * @return the property descriptors keyed by property name + */ + @Schema(description = "The property descriptors for this property group, keyed by property name.") + public Map getPropertyDescriptors() { + return propertyDescriptors; + } + + public void setPropertyDescriptors(final Map propertyDescriptors) { + this.propertyDescriptors = propertyDescriptors; + } + + /** + * @return the property values + */ + @Schema(description = "The property values for this property group.") + public Map getPropertyValues() { + return propertyValues; + } + + public void setPropertyValues(final Map propertyValues) { + this.propertyValues = propertyValues; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/SecretDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/SecretDTO.java new file mode 100644 index 000000000000..dbbf0f36a0bd --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/SecretDTO.java @@ -0,0 +1,89 @@ +/* + * 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; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlType; + +/** + * DTO representing a secret's metadata. Note: The actual secret value is never exposed via the REST API. + */ +@XmlType(name = "secret") +public class SecretDTO { + + private String providerId; + private String providerName; + private String groupName; + private String name; + private String fullyQualifiedName; + private String description; + + @Schema(description = "The identifier of the secret provider that manages this secret.") + public String getProviderId() { + return providerId; + } + + public void setProviderId(final String providerId) { + this.providerId = providerId; + } + + @Schema(description = "The name of the secret provider that manages this secret.") + public String getProviderName() { + return providerName; + } + + public void setProviderName(final String providerName) { + this.providerName = providerName; + } + + @Schema(description = "The name of the group this secret belongs to.") + public String getGroupName() { + return groupName; + } + + public void setGroupName(final String groupName) { + this.groupName = groupName; + } + + @Schema(description = "The name of the secret.") + public String getName() { + return name; + } + + public void setName(final String name) { + this.name = name; + } + + @Schema(description = "The fully qualified name of the secret, including the group name.") + public String getFullyQualifiedName() { + return fullyQualifiedName; + } + + public void setFullyQualifiedName(final String fullyQualifiedName) { + this.fullyQualifiedName = fullyQualifiedName; + } + + @Schema(description = "A description of the secret.") + public String getDescription() { + return description; + } + + public void setDescription(final String description) { + this.description = description; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VerifyConnectorConfigStepRequestDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VerifyConnectorConfigStepRequestDTO.java new file mode 100644 index 000000000000..c42edd31c0f3 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VerifyConnectorConfigStepRequestDTO.java @@ -0,0 +1,69 @@ +/* + * 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; + +import io.swagger.v3.oas.annotations.media.Schema; +import io.swagger.v3.oas.annotations.media.Schema.AccessMode; +import jakarta.xml.bind.annotation.XmlType; + +import java.util.List; + +@XmlType(name = "verifyConnectorConfigStepRequest") +public class VerifyConnectorConfigStepRequestDTO extends AsynchronousRequestDTO { + + private String connectorId; + private String configurationStepName; + private ConfigurationStepConfigurationDTO configurationStep; + private List results; + + @Schema(description = "The ID of the connector whose configuration step is being verified") + public String getConnectorId() { + return connectorId; + } + + public void setConnectorId(final String connectorId) { + this.connectorId = connectorId; + } + + @Schema(description = "The name of the configuration step being verified") + public String getConfigurationStepName() { + return configurationStepName; + } + + public void setConfigurationStepName(final String configurationStepName) { + this.configurationStepName = configurationStepName; + } + + @Schema(description = "The configuration step being verified") + public ConfigurationStepConfigurationDTO getConfigurationStep() { + return configurationStep; + } + + public void setConfigurationStep(final ConfigurationStepConfigurationDTO configurationStep) { + this.configurationStep = configurationStep; + } + + @Schema(description = "The results of the verification", accessMode = AccessMode.READ_ONLY) + public List getResults() { + return results; + } + + public void setResults(final List results) { + this.results = results; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.java index 6c56c31258f6..88eeaa8a818f 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.java +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.java @@ -49,6 +49,7 @@ public class ProvenanceEventDTO { private String clusterNodeAddress; // include when clustered private String groupId; + private String connectorId; private String componentId; private String componentType; private String componentName; @@ -217,6 +218,19 @@ public void setGroupId(String groupId) { this.groupId = groupId; } + /** + * @return id of the Connector that manages the component that generated this event, or {@code null} if the component is not managed by a Connector + */ + @Schema(description = "The id of the connector that manages the component that generated the event. If the component is not managed by a connector, this will not be set." + ) + public String getConnectorId() { + return connectorId; + } + + public void setConnectorId(String connectorId) { + this.connectorId = connectorId; + } + /** * @return id of the component that generated this event */ diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectorStatusDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectorStatusDTO.java new file mode 100644 index 000000000000..388c22dff540 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectorStatusDTO.java @@ -0,0 +1,186 @@ +/* + * 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.status; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlType; +import jakarta.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +import java.util.ArrayList; +import java.util.Date; +import java.util.List; + +/** + * DTO for serializing the status of a connector. + */ +@XmlType(name = "connectorStatus") +public class ConnectorStatusDTO implements Cloneable { + + private String id; + private String groupId; + private String name; + private String type; + private String runStatus; + private String validationStatus; + private Date statsLastRefreshed; + + private ConnectorStatusSnapshotDTO aggregateSnapshot; + private List nodeSnapshots; + + /** + * The id of the connector. + * + * @return The connector id + */ + @Schema(description = "The id of the connector.") + public String getId() { + return id; + } + + public void setId(final String id) { + this.id = id; + } + + /** + * The id of the group this connector belongs to. + * + * @return The group id + */ + @Schema(description = "The id of the group this connector belongs to.") + public String getGroupId() { + return groupId; + } + + public void setGroupId(final String groupId) { + this.groupId = groupId; + } + + /** + * The name of the connector. + * + * @return The connector name + */ + @Schema(description = "The name of the connector.") + public String getName() { + return name; + } + + public void setName(final String name) { + this.name = name; + } + + /** + * The type of the connector. + * + * @return The connector type + */ + @Schema(description = "The type of the connector.") + public String getType() { + return type; + } + + public void setType(final String type) { + this.type = type; + } + + /** + * The run status of the connector. + * + * @return The run status + */ + @Schema(description = "The run status of the connector.") + public String getRunStatus() { + return runStatus; + } + + public void setRunStatus(final String runStatus) { + this.runStatus = runStatus; + } + + /** + * The validation status of the connector. + * + * @return The validation status + */ + @Schema(description = "The validation status of the connector.", + allowableValues = {"VALID", "INVALID", "VALIDATING"}) + public String getValidationStatus() { + return validationStatus; + } + + public void setValidationStatus(final String validationStatus) { + this.validationStatus = validationStatus; + } + + @XmlJavaTypeAdapter(TimeAdapter.class) + @Schema(description = "The timestamp of when the stats were last refreshed.", type = "string") + public Date getStatsLastRefreshed() { + return statsLastRefreshed; + } + + public void setStatsLastRefreshed(final Date statsLastRefreshed) { + this.statsLastRefreshed = statsLastRefreshed; + } + + @Schema(description = "A status snapshot that represents the aggregate stats of all nodes in the cluster. If the NiFi instance is " + + "a standalone instance, rather than a cluster, this represents the stats of the single instance.") + public ConnectorStatusSnapshotDTO getAggregateSnapshot() { + return aggregateSnapshot; + } + + public void setAggregateSnapshot(final ConnectorStatusSnapshotDTO aggregateSnapshot) { + this.aggregateSnapshot = aggregateSnapshot; + } + + @Schema(description = "A status snapshot for each node in the cluster. If the NiFi instance is a standalone instance, rather than " + + "a cluster, this may be null.") + public List getNodeSnapshots() { + return nodeSnapshots; + } + + public void setNodeSnapshots(final List nodeSnapshots) { + this.nodeSnapshots = nodeSnapshots; + } + + @Override + public ConnectorStatusDTO clone() { + final ConnectorStatusDTO other = new ConnectorStatusDTO(); + other.setId(getId()); + other.setGroupId(getGroupId()); + other.setName(getName()); + other.setType(getType()); + other.setRunStatus(getRunStatus()); + other.setValidationStatus(getValidationStatus()); + other.setStatsLastRefreshed(getStatsLastRefreshed()); + + if (getAggregateSnapshot() != null) { + other.setAggregateSnapshot(getAggregateSnapshot().clone()); + } + + final List snapshots = getNodeSnapshots(); + if (snapshots != null) { + final List snapshotClones = new ArrayList<>(snapshots.size()); + for (final NodeConnectorStatusSnapshotDTO snapshot : snapshots) { + snapshotClones.add(snapshot.clone()); + } + other.setNodeSnapshots(snapshotClones); + } + + return other; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectorStatusSnapshotDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectorStatusSnapshotDTO.java new file mode 100644 index 000000000000..23833dfb9733 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectorStatusSnapshotDTO.java @@ -0,0 +1,322 @@ +/* + * 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.status; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlType; + +/** + * DTO for serializing the status snapshot of a connector. + */ +@XmlType(name = "connectorStatusSnapshot") +public class ConnectorStatusSnapshotDTO implements Cloneable { + + private String id; + private String groupId; + private String name; + private String type; + private String runStatus; + + private Integer flowFilesSent = 0; + private Long bytesSent = 0L; + private Integer flowFilesReceived = 0; + private Long bytesReceived = 0L; + private Long bytesRead = 0L; + private Long bytesWritten = 0L; + private String sent; + private String received; + private String read; + private String written; + + private Integer flowFilesQueued = 0; + private Long bytesQueued = 0L; + private String queued; + private String queuedCount; + private String queuedSize; + + private Integer activeThreadCount = 0; + + private ProcessingPerformanceStatusDTO processingPerformanceStatus; + + private Boolean idle; + private Long idleDurationMillis; + private String idleDuration; + + @Schema(description = "The id of the connector.") + public String getId() { + return id; + } + + public void setId(final String id) { + this.id = id; + } + + @Schema(description = "The id of the parent process group to which the connector belongs.") + public String getGroupId() { + return groupId; + } + + public void setGroupId(final String groupId) { + this.groupId = groupId; + } + + @Schema(description = "The name of the connector.") + public String getName() { + return name; + } + + public void setName(final String name) { + this.name = name; + } + + @Schema(description = "The type of the connector.") + public String getType() { + return type; + } + + public void setType(final String type) { + this.type = type; + } + + @Schema(description = "The run status of the connector.") + public String getRunStatus() { + return runStatus; + } + + public void setRunStatus(final String runStatus) { + this.runStatus = runStatus; + } + + @Schema(description = "The number of FlowFiles sent by this connector's managed process group.") + public Integer getFlowFilesSent() { + return flowFilesSent; + } + + public void setFlowFilesSent(final Integer flowFilesSent) { + this.flowFilesSent = flowFilesSent; + } + + @Schema(description = "The number of bytes sent by this connector's managed process group.") + public Long getBytesSent() { + return bytesSent; + } + + public void setBytesSent(final Long bytesSent) { + this.bytesSent = bytesSent; + } + + @Schema(description = "The number of FlowFiles received by this connector's managed process group.") + public Integer getFlowFilesReceived() { + return flowFilesReceived; + } + + public void setFlowFilesReceived(final Integer flowFilesReceived) { + this.flowFilesReceived = flowFilesReceived; + } + + @Schema(description = "The number of bytes received by this connector's managed process group.") + public Long getBytesReceived() { + return bytesReceived; + } + + public void setBytesReceived(final Long bytesReceived) { + this.bytesReceived = bytesReceived; + } + + @Schema(description = "The number of bytes read by processors in this connector's managed process group.") + public Long getBytesRead() { + return bytesRead; + } + + public void setBytesRead(final Long bytesRead) { + this.bytesRead = bytesRead; + } + + @Schema(description = "The number of bytes written by processors in this connector's managed process group.") + public Long getBytesWritten() { + return bytesWritten; + } + + public void setBytesWritten(final Long bytesWritten) { + this.bytesWritten = bytesWritten; + } + + @Schema(description = "The count/size of data that has been sent by this connector, pretty-printed.") + public String getSent() { + return sent; + } + + public void setSent(final String sent) { + this.sent = sent; + } + + @Schema(description = "The count/size of data that has been received by this connector, pretty-printed.") + public String getReceived() { + return received; + } + + public void setReceived(final String received) { + this.received = received; + } + + @Schema(description = "The number of bytes read, pretty-printed.") + public String getRead() { + return read; + } + + public void setRead(final String read) { + this.read = read; + } + + @Schema(description = "The number of bytes written, pretty-printed.") + public String getWritten() { + return written; + } + + public void setWritten(final String written) { + this.written = written; + } + + @Schema(description = "The number of FlowFiles queued in this connector's managed process group.") + public Integer getFlowFilesQueued() { + return flowFilesQueued; + } + + public void setFlowFilesQueued(final Integer flowFilesQueued) { + this.flowFilesQueued = flowFilesQueued; + } + + @Schema(description = "The number of bytes queued in this connector's managed process group.") + public Long getBytesQueued() { + return bytesQueued; + } + + public void setBytesQueued(final Long bytesQueued) { + this.bytesQueued = bytesQueued; + } + + @Schema(description = "The count/size of queued data, pretty-printed.") + public String getQueued() { + return queued; + } + + public void setQueued(final String queued) { + this.queued = queued; + } + + @Schema(description = "The count of queued FlowFiles, pretty-printed.") + public String getQueuedCount() { + return queuedCount; + } + + public void setQueuedCount(final String queuedCount) { + this.queuedCount = queuedCount; + } + + @Schema(description = "The size of queued data, pretty-printed.") + public String getQueuedSize() { + return queuedSize; + } + + public void setQueuedSize(final String queuedSize) { + this.queuedSize = queuedSize; + } + + @Schema(description = "The number of active threads for the connector.") + public Integer getActiveThreadCount() { + return activeThreadCount; + } + + public void setActiveThreadCount(final Integer activeThreadCount) { + this.activeThreadCount = activeThreadCount; + } + + @Schema(description = "The processing performance status of the processors in this connector's managed process group.") + public ProcessingPerformanceStatusDTO getProcessingPerformanceStatus() { + return processingPerformanceStatus; + } + + public void setProcessingPerformanceStatus(final ProcessingPerformanceStatusDTO processingPerformanceStatus) { + this.processingPerformanceStatus = processingPerformanceStatus; + } + + @Schema(description = "Whether or not the connector is currently idle (no FlowFiles queued and no FlowFiles processed recently).") + public Boolean getIdle() { + return idle; + } + + public void setIdle(final Boolean idle) { + this.idle = idle; + } + + @Schema(description = "The number of milliseconds the connector has been idle, or null if the connector is not idle.") + public Long getIdleDurationMillis() { + return idleDurationMillis; + } + + public void setIdleDurationMillis(final Long idleDurationMillis) { + this.idleDurationMillis = idleDurationMillis; + } + + @Schema(description = "A human-readable representation of how long the connector has been idle, or null if the connector is not idle.") + public String getIdleDuration() { + return idleDuration; + } + + public void setIdleDuration(final String idleDuration) { + this.idleDuration = idleDuration; + } + + @Override + public ConnectorStatusSnapshotDTO clone() { + final ConnectorStatusSnapshotDTO other = new ConnectorStatusSnapshotDTO(); + other.setId(getId()); + other.setGroupId(getGroupId()); + other.setName(getName()); + other.setType(getType()); + other.setRunStatus(getRunStatus()); + + other.setFlowFilesSent(getFlowFilesSent()); + other.setBytesSent(getBytesSent()); + other.setFlowFilesReceived(getFlowFilesReceived()); + other.setBytesReceived(getBytesReceived()); + other.setBytesRead(getBytesRead()); + other.setBytesWritten(getBytesWritten()); + other.setSent(getSent()); + other.setReceived(getReceived()); + other.setRead(getRead()); + other.setWritten(getWritten()); + + other.setFlowFilesQueued(getFlowFilesQueued()); + other.setBytesQueued(getBytesQueued()); + other.setQueued(getQueued()); + other.setQueuedCount(getQueuedCount()); + other.setQueuedSize(getQueuedSize()); + + other.setActiveThreadCount(getActiveThreadCount()); + + if (getProcessingPerformanceStatus() != null) { + other.setProcessingPerformanceStatus(getProcessingPerformanceStatus().clone()); + } + + other.setIdle(getIdle()); + other.setIdleDurationMillis(getIdleDurationMillis()); + other.setIdleDuration(getIdleDuration()); + + return other; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectorStatusSnapshotDTO.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectorStatusSnapshotDTO.java new file mode 100644 index 000000000000..de9cc43a6c83 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectorStatusSnapshotDTO.java @@ -0,0 +1,79 @@ +/* + * 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.status; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlType; + +/** + * DTO for serializing the connector status snapshot for a particular node. + */ +@XmlType(name = "nodeConnectorStatusSnapshot") +public class NodeConnectorStatusSnapshotDTO implements Cloneable { + + private String nodeId; + private String address; + private Integer apiPort; + + private ConnectorStatusSnapshotDTO statusSnapshot; + + @Schema(description = "The unique ID that identifies the node") + public String getNodeId() { + return nodeId; + } + + public void setNodeId(final String nodeId) { + this.nodeId = nodeId; + } + + @Schema(description = "The API address of the node") + public String getAddress() { + return address; + } + + public void setAddress(final String address) { + this.address = address; + } + + @Schema(description = "The API port used to communicate with the node") + public Integer getApiPort() { + return apiPort; + } + + public void setApiPort(final Integer apiPort) { + this.apiPort = apiPort; + } + + @Schema(description = "The connector status snapshot from the node.") + public ConnectorStatusSnapshotDTO getStatusSnapshot() { + return statusSnapshot; + } + + public void setStatusSnapshot(final ConnectorStatusSnapshotDTO statusSnapshot) { + this.statusSnapshot = statusSnapshot; + } + + @Override + public NodeConnectorStatusSnapshotDTO clone() { + final NodeConnectorStatusSnapshotDTO other = new NodeConnectorStatusSnapshotDTO(); + other.setNodeId(getNodeId()); + other.setAddress(getAddress()); + other.setApiPort(getApiPort()); + other.setStatusSnapshot(getStatusSnapshot().clone()); + return other; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConfigurationStepEntity.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConfigurationStepEntity.java new file mode 100644 index 000000000000..689b1ddf96fc --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConfigurationStepEntity.java @@ -0,0 +1,77 @@ +/* + * 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 io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO; +import org.apache.nifi.web.api.dto.RevisionDTO; + +/** + * A serialized representation of this class can be placed in the entity body of a request or response to or from the API. + * This particular entity holds a reference to a ConfigurationStepConfigurationDTO. + * Note that this entity does not extend ComponentEntity because a configuration step is not itself an updatable component. + * Instead, it is updated through the parent Connector, so the client must provide the parent connector ID and revision. + * Permissions are not included on this entity since they are managed at the parent Connector level. + */ +@XmlRootElement(name = "configurationStepEntity") +public class ConfigurationStepEntity extends Entity { + + private ConfigurationStepConfigurationDTO configurationStep; + private String parentConnectorId; + private RevisionDTO parentConnectorRevision; + private Boolean disconnectedNodeAcknowledged; + + /** + * @return the configuration step configuration + */ + @Schema(description = "The configuration step configuration.") + public ConfigurationStepConfigurationDTO getConfigurationStep() { + return configurationStep; + } + + public void setConfigurationStep(final ConfigurationStepConfigurationDTO configurationStep) { + this.configurationStep = configurationStep; + } + + @Schema(description = "The id of the parent connector.") + public String getParentConnectorId() { + return parentConnectorId; + } + + public void setParentConnectorId(final String parentConnectorId) { + this.parentConnectorId = parentConnectorId; + } + + @Schema(description = "The revision of the parent connector that this configuration step belongs to.") + public RevisionDTO getParentConnectorRevision() { + return parentConnectorRevision; + } + + public void setParentConnectorRevision(final RevisionDTO parentConnectorRevision) { + this.parentConnectorRevision = parentConnectorRevision; + } + + @Schema(description = "Acknowledges that this node is disconnected to allow for mutable requests to proceed.") + public Boolean isDisconnectedNodeAcknowledged() { + return disconnectedNodeAcknowledged; + } + + public void setDisconnectedNodeAcknowledged(final Boolean disconnectedNodeAcknowledged) { + this.disconnectedNodeAcknowledged = disconnectedNodeAcknowledged; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConfigurationStepNamesEntity.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConfigurationStepNamesEntity.java new file mode 100644 index 000000000000..0dc2da053655 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConfigurationStepNamesEntity.java @@ -0,0 +1,44 @@ +/* + * 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 io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlRootElement; + +import java.util.List; + +/** + * A serialized representation of this class can be placed in the entity body of a request or response to or from the API. + * This particular entity holds a reference to a list of configuration step names. + */ +@XmlRootElement(name = "configurationStepNamesEntity") +public class ConfigurationStepNamesEntity extends Entity { + + private List configurationStepNames; + + /** + * @return the configuration step names + */ + @Schema(description = "The list of configuration step names for a connector.") + public List getConfigurationStepNames() { + return configurationStepNames; + } + + public void setConfigurationStepNames(final List configurationStepNames) { + this.configurationStepNames = configurationStepNames; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorEntity.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorEntity.java new file mode 100644 index 000000000000..0cf02aaa00d3 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorEntity.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.web.api.entity; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlRootElement; +import jakarta.xml.bind.annotation.XmlType; +import org.apache.nifi.web.api.dto.ConnectorDTO; +import org.apache.nifi.web.api.dto.PermissionsDTO; +import org.apache.nifi.web.api.dto.status.ConnectorStatusDTO; + +@XmlRootElement(name = "connectorEntity") +@XmlType(name = "connectorEntity") +public class ConnectorEntity extends ComponentEntity implements Permissible, OperationPermissible { + private ConnectorDTO component; + private ConnectorStatusDTO status; + private PermissionsDTO operatePermissions; + + @Override + @Schema(description = "The Connector DTO") + public ConnectorDTO getComponent() { + return component; + } + + @Override + public void setComponent(final ConnectorDTO component) { + this.component = component; + } + + @Schema(description = "The permissions for this component operations.") + @Override + public PermissionsDTO getOperatePermissions() { + return operatePermissions; + } + + @Override + public void setOperatePermissions(final PermissionsDTO operatePermissions) { + this.operatePermissions = operatePermissions; + } + + @Schema(description = "The status of the connector.") + public ConnectorStatusDTO getStatus() { + return status; + } + + public void setStatus(final ConnectorStatusDTO status) { + this.status = status; + } +} + + diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorPropertyAllowableValuesEntity.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorPropertyAllowableValuesEntity.java new file mode 100644 index 000000000000..a2be3a0483cf --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorPropertyAllowableValuesEntity.java @@ -0,0 +1,71 @@ +/* + * 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 io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlRootElement; + +import java.util.List; + +/** + * Entity for holding allowable values for a connector property. + */ +@XmlRootElement(name = "connectorPropertyAllowableValuesEntity") +public class ConnectorPropertyAllowableValuesEntity extends Entity { + + private String configurationStepName; + private String propertyGroupName; + private String propertyName; + private List allowableValues; + + @Schema(description = "The name of the configuration step.") + public String getConfigurationStepName() { + return configurationStepName; + } + + public void setConfigurationStepName(final String configurationStepName) { + this.configurationStepName = configurationStepName; + } + + @Schema(description = "The name of the property group.") + public String getPropertyGroupName() { + return propertyGroupName; + } + + public void setPropertyGroupName(final String propertyGroupName) { + this.propertyGroupName = propertyGroupName; + } + + @Schema(description = "The name of the property.") + public String getPropertyName() { + return propertyName; + } + + public void setPropertyName(final String propertyName) { + this.propertyName = propertyName; + } + + @Schema(description = "The allowable values for the property.") + public List getAllowableValues() { + return allowableValues; + } + + public void setAllowableValues(final List allowableValues) { + this.allowableValues = allowableValues; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorRunStatusEntity.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorRunStatusEntity.java new file mode 100644 index 000000000000..cd08a7fa7603 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorRunStatusEntity.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.web.api.entity; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlType; + +@XmlType(name = "connectorRunStatus") +public class ConnectorRunStatusEntity extends ComponentRunStatusEntity { + private static final String[] SUPPORTED_STATE = {"RUNNING", "STOPPED", "DISABLED"}; + + @Override + protected String[] getSupportedState() { + return SUPPORTED_STATE; + } + + @Schema(description = "The run status of the Connector.", allowableValues = {"RUNNING", "STOPPED", "DISABLED"}) + @Override + public String getState() { + return super.getState(); + } +} + + diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorTypesEntity.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorTypesEntity.java new file mode 100644 index 000000000000..3192d3cab6dc --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorTypesEntity.java @@ -0,0 +1,44 @@ +/* + * 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 jakarta.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.DocumentedTypeDTO; + +import java.util.Set; + +/** + * A serialized representation of this class can be placed in the entity body of a response to the API. This particular entity holds a reference to a list of connector types. + */ +@XmlRootElement(name = "connectorTypesEntity") +public class ConnectorTypesEntity extends Entity { + + private Set connectorTypes; + + /** + * @return list of connector types that are being serialized + */ + public Set getConnectorTypes() { + return connectorTypes; + } + + public void setConnectorTypes(final Set connectorTypes) { + this.connectorTypes = connectorTypes; + } + +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorsEntity.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorsEntity.java new file mode 100644 index 000000000000..46219d6b1261 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectorsEntity.java @@ -0,0 +1,55 @@ +/* + * 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 io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlRootElement; +import jakarta.xml.bind.annotation.XmlType; +import jakarta.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +import java.util.Date; +import java.util.Set; + +@XmlRootElement(name = "connectorsEntity") +@XmlType(name = "connectorsEntity") +public class ConnectorsEntity extends Entity { + private Date currentTime; + private Set connectors; + + @XmlJavaTypeAdapter(TimeAdapter.class) + @Schema(description = "The current time on the system.", + type = "string" + ) + public Date getCurrentTime() { + return currentTime; + } + + public void setCurrentTime(final Date currentTime) { + this.currentTime = currentTime; + } + + public Set getConnectors() { + return connectors; + } + + public void setConnectors(final Set connectors) { + this.connectors = connectors; + } +} + + diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/CurrentUserEntity.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/CurrentUserEntity.java index 0756eaf66dd2..8bc5b65878a3 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/CurrentUserEntity.java +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/CurrentUserEntity.java @@ -40,6 +40,7 @@ public class CurrentUserEntity extends Entity { private PermissionsDTO policiesPermissions; private PermissionsDTO systemPermissions; private PermissionsDTO parameterContextPermissions; + private PermissionsDTO connectorsPermissions; private PermissionsDTO restrictedComponentsPermissions; private Set componentRestrictionPermissions; @@ -165,6 +166,18 @@ public void setParameterContextPermissions(PermissionsDTO parameterContextPermis this.parameterContextPermissions = parameterContextPermissions; } + /** + * @return permissions for accessing connectors + */ + @Schema(description = "Permissions for accessing connectors.") + public PermissionsDTO getConnectorsPermissions() { + return connectorsPermissions; + } + + public void setConnectorsPermissions(PermissionsDTO connectorsPermissions) { + this.connectorsPermissions = connectorsPermissions; + } + /** * @return permissions for accessing the restricted components */ diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/NarDetailsEntity.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/NarDetailsEntity.java index 799677beeea1..35c4735e47b8 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/NarDetailsEntity.java +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/NarDetailsEntity.java @@ -36,6 +36,7 @@ public class NarDetailsEntity extends Entity { private Set parameterProviderTypes; private Set flowRegistryClientTypes; private Set flowAnalysisRuleTypes; + private Set connectorTypes; @Schema(description = "The NAR summary") public NarSummaryDTO getNarSummary() { @@ -108,4 +109,13 @@ public Set getFlowAnalysisRuleTypes() { public void setFlowAnalysisRuleTypes(final Set flowAnalysisRuleTypes) { this.flowAnalysisRuleTypes = flowAnalysisRuleTypes; } + + @Schema(description = "The Connector types contained in the NAR") + public Set getConnectorTypes() { + return connectorTypes; + } + + public void setConnectorTypes(final Set connectorTypes) { + this.connectorTypes = connectorTypes; + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/SecretsEntity.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/SecretsEntity.java new file mode 100644 index 000000000000..0706dde05c97 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/SecretsEntity.java @@ -0,0 +1,43 @@ +/* + * 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 io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.SecretDTO; + +import java.util.List; + +/** + * A serialized representation of this class can be placed in the entity body of a response to the API. + * This particular entity holds a list of secrets. + */ +@XmlRootElement(name = "secretsEntity") +public class SecretsEntity extends Entity { + + private List secrets; + + @Schema(description = "The list of secrets available from all secret providers.") + public List getSecrets() { + return secrets; + } + + public void setSecrets(final List secrets) { + this.secrets = secrets; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/StepDocumentationEntity.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/StepDocumentationEntity.java new file mode 100644 index 000000000000..e01da34a1dcd --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/StepDocumentationEntity.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.api.entity; + +import jakarta.xml.bind.annotation.XmlRootElement; + +@XmlRootElement(name = "stepDocumentationEntity") +public class StepDocumentationEntity extends Entity { + + private String stepDocumentation; + + public String getStepDocumentation() { + return stepDocumentation; + } + + public void setStepDocumentation(final String stepDocumentation) { + this.stepDocumentation = stepDocumentation; + } + +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VerifyConnectorConfigStepRequestEntity.java b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VerifyConnectorConfigStepRequestEntity.java new file mode 100644 index 000000000000..cacfe93c3306 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VerifyConnectorConfigStepRequestEntity.java @@ -0,0 +1,37 @@ +/* + * 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 io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.VerifyConnectorConfigStepRequestDTO; + +@XmlRootElement(name = "verifyConnectorConfigStepRequestEntity") +public class VerifyConnectorConfigStepRequestEntity extends Entity { + + private VerifyConnectorConfigStepRequestDTO request; + + @Schema(description = "The verification request") + public VerifyConnectorConfigStepRequestDTO getRequest() { + return request; + } + + public void setRequest(final VerifyConnectorConfigStepRequestDTO request) { + this.request = request; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/ReconstitutedSerializedRepositoryRecord.java b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/ReconstitutedSerializedRepositoryRecord.java index bda74586d8b6..55fc92024ada 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/ReconstitutedSerializedRepositoryRecord.java +++ b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/ReconstitutedSerializedRepositoryRecord.java @@ -24,12 +24,14 @@ public class ReconstitutedSerializedRepositoryRecord implements SerializedReposi private final RepositoryRecordType type; private final FlowFileRecord flowFile; private final String swapLocation; + private final String originalSwapLocation; private ReconstitutedSerializedRepositoryRecord(final Builder builder) { this.queueIdentifier = builder.queueIdentifier; this.type = builder.type; this.flowFile = builder.flowFile; this.swapLocation = builder.swapLocation; + this.originalSwapLocation = builder.originalSwapLocation; } @Override @@ -72,6 +74,11 @@ public String getSwapLocation() { return swapLocation; } + @Override + public String getOriginalSwapLocation() { + return originalSwapLocation; + } + @Override public String toString() { return "ReconstitutedSerializedRepositoryRecord[recordType=" + type + ", queueId=" + queueIdentifier + ", flowFileUuid=" + flowFile.getAttribute(CoreAttributes.UUID.key()) @@ -83,6 +90,7 @@ public static class Builder { private RepositoryRecordType type; private FlowFileRecord flowFile; private String swapLocation; + private String originalSwapLocation; public Builder queueIdentifier(final String queueIdentifier) { this.queueIdentifier = queueIdentifier; @@ -104,6 +112,11 @@ public Builder swapLocation(final String swapLocation) { return this; } + public Builder originalSwapLocation(final String originalSwapLocation) { + this.originalSwapLocation = originalSwapLocation; + return this; + } + public ReconstitutedSerializedRepositoryRecord build() { return new ReconstitutedSerializedRepositoryRecord(this); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java index b3faad6c2811..dbee510fa7b0 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java +++ b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java @@ -24,7 +24,11 @@ public abstract class RepositoryRecordSerde implements SerDe UpdateType.UPDATE; case SWAP_OUT -> UpdateType.SWAP_OUT; case SWAP_IN -> UpdateType.SWAP_IN; + case SWAP_FILE_DELETED -> UpdateType.SWAP_FILE_DELETED; + case SWAP_FILE_RENAMED -> UpdateType.SWAP_FILE_RENAMED; default -> null; }; } @@ -43,4 +49,9 @@ public UpdateType getUpdateType(final SerializedRepositoryRecord record) { public String getLocation(final SerializedRepositoryRecord record) { return record.getSwapLocation(); } + + @Override + public String getOriginalLocation(final SerializedRepositoryRecord record) { + return record.getOriginalSwapLocation(); + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java index 767fe5d0d45f..b2ea001708b9 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java +++ b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java @@ -44,9 +44,9 @@ import java.util.Map; public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implements SerDe { - private static final int MAX_ENCODING_VERSION = 2; + private static final int MAX_ENCODING_VERSION = 3; - private final RecordSchema writeSchema = RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V2; + private final RecordSchema writeSchema = RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V3; private final RecordSchema contentClaimSchema = ContentClaimSchema.CONTENT_CLAIM_SCHEMA_V1; private final ResourceClaimManager resourceClaimManager; @@ -76,11 +76,13 @@ public void serializeRecord(final SerializedRepositoryRecord record, final DataO case CONTENTMISSING, DELETE -> RepositoryRecordSchema.DELETE_SCHEMA_V2; case SWAP_IN -> RepositoryRecordSchema.SWAP_IN_SCHEMA_V2; case SWAP_OUT -> RepositoryRecordSchema.SWAP_OUT_SCHEMA_V2; + case SWAP_FILE_DELETED -> RepositoryRecordSchema.SWAP_FILE_DELETED_SCHEMA_V3; + case SWAP_FILE_RENAMED -> RepositoryRecordSchema.SWAP_FILE_RENAMED_SCHEMA_V3; default -> throw new IllegalArgumentException("Received Repository Record with unknown Update Type: " + record.getType()); // won't happen. }; - serializeRecord(record, out, schema, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V2); + serializeRecord(record, out, schema, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V3); } protected void serializeRecord(final SerializedRepositoryRecord record, final DataOutputStream out, RecordSchema schema, RecordSchema repositoryRecordSchema) throws IOException { @@ -163,6 +165,8 @@ private SerializedRepositoryRecord createRepositoryRecord(final Record updateRec case SWAP_IN -> swapInRecord(record); case SWAP_OUT -> swapOutRecord(record); case UPDATE -> updateRecord(record); + case SWAP_FILE_DELETED -> swapFileDeletedRecord(record); + case SWAP_FILE_RENAMED -> swapFileRenamedRecord(record); default -> throw new IOException("Found unrecognized Update Type '" + actionType + "'"); }; @@ -254,6 +258,30 @@ private SerializedRepositoryRecord swapOutRecord(final Record record) { .build(); } + private SerializedRepositoryRecord swapFileDeletedRecord(final Record record) { + final String queueId = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE)); + final String swapLocation = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE)); + + return new ReconstitutedSerializedRepositoryRecord.Builder() + .type(RepositoryRecordType.SWAP_FILE_DELETED) + .swapLocation(swapLocation) + .queueIdentifier(queueId) + .build(); + } + + private SerializedRepositoryRecord swapFileRenamedRecord(final Record record) { + final String queueId = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE)); + final String originalSwapLocation = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.ORIGINAL_SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE)); + final String swapLocation = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE)); + + return new ReconstitutedSerializedRepositoryRecord.Builder() + .type(RepositoryRecordType.SWAP_FILE_RENAMED) + .originalSwapLocation(originalSwapLocation) + .swapLocation(swapLocation) + .queueIdentifier(queueId) + .build(); + } + @Override public int getVersion() { return MAX_ENCODING_VERSION; diff --git a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SerializedRepositoryRecord.java b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SerializedRepositoryRecord.java index 51eb034cd92f..5d87852fec8b 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SerializedRepositoryRecord.java +++ b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SerializedRepositoryRecord.java @@ -45,6 +45,14 @@ public interface SerializedRepositoryRecord { */ String getSwapLocation(); + /** + * @return For SWAP_FILE_RENAMED records, provides the original swap location before the rename. + * For other record types, returns null. + */ + default String getOriginalSwapLocation() { + return null; + } + /** * @return FlowFile being encapsulated by this record */ diff --git a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecordSerdeFactory.java b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecordSerdeFactory.java index d466ffa8d0ac..effcc57ef396 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecordSerdeFactory.java +++ b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecordSerdeFactory.java @@ -55,7 +55,11 @@ public SerDe createSerDe(final String encodingName) @Override public Long getRecordIdentifier(final SerializedRepositoryRecord record) { - return record.getFlowFileRecord().getId(); + final FlowFileRecord flowFileRecord = record.getFlowFileRecord(); + if (flowFileRecord == null) { + return null; + } + return flowFileRecord.getId(); } @Override @@ -66,6 +70,8 @@ public UpdateType getUpdateType(final SerializedRepositoryRecord record) { case UPDATE -> UpdateType.UPDATE; case SWAP_OUT -> UpdateType.SWAP_OUT; case SWAP_IN -> UpdateType.SWAP_IN; + case SWAP_FILE_DELETED -> UpdateType.SWAP_FILE_DELETED; + case SWAP_FILE_RENAMED -> UpdateType.SWAP_FILE_RENAMED; default -> null; }; } @@ -75,4 +81,9 @@ public String getLocation(final SerializedRepositoryRecord record) { return record.getSwapLocation(); } + @Override + public String getOriginalLocation(final SerializedRepositoryRecord record) { + return record.getOriginalSwapLocation(); + } + } diff --git a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java index 83c4b0791151..221b6860e1ab 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java +++ b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java @@ -39,15 +39,16 @@ public RepositoryRecordFieldMap(final SerializedRepositoryRecord record, final R public Object getFieldValue(final String fieldName) { return switch (fieldName) { case RepositoryRecordSchema.ACTION_TYPE -> record.getType().name(); - case RepositoryRecordSchema.RECORD_ID -> record.getFlowFileRecord().getId(); + case RepositoryRecordSchema.RECORD_ID -> flowFile == null ? null : flowFile.getId(); case RepositoryRecordSchema.SWAP_LOCATION -> record.getSwapLocation(); - case FlowFileSchema.ATTRIBUTES -> flowFile.getAttributes(); - case FlowFileSchema.ENTRY_DATE -> flowFile.getEntryDate(); - case FlowFileSchema.FLOWFILE_SIZE -> flowFile.getSize(); - case FlowFileSchema.LINEAGE_START_DATE -> flowFile.getLineageStartDate(); - case FlowFileSchema.LINEAGE_START_INDEX -> flowFile.getLineageStartIndex(); - case FlowFileSchema.QUEUE_DATE -> flowFile.getLastQueueDate(); - case FlowFileSchema.QUEUE_DATE_INDEX -> flowFile.getQueueDateIndex(); + case RepositoryRecordSchema.ORIGINAL_SWAP_LOCATION -> record.getOriginalSwapLocation(); + case FlowFileSchema.ATTRIBUTES -> flowFile == null ? null : flowFile.getAttributes(); + case FlowFileSchema.ENTRY_DATE -> flowFile == null ? null : flowFile.getEntryDate(); + case FlowFileSchema.FLOWFILE_SIZE -> flowFile == null ? null : flowFile.getSize(); + case FlowFileSchema.LINEAGE_START_DATE -> flowFile == null ? null : flowFile.getLineageStartDate(); + case FlowFileSchema.LINEAGE_START_INDEX -> flowFile == null ? null : flowFile.getLineageStartIndex(); + case FlowFileSchema.QUEUE_DATE -> flowFile == null ? null : flowFile.getLastQueueDate(); + case FlowFileSchema.QUEUE_DATE_INDEX -> flowFile == null ? null : flowFile.getQueueDateIndex(); case FlowFileSchema.CONTENT_CLAIM -> { final ContentClaimFieldMap contentClaimFieldMap = record.getContentClaim() == null ? null : new ContentClaimFieldMap(record.getContentClaim(), record.getClaimOffset(), contentClaimSchema); diff --git a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordSchema.java b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordSchema.java index db77c8baf487..935a3046251b 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordSchema.java +++ b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordSchema.java @@ -32,18 +32,22 @@ public class RepositoryRecordSchema { public static final String REPOSITORY_RECORD_UPDATE_V1 = "Repository Record Update"; // top level field name public static final String REPOSITORY_RECORD_UPDATE_V2 = "Repository Record Update"; // top level field name + public static final String REPOSITORY_RECORD_UPDATE_V3 = "Repository Record Update"; // top level field name // repository record fields public static final String ACTION_TYPE = "Action"; public static final String RECORD_ID = "Record ID"; public static final String QUEUE_IDENTIFIER = "Queue Identifier"; public static final String SWAP_LOCATION = "Swap Location"; + public static final String ORIGINAL_SWAP_LOCATION = "Original Swap Location"; // Update types public static final String CREATE_OR_UPDATE_ACTION = "Create or Update"; public static final String DELETE_ACTION = "Delete"; public static final String SWAP_IN_ACTION = "Swap In"; public static final String SWAP_OUT_ACTION = "Swap Out"; + public static final String SWAP_FILE_DELETED_ACTION = "Swap File Deleted"; + public static final String SWAP_FILE_RENAMED_ACTION = "Swap File Renamed"; public static final RecordSchema REPOSITORY_RECORD_SCHEMA_V1; public static final RecordSchema CREATE_OR_UPDATE_SCHEMA_V1; @@ -57,6 +61,10 @@ public class RepositoryRecordSchema { public static final RecordSchema SWAP_IN_SCHEMA_V2; public static final RecordSchema SWAP_OUT_SCHEMA_V2; + public static final RecordSchema REPOSITORY_RECORD_SCHEMA_V3; + public static final RecordSchema SWAP_FILE_DELETED_SCHEMA_V3; + public static final RecordSchema SWAP_FILE_RENAMED_SCHEMA_V3; + public static final RecordField ACTION_TYPE_FIELD = new SimpleRecordField(ACTION_TYPE, FieldType.STRING, Repetition.EXACTLY_ONE); public static final RecordField RECORD_ID_FIELD = new SimpleRecordField(RECORD_ID, FieldType.LONG, Repetition.EXACTLY_ONE); @@ -132,7 +140,62 @@ public class RepositoryRecordSchema { SWAP_IN_SCHEMA_V2 = new RecordSchema(swapInFields); // Union Field that creates the top-level field type - final UnionRecordField repoUpdateField = new UnionRecordField(REPOSITORY_RECORD_UPDATE_V2, Repetition.EXACTLY_ONE, createOrUpdate, delete, swapOut, swapIn); + final UnionRecordField repoUpdateField = new UnionRecordField(REPOSITORY_RECORD_UPDATE_V2, Repetition.EXACTLY_ONE, + createOrUpdate, delete, swapOut, swapIn); REPOSITORY_RECORD_SCHEMA_V2 = new RecordSchema(Collections.singletonList(repoUpdateField)); } + + static { + // V3 reuses V2 schemas for Create/Update, Delete, Swap Out, Swap In + // and adds new schemas for Swap File Deleted and Swap File Renamed + + // Fields for "Create" or "Update" records (same as V2) + final List createOrUpdateFields = new ArrayList<>(); + createOrUpdateFields.add(ACTION_TYPE_FIELD); + createOrUpdateFields.addAll(FlowFileSchema.FLOWFILE_SCHEMA_V2.getFields()); + createOrUpdateFields.add(new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE)); + createOrUpdateFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.ZERO_OR_ONE)); + final ComplexRecordField createOrUpdate = new ComplexRecordField(CREATE_OR_UPDATE_ACTION, Repetition.EXACTLY_ONE, createOrUpdateFields); + + // Fields for "Delete" records (same as V2) + final List deleteFields = new ArrayList<>(); + deleteFields.add(ACTION_TYPE_FIELD); + deleteFields.add(RECORD_ID_FIELD); + final ComplexRecordField delete = new ComplexRecordField(DELETE_ACTION, Repetition.EXACTLY_ONE, deleteFields); + + // Fields for "Swap Out" records (same as V2) + final List swapOutFields = new ArrayList<>(); + swapOutFields.add(ACTION_TYPE_FIELD); + swapOutFields.add(RECORD_ID_FIELD); + swapOutFields.add(new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE)); + swapOutFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE)); + final ComplexRecordField swapOut = new ComplexRecordField(SWAP_OUT_ACTION, Repetition.EXACTLY_ONE, swapOutFields); + + // Fields for "Swap In" records (same as V2) + final List swapInFields = new ArrayList<>(createOrUpdateFields); + swapInFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE)); + final ComplexRecordField swapIn = new ComplexRecordField(SWAP_IN_ACTION, Repetition.EXACTLY_ONE, swapInFields); + + // Fields for "Swap File Deleted" records (new in V3) + final List swapFileDeletedFields = new ArrayList<>(); + swapFileDeletedFields.add(ACTION_TYPE_FIELD); + swapFileDeletedFields.add(new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE)); + swapFileDeletedFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE)); + final ComplexRecordField swapFileDeleted = new ComplexRecordField(SWAP_FILE_DELETED_ACTION, Repetition.EXACTLY_ONE, swapFileDeletedFields); + SWAP_FILE_DELETED_SCHEMA_V3 = new RecordSchema(swapFileDeletedFields); + + // Fields for "Swap File Renamed" records (new in V3) + final List swapFileRenamedFields = new ArrayList<>(); + swapFileRenamedFields.add(ACTION_TYPE_FIELD); + swapFileRenamedFields.add(new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE)); + swapFileRenamedFields.add(new SimpleRecordField(ORIGINAL_SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE)); + swapFileRenamedFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE)); + final ComplexRecordField swapFileRenamed = new ComplexRecordField(SWAP_FILE_RENAMED_ACTION, Repetition.EXACTLY_ONE, swapFileRenamedFields); + SWAP_FILE_RENAMED_SCHEMA_V3 = new RecordSchema(swapFileRenamedFields); + + // Union Field that creates the top-level field type (V3 includes all V2 types plus new swap file types) + final UnionRecordField repoUpdateField = new UnionRecordField(REPOSITORY_RECORD_UPDATE_V3, Repetition.EXACTLY_ONE, + createOrUpdate, delete, swapOut, swapIn, swapFileDeleted, swapFileRenamed); + REPOSITORY_RECORD_SCHEMA_V3 = new RecordSchema(Collections.singletonList(repoUpdateField)); + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java index d93b23d73e97..5812561f45c8 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java +++ b/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java @@ -21,7 +21,6 @@ import org.apache.nifi.repository.schema.NamedValue; import org.apache.nifi.repository.schema.Record; import org.apache.nifi.repository.schema.RecordSchema; -import org.wali.UpdateType; public class RepositoryRecordUpdate implements Record { private final RecordSchema schema; @@ -40,17 +39,17 @@ public RecordSchema getSchema() { @Override public Object getFieldValue(final String fieldName) { if (RepositoryRecordSchema.REPOSITORY_RECORD_UPDATE_V2.equals(fieldName)) { - String actionType = (String) fieldMap.getFieldValue(RepositoryRecordSchema.ACTION_TYPE); - if (RepositoryRecordType.CONTENTMISSING.name().equals(actionType)) { - actionType = RepositoryRecordType.DELETE.name(); - } - final UpdateType updateType = UpdateType.valueOf(actionType); + final String actionType = (String) fieldMap.getFieldValue(RepositoryRecordSchema.ACTION_TYPE); + final RepositoryRecordType recordType = RepositoryRecordType.valueOf(actionType); - final String actionName = switch (updateType) { + final String actionName = switch (recordType) { case CREATE, UPDATE -> RepositoryRecordSchema.CREATE_OR_UPDATE_ACTION; - case DELETE -> RepositoryRecordSchema.DELETE_ACTION; + case DELETE, CONTENTMISSING -> RepositoryRecordSchema.DELETE_ACTION; case SWAP_IN -> RepositoryRecordSchema.SWAP_IN_ACTION; case SWAP_OUT -> RepositoryRecordSchema.SWAP_OUT_ACTION; + case SWAP_FILE_DELETED -> RepositoryRecordSchema.SWAP_FILE_DELETED_ACTION; + case SWAP_FILE_RENAMED -> RepositoryRecordSchema.SWAP_FILE_RENAMED_ACTION; + default -> throw new IllegalArgumentException("Unknown record type: " + recordType); }; return new NamedValue(actionName, fieldMap); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceFactory.java b/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceFactory.java index 1dacb0fa37d9..ff74a2cceaab 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceFactory.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceFactory.java @@ -298,6 +298,24 @@ public String getSafeDescription() { } }; + private static final Resource CONNECTORS_RESOURCE = new Resource() { + + @Override + public String getIdentifier() { + return ResourceType.Connector.getValue(); + } + + @Override + public String getName() { + return "Connectors"; + } + + @Override + public String getSafeDescription() { + return "connectors"; + } + }; + /** * Gets the Resource for accessing the Controller. This includes Controller level configuration, bulletins, reporting tasks, and the cluster. * @@ -341,6 +359,14 @@ public static Resource getParameterProvidersResource() { return PARAMETER_PROVIDERS_RESOURCE; } + /** + * Gets the Resource for accessing Connectors. + * @return The resource for access Connectors + */ + public static Resource getConnectorsResource() { + return CONNECTORS_RESOURCE; + } + /** * Gets the Resource for accessing provenance. Access to this Resource allows the user to access data provenance. However, additional authorization * is required based on the component that generated the event and the attributes of the event. @@ -559,6 +585,7 @@ public String getSafeDescription() { case Label -> "Label"; case ParameterContext -> "Parameter Context"; case ParameterProvider -> "Parameter Provider"; + case Connector -> "Connector"; default -> "Component"; }; return componentType + " with ID " + identifier; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceType.java b/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceType.java index 0856573c9d13..33a29d880b49 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceType.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-authorization/src/main/java/org/apache/nifi/authorization/resource/ResourceType.java @@ -44,7 +44,8 @@ public enum ResourceType { Operation("/operation"), Tenant("/tenants"), ParameterContext("/parameter-contexts"), - ParameterProvider("/parameter-providers"); + ParameterProvider("/parameter-providers"), + Connector("/connectors"); final String value; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java index f16d4eedc2d7..efa017ae5f77 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java @@ -26,6 +26,12 @@ import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionStatusEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionsEndpointMerger; +import org.apache.nifi.cluster.coordination.http.endpoints.ConnectorEndpointMerger; +import org.apache.nifi.cluster.coordination.http.endpoints.ConnectorFlowEndpointMerger; +import org.apache.nifi.cluster.coordination.http.endpoints.ConnectorPropertyGroupEndpointMerger; +import org.apache.nifi.cluster.coordination.http.endpoints.ConnectorPropertyGroupNamesEndpointMerger; +import org.apache.nifi.cluster.coordination.http.endpoints.ConnectorStatusEndpointMerger; +import org.apache.nifi.cluster.coordination.http.endpoints.ConnectorsEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ControllerBulletinsEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ControllerConfigurationEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ControllerEndpointMerger; @@ -98,6 +104,7 @@ import org.apache.nifi.cluster.coordination.http.endpoints.UserGroupsEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.UsersEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.VerifyConfigEndpointMerger; +import org.apache.nifi.cluster.coordination.http.endpoints.VerifyConnectorConfigStepEndpointMerger; import org.apache.nifi.cluster.manager.NodeResponse; import org.apache.nifi.stream.io.NullOutputStream; import org.apache.nifi.util.FormatUtils; @@ -138,6 +145,13 @@ public StandardHttpResponseMapper(final NiFiProperties nifiProperties) { endpointMergers.add(new ProcessorEndpointMerger()); endpointMergers.add(new ProcessorsEndpointMerger()); endpointMergers.add(new ProcessorRunStatusDetailsEndpointMerger()); + endpointMergers.add(new ConnectorEndpointMerger()); + endpointMergers.add(new ConnectorsEndpointMerger()); + endpointMergers.add(new ConnectorStatusEndpointMerger()); + endpointMergers.add(new ConnectorFlowEndpointMerger()); + endpointMergers.add(new ConnectorPropertyGroupEndpointMerger()); + endpointMergers.add(new ConnectorPropertyGroupNamesEndpointMerger()); + endpointMergers.add(new VerifyConnectorConfigStepEndpointMerger()); endpointMergers.add(new ConnectionEndpointMerger()); endpointMergers.add(new ConnectionsEndpointMerger()); endpointMergers.add(new PortEndpointMerger()); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AssetsEndpointMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AssetsEndpointMerger.java index 422ade1492d0..bcf5e0e06605 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AssetsEndpointMerger.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AssetsEndpointMerger.java @@ -31,7 +31,7 @@ public class AssetsEndpointMerger extends AbstractSingleDTOEndpoint> { - private static final Pattern ASSETS_URI = Pattern.compile("/nifi-api/parameter-contexts/[a-f0-9\\-]{36}/assets"); + private static final Pattern ASSETS_URI = Pattern.compile("/nifi-api/(parameter-contexts|connectors)/[a-f0-9\\-]{36}/assets"); @Override public boolean canHandle(final URI uri, final String method) { diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java index 60686c536973..02e06b93d85b 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java @@ -38,6 +38,8 @@ public class ComponentStateEndpointMerger extends AbstractSingleDTOEndpoint implements EndpointResponseMerger { + public static final Pattern CONNECTORS_URI_PATTERN = Pattern.compile("/nifi-api/connectors"); + public static final Pattern CONNECTOR_URI_PATTERN = Pattern.compile("/nifi-api/connectors/[a-f0-9\\-]{36}"); + public static final Pattern CONNECTOR_RUN_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/connectors/[a-f0-9\\-]{36}/run-status"); + + @Override + public boolean canHandle(final URI uri, final String method) { + if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && (CONNECTOR_URI_PATTERN.matcher(uri.getPath()).matches())) { + return true; + } else if ("PUT".equalsIgnoreCase(method) && CONNECTOR_RUN_STATUS_URI_PATTERN.matcher(uri.getPath()).matches()) { + return true; + } else if ("POST".equalsIgnoreCase(method) && CONNECTORS_URI_PATTERN.matcher(uri.getPath()).matches()) { + return true; + } + + return false; + } + + @Override + protected Class getEntityClass() { + return ConnectorEntity.class; + } + + @Override + protected void mergeResponses(final ConnectorEntity clientEntity, final Map entityMap, final Set successfulResponses, + final Set problematicResponses) { + + ConnectorEntityMerger.merge(clientEntity, entityMap); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorFlowEndpointMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorFlowEndpointMerger.java new file mode 100644 index 000000000000..a933ac469605 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorFlowEndpointMerger.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.cluster.coordination.http.endpoints; + +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO; +import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity; + +import java.net.URI; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +public class ConnectorFlowEndpointMerger extends AbstractSingleDTOEndpoint { + public static final Pattern CONNECTOR_FLOW_URI_PATTERN = Pattern.compile("/nifi-api/connectors/[a-f0-9\\-]{36}/flow"); + + @Override + public boolean canHandle(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && CONNECTOR_FLOW_URI_PATTERN.matcher(uri.getPath()).matches(); + } + + @Override + protected Class getEntityClass() { + return ProcessGroupFlowEntity.class; + } + + @Override + protected ProcessGroupFlowDTO getDto(final ProcessGroupFlowEntity entity) { + return entity.getProcessGroupFlow(); + } + + @Override + protected void mergeResponses(final ProcessGroupFlowDTO clientDto, final Map dtoMap, + final Set successfulResponses, final Set problematicResponses) { + + // Delegate to the shared flow merging logic + FlowMerger.mergeProcessGroupFlowDto(clientDto, dtoMap); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorPropertyGroupEndpointMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorPropertyGroupEndpointMerger.java new file mode 100644 index 000000000000..bfda6dcab871 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorPropertyGroupEndpointMerger.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.cluster.coordination.http.endpoints; + +import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger; +import org.apache.nifi.cluster.manager.ConfigurationStepEntityMerger; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.entity.ConfigurationStepEntity; + +import java.net.URI; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +public class ConnectorPropertyGroupEndpointMerger extends AbstractSingleEntityEndpoint implements EndpointResponseMerger { + public static final Pattern CONNECTOR_PROPERTY_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/connectors/[a-f0-9\\-]{36}/configuration-steps/[^/]+"); + + @Override + public boolean canHandle(final URI uri, final String method) { + return ("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) + && CONNECTOR_PROPERTY_GROUP_URI_PATTERN.matcher(uri.getPath()).matches(); + } + + @Override + protected Class getEntityClass() { + return ConfigurationStepEntity.class; + } + + @Override + protected void mergeResponses(final ConfigurationStepEntity clientEntity, final Map entityMap, + final Set successfulResponses, final Set problematicResponses) { + + ConfigurationStepEntityMerger.merge(clientEntity, entityMap); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorPropertyGroupNamesEndpointMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorPropertyGroupNamesEndpointMerger.java new file mode 100644 index 000000000000..fdc932c1bbf5 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorPropertyGroupNamesEndpointMerger.java @@ -0,0 +1,63 @@ +/* + * 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 org.apache.nifi.cluster.coordination.http.EndpointResponseMerger; +import org.apache.nifi.cluster.manager.ConfigurationStepNamesEntityMerger; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.entity.ConfigurationStepNamesEntity; + +import java.net.URI; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +public class ConnectorPropertyGroupNamesEndpointMerger implements EndpointResponseMerger { + public static final Pattern CONNECTOR_PROPERTY_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/connectors/[a-f0-9\\-]{36}/configuration-steps"); + + @Override + public boolean canHandle(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && CONNECTOR_PROPERTY_GROUPS_URI_PATTERN.matcher(uri.getPath()).matches(); + } + + @Override + public final NodeResponse merge(final URI uri, final String method, final Set successfulResponses, + final Set problematicResponses, final NodeResponse clientResponse) { + if (!canHandle(uri, method)) { + throw new IllegalArgumentException("Cannot use Endpoint Mapper of type " + getClass().getSimpleName() + + " to map responses for URI " + uri + ", HTTP Method " + method); + } + + final ConfigurationStepNamesEntity responseEntity = clientResponse.getClientResponse().readEntity(ConfigurationStepNamesEntity.class); + + final Map entityMap = new HashMap<>(); + for (final NodeResponse nodeResponse : successfulResponses) { + final ConfigurationStepNamesEntity nodeResponseEntity = nodeResponse == clientResponse + ? responseEntity + : nodeResponse.getClientResponse().readEntity(ConfigurationStepNamesEntity.class); + entityMap.put(nodeResponse.getNodeId(), nodeResponseEntity); + } + + ConfigurationStepNamesEntityMerger.merge(responseEntity, entityMap); + + // create a new client response + return new NodeResponse(clientResponse, responseEntity); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorStatusEndpointMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorStatusEndpointMerger.java new file mode 100644 index 000000000000..bff04b3fe719 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorStatusEndpointMerger.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.cluster.coordination.http.endpoints; + +import org.apache.nifi.cluster.manager.ComponentEntityStatusMerger; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.manager.StatusMerger; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.dto.status.NodeProcessGroupStatusSnapshotDTO; +import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO; +import org.apache.nifi.web.api.entity.ProcessGroupStatusEntity; + +import java.net.URI; +import java.util.ArrayList; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +public class ConnectorStatusEndpointMerger extends AbstractSingleEntityEndpoint implements ComponentEntityStatusMerger { + public static final Pattern CONNECTOR_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/connectors/[a-f0-9\\-]{36}/status"); + + @Override + public boolean canHandle(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && CONNECTOR_STATUS_URI_PATTERN.matcher(uri.getPath()).matches(); + } + + @Override + protected Class getEntityClass() { + return ProcessGroupStatusEntity.class; + } + + @Override + protected void mergeResponses(final ProcessGroupStatusEntity clientEntity, final Map entityMap, + final Set successfulResponses, final Set problematicResponses) { + final ProcessGroupStatusDTO mergedProcessGroupStatus = clientEntity.getProcessGroupStatus(); + mergedProcessGroupStatus.setNodeSnapshots(new ArrayList<>()); + + final NodeIdentifier selectedNodeId = entityMap.entrySet().stream() + .filter(e -> e.getValue() == clientEntity) + .map(Map.Entry::getKey) + .findFirst() + .orElse(null); + + final NodeProcessGroupStatusSnapshotDTO selectedNodeSnapshot = new NodeProcessGroupStatusSnapshotDTO(); + selectedNodeSnapshot.setStatusSnapshot(mergedProcessGroupStatus.getAggregateSnapshot().clone()); + selectedNodeSnapshot.setAddress(selectedNodeId.getApiAddress()); + selectedNodeSnapshot.setApiPort(selectedNodeId.getApiPort()); + selectedNodeSnapshot.setNodeId(selectedNodeId.getId()); + + mergedProcessGroupStatus.getNodeSnapshots().add(selectedNodeSnapshot); + + for (final Map.Entry entry : entityMap.entrySet()) { + final NodeIdentifier nodeId = entry.getKey(); + final ProcessGroupStatusEntity nodeProcessGroupStatusEntity = entry.getValue(); + final ProcessGroupStatusDTO nodeProcessGroupStatus = nodeProcessGroupStatusEntity.getProcessGroupStatus(); + if (nodeProcessGroupStatus == mergedProcessGroupStatus) { + continue; + } + + mergeStatus(mergedProcessGroupStatus, clientEntity.getCanRead(), nodeProcessGroupStatus, nodeProcessGroupStatusEntity.getCanRead(), nodeId); + } + } + + @Override + public void mergeStatus(final ProcessGroupStatusDTO clientStatus, final boolean clientStatusReadablePermission, + final ProcessGroupStatusDTO status, final boolean statusReadablePermission, + final NodeIdentifier statusNodeIdentifier) { + StatusMerger.merge(clientStatus, clientStatusReadablePermission, status, statusReadablePermission, + statusNodeIdentifier.getId(), statusNodeIdentifier.getApiAddress(), statusNodeIdentifier.getApiPort()); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorsEndpointMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorsEndpointMerger.java new file mode 100644 index 000000000000..9d89fb3b2d4d --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ConnectorsEndpointMerger.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.cluster.coordination.http.endpoints; + +import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger; +import org.apache.nifi.cluster.manager.ConnectorsEntityMerger; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ConnectorsEntity; + +import java.net.URI; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +public class ConnectorsEndpointMerger implements EndpointResponseMerger { + public static final Pattern CONNECTORS_URI_PATTERN = Pattern.compile("/nifi-api/connectors"); + + @Override + public boolean canHandle(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && CONNECTORS_URI_PATTERN.matcher(uri.getPath()).matches(); + } + + @Override + public final NodeResponse merge(final URI uri, final String method, final Set successfulResponses, final Set problematicResponses, final NodeResponse clientResponse) { + if (!canHandle(uri, method)) { + throw new IllegalArgumentException("Cannot use Endpoint Mapper of type " + getClass().getSimpleName() + " to map responses for URI " + uri + ", HTTP Method " + method); + } + + final ConnectorsEntity responseEntity = clientResponse.getClientResponse().readEntity(ConnectorsEntity.class); + final Set connectorEntities = responseEntity.getConnectors(); + + final Map> entityMap = new HashMap<>(); + for (final NodeResponse nodeResponse : successfulResponses) { + final ConnectorsEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().readEntity(ConnectorsEntity.class); + final Set nodeConnectorEntities = nodeResponseEntity.getConnectors(); + + for (final ConnectorEntity nodeConnectorEntity : nodeConnectorEntities) { + Map innerMap = entityMap.computeIfAbsent(nodeConnectorEntity.getId(), k -> new HashMap<>()); + innerMap.put(nodeResponse.getNodeId(), nodeConnectorEntity); + } + } + + ConnectorsEntityMerger.mergeConnectors(connectorEntities, entityMap); + + // create a new client response + return new NodeResponse(clientResponse, responseEntity); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/CurrentUserEndpointMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/CurrentUserEndpointMerger.java index f0344232a0c6..24754601bf39 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/CurrentUserEndpointMerger.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/CurrentUserEndpointMerger.java @@ -55,6 +55,7 @@ protected void mergeResponses(final CurrentUserEntity clientEntity, final Map clientEntityComponentRestrictionsPermissions = clientEntity.getComponentRestrictionPermissions(); final Set entityComponentRestrictionsPermissions = entity.getComponentRestrictionPermissions(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowMerger.java index ad15fa126954..cd84ddcdcec9 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowMerger.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowMerger.java @@ -65,6 +65,17 @@ protected ProcessGroupFlowDTO getDto(final ProcessGroupFlowEntity entity) { protected void mergeResponses(final ProcessGroupFlowDTO clientDto, final Map dtoMap, final Set successfulResponses, final Set problematicResponses) { + mergeProcessGroupFlowDto(clientDto, dtoMap); + } + + /** + * Merges the ProcessGroupFlowDTO responses from all nodes into the client DTO. + * This method is package-private to allow reuse by other endpoint mergers (e.g., ConnectorFlowEndpointMerger). + * + * @param clientDto the client DTO to merge responses into + * @param dtoMap the responses from all nodes + */ + static void mergeProcessGroupFlowDto(final ProcessGroupFlowDTO clientDto, final Map dtoMap) { final FlowDTO flowDto = clientDto.getFlow(); final Set clientConnections = flowDto.getConnections(); final Set clientProcessors = flowDto.getProcessors(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/VerifyConnectorConfigStepEndpointMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/VerifyConnectorConfigStepEndpointMerger.java new file mode 100644 index 000000000000..9f32b505617e --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/VerifyConnectorConfigStepEndpointMerger.java @@ -0,0 +1,72 @@ +/* + * 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 org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO; +import org.apache.nifi.web.api.dto.VerifyConnectorConfigStepRequestDTO; +import org.apache.nifi.web.api.entity.VerifyConnectorConfigStepRequestEntity; + +import java.net.URI; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +public class VerifyConnectorConfigStepEndpointMerger extends AbstractSingleEntityEndpoint { + public static final Pattern VERIFY_CONNECTOR_CONFIG_STEP_URI_PATTERN = + Pattern.compile("/nifi-api/connectors/[a-f0-9\\-]{36}/configuration-steps/[^/]+/verify-config(/[a-f0-9\\-]{36})?"); + + @Override + protected Class getEntityClass() { + return VerifyConnectorConfigStepRequestEntity.class; + } + + @Override + public boolean canHandle(final URI uri, final String method) { + return VERIFY_CONNECTOR_CONFIG_STEP_URI_PATTERN.matcher(uri.getPath()).matches(); + } + + @Override + protected void mergeResponses(final VerifyConnectorConfigStepRequestEntity clientEntity, + final Map entityMap, + final Set successfulResponses, + final Set problematicResponses) { + + final VerifyConnectorConfigStepRequestDTO requestDto = clientEntity.getRequest(); + final List results = requestDto.getResults(); + + if (results == null) { + return; + } + + final ConfigVerificationResultMerger resultMerger = new ConfigVerificationResultMerger(); + for (final Map.Entry entry : entityMap.entrySet()) { + final NodeIdentifier nodeId = entry.getKey(); + final VerifyConnectorConfigStepRequestEntity entity = entry.getValue(); + + final List nodeResults = entity.getRequest().getResults(); + resultMerger.addNodeResults(nodeId, nodeResults); + } + + final List aggregateResults = resultMerger.computeAggregateResults(); + + clientEntity.getRequest().setResults(aggregateResults); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ClusteredConnectorRequestReplicator.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ClusteredConnectorRequestReplicator.java new file mode 100644 index 000000000000..a3db59cf4f93 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ClusteredConnectorRequestReplicator.java @@ -0,0 +1,131 @@ +/* + * 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.replication; + +import jakarta.ws.rs.core.Response; +import jakarta.ws.rs.core.Response.Status; +import jakarta.ws.rs.core.Response.Status.Family; +import jakarta.ws.rs.core.Response.StatusType; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.authorization.user.StandardNiFiUser; +import org.apache.nifi.cluster.coordination.ClusterCoordinator; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.components.connector.ConnectorRequestReplicator; +import org.apache.nifi.components.connector.ConnectorState; +import org.apache.nifi.controller.flow.FlowManager; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.Entity; + +import java.io.IOException; +import java.net.URI; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.function.Supplier; + +public class ClusteredConnectorRequestReplicator implements ConnectorRequestReplicator { + private static final String GET = "GET"; + + private final Supplier requestReplicatorSupplier; + private final Supplier clusterCoordinatorSupplier; + private final String replicationScheme; + + public ClusteredConnectorRequestReplicator(final Supplier requestReplicatorSupplier, final Supplier clusterCoordinatorSupplier, + final boolean httpsEnabled) { + this.requestReplicatorSupplier = Objects.requireNonNull(requestReplicatorSupplier, "Request Replicator Supplier required"); + this.clusterCoordinatorSupplier = Objects.requireNonNull(clusterCoordinatorSupplier, "Cluster Coordinator Supplier required"); + this.replicationScheme = httpsEnabled ? "https" : "http"; + } + + @Override + public ConnectorState getState(final String connectorId) throws IOException { + final RequestReplicator requestReplicator = getRequestReplicator(); + final NiFiUser nodeUser = getNodeUser(); + final URI uri = URI.create(replicationScheme + "://localhost/nifi-api/connectors/" + connectorId); + final AsyncClusterResponse asyncResponse = requestReplicator.replicate(nodeUser, GET, uri, Map.of(), Map.of()); + + try { + final NodeResponse mergedNodeResponse = asyncResponse.awaitMergedResponse(); + final Response response = mergedNodeResponse.getClientResponse(); + verifyResponse(response.getStatusInfo(), connectorId); + + // Use the merged/updated entity if available, otherwise fall back to reading from the raw response. + // The updatedEntity contains the properly merged state from all nodes, while readEntity() would + // only return the state from whichever single node was selected as the "client response". + final ConnectorEntity connectorEntity; + final Entity updatedEntity = mergedNodeResponse.getUpdatedEntity(); + if (updatedEntity instanceof ConnectorEntity mergedConnectorEntity) { + connectorEntity = mergedConnectorEntity; + } else { + connectorEntity = response.readEntity(ConnectorEntity.class); + } + + final String stateName = connectorEntity.getComponent().getState(); + try { + return ConnectorState.valueOf(stateName); + } catch (final IllegalArgumentException e) { + throw new IOException("Received unknown Connector state '" + stateName + "' for Connector with ID " + connectorId); + } + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted while waiting for response for status of Connector with ID " + connectorId, e); + } + } + + @Override + public void setFlowManager(final FlowManager flowManager) { + } + + private RequestReplicator getRequestReplicator() { + final RequestReplicator requestReplicator = requestReplicatorSupplier.get(); + return Objects.requireNonNull(requestReplicator, "Request Replicator required"); + } + + private NiFiUser getNodeUser() { + final ClusterCoordinator clusterCoordinator = clusterCoordinatorSupplier.get(); + Objects.requireNonNull(clusterCoordinator, "Cluster Coordinator required"); + + final NodeIdentifier localNodeIdentifier = clusterCoordinator.getLocalNodeIdentifier(); + Objects.requireNonNull(localNodeIdentifier, "Local Node Identifier required"); + + final Set nodeIdentities = localNodeIdentifier.getNodeIdentities(); + final String nodeIdentity = nodeIdentities.isEmpty() ? localNodeIdentifier.getApiAddress() : nodeIdentities.iterator().next(); + return new StandardNiFiUser.Builder().identity(nodeIdentity).build(); + } + + private void verifyResponse(final StatusType responseStatusType, final String connectorId) throws IOException { + final int statusCode = responseStatusType.getStatusCode(); + final String reason = responseStatusType.getReasonPhrase(); + + if (responseStatusType.getStatusCode() == Status.NOT_FOUND.getStatusCode()) { + throw new IllegalArgumentException("Connector with ID + " + connectorId + " does not exist"); + } + + final Family responseFamily = responseStatusType.getFamily(); + if (responseFamily == Family.SERVER_ERROR) { + throw new IOException("Server-side error requesting State for Connector with ID + " + connectorId + ". Status code: " + statusCode + ", reason: " + reason); + } + if (responseFamily == Family.CLIENT_ERROR) { + throw new IOException("Client-side error requesting State for Connector with ID " + connectorId + ". Status code: " + statusCode + ", reason: " + reason); + } + if (responseFamily != Family.SUCCESSFUL) { + throw new IOException("Unexpected response code " + statusCode + " returned when requesting State for Connector with ID " + connectorId + ": " + reason); + } + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java index 5736b608246c..24bfda72225d 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java @@ -28,6 +28,7 @@ import org.apache.nifi.cluster.coordination.http.HttpResponseMapper; import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMapper; import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionEndpointMerger; +import org.apache.nifi.cluster.coordination.http.endpoints.ConnectorEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServiceEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.FlowRegistryClientEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.FunnelEndpointMerger; @@ -670,6 +671,7 @@ private boolean isDeleteComponent(final String method, final String uriPath) { // Processors, which is done by issuing a request to DELETE /processors//threads return ConnectionEndpointMerger.CONNECTION_URI_PATTERN.matcher(uriPath).matches() || ProcessorEndpointMerger.PROCESSOR_URI_PATTERN.matcher(uriPath).matches() + || ConnectorEndpointMerger.CONNECTOR_URI_PATTERN.matcher(uriPath).matches() || FunnelEndpointMerger.FUNNEL_URI_PATTERN.matcher(uriPath).matches() || PortEndpointMerger.INPUT_PORT_URI_PATTERN.matcher(uriPath).matches() || PortEndpointMerger.OUTPUT_PORT_URI_PATTERN.matcher(uriPath).matches() diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConfigurationStepEntityMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConfigurationStepEntityMerger.java new file mode 100644 index 000000000000..4ac0f52e3cf3 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConfigurationStepEntityMerger.java @@ -0,0 +1,160 @@ +/* + * 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 org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorPropertyDescriptorDTO; +import org.apache.nifi.web.api.dto.PropertyGroupConfigurationDTO; +import org.apache.nifi.web.api.entity.ConfigurationStepEntity; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ConfigurationStepEntityMerger { + + /** + * Merges the ConfigurationStepEntity responses. + * Note: Permissions are not merged here since they are managed at the parent Connector level. + * + * @param clientEntity the entity being returned to the client + * @param entityMap all node responses + */ + public static void merge(final ConfigurationStepEntity clientEntity, final Map entityMap) { + final ConfigurationStepConfigurationDTO clientDto = clientEntity.getConfigurationStep(); + final Map dtoMap = new HashMap<>(); + + for (final Map.Entry entry : entityMap.entrySet()) { + final ConfigurationStepEntity nodeConfigurationStepEntity = entry.getValue(); + final ConfigurationStepConfigurationDTO nodeConfigurationStepDto = nodeConfigurationStepEntity.getConfigurationStep(); + dtoMap.put(entry.getKey(), nodeConfigurationStepDto); + } + + mergeDtos(clientDto, dtoMap); + } + + private static void mergeDtos(final ConfigurationStepConfigurationDTO clientDto, final Map dtoMap) { + // if unauthorized for the client dto, simply return + if (clientDto == null) { + return; + } + + // Configuration step configurations should be consistent across all nodes + // We primarily need to ensure that the configuration step name is consistent + // The property values within property groups should also be consistent across nodes + + for (final Map.Entry nodeEntry : dtoMap.entrySet()) { + final ConfigurationStepConfigurationDTO nodeConfigurationStep = nodeEntry.getValue(); + + if (nodeConfigurationStep != null) { + // Verify that configuration step names are consistent + if (clientDto.getConfigurationStepName() != null && nodeConfigurationStep.getConfigurationStepName() != null) { + if (!clientDto.getConfigurationStepName().equals(nodeConfigurationStep.getConfigurationStepName())) { + // Log inconsistency but use the client's version + // In a properly functioning cluster, configuration step names should be consistent + } + } + + // Merge property group configurations + mergePropertyGroupConfigurations(clientDto.getPropertyGroupConfigurations(), nodeConfigurationStep.getPropertyGroupConfigurations()); + } + } + + // Merge property descriptors to handle dynamic allowable values + mergePropertyDescriptors(clientDto, dtoMap); + } + + private static void mergePropertyGroupConfigurations(final List clientPropertyGroups, final List nodePropertyGroups) { + if (clientPropertyGroups == null || nodePropertyGroups == null) { + return; + } + + // For property groups, we expect configurations to be consistent across nodes + // This merger primarily ensures that we have a consistent view + // Property values should not differ between nodes in a properly configured cluster + + for (final PropertyGroupConfigurationDTO clientPropertyGroup : clientPropertyGroups) { + for (final PropertyGroupConfigurationDTO nodePropertyGroup : nodePropertyGroups) { + if (clientPropertyGroup.getPropertyGroupName() != null + && clientPropertyGroup.getPropertyGroupName().equals(nodePropertyGroup.getPropertyGroupName())) { + + // Property values should be consistent across nodes + // In case of any inconsistency, we use the client's values + // This is primarily for defensive programming + break; + } + } + } + } + + /** + * Merges property descriptors across all nodes in the configuration step. + * This is necessary to handle dynamic allowable values that may differ between nodes. + * + * @param clientDto the configuration step DTO being returned to the client + * @param dtoMap all node configuration step DTOs + */ + static void mergePropertyDescriptors(final ConfigurationStepConfigurationDTO clientDto, final Map dtoMap) { + if (clientDto == null || clientDto.getPropertyGroupConfigurations() == null) { + return; + } + + for (final PropertyGroupConfigurationDTO clientPropertyGroup : clientDto.getPropertyGroupConfigurations()) { + if (clientPropertyGroup.getPropertyDescriptors() == null) { + continue; + } + + // Build a map of property descriptors from all nodes for this property group + final Map> propertyDescriptorMap = new HashMap<>(); + + for (final Map.Entry nodeEntry : dtoMap.entrySet()) { + final ConfigurationStepConfigurationDTO nodeConfigStep = nodeEntry.getValue(); + if (nodeConfigStep == null || nodeConfigStep.getPropertyGroupConfigurations() == null) { + continue; + } + + // Find the matching property group in the node's configuration + for (final PropertyGroupConfigurationDTO nodePropertyGroup : nodeConfigStep.getPropertyGroupConfigurations()) { + if (clientPropertyGroup.getPropertyGroupName() != null + && clientPropertyGroup.getPropertyGroupName().equals(nodePropertyGroup.getPropertyGroupName()) + && nodePropertyGroup.getPropertyDescriptors() != null) { + + // Aggregate property descriptors by name + for (final ConnectorPropertyDescriptorDTO propertyDescriptor : nodePropertyGroup.getPropertyDescriptors().values()) { + propertyDescriptorMap.computeIfAbsent(propertyDescriptor.getName(), name -> new HashMap<>()) + .put(nodeEntry.getKey(), propertyDescriptor); + } + } + } + } + + // Merge each property descriptor + for (final Map.Entry> entry : propertyDescriptorMap.entrySet()) { + final String propertyName = entry.getKey(); + final Map descriptorByNodeId = entry.getValue(); + + // Get the client's property descriptor directly from the map + final ConnectorPropertyDescriptorDTO clientPropertyDescriptor = clientPropertyGroup.getPropertyDescriptors().get(propertyName); + + if (clientPropertyDescriptor != null) { + ConnectorPropertyDescriptorDtoMerger.merge(clientPropertyDescriptor, descriptorByNodeId); + } + } + } + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConfigurationStepNamesEntityMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConfigurationStepNamesEntityMerger.java new file mode 100644 index 000000000000..66e541173df7 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConfigurationStepNamesEntityMerger.java @@ -0,0 +1,62 @@ +/* + * 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 org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.entity.ConfigurationStepNamesEntity; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class ConfigurationStepNamesEntityMerger { + + /** + * Merges multiple ConfigurationStepNamesEntity responses. + * + * @param clientEntity the entity being returned to the client + * @param entityMap all node responses + */ + public static void merge(final ConfigurationStepNamesEntity clientEntity, final Map entityMap) { + if (clientEntity == null) { + return; + } + + final Set allConfigurationStepNames = new HashSet<>(); + + // Add client's configuration step names + if (clientEntity.getConfigurationStepNames() != null) { + allConfigurationStepNames.addAll(clientEntity.getConfigurationStepNames()); + } + + // Merge configuration step names from all nodes + for (final Map.Entry entry : entityMap.entrySet()) { + final ConfigurationStepNamesEntity nodeEntity = entry.getValue(); + + if (nodeEntity != null && nodeEntity.getConfigurationStepNames() != null) { + allConfigurationStepNames.addAll(nodeEntity.getConfigurationStepNames()); + } + } + + // Set the merged configuration step names (sorted for consistency) + final List sortedConfigurationStepNames = new ArrayList<>(allConfigurationStepNames); + sortedConfigurationStepNames.sort(String::compareTo); + clientEntity.setConfigurationStepNames(sortedConfigurationStepNames); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConnectorEntityMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConnectorEntityMerger.java new file mode 100644 index 000000000000..baa0f632367c --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConnectorEntityMerger.java @@ -0,0 +1,212 @@ +/* + * 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 org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.components.connector.ConnectorState; +import org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorDTO; +import org.apache.nifi.web.api.dto.status.ConnectorStatusDTO; +import org.apache.nifi.web.api.dto.status.NodeConnectorStatusSnapshotDTO; +import org.apache.nifi.web.api.entity.ConnectorEntity; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; + +public class ConnectorEntityMerger { + /** + * Merges the ConnectorEntity responses. + * + * @param clientEntity the entity being returned to the client + * @param entityMap all node responses + */ + public static void merge(final ConnectorEntity clientEntity, final Map entityMap) { + final ConnectorDTO clientDto = clientEntity.getComponent(); + final Map dtoMap = new HashMap<>(); + for (final Map.Entry entry : entityMap.entrySet()) { + final ConnectorEntity nodeConnectorEntity = entry.getValue(); + final ConnectorDTO nodeConnectorDto = nodeConnectorEntity.getComponent(); + dtoMap.put(entry.getKey(), nodeConnectorDto); + } + + mergeDtos(clientDto, dtoMap); + + mergeStatus(clientEntity, entityMap); + } + + private static void mergeStatus(final ConnectorEntity clientEntity, final Map entityMap) { + final ConnectorStatusDTO mergedStatus = clientEntity.getStatus(); + if (mergedStatus == null) { + return; + } + + mergedStatus.setNodeSnapshots(new ArrayList<>()); + + // Identify the selected node (the one whose response was chosen as the client entity) + final NodeIdentifier selectedNodeId = entityMap.entrySet().stream() + .filter(e -> e.getValue() == clientEntity) + .map(Map.Entry::getKey) + .findFirst() + .orElse(null); + + // Add the selected node's snapshot to the node snapshots list + if (selectedNodeId != null && mergedStatus.getAggregateSnapshot() != null) { + final NodeConnectorStatusSnapshotDTO selectedNodeSnapshot = new NodeConnectorStatusSnapshotDTO(); + selectedNodeSnapshot.setStatusSnapshot(mergedStatus.getAggregateSnapshot().clone()); + selectedNodeSnapshot.setAddress(selectedNodeId.getApiAddress()); + selectedNodeSnapshot.setApiPort(selectedNodeId.getApiPort()); + selectedNodeSnapshot.setNodeId(selectedNodeId.getId()); + mergedStatus.getNodeSnapshots().add(selectedNodeSnapshot); + } + + // Merge snapshots from other nodes + for (final Map.Entry entry : entityMap.entrySet()) { + final NodeIdentifier nodeId = entry.getKey(); + final ConnectorEntity nodeEntity = entry.getValue(); + if (nodeEntity == clientEntity || nodeEntity == null) { + continue; + } + + final ConnectorStatusDTO nodeStatus = nodeEntity.getStatus(); + if (nodeStatus == null) { + continue; + } + + final boolean clientReadable = clientEntity.getPermissions() != null && Boolean.TRUE.equals(clientEntity.getPermissions().getCanRead()); + final boolean nodeReadable = nodeEntity.getPermissions() != null && Boolean.TRUE.equals(nodeEntity.getPermissions().getCanRead()); + StatusMerger.merge(mergedStatus, clientReadable, nodeStatus, nodeReadable, + nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort()); + } + } + + private static void mergeDtos(final ConnectorDTO clientDto, final Map dtoMap) { + // if unauthorized for the client dto, simple return + if (clientDto == null) { + return; + } + + mergeState(clientDto, dtoMap); + + // Merge configuration steps to handle dynamic property descriptors + mergeActiveConfiguration(clientDto, dtoMap); + mergeWorkingConfiguration(clientDto, dtoMap); + } + + /** + * Merges state from cluster nodes using priority-based selection. + * Priority (highest to lowest): + * 1. UPDATE_FAILED - indicates a failed update that needs attention + * 2. PREPARING_FOR_UPDATE - connector is preparing for update + * 3. UPDATING - connector is actively updating + * 4. UPDATED - connector has been updated + * 5. STARTING/STOPPING - connector is transitioning between run states + * 6. RUNNING/STOPPED/DISABLED - stable states + */ + private static void mergeState(final ConnectorDTO clientDto, final Map dtoMap) { + String mergedState = clientDto.getState(); + + for (final ConnectorDTO nodeConnector : dtoMap.values()) { + if (nodeConnector != null) { + final String nodeState = nodeConnector.getState(); + if (nodeState != null) { + if (getStatePriority(nodeState) > getStatePriority(mergedState)) { + mergedState = nodeState; + } + } + } + } + + clientDto.setState(mergedState); + } + + private static int getStatePriority(final String state) { + if (state == null) { + return 0; + } + + final ConnectorState connectorState = ConnectorState.valueOf(state); + return switch (connectorState) { + case UPDATE_FAILED -> 7; + case PREPARING_FOR_UPDATE -> 6; + case UPDATING -> 5; + case DRAINING, PURGING -> 4; + case UPDATED -> 3; + case STARTING, STOPPING -> 2; + default -> 1; // RUNNING, STOPPED + }; + } + + private static void mergeActiveConfiguration(final ConnectorDTO clientDto, final Map dtoMap) { + final ConnectorConfigurationDTO clientConfig = clientDto.getActiveConfiguration(); + if (clientConfig == null || clientConfig.getConfigurationStepConfigurations() == null) { + return; + } + + // For each configuration step in the client's active configuration, merge it with the corresponding steps from all nodes + for (final ConfigurationStepConfigurationDTO clientStep : clientConfig.getConfigurationStepConfigurations()) { + final Map stepDtoMap = new HashMap<>(); + + for (final Map.Entry nodeEntry : dtoMap.entrySet()) { + final ConnectorDTO nodeDto = nodeEntry.getValue(); + if (nodeDto == null || nodeDto.getActiveConfiguration() == null || nodeDto.getActiveConfiguration().getConfigurationStepConfigurations() == null) { + continue; + } + + // Find the matching configuration step in the node's active configuration + nodeDto.getActiveConfiguration().getConfigurationStepConfigurations().stream() + .filter(step -> clientStep.getConfigurationStepName() != null + && clientStep.getConfigurationStepName().equals(step.getConfigurationStepName())) + .findFirst() + .ifPresent(step -> stepDtoMap.put(nodeEntry.getKey(), step)); + } + + // Merge property descriptors for this configuration step + ConfigurationStepEntityMerger.mergePropertyDescriptors(clientStep, stepDtoMap); + } + } + + private static void mergeWorkingConfiguration(final ConnectorDTO clientDto, final Map dtoMap) { + final ConnectorConfigurationDTO clientConfig = clientDto.getWorkingConfiguration(); + if (clientConfig == null || clientConfig.getConfigurationStepConfigurations() == null) { + return; + } + + // For each configuration step in the client's working configuration, merge it with the corresponding steps from all nodes + for (final ConfigurationStepConfigurationDTO clientStep : clientConfig.getConfigurationStepConfigurations()) { + final Map stepDtoMap = new HashMap<>(); + + for (final Map.Entry nodeEntry : dtoMap.entrySet()) { + final ConnectorDTO nodeDto = nodeEntry.getValue(); + if (nodeDto == null || nodeDto.getWorkingConfiguration() == null || nodeDto.getWorkingConfiguration().getConfigurationStepConfigurations() == null) { + continue; + } + + // Find the matching configuration step in the node's working configuration + nodeDto.getWorkingConfiguration().getConfigurationStepConfigurations().stream() + .filter(step -> clientStep.getConfigurationStepName() != null + && clientStep.getConfigurationStepName().equals(step.getConfigurationStepName())) + .findFirst() + .ifPresent(step -> stepDtoMap.put(nodeEntry.getKey(), step)); + } + + // Merge property descriptors for this configuration step + ConfigurationStepEntityMerger.mergePropertyDescriptors(clientStep, stepDtoMap); + } + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConnectorPropertyDescriptorDtoMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConnectorPropertyDescriptorDtoMerger.java new file mode 100644 index 000000000000..ce6b29bffa1e --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConnectorPropertyDescriptorDtoMerger.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.manager; + +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.dto.AllowableValueDTO; +import org.apache.nifi.web.api.dto.ConnectorPropertyDescriptorDTO; +import org.apache.nifi.web.api.entity.AllowableValueEntity; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class ConnectorPropertyDescriptorDtoMerger { + public static void merge(final ConnectorPropertyDescriptorDTO clientPropertyDescriptor, final Map dtoMap) { + if (clientPropertyDescriptor.getAllowableValues() != null) { + final Map> allowableValueDtoToEntities = new LinkedHashMap<>(); + + addEntities(clientPropertyDescriptor, allowableValueDtoToEntities); + dtoMap.values().forEach(propertyDescriptorDTO -> addEntities(propertyDescriptorDTO, allowableValueDtoToEntities)); + + // Consider each allowable value from the client property descriptor. In the event of a duplicate allowable value, each entry will still + // be included in the merged list. Also ensure that each allowable value is present in all nodes property descriptors. + final List mergedAllowableValues = clientPropertyDescriptor.getAllowableValues().stream() + .map(allowableValueEntity -> allowableValueDtoToEntities.getOrDefault(allowableValueEntity.getAllowableValue(), Collections.emptyList())) + .filter(entities -> !entities.isEmpty() && allNodesHaveAllowableValue(entities.getFirst(), dtoMap)) + .map(AllowableValueEntityMerger::merge) + .collect(Collectors.toList()); + + clientPropertyDescriptor.setAllowableValues(mergedAllowableValues); + } + } + + private static boolean allNodesHaveAllowableValue(final AllowableValueEntity entity, final Map dtoMap) { + return dtoMap.values().stream() + .allMatch(propertyDescriptorDTO -> + propertyDescriptorDTO.getAllowableValues() != null && propertyDescriptorDTO.getAllowableValues().stream() + .anyMatch(allowableValueEntity -> allowableValueEntity.getAllowableValue().equals(entity.getAllowableValue()))); + } + + private static void addEntities(final ConnectorPropertyDescriptorDTO propertyDescriptorDTO, final Map> dtoToEntities) { + propertyDescriptorDTO.getAllowableValues().forEach( + allowableValueEntity -> dtoToEntities.computeIfAbsent(allowableValueEntity.getAllowableValue(), __ -> new ArrayList<>()).add(allowableValueEntity) + ); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConnectorsEntityMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConnectorsEntityMerger.java new file mode 100644 index 000000000000..a4d2e7692565 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ConnectorsEntityMerger.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.cluster.manager; + +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.entity.ConnectorEntity; + +import java.util.Map; +import java.util.Set; + +public class ConnectorsEntityMerger { + + private static final ConnectorEntityMerger connectorEntityMerger = new ConnectorEntityMerger(); + + /** + * Merges multiple ConnectorEntity responses. + * + * @param connectorEntities entities being returned to the client + * @param entityMap all node responses + */ + public static void mergeConnectors(final Set connectorEntities, final Map> entityMap) { + for (final ConnectorEntity entity : connectorEntities) { + connectorEntityMerger.merge(entity, entityMap.get(entity.getId())); + } + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java index 06c2a970ba93..bd351db8fedb 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/StatusMerger.java @@ -43,10 +43,13 @@ import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO; import org.apache.nifi.web.api.dto.status.ConnectionStatusPredictionsSnapshotDTO; import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO; +import org.apache.nifi.web.api.dto.status.ConnectorStatusDTO; +import org.apache.nifi.web.api.dto.status.ConnectorStatusSnapshotDTO; import org.apache.nifi.web.api.dto.status.ControllerServiceStatusDTO; import org.apache.nifi.web.api.dto.status.ControllerStatusDTO; import org.apache.nifi.web.api.dto.status.FlowAnalysisRuleStatusDTO; import org.apache.nifi.web.api.dto.status.NodeConnectionStatusSnapshotDTO; +import org.apache.nifi.web.api.dto.status.NodeConnectorStatusSnapshotDTO; import org.apache.nifi.web.api.dto.status.NodePortStatusSnapshotDTO; import org.apache.nifi.web.api.dto.status.NodeProcessGroupStatusSnapshotDTO; import org.apache.nifi.web.api.dto.status.NodeProcessorStatusSnapshotDTO; @@ -1075,4 +1078,96 @@ public static void merge(final FlowAnalysisRuleStatusDTO target, final FlowAnaly target.setValidationStatus(ValidationStatus.INVALID.name()); } } + + public static void merge(final ConnectorStatusDTO target, final boolean targetReadablePermission, final ConnectorStatusDTO toMerge, final boolean toMergeReadablePermission, + final String nodeId, final String nodeAddress, final Integer nodeApiPort) { + if (target == null || toMerge == null) { + return; + } + + if (targetReadablePermission && !toMergeReadablePermission) { + target.setGroupId(toMerge.getGroupId()); + target.setId(toMerge.getId()); + target.setName(toMerge.getName()); + target.setType(toMerge.getType()); + } + + if (ValidationStatus.VALIDATING.name().equalsIgnoreCase(toMerge.getValidationStatus())) { + target.setValidationStatus(ValidationStatus.VALIDATING.name()); + } else if (ValidationStatus.INVALID.name().equalsIgnoreCase(toMerge.getValidationStatus())) { + target.setValidationStatus(ValidationStatus.INVALID.name()); + } + + merge(target.getAggregateSnapshot(), targetReadablePermission, toMerge.getAggregateSnapshot(), toMergeReadablePermission); + + if (target.getNodeSnapshots() != null) { + final NodeConnectorStatusSnapshotDTO nodeSnapshot = new NodeConnectorStatusSnapshotDTO(); + nodeSnapshot.setStatusSnapshot(toMerge.getAggregateSnapshot()); + nodeSnapshot.setAddress(nodeAddress); + nodeSnapshot.setApiPort(nodeApiPort); + nodeSnapshot.setNodeId(nodeId); + + target.getNodeSnapshots().add(nodeSnapshot); + } + } + + public static void merge(final ConnectorStatusSnapshotDTO target, final boolean targetReadablePermission, + final ConnectorStatusSnapshotDTO toMerge, final boolean toMergeReadablePermission) { + if (target == null || toMerge == null) { + return; + } + + if (targetReadablePermission && !toMergeReadablePermission) { + target.setId(toMerge.getId()); + target.setGroupId(toMerge.getGroupId()); + target.setName(toMerge.getName()); + target.setType(toMerge.getType()); + } + + target.setFlowFilesSent(target.getFlowFilesSent() + toMerge.getFlowFilesSent()); + target.setBytesSent(target.getBytesSent() + toMerge.getBytesSent()); + target.setFlowFilesReceived(target.getFlowFilesReceived() + toMerge.getFlowFilesReceived()); + target.setBytesReceived(target.getBytesReceived() + toMerge.getBytesReceived()); + target.setBytesRead(target.getBytesRead() + toMerge.getBytesRead()); + target.setBytesWritten(target.getBytesWritten() + toMerge.getBytesWritten()); + + target.setFlowFilesQueued(target.getFlowFilesQueued() + toMerge.getFlowFilesQueued()); + target.setBytesQueued(target.getBytesQueued() + toMerge.getBytesQueued()); + + target.setActiveThreadCount(target.getActiveThreadCount() + toMerge.getActiveThreadCount()); + + // For idle status, the connector is considered idle only if ALL nodes report it as idle. + // The idle duration is the minimum across nodes (the most recently active node determines the duration). + if (Boolean.TRUE.equals(target.getIdle()) && Boolean.TRUE.equals(toMerge.getIdle())) { + if (target.getIdleDurationMillis() != null && toMerge.getIdleDurationMillis() != null) { + target.setIdleDurationMillis(Math.min(target.getIdleDurationMillis(), toMerge.getIdleDurationMillis())); + } else if (toMerge.getIdleDurationMillis() != null) { + target.setIdleDurationMillis(toMerge.getIdleDurationMillis()); + } + } else { + target.setIdle(false); + target.setIdleDurationMillis(null); + target.setIdleDuration(null); + } + + ProcessingPerformanceStatusMerger.mergeStatus(target.getProcessingPerformanceStatus(), toMerge.getProcessingPerformanceStatus()); + + updatePrettyPrintedFields(target); + } + + public static void updatePrettyPrintedFields(final ConnectorStatusSnapshotDTO target) { + target.setSent(prettyPrint(target.getFlowFilesSent(), target.getBytesSent())); + target.setReceived(prettyPrint(target.getFlowFilesReceived(), target.getBytesReceived())); + target.setRead(formatDataSize(target.getBytesRead())); + target.setWritten(formatDataSize(target.getBytesWritten())); + target.setQueued(prettyPrint(target.getFlowFilesQueued(), target.getBytesQueued())); + target.setQueuedCount(formatCount(target.getFlowFilesQueued())); + target.setQueuedSize(formatDataSize(target.getBytesQueued())); + + if (Boolean.TRUE.equals(target.getIdle()) && target.getIdleDurationMillis() != null) { + target.setIdleDuration(FormatUtils.formatHoursMinutesSeconds(target.getIdleDurationMillis(), TimeUnit.MILLISECONDS)); + } else { + target.setIdleDuration(null); + } + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/framework/configuration/ConnectorRequestReplicatorConfiguration.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/framework/configuration/ConnectorRequestReplicatorConfiguration.java new file mode 100644 index 000000000000..747b253716d6 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/framework/configuration/ConnectorRequestReplicatorConfiguration.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.framework.configuration; + +import org.apache.nifi.cluster.coordination.ClusterCoordinator; +import org.apache.nifi.cluster.coordination.http.replication.ClusteredConnectorRequestReplicator; +import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator; +import org.apache.nifi.components.connector.ConnectorRequestReplicator; +import org.apache.nifi.components.connector.StandaloneConnectorRequestReplicator; +import org.apache.nifi.util.NiFiProperties; +import org.springframework.beans.factory.ObjectProvider; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; + +@Configuration +public class ConnectorRequestReplicatorConfiguration { + + @Bean + public ConnectorRequestReplicator connectorRequestReplicator( + final NiFiProperties nifiProperties, + final ObjectProvider requestReplicatorProvider, + final ObjectProvider clusterCoordinatorProvider) { + + if (nifiProperties.isClustered()) { + // We have to use an ObjectProvider here and obtain a Supplier because of a circular dependency. + // The request replicator will not be available when created. However, it will be + // available before attempting to use the ConnectorRequestReplicator. + final String httpsHostname = nifiProperties.getProperty(NiFiProperties.WEB_HTTPS_HOST); + final boolean httpsEnabled = httpsHostname != null; + return new ClusteredConnectorRequestReplicator(requestReplicatorProvider::getIfAvailable, clusterCoordinatorProvider::getIfAvailable, httpsEnabled); + } + + return new StandaloneConnectorRequestReplicator(); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/framework/configuration/FrameworkClusterConfiguration.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/framework/configuration/FrameworkClusterConfiguration.java index 1bdbde509377..b0eb1edd9ddb 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/framework/configuration/FrameworkClusterConfiguration.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/framework/configuration/FrameworkClusterConfiguration.java @@ -48,8 +48,6 @@ public class FrameworkClusterConfiguration { private EventReporter eventReporter; - private FlowController flowController; - private ClusterCoordinator clusterCoordinator; private WebClientService webClientService; @@ -59,11 +57,6 @@ public void setProperties(final NiFiProperties properties) { this.properties = properties; } - @Autowired - public void setFlowController(final FlowController flowController) { - this.flowController = flowController; - } - @Autowired public void setEventReporter(final EventReporter eventReporter) { this.eventReporter = eventReporter; @@ -106,7 +99,7 @@ public ThreadPoolRequestReplicator requestReplicator( } @Bean - public ClusterDecommissionTask decommissionTask() { + public ClusterDecommissionTask decommissionTask(final FlowController flowController) { return new ClusterDecommissionTask(clusterCoordinator, flowController); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/endpoints/CurrentUserEndpointMergerTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/endpoints/CurrentUserEndpointMergerTest.java index 48d3ce0a51c1..2fdaff121bb8 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/endpoints/CurrentUserEndpointMergerTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/endpoints/CurrentUserEndpointMergerTest.java @@ -48,6 +48,7 @@ public void testMergeUserPermissions() { userNode1.setRestrictedComponentsPermissions(buildPermissions(false, false)); userNode1.setSystemPermissions(buildPermissions(true, true)); userNode1.setTenantsPermissions(buildPermissions(false, true)); + userNode1.setConnectorsPermissions(buildPermissions(true, false)); final Set componentRestrictionsNode1 = new HashSet<>(); componentRestrictionsNode1.add(buildComponentRestriction(RequiredPermission.ACCESS_KEYTAB, true, true)); @@ -64,6 +65,7 @@ public void testMergeUserPermissions() { userNode2.setRestrictedComponentsPermissions(buildPermissions(true, true)); userNode2.setSystemPermissions(buildPermissions(false, false)); userNode2.setTenantsPermissions(buildPermissions(true, true)); + userNode2.setConnectorsPermissions(buildPermissions(false, true)); final Set componentRestrictionsNode2 = new HashSet<>(); componentRestrictionsNode2.add(buildComponentRestriction(RequiredPermission.ACCESS_KEYTAB, true, false)); @@ -92,6 +94,8 @@ public void testMergeUserPermissions() { assertFalse(userNode1.getSystemPermissions().getCanWrite()); assertFalse(userNode1.getTenantsPermissions().getCanRead()); assertTrue(userNode1.getTenantsPermissions().getCanWrite()); + assertFalse(userNode1.getConnectorsPermissions().getCanRead()); + assertFalse(userNode1.getConnectorsPermissions().getCanWrite()); userNode1.getComponentRestrictionPermissions().forEach(componentRestriction -> { if (RequiredPermission.ACCESS_KEYTAB.getPermissionIdentifier().equals(componentRestriction.getRequiredPermission().getId())) { diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java index 2baa903beaa6..808c65222932 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java @@ -268,7 +268,7 @@ public void testMultipleRequestWithTwoPhaseCommit() throws Exception { when(coordinator.getConnectionStatus(Mockito.any(NodeIdentifier.class))).thenReturn(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED)); final AtomicInteger requestCount = new AtomicInteger(0); - final NiFiProperties props = NiFiProperties.createBasicNiFiProperties(null); + final NiFiProperties props = NiFiProperties.createBasicNiFiProperties((String) null); final MockReplicationClient client = new MockReplicationClient(); final RequestCompletionCallback requestCompletionCallback = (uri, method, responses) -> { @@ -329,7 +329,7 @@ public void testOneNodeRejectsTwoPhaseCommit() { final ClusterCoordinator coordinator = createClusterCoordinator(); final AtomicInteger requestCount = new AtomicInteger(0); - final NiFiProperties props = NiFiProperties.createBasicNiFiProperties(null); + final NiFiProperties props = NiFiProperties.createBasicNiFiProperties((String) null); final MockReplicationClient client = new MockReplicationClient(); final RequestCompletionCallback requestCompletionCallback = (uri, method, responses) -> { @@ -527,7 +527,7 @@ private void withReplicator(final WithReplicator function, final Status status, private void withReplicator(final WithReplicator function, final Status status, final long delayMillis, final RuntimeException failure, final String expectedRequestChain, final String expectedProxiedEntityGroups) { final ClusterCoordinator coordinator = createClusterCoordinator(); - final NiFiProperties nifiProps = NiFiProperties.createBasicNiFiProperties(null); + final NiFiProperties nifiProps = NiFiProperties.createBasicNiFiProperties((String) null); final MockReplicationClient client = new MockReplicationClient(); final RequestCompletionCallback requestCompletionCallback = (uri, method, responses) -> { }; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/ConfigurationStepEntityMergerTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/ConfigurationStepEntityMergerTest.java new file mode 100644 index 000000000000..d32e21bf7447 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/ConfigurationStepEntityMergerTest.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.manager; + +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.dto.AllowableValueDTO; +import org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorPropertyDescriptorDTO; +import org.apache.nifi.web.api.dto.PropertyGroupConfigurationDTO; +import org.apache.nifi.web.api.entity.AllowableValueEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepEntity; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +class ConfigurationStepEntityMergerTest { + + @Test + void testMergePropertyDescriptorsAcrossNodes() { + final ConfigurationStepEntity clientEntity = createConfigurationStepEntity("step1", + Map.of("hostname", List.of("localhost", "server1", "server2"))); + + final ConfigurationStepEntity node1Entity = createConfigurationStepEntity("step1", + Map.of("hostname", List.of("localhost", "server1"))); + + final ConfigurationStepEntity node2Entity = createConfigurationStepEntity("step1", + Map.of("hostname", List.of("localhost", "server1", "server2"))); + + final Map entityMap = new HashMap<>(); + entityMap.put(getNodeIdentifier("client", 8000), clientEntity); + entityMap.put(getNodeIdentifier("node1", 8001), node1Entity); + entityMap.put(getNodeIdentifier("node2", 8002), node2Entity); + + ConfigurationStepEntityMerger.merge(clientEntity, entityMap); + + // Should only include allowable values present on all nodes + final PropertyGroupConfigurationDTO propertyGroup = clientEntity.getConfigurationStep().getPropertyGroupConfigurations().get(0); + final ConnectorPropertyDescriptorDTO descriptor = propertyGroup.getPropertyDescriptors().get("hostname"); + + assertNotNull(descriptor.getAllowableValues()); + assertEquals(2, descriptor.getAllowableValues().size()); + + final List values = descriptor.getAllowableValues().stream() + .map(AllowableValueEntity::getAllowableValue) + .map(AllowableValueDTO::getValue) + .sorted() + .toList(); + + assertEquals(List.of("localhost", "server1"), values); + } + + @Test + void testMergeWithNoOverlappingAllowableValues() { + final ConfigurationStepEntity clientEntity = createConfigurationStepEntity("step1", + Map.of("region", List.of("us-east-1", "us-west-1"))); + + final ConfigurationStepEntity node1Entity = createConfigurationStepEntity("step1", + Map.of("region", List.of("eu-west-1"))); + + final ConfigurationStepEntity node2Entity = createConfigurationStepEntity("step1", + Map.of("region", List.of("ap-south-1"))); + + final Map entityMap = new HashMap<>(); + entityMap.put(getNodeIdentifier("client", 8000), clientEntity); + entityMap.put(getNodeIdentifier("node1", 8001), node1Entity); + entityMap.put(getNodeIdentifier("node2", 8002), node2Entity); + + ConfigurationStepEntityMerger.merge(clientEntity, entityMap); + + final PropertyGroupConfigurationDTO propertyGroup = clientEntity.getConfigurationStep().getPropertyGroupConfigurations().get(0); + final ConnectorPropertyDescriptorDTO descriptor = propertyGroup.getPropertyDescriptors().get("region"); + + assertNotNull(descriptor.getAllowableValues()); + assertEquals(0, descriptor.getAllowableValues().size()); + } + + @Test + void testMergeMultiplePropertyGroups() { + final ConfigurationStepConfigurationDTO clientStep = new ConfigurationStepConfigurationDTO(); + clientStep.setConfigurationStepName("database-config"); + + final PropertyGroupConfigurationDTO group1 = createPropertyGroup("connection", + Map.of("hostname", List.of("localhost", "server1"))); + final PropertyGroupConfigurationDTO group2 = createPropertyGroup("credentials", + Map.of("auth-type", List.of("password", "certificate"))); + + clientStep.setPropertyGroupConfigurations(List.of(group1, group2)); + + final ConfigurationStepConfigurationDTO node1Step = new ConfigurationStepConfigurationDTO(); + node1Step.setConfigurationStepName("database-config"); + final PropertyGroupConfigurationDTO node1Group1 = createPropertyGroup("connection", + Map.of("hostname", List.of("localhost", "server1"))); + final PropertyGroupConfigurationDTO node1Group2 = createPropertyGroup("credentials", + Map.of("auth-type", List.of("password"))); + node1Step.setPropertyGroupConfigurations(List.of(node1Group1, node1Group2)); + + final Map dtoMap = new HashMap<>(); + dtoMap.put(getNodeIdentifier("client", 8000), clientStep); + dtoMap.put(getNodeIdentifier("node1", 8001), node1Step); + + ConfigurationStepEntityMerger.mergePropertyDescriptors(clientStep, dtoMap); + + // Verify first property group + final ConnectorPropertyDescriptorDTO hostnameDescriptor = group1.getPropertyDescriptors().get("hostname"); + assertEquals(2, hostnameDescriptor.getAllowableValues().size()); + + // Verify second property group - should only have "password" since node1 doesn't have "certificate" + final ConnectorPropertyDescriptorDTO authDescriptor = group2.getPropertyDescriptors().get("auth-type"); + assertEquals(1, authDescriptor.getAllowableValues().size()); + assertEquals("password", authDescriptor.getAllowableValues().get(0).getAllowableValue().getValue()); + } + + private NodeIdentifier getNodeIdentifier(final String id, final int port) { + return new NodeIdentifier(id, "localhost", port, "localhost", port + 1, "localhost", port + 2, port + 3, true); + } + + private ConfigurationStepEntity createConfigurationStepEntity(final String stepName, final Map> propertyAllowableValues) { + final ConfigurationStepConfigurationDTO stepDto = new ConfigurationStepConfigurationDTO(); + stepDto.setConfigurationStepName(stepName); + stepDto.setConfigurationStepDescription("Test configuration step"); + + final PropertyGroupConfigurationDTO propertyGroup = createPropertyGroup("default", propertyAllowableValues); + stepDto.setPropertyGroupConfigurations(List.of(propertyGroup)); + + final ConfigurationStepEntity entity = new ConfigurationStepEntity(); + entity.setConfigurationStep(stepDto); + + return entity; + } + + private PropertyGroupConfigurationDTO createPropertyGroup(final String groupName, final Map> propertyAllowableValues) { + final PropertyGroupConfigurationDTO propertyGroup = new PropertyGroupConfigurationDTO(); + propertyGroup.setPropertyGroupName(groupName); + + final Map descriptors = new HashMap<>(); + for (final Map.Entry> entry : propertyAllowableValues.entrySet()) { + final ConnectorPropertyDescriptorDTO descriptor = new ConnectorPropertyDescriptorDTO(); + descriptor.setName(entry.getKey()); + descriptor.setDescription("Test property"); + descriptor.setType("STRING"); + descriptor.setRequired(false); + + final List allowableValues = entry.getValue().stream() + .map(value -> { + final AllowableValueDTO allowableValueDto = new AllowableValueDTO(); + allowableValueDto.setValue(value); + allowableValueDto.setDisplayName(value); + + final AllowableValueEntity entity = new AllowableValueEntity(); + entity.setAllowableValue(allowableValueDto); + entity.setCanRead(true); + return entity; + }) + .toList(); + + descriptor.setAllowableValues(allowableValues); + descriptors.put(entry.getKey(), descriptor); + } + + propertyGroup.setPropertyDescriptors(descriptors); + propertyGroup.setPropertyValues(new HashMap<>()); + + return propertyGroup; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/ConnectorEntityMergerTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/ConnectorEntityMergerTest.java new file mode 100644 index 000000000000..1560afa564bb --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/ConnectorEntityMergerTest.java @@ -0,0 +1,404 @@ +/* + * 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 org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.dto.AllowableValueDTO; +import org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorDTO; +import org.apache.nifi.web.api.dto.ConnectorPropertyDescriptorDTO; +import org.apache.nifi.web.api.dto.PermissionsDTO; +import org.apache.nifi.web.api.dto.PropertyGroupConfigurationDTO; +import org.apache.nifi.web.api.dto.RevisionDTO; +import org.apache.nifi.web.api.dto.status.ConnectorStatusDTO; +import org.apache.nifi.web.api.dto.status.ConnectorStatusSnapshotDTO; +import org.apache.nifi.web.api.dto.status.NodeConnectorStatusSnapshotDTO; +import org.apache.nifi.web.api.entity.AllowableValueEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class ConnectorEntityMergerTest { + + @Test + void testMergeConnectorState() { + final ConnectorEntity clientEntity = createConnectorEntity("connector1", "STOPPED"); + final ConnectorEntity node1Entity = createConnectorEntity("connector1", "STOPPED"); + final ConnectorEntity node2Entity = createConnectorEntity("connector1", "STOPPED"); + + final Map entityMap = new HashMap<>(); + entityMap.put(getNodeIdentifier("client", 8000), clientEntity); + entityMap.put(getNodeIdentifier("node1", 8001), node1Entity); + entityMap.put(getNodeIdentifier("node2", 8002), node2Entity); + + ConnectorEntityMerger.merge(clientEntity, entityMap); + + assertEquals("STOPPED", clientEntity.getComponent().getState()); + } + + @Test + void testMergeConnectorStateWithNull() { + final ConnectorEntity clientEntity = createConnectorEntity("connector1", null); + final ConnectorEntity node1Entity = createConnectorEntity("connector1", "RUNNING"); + final ConnectorEntity node2Entity = createConnectorEntity("connector1", "RUNNING"); + + final Map entityMap = new HashMap<>(); + entityMap.put(getNodeIdentifier("client", 8000), clientEntity); + entityMap.put(getNodeIdentifier("node1", 8001), node1Entity); + entityMap.put(getNodeIdentifier("node2", 8002), node2Entity); + + ConnectorEntityMerger.merge(clientEntity, entityMap); + + assertEquals("RUNNING", clientEntity.getComponent().getState()); + } + + @Test + void testMergeConfigurationStepsWithDifferentAllowableValues() { + final ConnectorEntity clientEntity = createConnectorEntityWithConfig( + "connector1", "STOPPED", + Map.of("hostname", List.of("localhost", "server1", "server2")) + ); + + final ConnectorEntity node1Entity = createConnectorEntityWithConfig( + "connector1", "STOPPED", + Map.of("hostname", List.of("localhost", "server1")) + ); + + final ConnectorEntity node2Entity = createConnectorEntityWithConfig( + "connector1", "STOPPED", + Map.of("hostname", List.of("localhost", "server1", "server2")) + ); + + final Map entityMap = new HashMap<>(); + entityMap.put(getNodeIdentifier("client", 8000), clientEntity); + entityMap.put(getNodeIdentifier("node1", 8001), node1Entity); + entityMap.put(getNodeIdentifier("node2", 8002), node2Entity); + + ConnectorEntityMerger.merge(clientEntity, entityMap); + + // Verify working configuration was merged - should only have values present on all nodes + final ConnectorConfigurationDTO config = clientEntity.getComponent().getWorkingConfiguration(); + assertNotNull(config); + + final ConfigurationStepConfigurationDTO step = config.getConfigurationStepConfigurations().get(0); + final PropertyGroupConfigurationDTO propertyGroup = step.getPropertyGroupConfigurations().get(0); + final ConnectorPropertyDescriptorDTO descriptor = propertyGroup.getPropertyDescriptors().get("hostname"); + + assertNotNull(descriptor.getAllowableValues()); + assertEquals(2, descriptor.getAllowableValues().size()); + + final List values = descriptor.getAllowableValues().stream() + .map(AllowableValueEntity::getAllowableValue) + .map(AllowableValueDTO::getValue) + .sorted() + .toList(); + + assertEquals(List.of("localhost", "server1"), values); + } + + @Test + void testMergeMultipleConfigurationSteps() { + final ConnectorEntity clientEntity = createConnectorEntityWithMultipleSteps(); + final ConnectorEntity node1Entity = createConnectorEntityWithMultipleSteps(); + + final Map entityMap = new HashMap<>(); + entityMap.put(getNodeIdentifier("client", 8000), clientEntity); + entityMap.put(getNodeIdentifier("node1", 8001), node1Entity); + + ConnectorEntityMerger.merge(clientEntity, entityMap); + + final ConnectorConfigurationDTO config = clientEntity.getComponent().getWorkingConfiguration(); + assertNotNull(config); + assertEquals(2, config.getConfigurationStepConfigurations().size()); + } + + @Test + void testMergeConnectorStatusValidationStatusPriority() { + final ConnectorEntity clientEntity = createConnectorEntityWithStatus("connector1", "RUNNING", 1, "VALID", + 5, 100L, 10, 200L, 50L, 75L, 10, 1000L, true, 5000L); + final ConnectorEntity node1Entity = createConnectorEntityWithStatus("connector1", "RUNNING", 1, "VALIDATING", + 7, 150L, 12, 250L, 60L, 80L, 5, 500L, true, 3000L); + final ConnectorEntity node2Entity = createConnectorEntityWithStatus("connector1", "RUNNING", 1, "VALID", + 3, 200L, 8, 300L, 70L, 85L, 8, 800L, false, null); + + final Map entityMap = new HashMap<>(); + entityMap.put(getNodeIdentifier("client", 8000), clientEntity); + entityMap.put(getNodeIdentifier("node1", 8001), node1Entity); + entityMap.put(getNodeIdentifier("node2", 8002), node2Entity); + + ConnectorEntityMerger.merge(clientEntity, entityMap); + + assertEquals("VALIDATING", clientEntity.getStatus().getValidationStatus()); + } + + @Test + void testMergeStatusSnapshotsAggregated() { + final ConnectorEntity clientEntity = createConnectorEntityWithStatus("connector1", "RUNNING", 2, "VALID", + 5, 100L, 10, 200L, 50L, 75L, 10, 1000L, true, 5000L); + final ConnectorEntity node1Entity = createConnectorEntityWithStatus("connector1", "RUNNING", 3, "VALID", + 7, 150L, 12, 250L, 60L, 80L, 5, 500L, true, 3000L); + + final Map entityMap = new HashMap<>(); + entityMap.put(getNodeIdentifier("client", 8000), clientEntity); + entityMap.put(getNodeIdentifier("node1", 8001), node1Entity); + + ConnectorEntityMerger.merge(clientEntity, entityMap); + + final ConnectorStatusDTO mergedStatus = clientEntity.getStatus(); + assertNotNull(mergedStatus); + assertNotNull(mergedStatus.getAggregateSnapshot()); + assertNotNull(mergedStatus.getNodeSnapshots()); + + // Verify node snapshots contain both nodes + assertEquals(2, mergedStatus.getNodeSnapshots().size()); + + // Verify aggregate snapshot has summed values + final ConnectorStatusSnapshotDTO aggregate = mergedStatus.getAggregateSnapshot(); + assertEquals(Integer.valueOf(12), aggregate.getFlowFilesSent()); // 5 + 7 + assertEquals(Long.valueOf(250L), aggregate.getBytesSent()); // 100 + 150 + assertEquals(Integer.valueOf(22), aggregate.getFlowFilesReceived()); // 10 + 12 + assertEquals(Long.valueOf(450L), aggregate.getBytesReceived()); // 200 + 250 + assertEquals(Long.valueOf(110L), aggregate.getBytesRead()); // 50 + 60 + assertEquals(Long.valueOf(155L), aggregate.getBytesWritten()); // 75 + 80 + assertEquals(Integer.valueOf(15), aggregate.getFlowFilesQueued()); // 10 + 5 + assertEquals(Long.valueOf(1500L), aggregate.getBytesQueued()); // 1000 + 500 + assertEquals(Integer.valueOf(5), aggregate.getActiveThreadCount()); // 2 + 3 + + // Both nodes are idle, so aggregate should be idle with min duration + assertTrue(aggregate.getIdle()); + assertEquals(Long.valueOf(3000L), aggregate.getIdleDurationMillis()); // min(5000, 3000) + } + + @Test + void testMergeStatusSnapshotsIdleWhenOneNodeNotIdle() { + final ConnectorEntity clientEntity = createConnectorEntityWithStatus("connector1", "RUNNING", 2, "VALID", + 5, 100L, 10, 200L, 50L, 75L, 10, 1000L, true, 5000L); + final ConnectorEntity node1Entity = createConnectorEntityWithStatus("connector1", "RUNNING", 3, "VALID", + 7, 150L, 12, 250L, 60L, 80L, 5, 500L, false, null); + + final Map entityMap = new HashMap<>(); + entityMap.put(getNodeIdentifier("client", 8000), clientEntity); + entityMap.put(getNodeIdentifier("node1", 8001), node1Entity); + + ConnectorEntityMerger.merge(clientEntity, entityMap); + + final ConnectorStatusSnapshotDTO aggregate = clientEntity.getStatus().getAggregateSnapshot(); + + // One node is not idle, so aggregate should not be idle + assertFalse(aggregate.getIdle()); + assertNull(aggregate.getIdleDurationMillis()); + } + + @Test + void testMergeStatusNodeSnapshotsContainCorrectNodeInfo() { + final ConnectorEntity clientEntity = createConnectorEntityWithStatus("connector1", "RUNNING", 2, "VALID", + 5, 100L, 10, 200L, 50L, 75L, 10, 1000L, false, null); + final ConnectorEntity node1Entity = createConnectorEntityWithStatus("connector1", "RUNNING", 3, "VALID", + 7, 150L, 12, 250L, 60L, 80L, 5, 500L, false, null); + + final NodeIdentifier clientNodeId = getNodeIdentifier("client", 8000); + final NodeIdentifier node1NodeId = getNodeIdentifier("node1", 8001); + + final Map entityMap = new HashMap<>(); + entityMap.put(clientNodeId, clientEntity); + entityMap.put(node1NodeId, node1Entity); + + ConnectorEntityMerger.merge(clientEntity, entityMap); + + final List nodeSnapshots = clientEntity.getStatus().getNodeSnapshots(); + assertEquals(2, nodeSnapshots.size()); + + // Find the client node snapshot + final NodeConnectorStatusSnapshotDTO clientSnapshot = nodeSnapshots.stream() + .filter(s -> s.getNodeId().equals(clientNodeId.getId())) + .findFirst() + .orElse(null); + assertNotNull(clientSnapshot); + assertEquals(clientNodeId.getApiAddress(), clientSnapshot.getAddress()); + assertEquals(clientNodeId.getApiPort(), clientSnapshot.getApiPort()); + assertEquals(Long.valueOf(100L), clientSnapshot.getStatusSnapshot().getBytesSent()); + + // Find the other node snapshot + final NodeConnectorStatusSnapshotDTO node1Snapshot = nodeSnapshots.stream() + .filter(s -> s.getNodeId().equals(node1NodeId.getId())) + .findFirst() + .orElse(null); + assertNotNull(node1Snapshot); + assertEquals(node1NodeId.getApiAddress(), node1Snapshot.getAddress()); + assertEquals(node1NodeId.getApiPort(), node1Snapshot.getApiPort()); + assertEquals(Long.valueOf(150L), node1Snapshot.getStatusSnapshot().getBytesSent()); + } + + @Test + void testMergeStatusWithNullStatus() { + final ConnectorEntity clientEntity = createConnectorEntity("connector1", "STOPPED"); + final ConnectorEntity node1Entity = createConnectorEntity("connector1", "STOPPED"); + + final Map entityMap = new HashMap<>(); + entityMap.put(getNodeIdentifier("client", 8000), clientEntity); + entityMap.put(getNodeIdentifier("node1", 8001), node1Entity); + + // Both have null status - should not throw + ConnectorEntityMerger.merge(clientEntity, entityMap); + assertNull(clientEntity.getStatus()); + } + + private NodeIdentifier getNodeIdentifier(final String id, final int port) { + return new NodeIdentifier(id, "localhost", port, "localhost", port + 1, "localhost", port + 2, port + 3, true); + } + + private ConnectorEntity createConnectorEntity(final String id, final String state) { + final ConnectorDTO dto = new ConnectorDTO(); + dto.setId(id); + dto.setName("Test Connector"); + dto.setType("TestConnector"); + dto.setState(state); + + final PermissionsDTO permissions = new PermissionsDTO(); + permissions.setCanRead(true); + permissions.setCanWrite(true); + + final ConnectorEntity entity = new ConnectorEntity(); + entity.setComponent(dto); + entity.setRevision(createNewRevision()); + entity.setPermissions(permissions); + + return entity; + } + + private ConnectorEntity createConnectorEntityWithStatus(final String id, final String state, final int activeThreadCount, final String validationStatus, + final Integer flowFilesSent, final Long bytesSent, + final Integer flowFilesReceived, final Long bytesReceived, + final Long bytesRead, final Long bytesWritten, + final Integer flowFilesQueued, final Long bytesQueued, + final Boolean idle, final Long idleDurationMillis) { + final ConnectorEntity entity = createConnectorEntity(id, state); + + final ConnectorStatusSnapshotDTO snapshot = new ConnectorStatusSnapshotDTO(); + snapshot.setId(id); + snapshot.setName("Test Connector"); + snapshot.setType("TestConnector"); + snapshot.setRunStatus(state); + snapshot.setActiveThreadCount(activeThreadCount); + snapshot.setFlowFilesSent(flowFilesSent); + snapshot.setBytesSent(bytesSent); + snapshot.setFlowFilesReceived(flowFilesReceived); + snapshot.setBytesReceived(bytesReceived); + snapshot.setBytesRead(bytesRead); + snapshot.setBytesWritten(bytesWritten); + snapshot.setFlowFilesQueued(flowFilesQueued); + snapshot.setBytesQueued(bytesQueued); + snapshot.setIdle(idle); + snapshot.setIdleDurationMillis(idleDurationMillis); + + final ConnectorStatusDTO status = new ConnectorStatusDTO(); + status.setId(id); + status.setRunStatus(state); + status.setValidationStatus(validationStatus); + status.setAggregateSnapshot(snapshot); + entity.setStatus(status); + + return entity; + } + + private ConnectorEntity createConnectorEntityWithConfig(final String id, final String state, + final Map> propertyAllowableValues) { + final ConnectorEntity entity = createConnectorEntity(id, state); + + final ConnectorConfigurationDTO config = new ConnectorConfigurationDTO(); + final ConfigurationStepConfigurationDTO step = createConfigurationStep("step1", propertyAllowableValues); + config.setConfigurationStepConfigurations(List.of(step)); + + entity.getComponent().setActiveConfiguration(config); + entity.getComponent().setWorkingConfiguration(config); + + return entity; + } + + private ConnectorEntity createConnectorEntityWithMultipleSteps() { + final ConnectorEntity entity = createConnectorEntity("connector1", "STOPPED"); + + final ConnectorConfigurationDTO config = new ConnectorConfigurationDTO(); + final ConfigurationStepConfigurationDTO step1 = createConfigurationStep("step1", + Map.of("hostname", List.of("localhost"))); + final ConfigurationStepConfigurationDTO step2 = createConfigurationStep("step2", + Map.of("port", List.of("8080", "9090"))); + config.setConfigurationStepConfigurations(List.of(step1, step2)); + + entity.getComponent().setActiveConfiguration(config); + entity.getComponent().setWorkingConfiguration(config); + + return entity; + } + + private ConfigurationStepConfigurationDTO createConfigurationStep(final String stepName, + final Map> propertyAllowableValues) { + final ConfigurationStepConfigurationDTO step = new ConfigurationStepConfigurationDTO(); + step.setConfigurationStepName(stepName); + step.setConfigurationStepDescription("Test step"); + + final PropertyGroupConfigurationDTO propertyGroup = new PropertyGroupConfigurationDTO(); + propertyGroup.setPropertyGroupName("default"); + + final Map descriptors = new HashMap<>(); + for (final Map.Entry> entry : propertyAllowableValues.entrySet()) { + final ConnectorPropertyDescriptorDTO descriptor = new ConnectorPropertyDescriptorDTO(); + descriptor.setName(entry.getKey()); + descriptor.setDescription("Test property"); + descriptor.setType("STRING"); + + final List allowableValues = entry.getValue().stream() + .map(value -> { + final AllowableValueDTO allowableValueDto = new AllowableValueDTO(); + allowableValueDto.setValue(value); + + final AllowableValueEntity allowableEntity = new AllowableValueEntity(); + allowableEntity.setAllowableValue(allowableValueDto); + allowableEntity.setCanRead(true); + return allowableEntity; + }) + .toList(); + + descriptor.setAllowableValues(allowableValues); + descriptors.put(entry.getKey(), descriptor); + } + + propertyGroup.setPropertyDescriptors(descriptors); + propertyGroup.setPropertyValues(new HashMap<>()); + + step.setPropertyGroupConfigurations(List.of(propertyGroup)); + + return step; + } + + private RevisionDTO createNewRevision() { + final RevisionDTO revisionDto = new RevisionDTO(); + revisionDto.setClientId(getClass().getName()); + revisionDto.setVersion(0L); + return revisionDto; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/ConnectorPropertyDescriptorDtoMergerTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/ConnectorPropertyDescriptorDtoMergerTest.java new file mode 100644 index 000000000000..b2c4ad25277e --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/ConnectorPropertyDescriptorDtoMergerTest.java @@ -0,0 +1,171 @@ +/* + * 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 org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.util.EqualsWrapper; +import org.apache.nifi.web.api.dto.AllowableValueDTO; +import org.apache.nifi.web.api.dto.ConnectorPropertyDescriptorDTO; +import org.apache.nifi.web.api.entity.AllowableValueEntity; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + +class ConnectorPropertyDescriptorDtoMergerTest { + + @Test + void testMergeWithNoAllowableValues() { + final ConnectorPropertyDescriptorDTO clientPropertyDescriptor = new ConnectorPropertyDescriptorDTO(); + + final Map dtoMap = + Map.of(createNodeIdentifier("node1"), new ConnectorPropertyDescriptorDTO(), + createNodeIdentifier("node2"), new ConnectorPropertyDescriptorDTO()); + + ConnectorPropertyDescriptorDtoMerger.merge(clientPropertyDescriptor, dtoMap); + + assertNull(clientPropertyDescriptor.getAllowableValues()); + } + + @Test + void testMergeWithEmptyAllowableValuesList() { + testMerge( + createPropertyDescriptorDTO(), + Map.of(createNodeIdentifier("node1"), createPropertyDescriptorDTO(), + createNodeIdentifier("node2"), createPropertyDescriptorDTO()), + createPropertyDescriptorDTO() + ); + } + + @Test + void testMergeWithSingleNode() { + testMerge( + createPropertyDescriptorDTO(v("value1"), v("value2")), + Collections.emptyMap(), + createPropertyDescriptorDTO(v("value1"), v("value2")) + ); + } + + @Test + void testMergeWithNonOverlappingAllowableValues() { + testMerge( + createPropertyDescriptorDTO(v("value1"), v("value2")), + Map.of(createNodeIdentifier("node1"), createPropertyDescriptorDTO(v("value3")), + createNodeIdentifier("node2"), createPropertyDescriptorDTO(v("value4"), v("value5"), v("value6"))), + createPropertyDescriptorDTO() + ); + } + + @Test + void testMergeWithOverlappingAllowableValues() { + testMerge( + createPropertyDescriptorDTO(v("value1"), v("value2"), v("value3")), + Map.of(createNodeIdentifier("node1"), createPropertyDescriptorDTO(v("value1"), v("value2"), v("value3")), + createNodeIdentifier("node2"), createPropertyDescriptorDTO(v("value2"), v("value3", false))), + createPropertyDescriptorDTO(v("value2"), v("value3", false)) + ); + } + + @Test + void testMergeWithIdenticalAllowableValues() { + testMerge( + createPropertyDescriptorDTO(v("value1"), v("value2")), + Map.of(createNodeIdentifier("node1"), createPropertyDescriptorDTO(v("value1"), v("value2")), + createNodeIdentifier("node2"), createPropertyDescriptorDTO(v("value1"), v("value2"))), + createPropertyDescriptorDTO(v("value1"), v("value2")) + ); + } + + @Test + void testMergeWithDuplicateAllowableValues() { + testMerge( + createPropertyDescriptorDTO(v("value1"), v("value1")), + Map.of(createNodeIdentifier("node1"), createPropertyDescriptorDTO(v("value1"), v("value1")), + createNodeIdentifier("node2"), createPropertyDescriptorDTO(v("value1"), v("value1"))), + createPropertyDescriptorDTO(v("value1"), v("value1")) + ); + } + + private ConnectorPropertyDescriptorDTO createPropertyDescriptorDTO(final AllowableValueData... allowableValueData) { + final ConnectorPropertyDescriptorDTO clientPropertyDescriptor = new ConnectorPropertyDescriptorDTO(); + + final List allowableValueEntities = Arrays.stream(allowableValueData) + .map(AllowableValueData::toEntity) + .collect(Collectors.toList()); + + clientPropertyDescriptor.setAllowableValues(allowableValueEntities); + + return clientPropertyDescriptor; + } + + private NodeIdentifier createNodeIdentifier(final String id) { + return new NodeIdentifier(id, id, 1, id, 1, id, 1, null, false); + } + + private void testMerge(final ConnectorPropertyDescriptorDTO clientPropertyDescriptor, + final Map dtoMap, + final ConnectorPropertyDescriptorDTO expected) { + ConnectorPropertyDescriptorDtoMerger.merge(clientPropertyDescriptor, dtoMap); + + final List> equalsProperties = Arrays.asList( + AllowableValueEntity::getAllowableValue, + AllowableValueEntity::getCanRead + ); + + final List> expectedWrappers = EqualsWrapper.wrapList(expected.getAllowableValues(), equalsProperties); + final List> actualWrappers = EqualsWrapper.wrapList(clientPropertyDescriptor.getAllowableValues(), equalsProperties); + + assertEquals(expectedWrappers, actualWrappers); + } + + private static class AllowableValueData { + private final String value; + private final Boolean canRead; + + private AllowableValueData(final String value, final Boolean canRead) { + this.value = value; + this.canRead = canRead; + } + + private AllowableValueEntity toEntity() { + final AllowableValueEntity entity = new AllowableValueEntity(); + + final AllowableValueDTO allowableValueDTO = new AllowableValueDTO(); + allowableValueDTO.setValue(value); + + entity.setAllowableValue(allowableValueDTO); + entity.setCanRead(canRead); + + return entity; + } + } + + private static AllowableValueData v(final String value) { + return v(value, true); + } + + private static AllowableValueData v(final String value, final Boolean canRead) { + return new AllowableValueData(value, canRead); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/asset/StandardAsset.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/asset/StandardAsset.java index 77e2ef5b64de..ed0cea997b63 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/asset/StandardAsset.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/asset/StandardAsset.java @@ -23,14 +23,14 @@ public class StandardAsset implements Asset { private final String identifier; - private final String parameterContextIdentifier; + private final String ownerIdentifier; private final String name; private final File file; private final String digest; - public StandardAsset(final String identifier, final String paramContextIdentifier, final String name, final File file, final String digest) { + public StandardAsset(final String identifier, final String ownerIdentifier, final String name, final File file, final String digest) { this.identifier = Objects.requireNonNull(identifier, "Identifier is required"); - this.parameterContextIdentifier = Objects.requireNonNull(paramContextIdentifier, "Parameter Context Identifier is required"); + this.ownerIdentifier = Objects.requireNonNull(ownerIdentifier, "Owner Identifier is required"); this.name = Objects.requireNonNull(name, "Name is required"); this.file = Objects.requireNonNull(file, "File is required"); this.digest = digest; @@ -43,7 +43,12 @@ public String getIdentifier() { @Override public String getParameterContextIdentifier() { - return parameterContextIdentifier; + return ownerIdentifier; + } + + @Override + public String getOwnerIdentifier() { + return ownerIdentifier; } @Override diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/asset/StandardAssetManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/asset/StandardAssetManager.java index f643cccf3a73..28b905462216 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/asset/StandardAssetManager.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/asset/StandardAssetManager.java @@ -68,36 +68,20 @@ public void initialize(final AssetManagerInitializationContext context) { } @Override - public Asset createAsset(final String parameterContextId, final String assetName, final InputStream contents) throws IOException { - final String assetId = createAssetId(parameterContextId, assetName); - - final File file = getFile(parameterContextId, assetName); - final File dir = file.getParentFile(); - if (!dir.exists()) { - try { - Files.createDirectories(dir.toPath()); - } catch (final IOException ioe) { - throw new IOException("Could not create directory in order to store asset", ioe); - } - } - - // Write contents to a temporary file, then move it to the final location. - // This allows us to avoid a situation where we upload a file, then we attempt to overwrite it but fail, leaving a corrupt asset. - final File tempFile = new File(dir, file.getName() + ".tmp"); - logger.debug("Writing temp asset file [{}]", tempFile.getAbsolutePath()); + public Asset createAsset(final String ownerId, final String assetName, final InputStream contents) throws IOException { + final String assetId = createAssetId(ownerId, assetName); + final File assetFile = getFile(ownerId, assetName); + return saveAsset(ownerId, assetId, assetName, contents, assetFile); + } - try { - Files.copy(contents, tempFile.toPath(), StandardCopyOption.REPLACE_EXISTING); - } catch (final Exception e) { - throw new IOException("Failed to write asset to file " + tempFile.getAbsolutePath(), e); + @Override + public Asset saveAsset(final String ownerId, final String assetId, final String assetName, final InputStream contents) throws IOException { + final String generatedAssetId = createAssetId(ownerId, assetName); + if (!generatedAssetId.equals(assetId)) { + throw new IllegalArgumentException("The provided asset id [%s] does not match the id this asset manager generated for the given owner and name".formatted(assetId)); } - - Files.move(tempFile.toPath(), file.toPath(), StandardCopyOption.REPLACE_EXISTING); - - final String digest = computeDigest(file); - final Asset asset = new StandardAsset(assetId, parameterContextId, assetName, file, digest); - assets.put(assetId, asset); - return asset; + final File assetFile = getFile(ownerId, assetName); + return saveAsset(ownerId, assetId, assetName, contents, assetFile); } @Override @@ -106,22 +90,22 @@ public Optional getAsset(final String id) { } @Override - public List getAssets(final String parameterContextId) { + public List getAssets(final String ownerId) { final List allAssets = new ArrayList<>(assets.values()); - final List paramContextAssets = new ArrayList<>(); + final List ownerAssets = new ArrayList<>(); for (final Asset asset : allAssets) { - if (asset.getParameterContextIdentifier().equals(parameterContextId)) { - paramContextAssets.add(asset); + if (asset.getOwnerIdentifier().equals(ownerId)) { + ownerAssets.add(asset); } } - return paramContextAssets; + return ownerAssets; } @Override - public Asset createMissingAsset(final String parameterContextId, final String assetName) { - final String assetId = createAssetId(parameterContextId, assetName); - final File file = getFile(parameterContextId, assetName); - final Asset asset = new StandardAsset(assetId, parameterContextId, assetName, file, null); + public Asset createMissingAsset(final String ownerId, final String assetName) { + final String assetId = createAssetId(ownerId, assetName); + final File file = getFile(ownerId, assetName); + final Asset asset = new StandardAsset(assetId, ownerId, assetName, file, null); assets.put(assetId, asset); return asset; } @@ -155,14 +139,43 @@ public Optional deleteAsset(final String id) { return Optional.of(removed); } - private String createAssetId(final String parameterContextId, final String assetName) { - final String seed = parameterContextId + "/" + assetName; + private Asset saveAsset(final String ownerId, final String assetId, final String assetName, final InputStream contents, final File assetFile) throws IOException { + final File dir = assetFile.getParentFile(); + if (!dir.exists()) { + try { + Files.createDirectories(dir.toPath()); + } catch (final IOException ioe) { + throw new IOException("Could not create directory in order to store asset", ioe); + } + } + + // Write contents to a temporary file, then move it to the final location. + // This allows us to avoid a situation where we upload a file, then we attempt to overwrite it but fail, leaving a corrupt asset. + final File tempFile = new File(dir, assetFile.getName() + ".tmp"); + logger.debug("Writing temp asset file [{}]", tempFile.getAbsolutePath()); + + try { + Files.copy(contents, tempFile.toPath(), StandardCopyOption.REPLACE_EXISTING); + } catch (final Exception e) { + throw new IOException("Failed to write asset to file " + tempFile.getAbsolutePath(), e); + } + + Files.move(tempFile.toPath(), assetFile.toPath(), StandardCopyOption.REPLACE_EXISTING); + + final String digest = computeDigest(assetFile); + final Asset asset = new StandardAsset(assetId, ownerId, assetName, assetFile, digest); + assets.put(assetId, asset); + return asset; + } + + private String createAssetId(final String ownerId, final String assetName) { + final String seed = ownerId + "/" + assetName; return UUID.nameUUIDFromBytes(seed.getBytes(StandardCharsets.UTF_8)).toString(); } - private File getFile(final String paramContextId, final String assetName) { + private File getFile(final String ownerId, final String assetName) { final Path parentPath = assetStorageLocation.toPath().normalize(); - final Path assetPath = Paths.get(paramContextId, assetName).normalize(); + final Path assetPath = Paths.get(ownerId, assetName).normalize(); final Path fullPath = parentPath.resolve(assetPath); return fullPath.toFile(); } @@ -183,17 +196,17 @@ private void recoverLocalAssets() throws IOException { continue; } - final String contextId = file.getName(); + final String ownerId = file.getName(); final File[] assetFiles = file.listFiles(); if (assetFiles == null) { - logger.warn("Unable to determine which assets exist for Parameter Context {}", contextId); + logger.warn("Unable to determine which assets exist for owner {}", ownerId); continue; } for (final File assetFile : assetFiles) { - final String assetId = createAssetId(contextId, assetFile.getName()); + final String assetId = createAssetId(ownerId, assetFile.getName()); final String digest = computeDigest(assetFile); - final Asset asset = new StandardAsset(assetId, contextId, assetFile.getName(), assetFile, digest); + final Asset asset = new StandardAsset(assetId, ownerId, assetFile.getName(), assetFile, digest); assets.put(assetId, asset); } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/asset/StandardConnectorAssetManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/asset/StandardConnectorAssetManager.java new file mode 100644 index 000000000000..87b01a47b1c3 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/asset/StandardConnectorAssetManager.java @@ -0,0 +1,225 @@ +/* + * 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.asset; + +import org.apache.nifi.nar.FileDigestUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; +import java.util.ArrayList; +import java.util.HexFormat; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; + +/** + * AssetManager implementation for Connectors. This differs from the StandardAssetManager used for Parameter Contexts in that + * Connectors want every asset added to be treated as a new asset, even if the same filename is being saved to the same Connector. + * This is because all changes to the Connector needs to be stored separately in the working configuration until applied and can + * not interfere with the active configuration. + */ +public class StandardConnectorAssetManager implements AssetManager { + + private static final Logger logger = LoggerFactory.getLogger(StandardConnectorAssetManager.class); + + public static final String ASSET_STORAGE_LOCATION_PROPERTY = "directory"; + public static final String DEFAULT_ASSET_STORAGE_LOCATION = "./connector_assets"; + + private volatile File assetStorageLocation; + private final Map assets = new ConcurrentHashMap<>(); + + @Override + public void initialize(final AssetManagerInitializationContext context) { + final String storageLocation = getStorageLocation(context); + + assetStorageLocation = new File(storageLocation); + if (!assetStorageLocation.exists()) { + try { + Files.createDirectories(assetStorageLocation.toPath()); + } catch (IOException e) { + throw new RuntimeException("The Connector Asset Manager's [%s] property is set to [%s] but the directory does not exist and cannot be created" + .formatted(ASSET_STORAGE_LOCATION_PROPERTY, storageLocation), e); + } + } + + try { + recoverLocalAssets(); + } catch (final IOException e) { + throw new RuntimeException("Unable to access connector assets", e); + } + } + + @Override + public Asset createAsset(final String connectorId, final String assetName, final InputStream contents) throws IOException { + final String assetId = generateAssetId(); + final File assetFile = getFile(connectorId, assetId, assetName); + return saveAsset(connectorId, assetId, assetName, contents, assetFile); + } + + @Override + public Asset saveAsset(final String connectorId, final String assetId, final String assetName, final InputStream contents) throws IOException { + final File assetFile = getFile(connectorId, assetId, assetName); + return saveAsset(connectorId, assetId, assetName, contents, assetFile); + } + + @Override + public Optional getAsset(final String id) { + return Optional.ofNullable(assets.get(id)); + } + + @Override + public List getAssets(final String connectorId) { + final List allAssets = new ArrayList<>(assets.values()); + final List ownerAssets = new ArrayList<>(); + for (final Asset asset : allAssets) { + if (asset.getOwnerIdentifier().equals(connectorId)) { + ownerAssets.add(asset); + } + } + return ownerAssets; + } + + @Override + public Asset createMissingAsset(final String connectorId, final String assetName) { + final String assetId = generateAssetId(); + final File file = getFile(connectorId, assetId, assetName); + final Asset asset = new StandardAsset(assetId, connectorId, assetName, file, null); + assets.put(assetId, asset); + return asset; + } + + @Override + public Optional deleteAsset(final String id) { + final Asset removed = assets.remove(id); + if (removed == null) { + return Optional.empty(); + } + + final File assetFile = removed.getFile(); + if (assetFile.exists()) { + deleteFile(assetFile); + + final File assetIdDir = assetFile.getParentFile(); + deleteFile(assetIdDir); + + final File connectorDir = assetIdDir.getParentFile(); + final File[] children = connectorDir.listFiles(); + if (children != null && children.length == 0) { + deleteFile(connectorDir); + } + } + + return Optional.of(removed); + } + + private Asset saveAsset(final String connectorId, final String assetId, final String assetName, final InputStream contents, final File assetFile) throws IOException { + final File dir = assetFile.getParentFile(); + if (!dir.exists()) { + try { + Files.createDirectories(dir.toPath()); + } catch (final IOException ioe) { + throw new IOException("Could not create directory in order to store connector asset", ioe); + } + } + + try { + Files.copy(contents, assetFile.toPath(), StandardCopyOption.REPLACE_EXISTING); + } catch (final Exception e) { + throw new IOException("Failed to write connector asset to file " + assetFile.getAbsolutePath(), e); + } + + final String digest = computeDigest(assetFile); + final Asset asset = new StandardAsset(assetId, connectorId, assetName, assetFile, digest); + assets.put(assetId, asset); + return asset; + } + + private void deleteFile(final File file) { + try { + Files.delete(file.toPath()); + } catch (final IOException e) { + logger.warn("Failed to delete [{}]", file.getAbsolutePath(), e); + } + } + + private String generateAssetId() { + return UUID.randomUUID().toString(); + } + + private File getFile(final String connectorId, final String assetId, final String assetName) { + final Path parentPath = assetStorageLocation.toPath().normalize(); + final Path assetPath = Paths.get(connectorId, assetId, assetName).normalize(); + final Path fullPath = parentPath.resolve(assetPath); + return fullPath.toFile(); + } + + private String getStorageLocation(final AssetManagerInitializationContext initializationContext) { + final String storageLocation = initializationContext.getProperties().get(ASSET_STORAGE_LOCATION_PROPERTY); + return storageLocation == null ? DEFAULT_ASSET_STORAGE_LOCATION : storageLocation; + } + + private void recoverLocalAssets() throws IOException { + final File[] connectorDirectories = assetStorageLocation.listFiles(); + if (connectorDirectories == null) { + throw new IOException("Unable to list files for connector asset storage location %s".formatted(assetStorageLocation.getAbsolutePath())); + } + + for (final File connectorDirectory : connectorDirectories) { + if (!connectorDirectory.isDirectory()) { + continue; + } + + final String connectorId = connectorDirectory.getName(); + final File[] assetIdDirectories = connectorDirectory.listFiles(); + if (assetIdDirectories == null) { + logger.warn("Unable to list directory [{}]", connectorDirectory.getAbsolutePath()); + continue; + } + + for (final File assetIdDirectory : assetIdDirectories) { + final String assetId = assetIdDirectory.getName(); + + final File[] assetFiles = assetIdDirectory.listFiles(); + if (assetFiles == null) { + logger.warn("Unable to list directory [{}]", assetIdDirectory.getAbsolutePath()); + continue; + } + + for (final File assetFile : assetFiles) { + final String assetName = assetFile.getName(); + final String digest = computeDigest(assetFile); + final Asset asset = new StandardAsset(assetId, connectorId, assetName, assetFile, digest); + assets.put(assetId, asset); + } + } + } + } + + private String computeDigest(final File file) throws IOException { + return HexFormat.of().formatHex(FileDigestUtils.getDigest(file)); + } + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/StandardConnectorValidationTrigger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/StandardConnectorValidationTrigger.java new file mode 100644 index 000000000000..2e92671d2dda --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/StandardConnectorValidationTrigger.java @@ -0,0 +1,92 @@ +/* + * 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.components.connector; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.BooleanSupplier; + +/** + * Standard implementation of ConnectorValidationTrigger that submits validation + * tasks to an ScheduledExecutorService for asynchronous execution. + */ +public class StandardConnectorValidationTrigger implements ConnectorValidationTrigger { + private static final Logger logger = LoggerFactory.getLogger(StandardConnectorValidationTrigger.class); + + private final ScheduledExecutorService threadPool; + private final BooleanSupplier flowInitialized; + private final Set activelyValidating = Collections.synchronizedSet(new HashSet<>()); + + public StandardConnectorValidationTrigger(final ScheduledExecutorService threadPool, final BooleanSupplier flowInitialized) { + this.threadPool = threadPool; + this.flowInitialized = flowInitialized; + } + + @Override + public void triggerAsync(final ConnectorNode connector) { + // Avoid adding multiple validation tasks for the same connector concurrently. This is not 100% thread safe because when a task + // is rescheduled, there's a small window where a second thread could be scheduled after the Connector is removed from 'activelyValidating' and + // before the task is rescheduled. However, this is acceptable because having multiple threads validating concurrently is safe, it's just inefficient. + final boolean added = activelyValidating.add(connector); + if (!added) { + logger.debug("Connector {} is already undergoing validation; will not trigger another validation concurrently", connector); + return; + } + + if (!flowInitialized.getAsBoolean()) { + logger.debug("Triggered to perform validation on {} asynchronously but flow is not yet initialized so will ignore validation", connector); + reschedule(connector, Duration.ofSeconds(1)); + return; + } + + threadPool.submit(() -> { + try { + if (connector.isValidationPaused()) { + logger.debug("Connector {} is currently marked as having validation paused; will retry in 1 second", connector); + reschedule(connector, Duration.ofSeconds(1)); + return; + } + + trigger(connector); + + activelyValidating.remove(connector); + } catch (final Exception e) { + logger.error("Validation for connector {} failed; will retry in 5 seconds", connector, e); + reschedule(connector, Duration.ofSeconds(5)); + } + }); + } + + @Override + public void trigger(final ConnectorNode connector) { + connector.performValidation(); + } + + private void reschedule(final ConnectorNode connector, final Duration delay) { + activelyValidating.remove(connector); + threadPool.schedule(() -> triggerAsync(connector), delay.toMillis(), TimeUnit.MILLISECONDS); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/ParameterProviderSecretProvider.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/ParameterProviderSecretProvider.java new file mode 100644 index 000000000000..c44d5f10fd22 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/ParameterProviderSecretProvider.java @@ -0,0 +1,89 @@ +/* + * 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.components.connector.secrets; + +import org.apache.nifi.components.connector.Secret; +import org.apache.nifi.controller.ParameterProviderNode; +import org.apache.nifi.parameter.Parameter; +import org.apache.nifi.parameter.ParameterDescriptor; +import org.apache.nifi.parameter.ParameterGroup; + +import java.util.ArrayList; +import java.util.List; + +public class ParameterProviderSecretProvider implements SecretProvider { + private final ParameterProviderNode parameterProvider; + + public ParameterProviderSecretProvider(final ParameterProviderNode parameterProvider) { + this.parameterProvider = parameterProvider; + } + + @Override + public String getProviderId() { + return parameterProvider.getIdentifier(); + } + + @Override + public String getProviderName() { + return parameterProvider.getName(); + } + + @Override + public List getAllSecrets() { + final List secrets = new ArrayList<>(); + + final List parameterGroups = parameterProvider.fetchParameterValues(); + for (final ParameterGroup group : parameterGroups) { + for (final Parameter parameter : group.getParameters()) { + final Secret secret = createSecret(group.getGroupName(), parameter); + secrets.add(secret); + } + } + + return secrets; + } + + private Secret createSecret(final String groupName, final Parameter parameter) { + final ParameterDescriptor descriptor = parameter.getDescriptor(); + + return new StandardSecret.Builder() + .providerId(getProviderId()) + .providerName(getProviderName()) + .groupName(groupName) + .name(descriptor.getName()) + .fullyQualifiedName(getProviderName() + "." + groupName + "." + descriptor.getName()) + .description(descriptor.getDescription()) + .value(parameter.getValue()) + .authorizable(parameterProvider) + .build(); + } + + @Override + public List getSecrets(final List fullyQualifiedSecretNames) { + final List parameterGroups = parameterProvider.fetchParameterValues(fullyQualifiedSecretNames); + final List secrets = new ArrayList<>(); + for (final ParameterGroup group : parameterGroups) { + for (final Parameter parameter : group.getParameters()) { + final Secret secret = createSecret(group.getGroupName(), parameter); + secrets.add(secret); + } + } + + return secrets; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/ParameterProviderSecretsManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/ParameterProviderSecretsManager.java new file mode 100644 index 000000000000..524351a41370 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/ParameterProviderSecretsManager.java @@ -0,0 +1,180 @@ +/* + * 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.components.connector.secrets; + +import org.apache.nifi.components.connector.Secret; +import org.apache.nifi.components.connector.SecretReference; +import org.apache.nifi.components.validation.ValidationStatus; +import org.apache.nifi.controller.ParameterProviderNode; +import org.apache.nifi.controller.flow.FlowManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class ParameterProviderSecretsManager implements SecretsManager { + private static final Logger logger = LoggerFactory.getLogger(ParameterProviderSecretsManager.class); + private FlowManager flowManager; + + @Override + public void initialize(final SecretsManagerInitializationContext initializationContext) { + this.flowManager = initializationContext.getFlowManager(); + } + + @Override + public List getAllSecrets() { + final List secrets = new ArrayList<>(); + for (final SecretProvider provider : getSecretProviders()) { + secrets.addAll(provider.getAllSecrets()); + } + + // Sort secrets by Provider Name, then Group Name, then Secret Name + secrets.sort(Comparator.comparing(Secret::getProviderName) + .thenComparing(Secret::getGroupName) + .thenComparing(Secret::getName)); + + return secrets; + } + + @Override + public Set getSecretProviders() { + final Set providers = new HashSet<>(); + for (final ParameterProviderNode parameterProviderNode : flowManager.getAllParameterProviders()) { + ValidationStatus validationStatus = parameterProviderNode.getValidationStatus(); + if (validationStatus != ValidationStatus.VALID) { + validationStatus = parameterProviderNode.performValidation(); + } + if (validationStatus != ValidationStatus.VALID) { + logger.debug("Will not use Parameter Provider {} as a Secret Provider because it is not valid", parameterProviderNode.getName()); + continue; + } + + providers.add(new ParameterProviderSecretProvider(parameterProviderNode)); + } + + return providers; + } + + @Override + public Optional getSecret(final SecretReference secretReference) { + final SecretProvider provider = findProvider(secretReference); + if (provider == null) { + return Optional.empty(); + } + + final List secrets = provider.getSecrets(List.of(secretReference.getFullyQualifiedName())); + if (secrets.isEmpty()) { + return Optional.empty(); + } + + return Optional.of(secrets.getFirst()); + } + + @Override + public Map getSecrets(final Set secretReferences) { + if (secretReferences.isEmpty()) { + return Map.of(); + } + + // Partition secret references by Provider + final Map> referencesByProvider = new HashMap<>(); + for (final SecretReference secretReference : secretReferences) { + final SecretProvider provider = findProvider(secretReference); + referencesByProvider.computeIfAbsent(provider, k -> new HashSet<>()).add(secretReference); + } + + final Map secrets = new HashMap<>(); + for (final Map.Entry> entry : referencesByProvider.entrySet()) { + final SecretProvider provider = entry.getKey(); + final Set references = entry.getValue(); + + // If no provider found, be sure to map to a null Secret rather than skipping + if (provider == null) { + for (final SecretReference secretReference : references) { + secrets.put(secretReference, null); + } + + continue; + } + + final List secretNames = new ArrayList<>(); + references.forEach(ref -> secretNames.add(ref.getFullyQualifiedName())); + final List retrievedSecrets = provider.getSecrets(secretNames); + final Map secretsByName = retrievedSecrets.stream() + .collect(Collectors.toMap(Secret::getFullyQualifiedName, Function.identity())); + + for (final SecretReference secretReference : references) { + final Secret secret = secretsByName.get(secretReference.getFullyQualifiedName()); + secrets.put(secretReference, secret); + } + } + + return secrets; + } + + private SecretProvider findProvider(final SecretReference secretReference) { + final Set providers = getSecretProviders(); + + // Search first by Provider ID, if it's provided. + final String providerId = secretReference.getProviderId(); + if (providerId != null) { + for (final SecretProvider provider : providers) { + if (providerId.equals(provider.getProviderId())) { + return provider; + } + } + + // If ID is provided but doesn't match, do not consider name. + return null; + } + + // No Provider found by ID, extract Provider Name so we can search by it. + // If not explicitly provided, extract from FQN, if it is provided. + String providerName = secretReference.getProviderName(); + if (providerName == null) { + final String fqn = secretReference.getFullyQualifiedName(); + if (fqn != null) { + final int dotIndex = fqn.indexOf('.'); + if (dotIndex > 0) { + providerName = fqn.substring(0, dotIndex); + } + } + } + + // Search by Provider Name + if (providerName != null) { + for (final SecretProvider provider : providers) { + if (providerName.equals(provider.getProviderName())) { + return provider; + } + } + } + + // No Provider found + return null; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/StandardSecret.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/StandardSecret.java new file mode 100644 index 000000000000..0a81955fe5bc --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/StandardSecret.java @@ -0,0 +1,186 @@ +/* + * 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.components.connector.secrets; + +import org.apache.nifi.authorization.Resource; +import org.apache.nifi.authorization.resource.Authorizable; + +import java.util.Objects; + +public class StandardSecret implements AuthorizableSecret { + private final String providerId; + private final String providerName; + private final String groupName; + private final String name; + private final String fullyQualifiedName; + private final String description; + private final String value; + private final Authorizable authorizable; + + private StandardSecret(final Builder builder) { + this.providerId = builder.providerId; + this.providerName = builder.providerName; + this.groupName = builder.groupName; + this.name = builder.name; + this.description = builder.description; + this.value = builder.value; + this.authorizable = builder.authorizable; + this.fullyQualifiedName = builder.fullyQualifiedName == null ? groupName + "." + name : builder.fullyQualifiedName; + } + + @Override + public String getProviderId() { + return providerId; + } + + @Override + public String getProviderName() { + return providerName; + } + + @Override + public String getGroupName() { + return groupName; + } + + @Override + public String getName() { + return name; + } + + @Override + public String getDescription() { + return description; + } + + @Override + public String getValue() { + return value; + } + + @Override + public String getFullyQualifiedName() { + return fullyQualifiedName; + } + + @Override + public String toString() { + return "StandardSecret[providerName=%s, groupName=%s, name=%s]".formatted(providerName, groupName, name); + } + + @Override + public int hashCode() { + return Objects.hash(providerName, groupName, name); + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final StandardSecret other = (StandardSecret) obj; + return Objects.equals(this.providerName, other.providerName) + && Objects.equals(this.groupName, other.groupName) + && Objects.equals(this.name, other.name); + } + + @Override + public Authorizable getParentAuthorizable() { + return authorizable.getParentAuthorizable(); + } + + @Override + public Resource getResource() { + return authorizable.getResource(); + } + + public static class Builder { + private String providerId; + private String providerName; + private String groupName; + private String name; + private String fullyQualifiedName; + private String description; + private String value; + private Authorizable authorizable; + + public Builder providerId(final String providerId) { + this.providerId = providerId; + return this; + } + + public Builder providerName(final String providerName) { + this.providerName = providerName; + return this; + } + + public Builder groupName(final String groupName) { + this.groupName = groupName; + return this; + } + + public Builder name(final String name) { + this.name = name; + return this; + } + + public Builder description(final String description) { + this.description = description; + return this; + } + + public Builder value(final String value) { + this.value = value; + return this; + } + + public Builder authorizable(final Authorizable authorizable) { + this.authorizable = authorizable; + return this; + } + + public Builder fullyQualifiedName(final String fullyQualifiedName) { + this.fullyQualifiedName = fullyQualifiedName; + return this; + } + + public StandardSecret build() { + if (groupName == null) { + throw new IllegalStateException("Group name is required"); + } + if (authorizable == null) { + throw new IllegalStateException("Authorizable is required"); + } + if (name == null || name.isEmpty()) { + throw new IllegalStateException("Secret name must be provided"); + } + if (fullyQualifiedName == null || fullyQualifiedName.isEmpty()) { + throw new IllegalStateException("Secret fully qualified name must be provided"); + } + if ((providerId == null || providerId.isEmpty()) && (providerName == null || providerName.isEmpty())) { + throw new IllegalStateException("Either Secret provider ID or provider name must be provided"); + } + + return new StandardSecret(this); + } + } + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/StandardSecretsManagerInitializationContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/StandardSecretsManagerInitializationContext.java new file mode 100644 index 000000000000..ae5013768a81 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/components/connector/secrets/StandardSecretsManagerInitializationContext.java @@ -0,0 +1,33 @@ +/* + * 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.components.connector.secrets; + +import org.apache.nifi.controller.flow.FlowManager; + +public class StandardSecretsManagerInitializationContext implements SecretsManagerInitializationContext { + private final FlowManager flowManager; + + public StandardSecretsManagerInitializationContext(final FlowManager flowManager) { + this.flowManager = flowManager; + } + + @Override + public FlowManager getFlowManager() { + return flowManager; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index 82befdb40360..2268d7fa52a9 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -16,6 +16,10 @@ */ package org.apache.nifi.controller; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.lang3.builder.HashCodeBuilder; import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; import org.apache.nifi.annotation.behavior.Restricted; @@ -43,13 +47,19 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.InvocationFailedException; +import org.apache.nifi.components.connector.components.ComponentState; +import org.apache.nifi.components.connector.components.ConnectorMethod; +import org.apache.nifi.components.connector.components.MethodArgument; import org.apache.nifi.components.validation.ValidationState; import org.apache.nifi.components.validation.ValidationStatus; import org.apache.nifi.components.validation.ValidationTrigger; import org.apache.nifi.components.validation.VerifiableComponentFactory; import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.connectable.ConnectableFlowFileActivity; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; +import org.apache.nifi.connectable.FlowFileActivity; import org.apache.nifi.connectable.Position; import org.apache.nifi.controller.exception.ProcessorInstantiationException; import org.apache.nifi.controller.scheduling.LifecycleState; @@ -136,6 +146,9 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable { private static final Logger LOG = LoggerFactory.getLogger(StandardProcessorNode.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + .setDefaultPropertyInclusion(JsonInclude.Include.NON_NULL); public static final TimeUnit DEFAULT_TIME_UNIT = TimeUnit.MILLISECONDS; public static final String DEFAULT_YIELD_PERIOD = "1 sec"; @@ -180,6 +193,8 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable private volatile BackoffMechanism backoffMechanism; private volatile String maxBackoffPeriod; + private final ConnectableFlowFileActivity flowFileActivity = new ConnectableFlowFileActivity(); + public StandardProcessorNode(final LoggableComponent processor, final String uuid, final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler, final ControllerServiceProvider controllerServiceProvider, final ReloadComponent reloadComponent, @@ -979,7 +994,9 @@ List getIncomingNonLoopConnections() { } @Override - public List verifyConfiguration(final ProcessContext context, final ComponentLog logger, final Map attributes, final ExtensionManager extensionManager) { + public List verifyConfiguration(final ProcessContext context, final ComponentLog logger, final Map attributes, final ExtensionManager extensionManager, + final ParameterLookup parameterLookup) { + final List results = new ArrayList<>(); try { @@ -987,7 +1004,7 @@ public List verifyConfiguration(final ProcessContext c final long startNanos = System.nanoTime(); // Call super's verifyConfig, which will perform component validation - results.addAll(super.verifyConfig(context.getProperties(), context.getAnnotationData(), getProcessGroup().getParameterContext())); + results.addAll(super.verifyConfig(context.getProperties(), context.getAnnotationData(), parameterLookup)); final long validationComplete = System.nanoTime(); // If any invalid outcomes from validation, we do not want to perform additional verification, because we only run additional verification when the component is valid. @@ -1121,89 +1138,88 @@ protected Collection computeValidationErrors(final ValidationC } @Override - public List validateConfig() { + public List validateConfig(final ValidationContext validationContext) { final List results = new ArrayList<>(); - final ParameterContext parameterContext = getParameterContext(); - if (parameterContext == null && !this.parameterReferences.isEmpty()) { - results.add(new ValidationResult.Builder() - .subject(RUN_SCHEDULE) - .input("Parameter Context") - .valid(false) - .explanation("Processor configuration references one or more Parameters but no Parameter Context is currently set on the Process Group.") - .build()); - } else { - for (final ParameterReference paramRef : parameterReferences) { - final Optional parameterRef = parameterContext.getParameter(paramRef.getParameterName()); - if (!parameterRef.isPresent()) { - results.add(new ValidationResult.Builder() - .subject(RUN_SCHEDULE) - .input(paramRef.getParameterName()) - .valid(false) - .explanation("Processor configuration references Parameter '" + paramRef.getParameterName() + - "' but the currently selected Parameter Context does not have a Parameter with that name") - .build()); - } else { - final ParameterDescriptor parameterDescriptor = parameterRef.get().getDescriptor(); - if (parameterDescriptor.isSensitive()) { - results.add(new ValidationResult.Builder() - .subject(RUN_SCHEDULE) - .input(parameterDescriptor.getName()) - .valid(false) - .explanation("Processor configuration cannot reference sensitive parameters") - .build()); - } - } - } - - final String schedulingPeriod = getSchedulingPeriod(); - final String evaluatedSchedulingPeriod = evaluateParameters(schedulingPeriod); + for (final ParameterReference paramRef : parameterReferences) { + final String paramName = paramRef.getParameterName(); - if (evaluatedSchedulingPeriod != null) { - switch (schedulingStrategy) { - case CRON_DRIVEN: { - try { - CronExpression.parse(evaluatedSchedulingPeriod); - } catch (final Exception e) { + if (!validationContext.isParameterDefined(paramName)) { + results.add(new ValidationResult.Builder() + .subject(RUN_SCHEDULE) + .input(paramName) + .valid(false) + .explanation("Processor configuration references Parameter '" + paramName + + "' but the currently selected Parameter Context does not have a Parameter with that name") + .build()); + } else { + final ParameterContext parameterContext = getParameterContext(); + if (parameterContext != null) { + final Optional parameterFromContext = parameterContext.getParameter(paramName); + if (parameterFromContext.isPresent()) { + final ParameterDescriptor parameterDescriptor = parameterFromContext.get().getDescriptor(); + if (parameterDescriptor.isSensitive()) { results.add(new ValidationResult.Builder() .subject(RUN_SCHEDULE) - .input(schedulingPeriod) + .input(parameterDescriptor.getName()) .valid(false) - .explanation("Scheduling Period is not a valid cron expression") + .explanation("Processor configuration cannot reference sensitive parameters") .build()); } break; } - case TIMER_DRIVEN: { - try { - final long schedulingNanos = FormatUtils.getTimeDuration(Objects.requireNonNull(evaluatedSchedulingPeriod), - TimeUnit.NANOSECONDS); - - if (schedulingNanos < 0) { - results.add(new ValidationResult.Builder() - .subject(RUN_SCHEDULE) - .input(schedulingPeriod) - .valid(false) - .explanation("Scheduling Period must be positive") - .build()); - } + } + } + } + + final String schedulingPeriod = getSchedulingPeriod(); + final String evaluatedSchedulingPeriod = validationContext.evaluateParameters(schedulingPeriod); - this.schedulingNanos.set(Math.max(MINIMUM_SCHEDULING_NANOS, schedulingNanos)); + if (evaluatedSchedulingPeriod != null) { + switch (schedulingStrategy) { + case CRON_DRIVEN: { + try { + CronExpression.parse(evaluatedSchedulingPeriod); + } catch (final Exception e) { + results.add(new ValidationResult.Builder() + .subject(RUN_SCHEDULE) + .input(schedulingPeriod) + .valid(false) + .explanation("Scheduling Period is not a valid cron expression") + .build()); + } + } + break; + case TIMER_DRIVEN: { + try { + final long schedulingNanos = FormatUtils.getTimeDuration(Objects.requireNonNull(evaluatedSchedulingPeriod), + TimeUnit.NANOSECONDS); - } catch (final Exception e) { + if (schedulingNanos < 0) { results.add(new ValidationResult.Builder() .subject(RUN_SCHEDULE) .input(schedulingPeriod) .valid(false) - .explanation("Scheduling Period is not a valid time duration") + .explanation("Scheduling Period must be positive") .build()); } - break; + + this.schedulingNanos.set(Math.max(MINIMUM_SCHEDULING_NANOS, schedulingNanos)); + + } catch (final Exception e) { + results.add(new ValidationResult.Builder() + .subject(RUN_SCHEDULE) + .input(schedulingPeriod) + .valid(false) + .explanation("Scheduling Period is not a valid time duration") + .build()); } } + break; } } + return results; } @@ -1634,6 +1650,9 @@ private void initiateStart(final ScheduledExecutorService taskScheduler, final l // Completion Timestamp is set to MAX_VALUE because we don't want to timeout until the task has a chance to run. final AtomicLong completionTimestampRef = new AtomicLong(Long.MAX_VALUE); + // Mark current time as latest activity time so that we don't show as idle when the processor was stopped. + flowFileActivity.reset(); + // Create a task to invoke the @OnScheduled annotation of the processor final Callable startupTask = () -> { final ScheduledState currentScheduleState = scheduledState.get(); @@ -1657,10 +1676,10 @@ private void initiateStart(final ScheduledExecutorService taskScheduler, final l LOG.debug("Cannot start {} because Processor is currently not valid; will try again after 5 seconds", StandardProcessorNode.this); - startupAttemptCount.incrementAndGet(); - if (startupAttemptCount.get() == 240 || startupAttemptCount.get() % 7200 == 0) { + final long attempt = startupAttemptCount.getAndIncrement(); + if (attempt % 7200 == 0) { final ValidationState validationState = getValidationState(); - procLog.error("Encountering difficulty starting. (Validation State is {}: {}). Will continue trying to start.", + procLog.warn("Encountering difficulty starting. (Validation State is {}: {}). Will continue trying to start.", validationState, validationState.getValidationErrors()); } @@ -1937,6 +1956,99 @@ private void triggerLifecycleMethod(final ProcessContext processContext, final C } } + @Override + public List getConnectorMethods() { + return getConnectorMethods(getProcessor().getClass()); + } + + @Override + public String invokeConnectorMethod(final String methodName, final Map jsonArguments, final ProcessContext processContext) throws InvocationFailedException { + final ConfigurableComponent component = getComponent(); + + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(getExtensionManager(), component.getClass(), getIdentifier())) { + final Method implementationMethod = discoverConnectorMethod(component.getClass(), methodName); + final MethodArgument[] methodArguments = getConnectorMethodArguments(methodName, implementationMethod, component); + final List argumentValues = new ArrayList<>(); + + for (final MethodArgument methodArgument : methodArguments) { + if (ProcessContext.class.equals(methodArgument.type())) { + continue; + } + + final String jsonValue = jsonArguments.get(methodArgument.name()); + if (jsonValue == null && methodArgument.required()) { + throw new IllegalArgumentException("Cannot invoke Connector Method '" + methodName + "' on " + this + " because the required argument '" + + methodArgument.name() + "' was not provided"); + } + + if (jsonValue == null) { + argumentValues.add(null); + } else { + try { + final Object argumentValue = OBJECT_MAPPER.readValue(jsonValue, methodArgument.type()); + argumentValues.add(argumentValue); + } catch (final JsonProcessingException e) { + throw new InvocationFailedException("Failed to deserialize argument '" + methodArgument.name() + "' as type " + methodArgument.type().getName() + + " for Connector Method '" + methodName + "' on " + this, e); + } + } + } + + // Inject ProcessContext if the method signature supports it + final Class[] argumentTypes = implementationMethod.getParameterTypes(); + if (argumentTypes.length > 0 && ProcessContext.class.isAssignableFrom(argumentTypes[0])) { + argumentValues.addFirst(processContext); + } + if (argumentTypes.length > 1 && ProcessContext.class.isAssignableFrom(argumentTypes[argumentTypes.length - 1])) { + argumentValues.add(processContext); + } + + try { + implementationMethod.setAccessible(true); + final Object result = implementationMethod.invoke(component, argumentValues.toArray()); + if (result == null) { + return null; + } + + return OBJECT_MAPPER.writeValueAsString(result); + } catch (final JsonProcessingException e) { + throw new InvocationFailedException("Failed to serialize return value for Connector Method '" + methodName + "' on " + this, e); + } catch (final Exception e) { + throw new InvocationFailedException(e); + } + } + } + + private MethodArgument[] getConnectorMethodArguments(final String methodName, final Method implementationMethod, final ConfigurableComponent component) throws InvocationFailedException { + if (implementationMethod == null) { + throw new InvocationFailedException("No such connector method '" + methodName + "' exists for " + component.getClass().getName()); + } + + final ConnectorMethod connectorMethodDefinition = implementationMethod.getAnnotation(ConnectorMethod.class); + final ComponentState[] componentStates = connectorMethodDefinition.allowedStates(); + final ComponentState currentState = getComponentState(); + final boolean validState = Set.of(componentStates).contains(currentState); + if (!validState) { + throw new IllegalStateException("Cannot invoke Connector Method '" + methodName + "' on " + this + " because Processor is in state " + currentState + + " but the Connector Method does not allow invocation in this state"); + } + + final MethodArgument[] methodArguments = connectorMethodDefinition.arguments(); + return methodArguments; + } + + private ComponentState getComponentState() { + final ScheduledState scheduledState = getScheduledState(); + + return switch (scheduledState) { + case DISABLED -> ComponentState.PROCESSOR_DISABLED; + case STOPPED -> ComponentState.STOPPED; + case RUNNING, RUN_ONCE -> ComponentState.RUNNING; + case STARTING -> ComponentState.STARTING; + case STOPPING -> ComponentState.STOPPING; + }; + } + /** * Marks the processor as fully stopped, and completes any futures that are to be completed as a result */ @@ -2181,4 +2293,8 @@ private void updateControllerServiceReferences() { } } + @Override + public FlowFileActivity getFlowFileActivity() { + return flowFileActivity; + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flow/AbstractFlowManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flow/AbstractFlowManager.java index 88c55c1a9b18..98382d5f17ef 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flow/AbstractFlowManager.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flow/AbstractFlowManager.java @@ -57,11 +57,13 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.UUID; @@ -121,6 +123,21 @@ public ProcessGroup getGroup(final String id) { return allProcessGroups.get(requireNonNull(id)); } + @Override + public ProcessGroup getGroup(final String groupId, final String connectorId) { + final ProcessGroup group = allProcessGroups.get(requireNonNull(groupId)); + if (group == null) { + return null; + } + + // If we found the group, return it only if it has the correct connector ID + if (Objects.equals(group.getConnectorIdentifier().orElse(null), connectorId)) { + return group; + } + + return null; + } + @Override public void onProcessGroupAdded(final ProcessGroup group) { allProcessGroups.put(group.getIdentifier(), group); @@ -627,16 +644,27 @@ public ParameterContextManager getParameterContextManager() { @Override public ParameterContext createParameterContext(final String id, final String name, final String description, - final Map parameters, final List inheritedContextIds, - final ParameterProviderConfiguration parameterProviderConfiguration) { - final boolean namingConflict = parameterContextManager.getParameterContexts().stream() + final Map parameters, final List inheritedContextIds, + final ParameterProviderConfiguration parameterProviderConfiguration) { + + final ParameterReferenceManager referenceManager = new StandardParameterReferenceManager(this::getRootGroup); + return createParameterContext(id, name, description, parameters, inheritedContextIds, parameterProviderConfiguration, referenceManager, true); + } + + protected ParameterContext createParameterContext(final String id, final String name, final String description, + final Map parameters, final List inheritedContextIds, + final ParameterProviderConfiguration parameterProviderConfiguration, final ParameterReferenceManager referenceManager, + final boolean register) { + + if (register) { + final boolean namingConflict = parameterContextManager.getParameterContexts().stream() .anyMatch(paramContext -> paramContext.getName().equals(name)); - if (namingConflict) { - throw new IllegalStateException("Cannot create Parameter Context with name '" + name + "' because a Parameter Context already exists with that name"); + if (namingConflict) { + throw new IllegalStateException("Cannot create Parameter Context with name '" + name + "' because a Parameter Context already exists with that name"); + } } - final ParameterReferenceManager referenceManager = new StandardParameterReferenceManager(this); final ParameterContext parameterContext = new StandardParameterContext.Builder() .id(id) .name(name) @@ -659,10 +687,23 @@ public ParameterContext createParameterContext(final String id, final String nam parameterContext.setInheritedParameterContexts(parameterContextList); } - parameterContextManager.addParameterContext(parameterContext); + if (register) { + parameterContextManager.addParameterContext(parameterContext); + } + return parameterContext; } + @Override + public ParameterContext createEmptyParameterContext(final String id, final String name, final String description, final ProcessGroup rootGroup) { + final Map parameterMap = new HashMap<>(); + final List inheritedContextIds = new ArrayList<>(); + + final ParameterReferenceManager parameterReferenceManager = new StandardParameterReferenceManager(() -> rootGroup); + return createParameterContext(id, name, description, + parameterMap, inheritedContextIds, null, parameterReferenceManager, false); + } + @Override public void withParameterContextResolution(final Runnable parameterContextAction) { withParameterContextResolution.set(true); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flowanalysis/FlowAnalysisUtil.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flowanalysis/FlowAnalysisUtil.java index 7c8e8963b820..6e6c58bd9287 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flowanalysis/FlowAnalysisUtil.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/flowanalysis/FlowAnalysisUtil.java @@ -19,13 +19,13 @@ import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.registry.flow.mapping.ComponentIdLookup; import org.apache.nifi.registry.flow.mapping.FlowMappingOptions; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup; public class FlowAnalysisUtil { public static final String ENCRYPTED_SENSITIVE_VALUE_SUBSTITUTE = "*****"; - public static NiFiRegistryFlowMapper createMapper(ExtensionManager extensionManager) { + public static VersionedComponentFlowMapper createMapper(ExtensionManager extensionManager) { final FlowMappingOptions flowMappingOptions = new FlowMappingOptions.Builder() .mapPropertyDescriptors(true) .mapControllerServiceReferencesToVersionedId(true) @@ -36,7 +36,7 @@ public static NiFiRegistryFlowMapper createMapper(ExtensionManager extensionMana .mapAssetReferences(true) .build(); - final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(extensionManager, flowMappingOptions) { + final VersionedComponentFlowMapper mapper = new VersionedComponentFlowMapper(extensionManager, flowMappingOptions) { @Override public String getGroupId(String groupId) { return groupId; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/parameter/StandardParameterProviderNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/parameter/StandardParameterProviderNode.java index 0e8b616ad7b3..256fa991a53c 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/parameter/StandardParameterProviderNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/parameter/StandardParameterProviderNode.java @@ -266,7 +266,7 @@ public void removeReference(final ParameterContext reference) { } @Override - protected List validateConfig() { + protected List validateConfig(final ValidationContext validationContext) { return Collections.emptyList(); } @@ -279,16 +279,32 @@ public void verifyCanFetchParameters() { } @Override - public void fetchParameters() { + public List fetchParameterValues() { final ParameterProvider parameterProvider = parameterProviderRef.get().getParameterProvider(); final ConfigurationContext configurationContext = getConfigurationContext(); - List fetchedParameterGroups; + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(getExtensionManager(), parameterProvider.getClass(), parameterProvider.getIdentifier())) { - fetchedParameterGroups = parameterProvider.fetchParameters(configurationContext); + return parameterProvider.fetchParameters(configurationContext); } catch (final IOException | RuntimeException e) { throw new IllegalStateException(String.format("Error fetching parameters for %s: %s", this, e.getMessage()), e); } + } + + @Override + public List fetchParameterValues(final List fullyQualifiedParameterNames) { + final ParameterProvider parameterProvider = parameterProviderRef.get().getParameterProvider(); + final ConfigurationContext configurationContext = getConfigurationContext(); + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(getExtensionManager(), parameterProvider.getClass(), parameterProvider.getIdentifier())) { + return parameterProvider.fetchParameters(configurationContext, fullyQualifiedParameterNames); + } catch (final IOException | RuntimeException e) { + throw new IllegalStateException(String.format("Error fetching parameters for %s: %s", this, e.getMessage()), e); + } + } + + @Override + public void fetchParameters() { + final List fetchedParameterGroups = fetchParameterValues(); if (fetchedParameterGroups == null || fetchedParameterGroups.isEmpty()) { return; } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java index 29c0cf97aef0..9a9c66bc52d3 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java @@ -603,7 +603,10 @@ protected void commit(final Checkpoint checkpoint, final boolean asynchronous) { // Update the FlowFile Repository try { final Collection repoRecords = checkpoint.records.values(); - context.getFlowFileRepository().updateRepository((Collection) repoRecords); + if (!repoRecords.isEmpty()) { + context.getFlowFileRepository().updateRepository((Collection) repoRecords); + context.getConnectable().getFlowFileActivity().updateLatestActivityTime(); + } } catch (final IOException ioe) { // if we fail to commit the session, we need to roll back // the checkpoints as well because none of the checkpoints @@ -984,7 +987,7 @@ protected void updateProvenanceRepo(final Checkpoint checkpoint) { // Therefore, we create an Iterable that can iterate over each of these events, modifying them as needed, and returning them // in the appropriate order. This prevents an unnecessary step of creating an intermediate List and adding all of those values // to the List. - // This is done in a similar veign to how Java 8's streams work, iterating over the events and returning a processed version + // This is done in a similar vein to how Java 8's streams work, iterating over the events and returning a processed version // one-at-a-time as opposed to iterating over the entire Collection and putting the results in another Collection. However, // we don't want to change the Framework to require Java 8 at this time, because it's not yet as prevalent as we would desire final Map flowFileRecordMap = new HashMap<>(); @@ -993,6 +996,9 @@ protected void updateProvenanceRepo(final Checkpoint checkpoint) { flowFileRecordMap.put(flowFile.getAttribute(CoreAttributes.UUID.key()), flowFile); } + // Update connectable to indicate how much data was received and sent. + updateTransferCounts(recordsToSubmit); + final long commitNanos = System.nanoTime(); final List autoTermEvents = checkpoint.autoTerminatedEvents; final Iterable iterable = new Iterable<>() { @@ -1039,6 +1045,24 @@ public void remove() { provenanceRepo.registerEvents(iterable); } + private void updateTransferCounts(final Collection events) { + int createdCount = 0; + long createdBytes = 0L; + for (final ProvenanceEventRecord event : events) { + if (event.getEventType() == ProvenanceEventType.CREATE) { + createdCount++; + createdBytes += event.getFileSize(); + } + } + + context.getConnectable().getFlowFileActivity().updateTransferCounts( + provenanceReporter.getFlowFilesReceived() + createdCount, + // For bytes, consider received and fetched; we don't include fetched in count, as the FlowFile count didn't change. + provenanceReporter.getBytesReceived() + provenanceReporter.getBytesFetched() + createdBytes, + provenanceReporter.getFlowFilesSent(), + provenanceReporter.getBytesSent()); + } + private void updateEventContentClaims(final ProvenanceEventBuilder builder, final FlowFile flowFile, final StandardRepositoryRecord repoRecord) { final ContentClaim originalClaim = repoRecord.getOriginalClaim(); if (originalClaim == null) { @@ -1246,6 +1270,10 @@ protected synchronized void rollback(final boolean penalize, final boolean rollb resetWriteClaims(); resetReadClaim(); + if (!recordValues.isEmpty()) { + context.getConnectable().getFlowFileActivity().updateLatestActivityTime(); + } + if (recordsToHandle.isEmpty()) { LOG.trace("{} was rolled back, but no events were performed by this ProcessSession", this); acknowledgeRecords(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInvocationHandler.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInvocationHandler.java index 9b87040540c8..3cb82157cae2 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInvocationHandler.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceInvocationHandler.java @@ -17,6 +17,8 @@ package org.apache.nifi.controller.service; import org.apache.commons.lang3.ClassUtils; +import org.apache.nifi.components.connector.components.ComponentState; +import org.apache.nifi.components.connector.components.ConnectorMethod; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ControllerServiceProxyWrapper; import org.apache.nifi.nar.ExtensionManager; @@ -29,7 +31,6 @@ import java.lang.reflect.Method; import java.lang.reflect.Proxy; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -38,15 +39,7 @@ public class StandardControllerServiceInvocationHandler implements ControllerServiceInvocationHandler { private static final Logger logger = LoggerFactory.getLogger(StandardControllerServiceInvocationHandler.class); private static final Method PROXY_WRAPPER_GET_WRAPPED_METHOD; - - private static final Set validDisabledMethods; static { - // methods that are okay to be called when the service is disabled. - final Set validMethods = new HashSet<>(); - validMethods.addAll(Arrays.asList(ControllerService.class.getMethods())); - validMethods.addAll(Arrays.asList(Object.class.getMethods())); - validDisabledMethods = Collections.unmodifiableSet(validMethods); - try { PROXY_WRAPPER_GET_WRAPPED_METHOD = ControllerServiceProxyWrapper.class.getMethod("getWrapped"); } catch (final NoSuchMethodException e) { @@ -54,6 +47,7 @@ public class StandardControllerServiceInvocationHandler implements ControllerSer } } + private final Set validDisabledMethods; private final ControllerService originalService; private final AtomicReference serviceNodeHolder = new AtomicReference<>(null); private final ExtensionManager extensionManager; @@ -73,6 +67,34 @@ public StandardControllerServiceInvocationHandler(final ExtensionManager extensi this.extensionManager = extensionManager; this.originalService = originalService; this.serviceNodeHolder.set(serviceNode); + validDisabledMethods = determineDisabledAllowedMethods(originalService); + } + + private Set determineDisabledAllowedMethods(final ControllerService implementation) { + // methods that are okay to be called when the service is disabled. + final Set validMethods = new HashSet<>(); + validMethods.addAll(Arrays.asList(ControllerService.class.getMethods())); + validMethods.addAll(Arrays.asList(Object.class.getMethods())); + + // If there are any Connector Methods that are allowed to be called when the service is STOPPED or STOPPING, add those as well. + // This allows us to perform actions from Connectors, such as obtaining a list of available resources, during Connector configuration, etc. + // while the service is currently stopped. + for (final Method method : implementation.getClass().getDeclaredMethods()) { + final ConnectorMethod connectorMethod = method.getAnnotation(ConnectorMethod.class); + if (connectorMethod == null) { + continue; + } + + final ComponentState[] allowedStates = connectorMethod.allowedStates(); + for (final ComponentState allowedState : allowedStates) { + if (allowedState == ComponentState.STOPPED || allowedState == ComponentState.STOPPING) { + validMethods.add(method); + break; + } + } + } + + return validMethods; } @Override diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java index bc16e2cdb33d..ac5d12ef9f7e 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java @@ -16,6 +16,10 @@ */ package org.apache.nifi.controller.service; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.Restricted; import org.apache.nifi.annotation.documentation.DeprecationNotice; @@ -36,6 +40,10 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.InvocationFailedException; +import org.apache.nifi.components.connector.components.ComponentState; +import org.apache.nifi.components.connector.components.ConnectorMethod; +import org.apache.nifi.components.connector.components.MethodArgument; import org.apache.nifi.components.validation.ValidationState; import org.apache.nifi.components.validation.ValidationStatus; import org.apache.nifi.components.validation.ValidationTrigger; @@ -102,6 +110,9 @@ public class StandardControllerServiceNode extends AbstractComponentNode implements ControllerServiceNode { private static final Logger LOG = LoggerFactory.getLogger(StandardControllerServiceNode.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + .setDefaultPropertyInclusion(JsonInclude.Include.NON_NULL); private static final long INCREMENTAL_VALIDATION_DELAY_MS = 1000; private static final Duration MAXIMUM_DELAY = Duration.ofMinutes(10); @@ -507,7 +518,7 @@ public void verifyCanPerformVerification() { @Override public List verifyConfiguration(final ConfigurationContext context, final ComponentLog logger, final Map variables, - final ExtensionManager extensionManager) { + final ExtensionManager extensionManager, final ParameterLookup parameterLookup) { final List results = new ArrayList<>(); @@ -516,7 +527,7 @@ public List verifyConfiguration(final ConfigurationCon final long startNanos = System.nanoTime(); // Call super's verifyConfig, which will perform component validation - results.addAll(super.verifyConfig(context.getProperties(), context.getAnnotationData(), getProcessGroup() == null ? null : getProcessGroup().getParameterContext())); + results.addAll(super.verifyConfig(context.getProperties(), context.getAnnotationData(), parameterLookup)); final long validationComplete = System.nanoTime(); // If any invalid outcomes from validation, we do not want to perform additional verification, because we only run additional verification when the component is valid. @@ -607,7 +618,7 @@ public ValidationState performValidation(final ValidationContext validationConte } @Override - protected List validateConfig() { + protected List validateConfig(final ValidationContext validationContext) { return Collections.emptyList(); } @@ -632,6 +643,13 @@ protected List validateConfig() { */ @Override public CompletableFuture enable(final ScheduledExecutorService scheduler, final long administrativeYieldMillis, final boolean completeExceptionallyOnFailure) { + return enable(scheduler, administrativeYieldMillis, completeExceptionallyOnFailure, null); + } + + @Override + public CompletableFuture enable(final ScheduledExecutorService scheduler, final long administrativeYieldMillis, final boolean completeExceptionallyOnFailure, + final ConfigurationContext providedConfigurationContext) { + final CompletableFuture future = new CompletableFuture<>(); if (!stateTransition.transitionToEnabling(ControllerServiceState.DISABLED, future)) { @@ -650,7 +668,9 @@ public CompletableFuture enable(final ScheduledExecutorService scheduler, scheduler.execute(new Runnable() { @Override public void run() { - final ConfigurationContext configContext = new StandardConfigurationContext(serviceNode, controllerServiceProvider, null); + final ConfigurationContext configContext = providedConfigurationContext == null + ? new StandardConfigurationContext(serviceNode, controllerServiceProvider, null) + : providedConfigurationContext; if (!isActive()) { LOG.warn("Enabling {} stopped: no active status", serviceNode); @@ -659,9 +679,17 @@ public void run() { return; } - // Perform Validation and evaluate status before continuing - performValidation(); - final ValidationState validationState = getValidationState(); + // Perform validation - if a ConfigurationContext was provided, validate against its properties + final ValidationState validationState; + if (providedConfigurationContext == null) { + performValidation(); + validationState = getValidationState(); + } else { + final Map properties = providedConfigurationContext.getAllProperties(); + final ValidationContext validationContext = createValidationContext(properties, getAnnotationData(), getParameterLookup(), true); + validationState = performValidation(validationContext); + } + final ValidationStatus validationStatus = validationState.getStatus(); if (validationStatus == ValidationStatus.VALID) { LOG.debug("Enabling {} proceeding after performing validation", serviceNode); @@ -685,6 +713,7 @@ public void run() { LOG.debug("Validation rescheduling rejected for {}", serviceNode, e); future.completeExceptionally(new IllegalStateException("Enabling %s rejected: Validation Status [%s] Errors %s".formatted(serviceNode, validationStatus, errors))); } + // Enable command rescheduled or rejected return; } @@ -761,6 +790,12 @@ public CompletableFuture disable(final ScheduledExecutorService scheduler) } final CompletableFuture future = new CompletableFuture<>(); + // If already disabled, complete immediately + if (getState() == ControllerServiceState.DISABLED) { + future.complete(null); + return future; + } + final boolean transitioned = this.stateTransition.transitionToDisabling(ControllerServiceState.ENABLING, future); if (transitioned) { // If we transitioned from ENABLING to DISABLING, we need to immediately complete the disable @@ -910,6 +945,98 @@ public void migrateConfiguration(final Map originalPropertyValue } } + @Override + public String invokeConnectorMethod(final String methodName, final Map jsonArguments, final ConfigurationContext configurationContext) throws InvocationFailedException { + final ConfigurableComponent component = getComponent(); + + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(getExtensionManager(), component.getClass(), getIdentifier())) { + final Method implementationMethod = discoverConnectorMethod(component.getClass(), methodName); + final MethodArgument[] methodArguments = getConnectorMethodArguments(methodName, implementationMethod, component); + final List argumentValues = new ArrayList<>(); + + for (final MethodArgument methodArgument : methodArguments) { + if (ConfigurationContext.class.equals(methodArgument.type())) { + continue; + } + + final String jsonValue = jsonArguments.get(methodArgument.name()); + if (jsonValue == null && methodArgument.required()) { + throw new IllegalArgumentException("Cannot invoke Connector Method '" + methodName + "' on " + this + " because the required argument '" + + methodArgument.name() + "' was not provided"); + } + + if (jsonValue == null) { + argumentValues.add(null); + } else { + try { + final Object argumentValue = OBJECT_MAPPER.readValue(jsonValue, methodArgument.type()); + argumentValues.add(argumentValue); + } catch (final JsonProcessingException e) { + throw new InvocationFailedException("Failed to deserialize argument '" + methodArgument.name() + "' as type " + methodArgument.type().getName() + + " for Connector Method '" + methodName + "' on " + this, e); + } + } + } + + // Inject ConfigurationContext if the method signature supports it + final Class[] argumentTypes = implementationMethod.getParameterTypes(); + if (argumentTypes.length > 0 && ConfigurationContext.class.isAssignableFrom(argumentTypes[0])) { + argumentValues.addFirst(configurationContext); + } + if (argumentTypes.length > 1 && ConfigurationContext.class.isAssignableFrom(argumentTypes[argumentTypes.length - 1])) { + argumentValues.add(configurationContext); + } + + try { + implementationMethod.setAccessible(true); + final Object result = implementationMethod.invoke(component, argumentValues.toArray()); + if (result == null) { + return null; + } + + return OBJECT_MAPPER.writeValueAsString(result); + } catch (final JsonProcessingException e) { + throw new InvocationFailedException("Failed to serialize return value for Connector Method '" + methodName + "' on " + this, e); + } catch (final Exception e) { + throw new InvocationFailedException(e); + } + } + } + + @Override + public List getConnectorMethods() { + return getConnectorMethods(getControllerServiceImplementation().getClass()); + } + + private MethodArgument[] getConnectorMethodArguments(final String methodName, final Method implementationMethod, final ConfigurableComponent component) throws InvocationFailedException { + if (implementationMethod == null) { + throw new InvocationFailedException("No such connector method '" + methodName + "' exists for " + component.getClass().getName()); + } + + final ConnectorMethod connectorMethodDefinition = implementationMethod.getAnnotation(ConnectorMethod.class); + final ComponentState[] componentStates = connectorMethodDefinition.allowedStates(); + final ComponentState currentState = getComponentState(); + final boolean validState = Set.of(componentStates).contains(currentState); + if (!validState) { + throw new IllegalStateException("Cannot invoke Connector Method '" + methodName + "' on " + this + " because Processor is in state " + currentState + + " but the Connector Method does not allow invocation in this state"); + } + + final MethodArgument[] methodArguments = connectorMethodDefinition.arguments(); + return methodArguments; + } + + private ComponentState getComponentState() { + final ControllerServiceState scheduledState = getState(); + + return switch (scheduledState) { + case DISABLED -> ComponentState.STOPPED; + case DISABLING -> ComponentState.STOPPING; + case ENABLING -> ComponentState.STARTING; + case ENABLED -> ComponentState.RUNNING; + }; + } + @Override protected void performFlowAnalysisOnThis() { getValidationContextFactory().getFlowAnalyzer().ifPresent(flowAnalyzer -> flowAnalyzer.analyzeControllerService(this)); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java index 243a3c36c927..baec4ac365e1 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java @@ -269,7 +269,6 @@ public CompletableFuture enableControllerService(final ControllerServiceNo } serviceNode.verifyCanEnable(); - serviceNode.reloadAdditionalResourcesIfNecessary(); return processScheduler.enableControllerService(serviceNode); } @@ -320,7 +319,7 @@ private Collection removeControllerServicesWithUnavailabl } @Override - public Future enableControllerServicesAsync(final Collection serviceNodes) { + public CompletableFuture enableControllerServicesAsync(final Collection serviceNodes) { final CompletableFuture future = new CompletableFuture<>(); processScheduler.submitFrameworkTask(() -> { try { @@ -392,7 +391,7 @@ private void enableControllerServices(final Collection se } @Override - public Future enableControllerServiceAndDependencies(final ControllerServiceNode serviceNode) { + public CompletableFuture enableControllerServiceAndDependencies(final ControllerServiceNode serviceNode) { if (serviceNode.isActive()) { logger.debug("Enabling of Controller Service {} triggered but service already enabled", serviceNode); return CompletableFuture.completedFuture(null); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceResolver.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceResolver.java index 7b165b8a6cbb..442d301fddc9 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceResolver.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceResolver.java @@ -30,7 +30,7 @@ import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.registry.flow.FlowSnapshotContainer; import org.apache.nifi.registry.flow.RegisteredFlowSnapshot; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import java.util.ArrayList; import java.util.Collection; @@ -47,13 +47,13 @@ public class StandardControllerServiceResolver implements ControllerServiceResol private final Authorizer authorizer; private final FlowManager flowManager; - private final NiFiRegistryFlowMapper flowMapper; + private final VersionedComponentFlowMapper flowMapper; private final ControllerServiceProvider controllerServiceProvider; private final ControllerServiceApiLookup controllerServiceApiLookup; public StandardControllerServiceResolver(final Authorizer authorizer, final FlowManager flowManager, - final NiFiRegistryFlowMapper flowMapper, + final VersionedComponentFlowMapper flowMapper, final ControllerServiceProvider controllerServiceProvider, final ControllerServiceApiLookup controllerServiceApiLookup) { this.authorizer = authorizer; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java index 7ea59751e8ca..3779f5b34452 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java @@ -290,6 +290,7 @@ private String getBulletinStoreKey(final Bulletin bulletin) { case FLOW_ANALYSIS_RULE -> FLOW_ANALYSIS_RULE_BULLETIN_STORE_KEY; case PARAMETER_PROVIDER -> PARAMETER_PROVIDER_BULLETIN_STORE_KEY; case FLOW_REGISTRY_CLIENT -> FLOW_REGISTRY_CLIENT_STORE_KEY; + case CONNECTOR -> bulletin.getSourceId(); default -> bulletin.getGroupId(); }; } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java index 4d9e6bbeaf97..b67f79e56bf4 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java @@ -113,7 +113,7 @@ import org.apache.nifi.registry.flow.diff.StandardComparableDataFlow; import org.apache.nifi.registry.flow.diff.StandardFlowComparator; import org.apache.nifi.registry.flow.diff.StaticDifferenceDescriptor; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import org.apache.nifi.remote.PublicPort; import org.apache.nifi.remote.RemoteGroupPort; import org.apache.nifi.remote.StandardRemoteProcessGroupPortDescriptor; @@ -312,7 +312,7 @@ public ComponentAdditions addVersionedComponentsToProcessGroup(final ProcessGrou @Override public void synchronize(final ProcessGroup group, final VersionedExternalFlow versionedExternalFlow, final FlowSynchronizationOptions options) { - final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(context.getExtensionManager(), context.getFlowMappingOptions()); + final VersionedComponentFlowMapper mapper = new VersionedComponentFlowMapper(context.getExtensionManager(), context.getFlowMappingOptions()); final VersionedProcessGroup versionedGroup = mapper.mapProcessGroup(group, context.getControllerServiceProvider(), context.getFlowManager(), true); final ComparableDataFlow localFlow = new StandardComparableDataFlow("Currently Loaded Flow", versionedGroup); @@ -998,7 +998,7 @@ private void removeTemporaryFunnel(final ProcessGroup group) { private Map componentsById(final ProcessGroup group, final Function> retrieveComponents) { return retrieveComponents.apply(group).stream() .collect(Collectors.toMap(component -> component.getVersionedComponentId().orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())), Function.identity())); + VersionedComponentFlowMapper.generateVersionedComponentId(component.getIdentifier())), Function.identity())); } private Map componentsById(final ProcessGroup group, final Function> retrieveComponents, @@ -1006,7 +1006,7 @@ private Map componentsById(final ProcessGroup group, final Functi return retrieveComponents.apply(group).stream() .collect(Collectors.toMap(component -> retrieveVersionedComponentId.apply(component).orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(retrieveId.apply(component))), Function.identity())); + VersionedComponentFlowMapper.generateVersionedComponentId(retrieveId.apply(component))), Function.identity())); } private void synchronizeFunnels(final ProcessGroup group, final VersionedProcessGroup proposed, final Map funnelsByVersionedId) { @@ -1242,7 +1242,7 @@ public void verifyCanSynchronize(final ProcessGroup group, final VersionedProces final Map removedInputPortsByVersionId = new HashMap<>(); group.getInputPorts() .forEach(port -> removedInputPortsByVersionId.put(port.getVersionedComponentId().orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(port.getIdentifier())), port)); + VersionedComponentFlowMapper.generateVersionedComponentId(port.getIdentifier())), port)); flowContents.getInputPorts().stream() .map(VersionedPort::getIdentifier) @@ -1261,7 +1261,7 @@ public void verifyCanSynchronize(final ProcessGroup group, final VersionedProces final Map removedOutputPortsByVersionId = new HashMap<>(); group.getOutputPorts() .forEach(port -> removedOutputPortsByVersionId.put(port.getVersionedComponentId().orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(port.getIdentifier())), port)); + VersionedComponentFlowMapper.generateVersionedComponentId(port.getIdentifier())), port)); flowContents.getOutputPorts().stream() .map(VersionedPort::getIdentifier) @@ -1288,7 +1288,7 @@ private void verifyCanInstantiateProcessors(final ProcessGroup group, final Set< group.findAllProcessors() .forEach(proc -> proposedProcessors.remove(proc.getVersionedComponentId().orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(proc.getIdentifier())))); + VersionedComponentFlowMapper.generateVersionedComponentId(proc.getIdentifier())))); for (final VersionedProcessor processorToAdd : proposedProcessors.values()) { final String processorToAddClass = processorToAdd.getType(); @@ -1319,7 +1319,7 @@ private void verifyCanInstantiateControllerServices(final ProcessGroup group, fi group.findAllControllerServices() .forEach(service -> proposedServices.remove(service.getVersionedComponentId().orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(service.getIdentifier())))); + VersionedComponentFlowMapper.generateVersionedComponentId(service.getIdentifier())))); for (final VersionedControllerService serviceToAdd : proposedServices.values()) { final String serviceToAddClass = serviceToAdd.getType(); @@ -1348,7 +1348,7 @@ private void verifyCanInstantiateConnections(final ProcessGroup group, final Set group.findAllConnections() .forEach(conn -> proposedConnections.remove(conn.getVersionedComponentId().orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(conn.getIdentifier())))); + VersionedComponentFlowMapper.generateVersionedComponentId(conn.getIdentifier())))); for (final VersionedConnection connectionToAdd : proposedConnections.values()) { if (connectionToAdd.getPrioritizers() != null) { @@ -1377,13 +1377,18 @@ private ProcessGroup addProcessGroup(final ProcessGroup destination, final Versi final Map versionedParameterContexts, final Map parameterProviderReferences, ProcessGroup topLevelGroup) throws ProcessorInstantiationException { final String id = componentIdGenerator.generateUuid(proposed.getIdentifier(), proposed.getInstanceIdentifier(), destination.getIdentifier()); - final ProcessGroup group = context.getFlowManager().createProcessGroup(id); + final String connectorId = destination.getConnectorIdentifier().orElse(null); + final ProcessGroup group = context.getFlowManager().createProcessGroup(id, connectorId); group.setVersionedComponentId(proposed.getIdentifier()); group.setParent(destination); group.setName(proposed.getName()); destination.addProcessGroup(group); + // Connectors will have a single parameter context so if we are creating a group set the context of the parent process group. + if (connectorId != null) { + group.setParameterContext(destination.getParameterContext()); + } synchronize(group, proposed, versionedParameterContexts, parameterProviderReferences, topLevelGroup, true); return group; @@ -1949,7 +1954,8 @@ public void synchronizeProcessGroupSettings(final ProcessGroup processGroup, fin final ProcessGroup groupToUpdate; if (processGroup == null) { final String groupId = synchronizationOptions.getComponentIdGenerator().generateUuid(proposed.getIdentifier(), proposed.getInstanceIdentifier(), parentGroup.getIdentifier()); - final ProcessGroup group = context.getFlowManager().createProcessGroup(groupId); + final String connectorId = parentGroup.getConnectorIdentifier().orElse(null); + final ProcessGroup group = context.getFlowManager().createProcessGroup(groupId, connectorId); group.setVersionedComponentId(proposed.getIdentifier()); group.setParent(parentGroup); group.setName(proposed.getName()); @@ -2139,7 +2145,7 @@ private void verifyNotInherited(final String parameterContextId) { } private void updateParameterContext(final ProcessGroup group, final VersionedProcessGroup proposed, final Map versionedParameterContexts, - final Map parameterProviderReferences, final ComponentIdGenerator componentIdGenerator) { + final Map parameterProviderReferences, final ComponentIdGenerator componentIdGenerator) { // Update the Parameter Context final ParameterContext currentParamContext = group.getParameterContext(); final String proposedParameterContextName = proposed.getParameterContextName(); @@ -2164,9 +2170,9 @@ private void updateParameterContext(final ProcessGroup group, final VersionedPro final ParameterContext contextByName = getParameterContextByName(versionedParameterContext.getName()); if (contextByName == null) { final String parameterContextId = componentIdGenerator.generateUuid(versionedParameterContext.getName(), - versionedParameterContext.getName(), versionedParameterContext.getName()); + versionedParameterContext.getName(), versionedParameterContext.getName()); selectedParameterContext = createParameterContext(versionedParameterContext, parameterContextId, versionedParameterContexts, - parameterProviderReferences, componentIdGenerator); + parameterProviderReferences, componentIdGenerator); } else { selectedParameterContext = contextByName; addMissingConfiguration(versionedParameterContext, selectedParameterContext, versionedParameterContexts, parameterProviderReferences, componentIdGenerator); @@ -3072,7 +3078,7 @@ private void updateProcessor(final ProcessorNode processor, final VersionedProce private String getServiceInstanceId(final String serviceVersionedComponentId, final ProcessGroup group) { for (final ControllerServiceNode serviceNode : group.getControllerServices(false)) { final String versionedId = serviceNode.getVersionedComponentId().orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(serviceNode.getIdentifier())); + VersionedComponentFlowMapper.generateVersionedComponentId(serviceNode.getIdentifier())); if (versionedId.equals(serviceVersionedComponentId)) { return serviceNode.getIdentifier(); } @@ -3691,7 +3697,7 @@ private Connectable getConnectable(final ProcessGroup group, final ConnectableCo final String rpgId = connectableComponent.getGroupId(); final Optional rpgOption = group.getRemoteProcessGroups().stream() .filter(component -> rpgId.equals(component.getIdentifier()) || rpgId.equals(component.getVersionedComponentId().orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())))) + VersionedComponentFlowMapper.generateVersionedComponentId(component.getIdentifier())))) .findAny(); if (rpgOption.isEmpty()) { @@ -3717,7 +3723,7 @@ private Connectable getConnectable(final ProcessGroup group, final ConnectableCo final String rpgId = connectableComponent.getGroupId(); final Optional rpgOption = group.getRemoteProcessGroups().stream() .filter(component -> rpgId.equals(component.getIdentifier()) || rpgId.equals(component.getVersionedComponentId().orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier())))) + VersionedComponentFlowMapper.generateVersionedComponentId(component.getIdentifier())))) .findAny(); if (rpgOption.isEmpty()) { @@ -3918,12 +3924,12 @@ private void updateFlowAnalysisRule(final FlowAnalysisRuleNode flowAnalysisRule, } private boolean matchesId(final T component, final String id) { - return id.equals(component.getIdentifier()) || id.equals(component.getVersionedComponentId().orElse(NiFiRegistryFlowMapper.generateVersionedComponentId(component.getIdentifier()))); + return id.equals(component.getIdentifier()) || id.equals(component.getVersionedComponentId().orElse(VersionedComponentFlowMapper.generateVersionedComponentId(component.getIdentifier()))); } private boolean matchesGroupId(final ProcessGroup group, final String groupId) { return groupId.equals(group.getIdentifier()) || group.getVersionedComponentId().orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(group.getIdentifier())).equals(groupId); + VersionedComponentFlowMapper.generateVersionedComponentId(group.getIdentifier())).equals(groupId); } private void findAllProcessors(final Set processors, final Set childGroups, final Map map) { @@ -3975,7 +3981,7 @@ private void verifyCanRemoveMissingComponents(final ProcessGroup processGroup, f // match group's current connections to proposed connections to determine if they've been removed for (final Connection connection : processGroup.getConnections()) { final String versionedId = connection.getVersionedComponentId().orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(connection.getIdentifier())); + VersionedComponentFlowMapper.generateVersionedComponentId(connection.getIdentifier())); final VersionedConnection proposedConnection = proposedConnectionsByVersionedId.get(versionedId); if (proposedConnection == null) { // connection doesn't exist in proposed connections, make sure it doesn't have any data in it @@ -3994,7 +4000,7 @@ private void verifyCanRemoveMissingComponents(final ProcessGroup processGroup, f // match current child groups to proposed child groups to determine if they've been removed for (final ProcessGroup childGroup : processGroup.getProcessGroups()) { final String versionedId = childGroup.getVersionedComponentId().orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(childGroup.getIdentifier())); + VersionedComponentFlowMapper.generateVersionedComponentId(childGroup.getIdentifier())); final VersionedProcessGroup proposedChildGroup = proposedGroupsByVersionedId.get(versionedId); if (proposedChildGroup == null) { if (verifyConnectionRemoval) { @@ -4020,7 +4026,7 @@ private ControllerServiceNode getVersionedControllerService(final ProcessGroup g for (final ControllerServiceNode serviceNode : group.getControllerServices(false)) { final String serviceNodeVersionedComponentId = serviceNode.getVersionedComponentId().orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(serviceNode.getIdentifier())); + VersionedComponentFlowMapper.generateVersionedComponentId(serviceNode.getIdentifier())); if (serviceNodeVersionedComponentId.equals(versionedComponentId)) { return serviceNode; } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java index 1e6deb27ae08..1dda6c268432 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java @@ -34,11 +34,13 @@ import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; +import org.apache.nifi.connectable.FlowFileActivity; import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.LocalPort; import org.apache.nifi.connectable.Port; import org.apache.nifi.connectable.Position; import org.apache.nifi.connectable.Positionable; +import org.apache.nifi.connectable.ProcessGroupFlowFileActivity; import org.apache.nifi.controller.ComponentNode; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.controller.ControllerService; @@ -105,7 +107,7 @@ import org.apache.nifi.registry.flow.diff.StandardFlowComparator; import org.apache.nifi.registry.flow.mapping.ComponentIdLookup; import org.apache.nifi.registry.flow.mapping.FlowMappingOptions; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup; import org.apache.nifi.remote.PublicPort; import org.apache.nifi.remote.RemoteGroupPort; @@ -138,6 +140,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -170,6 +173,7 @@ public final class StandardProcessGroup implements ProcessGroup { private final AtomicReference versionedComponentId = new AtomicReference<>(); private final AtomicReference versionControlInfo = new AtomicReference<>(); private static final SecureRandom randomGenerator = new SecureRandom(); + private final String connectorId; private final ProcessScheduler scheduler; private final ControllerServiceProvider controllerServiceProvider; @@ -196,6 +200,8 @@ public final class StandardProcessGroup implements ProcessGroup { private volatile ExecutionEngine executionEngine = ExecutionEngine.INHERITED; private volatile int maxConcurrentTasks = 1; private volatile String statelessFlowTimeout = "1 min"; + private volatile Authorizable explicitParentAuthorizable; + private final FlowFileActivity flowFileActivity = new ProcessGroupFlowFileActivity(this); private FlowFileConcurrency flowFileConcurrency = FlowFileConcurrency.UNBOUNDED; private volatile FlowFileGate flowFileGate = new UnboundedFlowFileGate(); @@ -225,7 +231,7 @@ public StandardProcessGroup(final String id, final ControllerServiceProvider ser final StateManagerProvider stateManagerProvider, final FlowManager flowManager, final ReloadComponent reloadComponent, final NodeTypeProvider nodeTypeProvider, final NiFiProperties nifiProperties, final StatelessGroupNodeFactory statelessGroupNodeFactory, - final AssetManager assetManager) { + final AssetManager assetManager, final String connectorId) { this.id = id; this.controllerServiceProvider = serviceProvider; @@ -239,6 +245,7 @@ public StandardProcessGroup(final String id, final ControllerServiceProvider ser this.reloadComponent = reloadComponent; this.nodeTypeProvider = nodeTypeProvider; this.assetManager = assetManager; + this.connectorId = connectorId; name = new AtomicReference<>(); position = new AtomicReference<>(new Position(0D, 0D)); @@ -294,7 +301,7 @@ public void setParent(final ProcessGroup newParent) { @Override public Authorizable getParentAuthorizable() { - return getParent(); + return explicitParentAuthorizable == null ? getParent() : explicitParentAuthorizable; } @Override @@ -332,6 +339,11 @@ public void setName(final String name) { setLoggingAttributes(); } + @Override + public Optional getConnectorIdentifier() { + return Optional.ofNullable(connectorId); + } + @Override public void setPosition(final Position position) { this.position.set(position); @@ -1574,6 +1586,7 @@ private DropFlowFileStatus handleDropAllFlowFiles(String dropRequestId, Function aggregateDropFlowFileStatus.setState(null); AtomicBoolean processedAtLeastOne = new AtomicBoolean(false); + final List> completionFutures = new ArrayList<>(); connections.stream() .map(Connection::getFlowFileQueue) @@ -1581,10 +1594,21 @@ private DropFlowFileStatus handleDropAllFlowFiles(String dropRequestId, Function .forEach(additionalDropFlowFileStatus -> { aggregate(aggregateDropFlowFileStatus, additionalDropFlowFileStatus); processedAtLeastOne.set(true); + completionFutures.add(additionalDropFlowFileStatus.getCompletionFuture()); }); if (processedAtLeastOne.get()) { resultDropFlowFileStatus = aggregateDropFlowFileStatus; + + // When all individual drop requests complete, mark the aggregate as complete + CompletableFuture.allOf(completionFutures.toArray(new CompletableFuture[0])) + .whenComplete((result, throwable) -> { + if (throwable != null) { + aggregateDropFlowFileStatus.setState(DropFlowFileState.FAILURE, throwable.getMessage()); + } else { + aggregateDropFlowFileStatus.setState(DropFlowFileState.COMPLETE); + } + }); } else { resultDropFlowFileStatus = null; } @@ -1715,7 +1739,7 @@ public RemoteProcessGroup getRemoteProcessGroup(final String id) { } @Override - public Future startProcessor(final ProcessorNode processor, final boolean failIfStopping) { + public CompletableFuture startProcessor(final ProcessorNode processor, final boolean failIfStopping) { readLock.lock(); try { if (getProcessor(processor.getIdentifier()) == null) { @@ -1731,8 +1755,6 @@ public Future startProcessor(final ProcessorNode processor, final boolean return CompletableFuture.completedFuture(null); } - processor.reloadAdditionalResourcesIfNecessary(); - return scheduler.startProcessor(processor, failIfStopping); } finally { readLock.unlock(); @@ -2095,7 +2117,7 @@ public ProcessGroup findProcessGroup(final String id) { return this; } - final ProcessGroup group = flowManager.getGroup(id); + final ProcessGroup group = flowManager.getGroup(id, getConnectorIdentifier().orElse(null)); if (group == null) { return null; } @@ -3842,16 +3864,22 @@ public void updateFlow(final VersionedExternalFlow proposedSnapshot, final Strin final ComponentScheduler defaultComponentScheduler = new DefaultComponentScheduler(controllerServiceProvider, stateLookup); final ComponentScheduler retainExistingStateScheduler = new RetainExistingStateComponentScheduler(this, defaultComponentScheduler); - final FlowSynchronizationOptions synchronizationOptions = new FlowSynchronizationOptions.Builder() + final FlowSynchronizationOptions.Builder flowSynchronizationBuilder = new FlowSynchronizationOptions.Builder() .componentIdGenerator(idGenerator) .componentComparisonIdLookup(VersionedComponent::getIdentifier) .componentScheduler(retainExistingStateScheduler) .ignoreLocalModifications(!verifyNotDirty) .updateDescendantVersionedFlows(updateDescendantVersionedFlows) .updateGroupSettings(updateSettings) - .updateRpgUrls(false) - .propertyDecryptor(value -> null) - .build(); + .updateRpgUrls(false); + // Connectors should not have encrypted values copied from versioned flow. However we do need to decrypt parameter references. + if (getConnectorIdentifier().isPresent()) { + flowSynchronizationBuilder.propertyDecryptor(value -> value); + } else { + flowSynchronizationBuilder.propertyDecryptor(value -> null); + } + + final FlowSynchronizationOptions synchronizationOptions = flowSynchronizationBuilder.build(); final FlowMappingOptions flowMappingOptions = new FlowMappingOptions.Builder() .mapSensitiveConfiguration(false) @@ -3924,7 +3952,7 @@ public Set getAncestorServiceIds() { // because this allows us to find the Controller Service when doing a Flow Diff. ancestorServiceIds = parentGroup.getControllerServices(true).stream() .map(cs -> cs.getVersionedComponentId().orElse( - NiFiRegistryFlowMapper.generateVersionedComponentId(cs.getIdentifier()))) + VersionedComponentFlowMapper.generateVersionedComponentId(cs.getIdentifier()))) .collect(Collectors.toSet()); } @@ -3969,7 +3997,7 @@ private Set getModifications() { } try { - final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(extensionManager); + final VersionedComponentFlowMapper mapper = new VersionedComponentFlowMapper(extensionManager); final VersionedProcessGroup versionedGroup = mapper.mapProcessGroup(this, controllerServiceProvider, flowManager, false); final ComparableDataFlow currentFlow = new StandardComparableDataFlow("Local Flow", versionedGroup); @@ -4544,6 +4572,48 @@ public String getStatelessFlowTimeout() { return statelessFlowTimeout; } + @Override + public FlowFileActivity getFlowFileActivity() { + return flowFileActivity; + } + + @Override + public void setExplicitParentAuthorizable(final Authorizable parent) { + this.explicitParentAuthorizable = parent; + } + + @Override + public CompletableFuture purge() { + final CompletableFuture purgeFuture = new CompletableFuture<>(); + + Thread.ofVirtual().name("Purge " + this).start(() -> { + try { + stopProcessing().get(); + controllerServiceProvider.disableControllerServicesAsync(getControllerServices(true)).get(); + purgeQueues(); + removeComponents(this); + + purgeFuture.complete(null); + } catch (final Throwable t) { + purgeFuture.completeExceptionally(t); + } + }); + + return purgeFuture; + } + + private void purgeQueues() throws ExecutionException, InterruptedException { + for (final Connection connection : getConnections()) { + final FlowFileQueue flowFileQueue = connection.getFlowFileQueue(); + if (flowFileQueue.isEmpty()) { + continue; + } + + final DropFlowFileStatus status = connection.getFlowFileQueue().dropFlowFiles("purge-queues-" + getIdentifier(), "Framework"); + status.getCompletionFuture().get(); + } + } + @Override public void setStatelessFlowTimeout(final String statelessFlowTimeout) { if (statelessFlowTimeout == null) { diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/ConnectorLogObserver.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/ConnectorLogObserver.java new file mode 100644 index 000000000000..d6eba91f989a --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/ConnectorLogObserver.java @@ -0,0 +1,49 @@ +/* + * 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.logging; + +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.events.BulletinFactory; +import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.reporting.Severity; + +public class ConnectorLogObserver implements LogObserver { + private static final String CATEGORY = "Log Message"; + + private final BulletinRepository bulletinRepository; + private final ConnectorNode connectorNode; + + public ConnectorLogObserver(final BulletinRepository bulletinRepository, final ConnectorNode connectorNode) { + this.bulletinRepository = bulletinRepository; + this.connectorNode = connectorNode; + } + + @Override + public void onLogMessage(final LogMessage message) { + // Map LogLevel.WARN to Severity.WARNING so that we are consistent with the Severity enumeration. Else, just use whatever + // the LogLevel is (INFO and ERROR map directly and all others we will just accept as they are). + final String bulletinLevel = (message.getLogLevel() == LogLevel.WARN) ? Severity.WARNING.name() : message.getLogLevel().toString(); + bulletinRepository.addBulletin(BulletinFactory.createBulletin(connectorNode, CATEGORY, bulletinLevel, message.getMessage())); + } + + @Override + public String getComponentDescription() { + return connectorNode.toString(); + } + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java index 7209492ca31d..597b3c8c1094 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/ProcessorLogObserver.java @@ -31,7 +31,7 @@ public class ProcessorLogObserver implements LogObserver { private final BulletinRepository bulletinRepository; private final ProcessorNode processorNode; - public ProcessorLogObserver(BulletinRepository bulletinRepository, ProcessorNode processorNode) { + public ProcessorLogObserver(final BulletinRepository bulletinRepository, final ProcessorNode processorNode) { this.bulletinRepository = bulletinRepository; this.processorNode = processorNode; } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterContext.java index 05636a3a45fe..d5445d2a8c2e 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterContext.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterContext.java @@ -46,6 +46,7 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.Stack; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; @@ -147,6 +148,7 @@ public void setParameters(final Map updatedParameters) { } finally { writeLock.unlock(); } + alertReferencingComponents(parameterUpdates); } @@ -172,21 +174,25 @@ private Map getProposedParameters(final Map parameterUpdates) { - if (!parameterUpdates.isEmpty()) { - logger.debug("Parameter Context {} was updated. {} parameters changed ({}). Notifying all affected components.", this, parameterUpdates.size(), parameterUpdates); - - for (final ProcessGroup processGroup : parameterReferenceManager.getProcessGroupsBound(this)) { - try { - processGroup.onParameterContextUpdated(parameterUpdates); - } catch (final Exception e) { - logger.error("Failed to notify {} that Parameter Context was updated", processGroup, e); - } + if (parameterUpdates.isEmpty()) { + logger.debug("{} updated. No parameters changed so no existing components are affected.", this); + return; + } + + logger.debug("Parameter Context {} was updated. {} parameters changed ({}). Notifying all affected components.", this, parameterUpdates.size(), parameterUpdates); + for (final ProcessGroup processGroup : getBoundProcessGroups()) { + try { + processGroup.onParameterContextUpdated(parameterUpdates); + } catch (final Exception e) { + logger.error("Failed to notify {} that Parameter Context was updated", processGroup, e); } - } else { - logger.debug("Parameter Context {} was updated. {} parameters changed ({}). No existing components are affected.", this, parameterUpdates.size(), parameterUpdates); } } + protected Set getBoundProcessGroups() { + return parameterReferenceManager.getProcessGroupsBound(this); + } + /** * Returns a map from parameter name to ParameterUpdate for any actual updates to parameters. * @param currentParameters The current parameters diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterReferenceManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterReferenceManager.java index 117144986288..1d082963aea1 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterReferenceManager.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/parameter/StandardParameterReferenceManager.java @@ -22,7 +22,6 @@ import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.PropertyConfiguration; -import org.apache.nifi.controller.flow.FlowManager; import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.groups.ProcessGroup; @@ -33,14 +32,15 @@ import java.util.Map; import java.util.Set; import java.util.function.Function; +import java.util.function.Supplier; public class StandardParameterReferenceManager implements ParameterReferenceManager { - private final FlowManager flowManager; + private final Supplier rootGroupLookup; // TODO: Consider reworking this so that we don't have to recurse through all components all the time and instead // have a 'caching' impl that AbstractComponentNode.setProperties() adds to/subtracts from. - public StandardParameterReferenceManager(final FlowManager flowManager) { - this.flowManager = flowManager; + public StandardParameterReferenceManager(final Supplier rootGroupLookup) { + this.rootGroupLookup = rootGroupLookup; } @Override @@ -57,13 +57,13 @@ public Set getControllerServicesReferencing(final Paramet public List getReferencedControllerServiceData(final ParameterContext parameterContext, final String parameterName) { final List referencedControllerServiceData = new ArrayList<>(); + final ProcessGroup rootGroup = rootGroupLookup.get(); final String versionedServiceId = parameterContext.getParameter(parameterName) .map(Parameter::getValue) - .map(this.flowManager::getControllerServiceNode) + .map(serviceId -> rootGroup.findControllerService(serviceId, true, true)) .flatMap(VersionedComponent::getVersionedComponentId) .orElse(null); - final ProcessGroup rootGroup = flowManager.getRootGroup(); final List referencingGroups = rootGroup.findAllProcessGroups(group -> group.referencesParameterContext(parameterContext)); for (final ProcessGroup group : referencingGroups) { @@ -89,7 +89,7 @@ public List getReferencedControllerSer @Override public Set getProcessGroupsBound(final ParameterContext parameterContext) { - final ProcessGroup rootGroup = flowManager.getRootGroup(); + final ProcessGroup rootGroup = rootGroupLookup.get(); final List referencingGroups = rootGroup.findAllProcessGroups(group -> group.referencesParameterContext(parameterContext)); return new HashSet<>(referencingGroups); @@ -99,14 +99,13 @@ private Set getComponentsReferencing(final Paramete final Function> componentFunction) { final Set referencingComponents = new HashSet<>(); - final ProcessGroup rootGroup = flowManager.getRootGroup(); + final ProcessGroup rootGroup = rootGroupLookup.get(); final List referencingGroups = rootGroup.findAllProcessGroups(group -> group.referencesParameterContext(parameterContext)); for (final ProcessGroup group : referencingGroups) { for (final T componentNode : componentFunction.apply(group)) { if (componentNode.isReferencingParameter(parameterName)) { referencingComponents.add(componentNode); - continue; } } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/StandardValidationContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/StandardValidationContext.java index 5edcb7c02637..9ee15d3eff1c 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/StandardValidationContext.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/StandardValidationContext.java @@ -36,9 +36,12 @@ import org.apache.nifi.controller.service.ControllerServiceState; import org.apache.nifi.expression.ExpressionLanguageCompiler; import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.parameter.ExpressionLanguageAgnosticParameterParser; import org.apache.nifi.parameter.Parameter; -import org.apache.nifi.parameter.ParameterContext; +import org.apache.nifi.parameter.ParameterLookup; +import org.apache.nifi.parameter.ParameterParser; import org.apache.nifi.parameter.ParameterReference; +import org.apache.nifi.parameter.ParameterTokenList; import java.util.Collection; import java.util.Collections; @@ -59,7 +62,7 @@ public class StandardValidationContext implements ValidationContext { private final String annotationData; private final String groupId; private final String componentId; - private final ParameterContext parameterContext; + private final ParameterLookup parameterLookup; private final AtomicReference> effectiveValuesRef = new AtomicReference<>(); private final boolean validateConnections; @@ -69,7 +72,7 @@ public StandardValidationContext( final String annotationData, final String groupId, final String componentId, - final ParameterContext parameterContext, + final ParameterLookup parameterLookup, final boolean validateConnections) { this.controllerServiceProvider = controllerServiceProvider; @@ -77,7 +80,7 @@ public StandardValidationContext( this.annotationData = annotationData; this.groupId = groupId; this.componentId = componentId; - this.parameterContext = parameterContext; + this.parameterLookup = parameterLookup; this.validateConnections = validateConnections; preparedQueries = new HashMap<>(properties.size()); @@ -85,7 +88,7 @@ public StandardValidationContext( final PropertyDescriptor desc = entry.getKey(); final PropertyConfiguration configuration = entry.getValue(); - String value = (configuration == null) ? null : configuration.getEffectiveValue(parameterContext); + String value = (configuration == null) ? null : configuration.getEffectiveValue(parameterLookup); if (value == null) { value = desc.getDefaultValue(); } @@ -103,12 +106,12 @@ public StandardValidationContext( @Override public PropertyValue newPropertyValue(final String rawValue) { final ResourceContext resourceContext = new StandardResourceContext(new StandardResourceReferenceFactory(), null); - return new StandardPropertyValue(resourceContext, rawValue, controllerServiceProvider, parameterContext, Query.prepareWithParametersPreEvaluated(rawValue)); + return new StandardPropertyValue(resourceContext, rawValue, controllerServiceProvider, parameterLookup, Query.prepareWithParametersPreEvaluated(rawValue)); } @Override public ExpressionLanguageCompiler newExpressionLanguageCompiler() { - return new StandardExpressionLanguageCompiler(parameterContext); + return new StandardExpressionLanguageCompiler(parameterLookup); } @Override @@ -123,9 +126,9 @@ public ValidationContext getControllerServiceValidationContext(final ControllerS @Override public PropertyValue getProperty(final PropertyDescriptor property) { final PropertyConfiguration configuredValue = properties.get(property); - final String effectiveValue = configuredValue == null ? property.getDefaultValue() : configuredValue.getEffectiveValue(parameterContext); + final String effectiveValue = configuredValue == null ? property.getDefaultValue() : configuredValue.getEffectiveValue(parameterLookup); final ResourceContext resourceContext = new StandardResourceContext(new StandardResourceReferenceFactory(), property); - return new StandardPropertyValue(resourceContext, effectiveValue, controllerServiceProvider, parameterContext, preparedQueries.get(property)); + return new StandardPropertyValue(resourceContext, effectiveValue, controllerServiceProvider, parameterLookup, preparedQueries.get(property)); } @Override @@ -139,7 +142,7 @@ public Map getProperties() { for (final Map.Entry entry : this.properties.entrySet()) { final PropertyDescriptor descriptor = entry.getKey(); final PropertyConfiguration configuration = entry.getValue(); - final String value = configuration == null ? descriptor.getDefaultValue() : configuration.getEffectiveValue(parameterContext); + final String value = configuration == null ? descriptor.getDefaultValue() : configuration.getEffectiveValue(parameterLookup); valueMap.put(entry.getKey(), value); } @@ -215,20 +218,20 @@ public Collection getReferencedParameters(final String propertyName) { @Override public boolean isParameterDefined(final String parameterName) { - if (parameterContext == null) { + if (parameterLookup == null) { return false; } - return parameterContext.getParameter(parameterName).isPresent(); + return parameterLookup.getParameter(parameterName).isPresent(); } @Override public boolean isParameterSet(final String parameterName) { - if (parameterContext == null) { + if (parameterLookup == null) { return false; } - final Optional parameterOption = parameterContext.getParameter(parameterName); + final Optional parameterOption = parameterLookup.getParameter(parameterName); if (!parameterOption.isPresent()) { return false; } @@ -242,6 +245,13 @@ public boolean isValidateConnections() { return validateConnections; } + @Override + public String evaluateParameters(final String value) { + final ParameterParser parameterParser = new ExpressionLanguageAgnosticParameterParser(); + final ParameterTokenList parameterTokenList = parameterParser.parseTokens(value); + return parameterTokenList.substitute(parameterLookup); + } + @Override public String toString() { return "StandardValidationContext[componentId=" + componentId + ", properties=" + properties + "]"; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java index 00a7b6852a71..a1d8178f91a4 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java @@ -22,7 +22,7 @@ import org.apache.nifi.controller.ValidationContextFactory; import org.apache.nifi.controller.flowanalysis.FlowAnalyzer; import org.apache.nifi.controller.service.ControllerServiceProvider; -import org.apache.nifi.parameter.ParameterContext; +import org.apache.nifi.parameter.ParameterLookup; import org.apache.nifi.validation.RuleViolationsManager; import java.util.Map; @@ -50,8 +50,8 @@ public StandardValidationContextFactory( @Override public ValidationContext newValidationContext(final Map properties, final String annotationData, final String groupId, final String componentId, - final ParameterContext parameterContext, final boolean validateConnections) { - return new StandardValidationContext(serviceProvider, properties, annotationData, groupId, componentId, parameterContext, validateConnections); + final ParameterLookup parameterLookup, final boolean validateConnections) { + return new StandardValidationContext(serviceProvider, properties, annotationData, groupId, componentId, parameterLookup, validateConnections); } @Override diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/FlowAnalyzingRegistryClientNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/FlowAnalyzingRegistryClientNode.java index db1949e06f49..2d3bb71a3e49 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/FlowAnalyzingRegistryClientNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/FlowAnalyzingRegistryClientNode.java @@ -45,7 +45,7 @@ import org.apache.nifi.parameter.ParameterLookup; import org.apache.nifi.parameter.ParameterUpdate; import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedProcessGroup; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import org.apache.nifi.validation.RuleViolation; import org.apache.nifi.validation.RuleViolationsManager; import org.slf4j.Logger; @@ -70,7 +70,7 @@ public final class FlowAnalyzingRegistryClientNode implements FlowRegistryClient private final FlowAnalyzer flowAnalyzer; private final RuleViolationsManager ruleViolationsManager; private final FlowManager flowManager; - private final NiFiRegistryFlowMapper flowMapper; + private final VersionedComponentFlowMapper flowMapper; public FlowAnalyzingRegistryClientNode( final FlowRegistryClientNode node, @@ -78,7 +78,7 @@ public FlowAnalyzingRegistryClientNode( final FlowAnalyzer flowAnalyzer, final RuleViolationsManager ruleViolationsManager, final FlowManager flowManager, - final NiFiRegistryFlowMapper flowMapper + final VersionedComponentFlowMapper flowMapper ) { this.node = Objects.requireNonNull(node); this.serviceProvider = Objects.requireNonNull(serviceProvider); @@ -112,7 +112,8 @@ public RegisteredFlowSnapshot registerFlowSnapshot( } private boolean analyzeProcessGroupToRegister(final VersionedProcessGroup snapshot) { - final InstantiatedVersionedProcessGroup nonVersionedProcessGroup = flowMapper.mapNonVersionedProcessGroup(flowManager.getGroup(snapshot.getInstanceIdentifier()), serviceProvider); + final ProcessGroup group = flowManager.getGroup(snapshot.getInstanceIdentifier(), null); + final InstantiatedVersionedProcessGroup nonVersionedProcessGroup = flowMapper.mapNonVersionedProcessGroup(group, serviceProvider); flowAnalyzer.analyzeProcessGroup(nonVersionedProcessGroup); final List ruleViolations = ruleViolationsManager.getRuleViolationsForGroup(snapshot.getInstanceIdentifier()).stream() @@ -179,6 +180,13 @@ public void verifyCanUpdateProperties(final Map properties) { node.verifyCanUpdateProperties(properties); } + @Override + public ValidationContext createValidationContext(final Map propertyValues, final String annotationData, + final ParameterLookup parameterLookup, final boolean validateConnections) { + + return node.createValidationContext(propertyValues, annotationData, parameterLookup, validateConnections); + } + @Override public Set getReferencedParameterNames() { return node.getReferencedParameterNames(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClientNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClientNode.java index 9eb895fcec14..e8e936df2708 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClientNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClientNode.java @@ -28,6 +28,7 @@ import org.apache.nifi.components.ConfigVerificationResult.Outcome; import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.validation.ValidationStatus; import org.apache.nifi.components.validation.ValidationTrigger; @@ -118,7 +119,7 @@ public String getProcessGroupIdentifier() { } @Override - protected List validateConfig() { + protected List validateConfig(final ValidationContext validationContext) { return Collections.emptyList(); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/VersionedComponentFlowMapper.java similarity index 92% rename from nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java rename to nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/VersionedComponentFlowMapper.java index 9f847a24649a..1d845265c9bc 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/VersionedComponentFlowMapper.java @@ -21,6 +21,16 @@ import org.apache.nifi.asset.Asset; import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.connector.AssetReference; +import org.apache.nifi.components.connector.ConnectorConfiguration; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorState; +import org.apache.nifi.components.connector.ConnectorValueReference; +import org.apache.nifi.components.connector.FrameworkFlowContext; +import org.apache.nifi.components.connector.NamedStepConfiguration; +import org.apache.nifi.components.connector.SecretReference; +import org.apache.nifi.components.connector.StepConfiguration; +import org.apache.nifi.components.connector.StringLiteralValue; import org.apache.nifi.components.listen.ListenPortDefinition; import org.apache.nifi.components.resource.ResourceCardinality; import org.apache.nifi.components.resource.ResourceDefinition; @@ -53,7 +63,10 @@ import org.apache.nifi.flow.PortType; import org.apache.nifi.flow.Position; import org.apache.nifi.flow.VersionedAsset; +import org.apache.nifi.flow.VersionedConfigurationStep; import org.apache.nifi.flow.VersionedConnection; +import org.apache.nifi.flow.VersionedConnector; +import org.apache.nifi.flow.VersionedConnectorValueReference; import org.apache.nifi.flow.VersionedControllerService; import org.apache.nifi.flow.VersionedFlowAnalysisRule; import org.apache.nifi.flow.VersionedFlowCoordinates; @@ -108,7 +121,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; -public class NiFiRegistryFlowMapper { +public class VersionedComponentFlowMapper { private static final String ENCRYPTED_PREFIX = "enc{"; private static final String ENCRYPTED_SUFFIX = "}"; @@ -122,11 +135,11 @@ public class NiFiRegistryFlowMapper { // created before attempting to create the connection, where the ConnectableDTO is converted. private final Map versionedComponentIds = new HashMap<>(); - public NiFiRegistryFlowMapper(final ExtensionManager extensionManager) { + public VersionedComponentFlowMapper(final ExtensionManager extensionManager) { this(extensionManager, FlowMappingOptions.DEFAULT_OPTIONS); } - public NiFiRegistryFlowMapper(final ExtensionManager extensionManager, final FlowMappingOptions flowMappingOptions) { + public VersionedComponentFlowMapper(final ExtensionManager extensionManager, final FlowMappingOptions flowMappingOptions) { this.extensionManager = extensionManager; this.flowMappingOptions = flowMappingOptions; @@ -1003,6 +1016,7 @@ private VersionedParameter mapParameter(final Parameter parameter, final String .toList(); versionedParameter.setReferencedAssets(assetIds); } + return versionedParameter; } @@ -1018,4 +1032,79 @@ private org.apache.nifi.flow.ScheduledState mapScheduledState(final ScheduledSta ? org.apache.nifi.flow.ScheduledState.DISABLED : org.apache.nifi.flow.ScheduledState.ENABLED; } + + public VersionedConnector mapConnector(final ConnectorNode connectorNode) { + final VersionedConnector versionedConnector = new VersionedConnector(); + versionedConnector.setInstanceIdentifier(connectorNode.getIdentifier()); + versionedConnector.setName(connectorNode.getName()); + versionedConnector.setScheduledState(mapConnectorState(connectorNode.getDesiredState())); + versionedConnector.setType(connectorNode.getCanonicalClassName()); + versionedConnector.setBundle(mapBundle(connectorNode.getBundleCoordinate())); + + final List activeFlowConfiguration = createVersionedConfigurationSteps(connectorNode.getActiveFlowContext()); + versionedConnector.setActiveFlowConfiguration(activeFlowConfiguration); + + final List workingFlowConfiguration = createVersionedConfigurationSteps(connectorNode.getWorkingFlowContext()); + versionedConnector.setWorkingFlowConfiguration(workingFlowConfiguration); + + return versionedConnector; + } + + private List createVersionedConfigurationSteps(final FrameworkFlowContext flowContext) { + if (flowContext == null) { + return Collections.emptyList(); + } + + final ConnectorConfiguration configuration = flowContext.getConfigurationContext().toConnectorConfiguration(); + final List configurationSteps = new ArrayList<>(); + + for (final NamedStepConfiguration stepConfiguration : configuration.getNamedStepConfigurations()) { + final VersionedConfigurationStep versionedConfigurationStep = new VersionedConfigurationStep(); + versionedConfigurationStep.setName(stepConfiguration.stepName()); + versionedConfigurationStep.setProperties(mapPropertyValues(stepConfiguration.configuration())); + + configurationSteps.add(versionedConfigurationStep); + } + + return configurationSteps; + } + + private Map mapPropertyValues(final StepConfiguration configuration) { + final Map versionedProperties = new HashMap<>(); + for (final Map.Entry entry : configuration.getPropertyValues().entrySet()) { + final ConnectorValueReference valueReference = entry.getValue(); + if (valueReference == null) { + continue; + } + + final VersionedConnectorValueReference versionedReference = new VersionedConnectorValueReference(); + versionedReference.setValueType(valueReference.getValueType().name()); + + switch (valueReference) { + case StringLiteralValue stringLiteral -> versionedReference.setValue(stringLiteral.getValue()); + case AssetReference assetRef -> versionedReference.setAssetIds(assetRef.getAssetIdentifiers()); + case SecretReference secretRef -> { + versionedReference.setProviderId(secretRef.getProviderId()); + versionedReference.setProviderName(secretRef.getProviderName()); + versionedReference.setSecretName(secretRef.getSecretName()); + versionedReference.setFullyQualifiedSecretName(secretRef.getFullyQualifiedName()); + } + } + + versionedProperties.put(entry.getKey(), versionedReference); + } + + return versionedProperties; + } + + private org.apache.nifi.flow.ScheduledState mapConnectorState(final ConnectorState connectorState) { + if (connectorState == null) { + return null; + } + + return switch (connectorState) { + case RUNNING, STARTING -> org.apache.nifi.flow.ScheduledState.RUNNING; + default -> org.apache.nifi.flow.ScheduledState.ENABLED; + }; + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java index edd34d4fac5e..7d63e2b66b9c 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java @@ -67,7 +67,7 @@ import java.util.Optional; import java.util.Set; import java.util.UUID; -import java.util.concurrent.Future; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -1102,7 +1102,7 @@ public void startTransmitting(final RemoteGroupPort port) { } @Override - public Future stopTransmitting() { + public CompletableFuture stopTransmitting() { writeLock.lock(); try { verifyCanStopTransmitting(); @@ -1117,13 +1117,15 @@ public Future stopTransmitting() { configuredToTransmit.set(false); - return scheduler.submitFrameworkTask(this::waitForPortShutdown); + final CompletableFuture completableFuture = new CompletableFuture<>(); + scheduler.submitFrameworkTask(() -> waitForPortShutdown(completableFuture)); + return completableFuture; } finally { writeLock.unlock(); } } - private void waitForPortShutdown() { + private void waitForPortShutdown(final CompletableFuture completableFuture) { // Wait for the ports to stop try { for (final RemoteGroupPort port : getInputPorts()) { @@ -1149,6 +1151,7 @@ private void waitForPortShutdown() { } } finally { transmitting.set(false); + completableFuture.complete(null); } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/resources/META-INF/services/org.apache.nifi.asset.AssetManager b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/resources/META-INF/services/org.apache.nifi.asset.AssetManager index 6fbf05d94972..3a438fb65dd7 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/resources/META-INF/services/org.apache.nifi.asset.AssetManager +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/resources/META-INF/services/org.apache.nifi.asset.AssetManager @@ -13,4 +13,5 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.nifi.asset.StandardAssetManager \ No newline at end of file +org.apache.nifi.asset.StandardAssetManager +org.apache.nifi.asset.StandardConnectorAssetManager \ No newline at end of file diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/resources/META-INF/services/org.apache.nifi.components.connector.secrets.SecretsManager b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/resources/META-INF/services/org.apache.nifi.components.connector.secrets.SecretsManager new file mode 100644 index 000000000000..082fffbd8f19 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/resources/META-INF/services/org.apache.nifi.components.connector.secrets.SecretsManager @@ -0,0 +1,16 @@ +# 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. + +org.apache.nifi.components.connector.secrets.ParameterProviderSecretsManager diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/asset/StandardConnectorAssetManagerTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/asset/StandardConnectorAssetManagerTest.java new file mode 100644 index 000000000000..fa13d28bba35 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/asset/StandardConnectorAssetManagerTest.java @@ -0,0 +1,178 @@ +/* + * 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.asset; + +import org.apache.nifi.controller.NodeTypeProvider; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; + +class StandardConnectorAssetManagerTest { + + @Test + void testInitializeRecoversExistingAssets(@TempDir final Path tempDir) throws Exception { + final Path storageDirectory = tempDir.resolve("assets"); + final String connectorIdentifier = "connector-1"; + final String assetIdentifier = "asset-1"; + final String assetName = "asset.txt"; + + final Path assetDirectory = storageDirectory.resolve(connectorIdentifier).resolve(assetIdentifier); + Files.createDirectories(assetDirectory); + final Path assetPath = assetDirectory.resolve(assetName); + Files.writeString(assetPath, "existing-content", StandardCharsets.UTF_8); + + final StandardConnectorAssetManager manager = new StandardConnectorAssetManager(); + final AssetManagerInitializationContext context = createInitializationContext(storageDirectory.toFile()); + manager.initialize(context); + + final List assets = manager.getAssets(connectorIdentifier); + assertEquals(1, assets.size()); + + final Asset recoveredAsset = assets.getFirst(); + assertEquals(assetName, recoveredAsset.getName()); + assertEquals(connectorIdentifier, recoveredAsset.getOwnerIdentifier()); + assertTrue(recoveredAsset.getDigest().isPresent()); + assertEquals(assetPath.toFile().getAbsolutePath(), recoveredAsset.getFile().getAbsolutePath()); + } + + @Test + void testCreateAssetAndGetAsset(@TempDir final Path tempDir) throws Exception { + final Path storageDirectory = tempDir.resolve("assets"); + final StandardConnectorAssetManager manager = new StandardConnectorAssetManager(); + final AssetManagerInitializationContext context = createInitializationContext(storageDirectory.toFile()); + manager.initialize(context); + + assertTrue(Files.isDirectory(storageDirectory)); + + final String connectorIdentifier = "connector-1"; + final String assetName = "created-asset.txt"; + final byte[] contents = "created-content".getBytes(StandardCharsets.UTF_8); + final InputStream inputStream = new ByteArrayInputStream(contents); + + final Asset createdAsset = manager.createAsset(connectorIdentifier, assetName, inputStream); + assertNotNull(createdAsset); + assertEquals(connectorIdentifier, createdAsset.getOwnerIdentifier()); + assertEquals(assetName, createdAsset.getName()); + assertTrue(createdAsset.getDigest().isPresent()); + assertTrue(createdAsset.getFile().exists()); + + final Optional retrievedByIdentifier = manager.getAsset(createdAsset.getIdentifier()); + assertTrue(retrievedByIdentifier.isPresent()); + assertEquals(createdAsset.getIdentifier(), retrievedByIdentifier.get().getIdentifier()); + + final List connectorAssets = manager.getAssets(connectorIdentifier); + assertEquals(1, connectorAssets.size()); + assertEquals(createdAsset.getIdentifier(), connectorAssets.get(0).getIdentifier()); + } + + @Test + void testCreateMissingAssetDoesNotCreateFile(@TempDir final Path tempDir) { + final Path storageDirectory = tempDir.resolve("assets"); + final StandardConnectorAssetManager manager = new StandardConnectorAssetManager(); + final AssetManagerInitializationContext context = createInitializationContext(storageDirectory.toFile()); + manager.initialize(context); + + final String connectorIdentifier = "connector-2"; + final String assetName = "missing-asset.txt"; + + final Asset missingAsset = manager.createMissingAsset(connectorIdentifier, assetName); + assertNotNull(missingAsset); + assertEquals(connectorIdentifier, missingAsset.getOwnerIdentifier()); + assertEquals(assetName, missingAsset.getName()); + assertFalse(missingAsset.getDigest().isPresent()); + assertFalse(missingAsset.getFile().exists()); + + final Optional retrieved = manager.getAsset(missingAsset.getIdentifier()); + assertTrue(retrieved.isPresent()); + assertEquals(missingAsset.getIdentifier(), retrieved.get().getIdentifier()); + + final List connectorAssets = manager.getAssets(connectorIdentifier); + assertEquals(1, connectorAssets.size()); + assertEquals(missingAsset.getIdentifier(), connectorAssets.getFirst().getIdentifier()); + } + + @Test + void testDeleteAssetRemovesFilesAndDirectories(@TempDir final Path tempDir) throws Exception { + final Path storageDirectory = tempDir.resolve("assets"); + final StandardConnectorAssetManager manager = new StandardConnectorAssetManager(); + final AssetManagerInitializationContext context = createInitializationContext(storageDirectory.toFile()); + manager.initialize(context); + + final String connectorIdentifier = "connector-3"; + final String assetName = "deletable-asset.txt"; + final byte[] contents = "deletable-content".getBytes(StandardCharsets.UTF_8); + final InputStream inputStream = new ByteArrayInputStream(contents); + + final Asset asset = manager.createAsset(connectorIdentifier, assetName, inputStream); + final File assetFile = asset.getFile(); + final File assetDirectory = assetFile.getParentFile(); + final File connectorDirectory = assetDirectory.getParentFile(); + + assertTrue(assetFile.exists()); + assertTrue(assetDirectory.exists()); + assertTrue(connectorDirectory.exists()); + + final Optional deleted = manager.deleteAsset(asset.getIdentifier()); + assertTrue(deleted.isPresent()); + assertFalse(assetFile.exists()); + assertFalse(assetDirectory.exists()); + assertFalse(connectorDirectory.exists()); + + assertTrue(manager.getAsset(asset.getIdentifier()).isEmpty()); + assertTrue(manager.getAssets(connectorIdentifier).isEmpty()); + } + + @Test + void testDeleteNonExistentAssetReturnsEmpty(@TempDir final Path tempDir) { + final Path storageDirectory = tempDir.resolve("assets"); + final StandardConnectorAssetManager manager = new StandardConnectorAssetManager(); + final AssetManagerInitializationContext context = createInitializationContext(storageDirectory.toFile()); + manager.initialize(context); + + final Optional deleted = manager.deleteAsset("non-existent-id"); + assertTrue(deleted.isEmpty()); + } + + private AssetManagerInitializationContext createInitializationContext(final File storageDirectory) { + final Map properties = new HashMap<>(); + properties.put(StandardConnectorAssetManager.ASSET_STORAGE_LOCATION_PROPERTY, storageDirectory.getAbsolutePath()); + + final AssetReferenceLookup assetReferenceLookup = mock(AssetReferenceLookup.class); + final NodeTypeProvider nodeTypeProvider = mock(NodeTypeProvider.class); + + return new StandardAssetManagerInitializationContext(assetReferenceLookup, properties, nodeTypeProvider); + } +} + + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/components/connector/secrets/TestParameterProviderSecretsManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/components/connector/secrets/TestParameterProviderSecretsManager.java new file mode 100644 index 000000000000..870576010efa --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/components/connector/secrets/TestParameterProviderSecretsManager.java @@ -0,0 +1,374 @@ +/* + * 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.components.connector.secrets; + +import org.apache.nifi.components.connector.Secret; +import org.apache.nifi.components.connector.SecretReference; +import org.apache.nifi.components.validation.ValidationStatus; +import org.apache.nifi.controller.ParameterProviderNode; +import org.apache.nifi.controller.flow.FlowManager; +import org.apache.nifi.parameter.Parameter; +import org.apache.nifi.parameter.ParameterDescriptor; +import org.apache.nifi.parameter.ParameterGroup; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestParameterProviderSecretsManager { + + private static final String PROVIDER_1_ID = "provider-1-id"; + private static final String PROVIDER_1_NAME = "Provider One"; + private static final String PROVIDER_2_ID = "provider-2-id"; + private static final String PROVIDER_2_NAME = "Provider Two"; + + private static final String GROUP_1_NAME = "Group One"; + private static final String GROUP_2_NAME = "Group Two"; + + private static final String SECRET_1_NAME = "secret-one"; + private static final String SECRET_1_DESCRIPTION = "First secret"; + private static final String SECRET_1_VALUE = "secret-value-one"; + + private static final String SECRET_2_NAME = "secret-two"; + private static final String SECRET_2_DESCRIPTION = "Second secret"; + private static final String SECRET_2_VALUE = "secret-value-two"; + + private static final String SECRET_3_NAME = "secret-three"; + private static final String SECRET_3_DESCRIPTION = "Third secret"; + private static final String SECRET_3_VALUE = "secret-value-three"; + + private ParameterProviderSecretsManager secretsManager; + + @BeforeEach + public void setup() { + final FlowManager flowManager = mock(FlowManager.class); + final ParameterProviderNode providerNode1 = createMockedParameterProviderNode(PROVIDER_1_ID, PROVIDER_1_NAME, GROUP_1_NAME, + createParameter(SECRET_1_NAME, SECRET_1_DESCRIPTION, SECRET_1_VALUE), + createParameter(SECRET_2_NAME, SECRET_2_DESCRIPTION, SECRET_2_VALUE)); + final ParameterProviderNode providerNode2 = createMockedParameterProviderNode(PROVIDER_2_ID, PROVIDER_2_NAME, GROUP_2_NAME, + createParameter(SECRET_3_NAME, SECRET_3_DESCRIPTION, SECRET_3_VALUE)); + + final Set providers = new HashSet<>(); + providers.add(providerNode1); + providers.add(providerNode2); + when(flowManager.getAllParameterProviders()).thenReturn(providers); + + secretsManager = new ParameterProviderSecretsManager(); + final SecretsManagerInitializationContext initContext = new StandardSecretsManagerInitializationContext(flowManager); + secretsManager.initialize(initContext); + } + + private ParameterProviderNode createMockedParameterProviderNode(final String id, final String name, final String groupName, final Parameter... parameters) { + return createMockedParameterProviderNode(id, name, groupName, ValidationStatus.VALID, parameters); + } + + private ParameterProviderNode createMockedParameterProviderNode(final String id, final String name, final String groupName, + final ValidationStatus validationStatus, final Parameter... parameters) { + final ParameterProviderNode node = mock(ParameterProviderNode.class); + when(node.getIdentifier()).thenReturn(id); + when(node.getName()).thenReturn(name); + when(node.getValidationStatus()).thenReturn(validationStatus); + + final List parameterList = List.of(parameters); + final ParameterGroup group = new ParameterGroup(groupName, parameterList); + final List groups = List.of(group); + + when(node.fetchParameterValues()).thenReturn(groups); + + when(node.fetchParameterValues(anyList())).thenAnswer(invocation -> { + final List requestedNames = invocation.getArgument(0); + final List matchingParameters = parameterList.stream() + .filter(p -> requestedNames.contains(name + "." + groupName + "." + p.getDescriptor().getName())) + .toList(); + if (matchingParameters.isEmpty()) { + return List.of(); + } + return List.of(new ParameterGroup(groupName, matchingParameters)); + }); + + return node; + } + + private Parameter createParameter(final String name, final String description, final String value) { + final ParameterDescriptor descriptor = new ParameterDescriptor.Builder() + .name(name) + .description(description) + .build(); + return new Parameter.Builder() + .descriptor(descriptor) + .value(value) + .build(); + } + + private SecretReference createSecretReference(final String providerId, final String providerName, final String secretName) { + final String effectiveProviderId = providerId != null ? providerId : getProviderIdFromName(providerName); + final String effectiveProviderName = getProviderNameFromId(effectiveProviderId); + final String groupName = getGroupNameForProvider(effectiveProviderId); + return new SecretReference(providerId, providerName, secretName, effectiveProviderName + "." + groupName + "." + secretName); + } + + private String getProviderIdFromName(final String providerName) { + if (PROVIDER_1_NAME.equals(providerName)) { + return PROVIDER_1_ID; + } else if (PROVIDER_2_NAME.equals(providerName)) { + return PROVIDER_2_ID; + } + return null; + } + + private String getProviderNameFromId(final String providerId) { + if (PROVIDER_1_ID.equals(providerId)) { + return PROVIDER_1_NAME; + } else if (PROVIDER_2_ID.equals(providerId)) { + return PROVIDER_2_NAME; + } + return "Unknown Provider"; + } + + private String getGroupNameForProvider(final String providerId) { + if (PROVIDER_1_ID.equals(providerId)) { + return GROUP_1_NAME; + } + return GROUP_2_NAME; + } + + @Test + public void testGetSecretProvidersReturnsOneProviderPerMockedParameterProvider() { + final Set providers = secretsManager.getSecretProviders(); + + assertEquals(2, providers.size()); + + boolean foundProvider1 = false; + boolean foundProvider2 = false; + for (final SecretProvider provider : providers) { + if (PROVIDER_1_ID.equals(provider.getProviderId())) { + assertEquals(PROVIDER_1_NAME, provider.getProviderName()); + foundProvider1 = true; + } else if (PROVIDER_2_ID.equals(provider.getProviderId())) { + assertEquals(PROVIDER_2_NAME, provider.getProviderName()); + foundProvider2 = true; + } + } + + assertTrue(foundProvider1); + assertTrue(foundProvider2); + } + + @Test + public void testGetAllSecretsRetrievesSecretsFromAllProviders() { + final List allSecrets = secretsManager.getAllSecrets(); + + assertEquals(3, allSecrets.size()); + + boolean foundSecret1 = false; + boolean foundSecret2 = false; + boolean foundSecret3 = false; + + for (final Secret secret : allSecrets) { + if (SECRET_1_NAME.equals(secret.getName())) { + assertEquals(PROVIDER_1_NAME, secret.getProviderName()); + assertEquals(GROUP_1_NAME, secret.getGroupName()); + assertEquals(SECRET_1_VALUE, secret.getValue()); + foundSecret1 = true; + } else if (SECRET_2_NAME.equals(secret.getName())) { + assertEquals(PROVIDER_1_NAME, secret.getProviderName()); + assertEquals(GROUP_1_NAME, secret.getGroupName()); + assertEquals(SECRET_2_VALUE, secret.getValue()); + foundSecret2 = true; + } else if (SECRET_3_NAME.equals(secret.getName())) { + assertEquals(PROVIDER_2_NAME, secret.getProviderName()); + assertEquals(GROUP_2_NAME, secret.getGroupName()); + assertEquals(SECRET_3_VALUE, secret.getValue()); + foundSecret3 = true; + } + } + + assertTrue(foundSecret1); + assertTrue(foundSecret2); + assertTrue(foundSecret3); + } + + @Test + public void testGetSecretReturnsPopulatedOptionalWhenSecretIsFoundById() { + final SecretReference reference = createSecretReference(PROVIDER_1_ID, null, SECRET_1_NAME); + + final Optional result = secretsManager.getSecret(reference); + + assertTrue(result.isPresent()); + final Secret secret = result.get(); + assertEquals(SECRET_1_NAME, secret.getName()); + assertEquals(SECRET_1_VALUE, secret.getValue()); + assertEquals(PROVIDER_1_NAME, secret.getProviderName()); + } + + @Test + public void testGetSecretReturnsPopulatedOptionalWhenSecretIsFoundByName() { + final SecretReference reference = createSecretReference(null, PROVIDER_2_NAME, SECRET_3_NAME); + + final Optional result = secretsManager.getSecret(reference); + + assertTrue(result.isPresent()); + final Secret secret = result.get(); + assertEquals(SECRET_3_NAME, secret.getName()); + assertEquals(SECRET_3_VALUE, secret.getValue()); + assertEquals(PROVIDER_2_NAME, secret.getProviderName()); + } + + @Test + public void testGetSecretReturnsEmptyOptionalWhenInvalidSecretNameProvided() { + final SecretReference reference = createSecretReference(PROVIDER_1_ID, PROVIDER_1_NAME, "non-existent-secret"); + final Optional result = secretsManager.getSecret(reference); + assertFalse(result.isPresent()); + } + + @Test + public void testGetSecretReturnsEmptyOptionalWhenProviderNotFound() { + final SecretReference reference = createSecretReference("invalid-provider-id", "Invalid Provider", SECRET_1_NAME); + final Optional result = secretsManager.getSecret(reference); + assertFalse(result.isPresent()); + } + + @Test + public void testGetSecretsReturnsAllSecretsProperlyMappedWhenAllAreFound() { + final SecretReference reference1 = createSecretReference(PROVIDER_1_ID, PROVIDER_1_NAME, SECRET_1_NAME); + final SecretReference reference2 = createSecretReference(PROVIDER_1_ID, PROVIDER_1_NAME, SECRET_2_NAME); + final SecretReference reference3 = createSecretReference(PROVIDER_2_ID, PROVIDER_2_NAME, SECRET_3_NAME); + final Set references = Set.of(reference1, reference2, reference3); + + final Map results = secretsManager.getSecrets(references); + assertEquals(3, results.size()); + + final Secret secret1 = results.get(reference1); + assertNotNull(secret1); + assertEquals(SECRET_1_NAME, secret1.getName()); + assertEquals(SECRET_1_VALUE, secret1.getValue()); + + final Secret secret2 = results.get(reference2); + assertNotNull(secret2); + assertEquals(SECRET_2_NAME, secret2.getName()); + assertEquals(SECRET_2_VALUE, secret2.getValue()); + + final Secret secret3 = results.get(reference3); + assertNotNull(secret3); + assertEquals(SECRET_3_NAME, secret3.getName()); + assertEquals(SECRET_3_VALUE, secret3.getValue()); + } + + @Test + public void testGetSecretsReturnsNullValueForSecretWithInvalidName() { + final SecretReference validReference = createSecretReference(PROVIDER_1_ID, PROVIDER_1_NAME, SECRET_1_NAME); + final SecretReference invalidReference = createSecretReference(PROVIDER_1_ID, PROVIDER_1_NAME, "non-existent-secret"); + final Set references = Set.of(validReference, invalidReference); + + final Map results = secretsManager.getSecrets(references); + assertEquals(2, results.size()); + + final Secret validSecret = results.get(validReference); + assertNotNull(validSecret); + assertEquals(SECRET_1_NAME, validSecret.getName()); + + assertTrue(results.containsKey(invalidReference)); + assertNull(results.get(invalidReference)); + } + + @Test + public void testGetSecretsReturnsNullValueForSecretWithInvalidProvider() { + final SecretReference validReference = createSecretReference(PROVIDER_1_ID, PROVIDER_1_NAME, SECRET_1_NAME); + final SecretReference invalidProviderReference = createSecretReference("invalid-id", "Invalid Provider", SECRET_1_NAME); + final Set references = Set.of(validReference, invalidProviderReference); + + final Map results = secretsManager.getSecrets(references); + assertEquals(2, results.size()); + + final Secret validSecret = results.get(validReference); + assertNotNull(validSecret); + + assertTrue(results.containsKey(invalidProviderReference)); + assertNull(results.get(invalidProviderReference)); + } + + @Test + public void testGetSecretsFromMultipleProvidersWithMixedResults() { + final SecretReference provider1Secret1 = createSecretReference(PROVIDER_1_ID, null, SECRET_1_NAME); + final SecretReference provider1Invalid = createSecretReference(PROVIDER_1_ID, null, "invalid-secret"); + final SecretReference provider2Secret3 = createSecretReference(PROVIDER_2_ID, null, SECRET_3_NAME); + final SecretReference invalidProvider = createSecretReference("invalid-id", null, SECRET_1_NAME); + + final Set references = Set.of(provider1Secret1, provider1Invalid, provider2Secret3, invalidProvider); + final Map results = secretsManager.getSecrets(references); + assertEquals(4, results.size()); + + assertNotNull(results.get(provider1Secret1)); + assertEquals(SECRET_1_NAME, results.get(provider1Secret1).getName()); + + assertTrue(results.containsKey(provider1Invalid)); + assertNull(results.get(provider1Invalid)); + + assertNotNull(results.get(provider2Secret3)); + assertEquals(SECRET_3_NAME, results.get(provider2Secret3).getName()); + + assertTrue(results.containsKey(invalidProvider)); + assertNull(results.get(invalidProvider)); + } + + @Test + public void testGetSecretProviderSearchesByIdFirst() { + final SecretReference referenceWithBothIdAndName = createSecretReference(PROVIDER_1_ID, PROVIDER_2_NAME, SECRET_1_NAME); + final Optional result = secretsManager.getSecret(referenceWithBothIdAndName); + assertTrue(result.isPresent()); + assertEquals(PROVIDER_1_NAME, result.get().getProviderName()); + } + + @Test + public void testGetSecretProvidersFiltersOutInvalidProviders() { + final FlowManager flowManager = mock(FlowManager.class); + final ParameterProviderNode validProvider = createMockedParameterProviderNode("valid-id", "Valid Provider", "Group", + ValidationStatus.VALID, createParameter("secret", "description", "value")); + final ParameterProviderNode invalidProvider = createMockedParameterProviderNode("invalid-id", "Invalid Provider", "Group", + ValidationStatus.INVALID, createParameter("secret2", "description2", "value2")); + final ParameterProviderNode validatingProvider = createMockedParameterProviderNode("validating-id", "Validating Provider", "Group", + ValidationStatus.VALIDATING, createParameter("secret3", "description3", "value3")); + + final Set allProviders = new HashSet<>(); + allProviders.add(validProvider); + allProviders.add(invalidProvider); + allProviders.add(validatingProvider); + when(flowManager.getAllParameterProviders()).thenReturn(allProviders); + + final ParameterProviderSecretsManager manager = new ParameterProviderSecretsManager(); + manager.initialize(new StandardSecretsManagerInitializationContext(flowManager)); + + final Set secretProviders = manager.getSecretProviders(); + + assertEquals(1, secretProviders.size()); + assertEquals("valid-id", secretProviders.iterator().next().getProviderId()); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/controller/repository/StandardProcessSessionTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/controller/repository/StandardProcessSessionTest.java index 6c2171c3c861..c82ee1ca5771 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/controller/repository/StandardProcessSessionTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/controller/repository/StandardProcessSessionTest.java @@ -17,6 +17,7 @@ package org.apache.nifi.controller.repository; import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.connectable.FlowFileActivity; import org.apache.nifi.controller.lifecycle.TaskTermination; import org.apache.nifi.controller.metrics.GaugeRecord; import org.apache.nifi.controller.repository.claim.ContentClaim; @@ -34,6 +35,8 @@ import org.mockito.Captor; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -48,10 +51,12 @@ import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.isA; import static org.mockito.ArgumentMatchers.isNull; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) class StandardProcessSessionTest { private static final String BACKOFF_PERIOD = "5 s"; @@ -116,6 +121,8 @@ void setSession() { when(repositoryContext.createContentClaimWriteCache(isA(PerformanceTracker.class))).thenReturn(contentClaimWriteCache); when(repositoryContext.getConnectable()).thenReturn(connectable); when(connectable.getIdentifier()).thenReturn(Connectable.class.getSimpleName()); + final FlowFileActivity flowFileActivity = mock(FlowFileActivity.class); + when(connectable.getFlowFileActivity()).thenReturn(flowFileActivity); session = new StandardProcessSession(repositoryContext, taskTermination, performanceTracker); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceResolverTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceResolverTest.java index e89075c5722e..658b50c87dc6 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceResolverTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceResolverTest.java @@ -29,7 +29,7 @@ import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.registry.flow.FlowSnapshotContainer; import org.apache.nifi.registry.flow.RegisteredFlowSnapshot; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -57,7 +57,7 @@ public class StandardControllerServiceResolverTest { private static final String CHILD_REFERENCES_SERVICES_FROM_PARENT_LOCATION = BASE_SNAPSHOT_LOCATION + "/versioned-child-services-from-parent"; private static final String STANDARD_EXTERNAL_SERVICE_REFERENCE = BASE_SNAPSHOT_LOCATION + "/standard-external-service-reference"; - private NiFiRegistryFlowMapper flowMapper; + private VersionedComponentFlowMapper flowMapper; private ControllerServiceProvider controllerServiceProvider; private ControllerServiceApiLookup controllerServiceApiLookup; @@ -72,7 +72,7 @@ public class StandardControllerServiceResolverTest { public void setup() { Authorizer authorizer = mock(Authorizer.class); FlowManager flowManager = mock(FlowManager.class); - flowMapper = mock(NiFiRegistryFlowMapper.class); + flowMapper = mock(VersionedComponentFlowMapper.class); controllerServiceProvider = mock(ControllerServiceProvider.class); controllerServiceApiLookup = mock(ControllerServiceApiLookup.class); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/groups/StandardProcessGroupTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/groups/StandardProcessGroupTest.java index b3f888a3397f..0887de6c90d7 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/groups/StandardProcessGroupTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/groups/StandardProcessGroupTest.java @@ -130,7 +130,8 @@ void setProcessGroup() throws IOException { nodeTypeProvider, properties, statelessGroupNodeFactory, - assetManager + assetManager, + null ); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/registry/flow/FlowAnalyzingRegistryClientNodeTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/registry/flow/FlowAnalyzingRegistryClientNodeTest.java index a135ab8d66cd..e69a27a05f6c 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/registry/flow/FlowAnalyzingRegistryClientNodeTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/registry/flow/FlowAnalyzingRegistryClientNodeTest.java @@ -23,7 +23,7 @@ import org.apache.nifi.flowanalysis.EnforcementPolicy; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedProcessGroup; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import org.apache.nifi.validation.RuleViolation; import org.apache.nifi.validation.RuleViolationsManager; import org.junit.jupiter.api.Assertions; @@ -64,7 +64,7 @@ class FlowAnalyzingRegistryClientNodeTest { FlowManager flowManager; @Mock - NiFiRegistryFlowMapper flowMapper; + VersionedComponentFlowMapper flowMapper; @Mock InstantiatedVersionedProcessGroup nonVersionedProcessGroup; @@ -90,7 +90,7 @@ class FlowAnalyzingRegistryClientNodeTest { @BeforeEach public void setUp() { Mockito.when(versionedProcessGroup.getInstanceIdentifier()).thenReturn(INSTANCE_IDENTIFIER); - Mockito.when(flowManager.getGroup(Mockito.anyString())).thenReturn(processGroup); + Mockito.when(flowManager.getGroup(Mockito.anyString(), Mockito.eq(null))).thenReturn(processGroup); Mockito.when(flowMapper.mapNonVersionedProcessGroup(Mockito.same(processGroup), Mockito.same(serviceProvider))).thenReturn(nonVersionedProcessGroup); Mockito.when(ruleViolation1.getEnforcementPolicy()).thenReturn(EnforcementPolicy.ENFORCE); Mockito.when(ruleViolation2.getEnforcementPolicy()).thenReturn(EnforcementPolicy.ENFORCE); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/registry/flow/mapping/TestNiFiRegistryFlowMapper.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/registry/flow/mapping/TestVersionedComponentFlowMapper.java similarity index 95% rename from nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/registry/flow/mapping/TestNiFiRegistryFlowMapper.java rename to nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/registry/flow/mapping/TestVersionedComponentFlowMapper.java index cd08afd42304..cf2635927781 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/registry/flow/mapping/TestNiFiRegistryFlowMapper.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/registry/flow/mapping/TestVersionedComponentFlowMapper.java @@ -55,14 +55,14 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -public class TestNiFiRegistryFlowMapper { +public class TestVersionedComponentFlowMapper { private static final SensitiveValueEncryptor ENCRYPTOR = value -> new StringBuilder(value).reverse().toString(); @Test public void testMappingProcessorWithSensitiveValuesGivesNullValue() { final ExtensionManager extensionManager = mock(ExtensionManager.class); final FlowMappingOptions mappingOptions = FlowMappingOptions.DEFAULT_OPTIONS; - final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(extensionManager, mappingOptions); + final VersionedComponentFlowMapper mapper = new VersionedComponentFlowMapper(extensionManager, mappingOptions); final ControllerServiceProvider serviceProvider = mock(ControllerServiceProvider.class); final Map properties = new HashMap<>(); @@ -82,7 +82,7 @@ public void testMappingProcessorWithSensitiveValuesGivesNullValue() { public void testMappingProcessorWithSensitiveValuesLeavesSensitiveParameterReference() { final ExtensionManager extensionManager = mock(ExtensionManager.class); final FlowMappingOptions mappingOptions = FlowMappingOptions.DEFAULT_OPTIONS; - final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(extensionManager, mappingOptions); + final VersionedComponentFlowMapper mapper = new VersionedComponentFlowMapper(extensionManager, mappingOptions); final ControllerServiceProvider serviceProvider = mock(ControllerServiceProvider.class); final Map properties = new HashMap<>(); @@ -108,7 +108,7 @@ public void testMappingProcessorWithSensitiveValuesProvidesEncryptedValue() { .sensitiveValueEncryptor(ENCRYPTOR) .build(); - final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(extensionManager, mappingOptions); + final VersionedComponentFlowMapper mapper = new VersionedComponentFlowMapper(extensionManager, mappingOptions); final ControllerServiceProvider serviceProvider = mock(ControllerServiceProvider.class); final Map properties = new HashMap<>(); @@ -142,7 +142,7 @@ private void testControllerServicesMapsToProperId(final boolean useVersionedId) .mapControllerServiceReferencesToVersionedId(useVersionedId) .build(); - final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(extensionManager, mappingOptions); + final VersionedComponentFlowMapper mapper = new VersionedComponentFlowMapper(extensionManager, mappingOptions); final ControllerServiceNode mockServiceNode = mock(ControllerServiceNode.class); when(mockServiceNode.getIdentifier()).thenReturn("1234"); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConfigurationUpdateResult.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConfigurationUpdateResult.java new file mode 100644 index 000000000000..0a9922219ee8 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConfigurationUpdateResult.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.components.connector; + +public enum ConfigurationUpdateResult { + NO_CHANGES, + + CHANGES_MADE; +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorAction.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorAction.java new file mode 100644 index 000000000000..60a4865269a4 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorAction.java @@ -0,0 +1,49 @@ +/* + * 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.components.connector; + +/** + * Represents an action that can be performed on a Connector. + */ +public interface ConnectorAction { + + /** + * Returns the name of this action. + * @return the action name + */ + String getName(); + + /** + * Returns a description of what this action does. + * @return the action description + */ + String getDescription(); + + /** + * Returns whether this action is currently allowed to be performed. + * @return true if the action is allowed, false otherwise + */ + boolean isAllowed(); + + /** + * Returns the reason why this action is not allowed, or null if the action is allowed. + * @return the reason the action is not allowed, or null if allowed + */ + String getReasonNotAllowed(); +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfiguration.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfiguration.java new file mode 100644 index 000000000000..4261e2af3d20 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorConfiguration.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.components.connector; + +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; + +public class ConnectorConfiguration { + + private final Set stepConfigurations; + + public ConnectorConfiguration(final Set stepConfigurations) { + this.stepConfigurations = new HashSet<>(stepConfigurations); + } + + public Set getNamedStepConfigurations() { + return stepConfigurations; + } + + public NamedStepConfiguration getNamedStepConfiguration(final String name) { + for (final NamedStepConfiguration stepConfiguration : stepConfigurations) { + if (stepConfiguration.stepName().equals(name)) { + return stepConfiguration; + } + } + return null; + } + + @Override + public boolean equals(final Object other) { + if (other == null || getClass() != other.getClass()) { + return false; + } + + final ConnectorConfiguration that = (ConnectorConfiguration) other; + return Objects.equals(stepConfigurations, that.stepConfigurations); + } + + @Override + public int hashCode() { + return Objects.hashCode(stepConfigurations); + } + + @Override + public String toString() { + return "ConnectorConfiguration[" + + "stepConfigurations=" + stepConfigurations + + "]"; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorNode.java new file mode 100644 index 000000000000..93e90c1adb7a --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorNode.java @@ -0,0 +1,281 @@ +/* + * 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.components.connector; + +import org.apache.nifi.authorization.resource.ComponentAuthorizable; +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.VersionedComponent; +import org.apache.nifi.components.validation.ValidationState; +import org.apache.nifi.components.validation.ValidationStatus; +import org.apache.nifi.connectable.FlowFileTransferCounts; +import org.apache.nifi.engine.FlowEngine; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.VersionedConfigurationStep; +import org.apache.nifi.logging.ComponentLog; + +import java.time.Duration; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.Future; + +public interface ConnectorNode extends ComponentAuthorizable, VersionedComponent { + + String getName(); + + ConnectorState getCurrentState(); + + ConnectorState getDesiredState(); + + /** + * Verifies that the Connector is in a state that allows it to be deleted. If the Connector is not in a state + * that allows it to be deleted, this method will throw an IllegalStateException. + * + * @throws IllegalStateException if the Connector is not in a state that allows it to be deleted + */ + void verifyCanDelete(); + + void verifyCanStart(); + + Connector getConnector(); + + /** + * @return the fully qualified class name of the underlying Connector implementation + */ + String getComponentType(); + + String getCanonicalClassName(); + + BundleCoordinate getBundleCoordinate(); + + /** + * Returns whether or not the underlying extension is missing (i.e., the Connector is a GhostConnector). + * @return true if the extension is missing, false otherwise + */ + boolean isExtensionMissing(); + + List fetchAllowableValues(String stepName, String propertyName); + + List fetchAllowableValues(String stepName, String propertyName, String filter); + + void initializeConnector(FrameworkConnectorInitializationContext initializationContext); + + void loadInitialFlow() throws FlowUpdateException; + + /** + *

+ * Pause triggering asynchronous validation to occur when the connector is updated. Often times, it is necessary + * to update several aspects of a connector, such as the properties and annotation data, at once. When this occurs, + * we don't want to trigger validation for each update, so we can follow the pattern: + *

+ * + *
+     * 
+     * connectorNode.pauseValidationTrigger();
+     * try {
+     *   connectorNode.setProperties(properties);
+     *   connectorNode.setAnnotationData(annotationData);
+     * } finally {
+     *   connectorNode.resumeValidationTrigger();
+     * }
+     * 
+     * 
+ * + *

+ * When calling this method, it is imperative that {@link #resumeValidationTrigger()} is always called within a {@code finally} block to + * ensure that validation occurs. + *

+ */ + void pauseValidationTrigger(); + + /** + * Resume triggering asynchronous validation to occur when the connector is updated. This method is to be used in conjunction + * with {@link #pauseValidationTrigger()} as illustrated in its documentation. When this method is called, if the connector's Validation Status + * is {@link ValidationStatus#VALIDATING}, connector validation will immediately be triggered asynchronously. + */ + void resumeValidationTrigger(); + + /** + * Indicates whether validation triggering is currently paused. + * @return true if validation triggering is paused, false otherwise + */ + boolean isValidationPaused(); + + List verifyConfigurationStep(String configurationStepName, StepConfiguration configurationOverrides); + + List verify(); + + ComponentLog getComponentLog(); + + List getConfigurationSteps(); + + FrameworkFlowContext getActiveFlowContext(); + + FrameworkFlowContext getWorkingFlowContext(); + + void discardWorkingConfiguration(); + + // ------------------- + // The following methods should always be called via the ConnectorRepository in order to maintain proper + // lifecycle management of the Connector. + + /** + * Sets the name of the Connector. This method should only be invoked via the ConnectorRepository. + * @param name the Connector's name + */ + void setName(String name); + + /** + * Performs validation logic that is defined by the Connector. + * @return the ValidationState indicating the results of the validation + */ + ValidationState performValidation(); + + /** + * Returns the current validation status of the connector. + * @return the current ValidationStatus + */ + ValidationStatus getValidationStatus(); + + /** + * Returns the validation errors for the connector, or an empty collection if the connector is valid. + * @return the validation errors, or an empty collection if valid + */ + Collection getValidationErrors(); + + + ValidationState getValidationState(); + + /** + * Returns an Optional Duration indicating how long the Connector has been idle (i.e., not processed any FlowFiles and with no FlowFiles queued). + * If there is any FlowFile queued, the Optional will be empty. Otherwise, the duration will indicate how long it has been since the + * last FlowFile was acted upon. + * + * @return an Optional Duration indicating how long the Connector has been idle + */ + Optional getIdleDuration(); + + /** + * Returns the FlowFileTransferCounts that represents that amount of data sent and received + * by this Connector since it was started. + * @return the FlowFileTransferCounts for this Connector + */ + FlowFileTransferCounts getFlowFileTransferCounts(); + + /** + * Starts the Connector. This method should only be invoked via the ConnectorRepository. + * @param scheduler the ScheduledExecutorService to use for scheduling any tasks that the Connector needs to perform + * @return a Future that will be completed when the Connector has started + */ + Future start(FlowEngine scheduler); + + /** + * Stops the Connector. This method should only be invoked via the ConnectorRepository. + * @param scheduler the ScheduledExecutorService to use for scheduling any tasks that the Connector needs to perform + * @return a Future that will be completed when the Connector has stopped + */ + Future stop(FlowEngine scheduler); + + /** + * Allows the Connector to drain any in-flight data while not accepting any new data. + */ + Future drainFlowFiles(); + + /** + * Cancels the draining of FlowFiles that is currently in progress. + * @throws IllegalStateException if the Connector is not currently draining FlowFiles + */ + void cancelDrainFlowFiles(); + + /** + * Verifies that the Connector can cancel draining FlowFiles. + * @throws IllegalStateException if not in a state where draining FlowFiles can be cancelled + */ + void verifyCancelDrainFlowFiles() throws IllegalStateException; + + /** + * Verifies that the Connector can have its FlowFiles purged. + * @throws IllegalStateException if not in a state where FlowFiles can be purged + */ + void verifyCanPurgeFlowFiles() throws IllegalStateException; + + /** + * Purges all FlowFiles from the Connector, immediately dropping the data. + * + * @param requestor the user requesting the purge. This will be recorded in the associated provenance DROP events. + * @return a Future that will be completed when the purge operation is finished + */ + Future purgeFlowFiles(String requestor); + + /** + * Updates the configuration of one of the configuration steps. This method should only be invoked via the ConnectorRepository. + * @param configurationStepName the name of the configuration step being set + * (must match one of the names returned by {@link Connector#getConfigurationSteps()}) + * @param configuration the configuration for the given configuration step + * @throws FlowUpdateException if unable to apply the configuration changes + */ + void setConfiguration(String configurationStepName, StepConfiguration configuration) throws FlowUpdateException; + + void transitionStateForUpdating(); + + void prepareForUpdate() throws FlowUpdateException; + + /** + * Aborts the update preparation process. This method should only be invoked via the ConnectorRepository. + * @param cause the reason for aborting the update preparation + */ + // TODO: Should this return a Future? + void abortUpdate(Throwable cause); + + /** + * Notifies the Connector that the update process is finished and it can apply any changes that were made during the + * update process. This method should only be invoked via the ConnectorRepository. + * @throws FlowUpdateException if unable to apply the changes made during the update process + */ + void applyUpdate() throws FlowUpdateException; + + /** + * Inherits the given flow configuration into this Connector's active flow configuration. + * @param activeFlowConfiguration the active flow configuration to inherit + * @param workingFlowConfiguration the working flow configuration to inherit + * @param flowContextBundle the bundle associated with the provided configuration + * @throws FlowUpdateException if unable to inherit the given flow configuration + * @throws IllegalStateException if the Connector is not in a state of UPDATING + */ + void inheritConfiguration(List activeFlowConfiguration, List workingFlowConfiguration, + Bundle flowContextBundle) throws FlowUpdateException; + + /** + * Marks the connector as invalid with the given subject and explanation. This is used when a flow update + * fails during initialization or flow synchronization to indicate the connector cannot operate. + * + * @param subject the subject of the validation failure + * @param explanation the reason the connector is invalid + */ + void markInvalid(final String subject, final String explanation); + + /** + * Returns the list of available actions that can be performed on this Connector. + * Each action includes whether it is currently allowed and, if not, the reason why. + * @return the list of available actions + */ + List getAvailableActions(); +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorRepository.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorRepository.java new file mode 100644 index 000000000000..cfa5ae5f9a68 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorRepository.java @@ -0,0 +1,181 @@ +/* + * 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.components.connector; + +import org.apache.nifi.asset.Asset; +import org.apache.nifi.components.connector.secrets.SecretsManager; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.VersionedConfigurationStep; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.Future; + +public interface ConnectorRepository { + + void initialize(ConnectorRepositoryInitializationContext context); + + /** + * Verifies that a connector with the given identifier can be created. This checks that the connector + * does not already exist in the repository, and if a ConnectorConfigurationProvider is configured, + * delegates to the provider's verifyCreate method to ensure the external store can support the operation. + * + * @param connectorId the identifier of the connector to be created + * @throws IllegalStateException if a connector with the given identifier already exists + * @throws ConnectorConfigurationProviderException if the provider rejects the create operation + */ + void verifyCreate(String connectorId); + + /** + * Adds the given Connector to the Repository + * @param connector the Connector to add + */ + void addConnector(ConnectorNode connector); + + /** + * Restores a previously added Connector to the Repository on restart. + * This is differentiated from addConnector in that this method is not called + * for newly created Connectors during the typical lifecycle of NiFi, but rather + * only to notify the Repository of Connectors that were present when NiFi was last shutdown. + * + * @param connector the Connector to restore + */ + void restoreConnector(ConnectorNode connector); + + /** + * Removes the given Connector from the Repository + * @param connectorId the identifier of the Connector to remove + */ + void removeConnector(String connectorId); + + /** + * Gets the Connector with the given identifier + * @param identifier the identifier of the Connector to get + * @return the Connector with the given identifier, or null if no such Connector exists + */ + ConnectorNode getConnector(String identifier); + + /** + * @return all Connectors in the Repository + */ + List getConnectors(); + + /** + * Starts the given Connector, managing any appropriate lifecycle events. + * @param connector the Connector to start + * @return a CompletableFuture that will be completed when the Connector has started + */ + Future startConnector(ConnectorNode connector); + + /** + * Stops the given Connector, managing any appropriate lifecycle events. + * @param connector the Connector to stop + * @return a CompletableFuture that will be completed when the Connector has stopped + */ + Future stopConnector(ConnectorNode connector); + + /** + * Restarts the given Connector, managing any appropriate lifecycle events. + * + * @param connector the Connector to restart + * @return a CompletableFuture that will be completed when the Connector has restarted + */ + Future restartConnector(ConnectorNode connector); + + /** + * Updates the metadata of a Connector, such as its name. This method should be used instead of calling + * {@link ConnectorNode#setName(String)} directly, so that the ConnectorRepository can synchronize + * changes with the ConnectorConfigurationProvider if one is configured. + * + * @param connector the Connector to update + * @param name the new name for the Connector + */ + void updateConnector(ConnectorNode connector, String name); + + void configureConnector(ConnectorNode connector, String stepName, StepConfiguration configuration) throws FlowUpdateException; + + void applyUpdate(ConnectorNode connector, ConnectorUpdateContext context) throws FlowUpdateException; + + void inheritConfiguration(ConnectorNode connector, List activeFlowConfiguration, + List workingFlowConfiguration, Bundle flowContextBundle) throws FlowUpdateException; + + void discardWorkingConfiguration(ConnectorNode connector); + + SecretsManager getSecretsManager(); + + /** + * Creates a new ConnectorStateTransition instance for managing the lifecycle state of a connector. + * + * @param type the connector type + * @param id the connector identifier + * @return a new ConnectorStateTransition instance + */ + ConnectorStateTransition createStateTransition(String type, String id); + + FrameworkConnectorInitializationContextBuilder createInitializationContextBuilder(); + + /** + * Stores an asset for the given connector. If a {@link ConnectorConfigurationProvider} is configured, + * the asset is also uploaded to the external store and a mapping between the NiFi asset ID and the + * external ID is recorded. If the provider upload fails, the local asset is rolled back. + * + * @param connectorId the identifier of the connector that owns the asset + * @param assetId the NiFi-assigned asset identifier (UUID) + * @param assetName the filename of the asset + * @param content the binary content of the asset + * @return the stored Asset + * @throws IOException if an I/O error occurs during storage + */ + Asset storeAsset(String connectorId, String assetId, String assetName, InputStream content) throws IOException; + + /** + * Retrieves an asset by its NiFi-assigned identifier. + * + * @param assetId the NiFi-assigned asset identifier (UUID) + * @return an Optional containing the asset if found, or empty if no asset exists with the given ID + */ + Optional getAsset(String assetId); + + /** + * Retrieves all assets belonging to the given connector. + * + * @param connectorId the identifier of the connector + * @return the list of assets for the connector + */ + List getAssets(String connectorId); + + /** + * Deletes all assets belonging to the given connector from the local store and, + * if a {@link ConnectorConfigurationProvider} is configured, from the external store as well. + * + * @param connectorId the identifier of the connector whose assets should be deleted + */ + void deleteAssets(String connectorId); + + /** + * Ensures that asset binaries for the given connector are available locally by downloading + * any missing or changed assets from the external {@link ConnectorConfigurationProvider}. + * This is a no-op if no provider is configured. This method should be called before operations + * that need local asset files to be present, such as configuration verification. + * + * @param connector the connector whose assets should be synced + */ + void syncAssetsFromProvider(ConnectorNode connector); +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorRepositoryInitializationContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorRepositoryInitializationContext.java new file mode 100644 index 000000000000..91bca1fcc95c --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorRepositoryInitializationContext.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.components.connector; + +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.connector.secrets.SecretsManager; +import org.apache.nifi.controller.NodeTypeProvider; +import org.apache.nifi.controller.flow.FlowManager; +import org.apache.nifi.nar.ExtensionManager; + +public interface ConnectorRepositoryInitializationContext { + + FlowManager getFlowManager(); + + ExtensionManager getExtensionManager(); + + SecretsManager getSecretsManager(); + + AssetManager getAssetManager(); + + NodeTypeProvider getNodeTypeProvider(); + + ConnectorRequestReplicator getRequestReplicator(); + + /** + * Returns the ConnectorConfigurationProvider to use for external management of connector working configuration, + * or null if no provider is configured. + * + * @return the ConnectorConfigurationProvider, or null if not configured + */ + default ConnectorConfigurationProvider getConnectorConfigurationProvider() { + return null; + } + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorRequestReplicator.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorRequestReplicator.java new file mode 100644 index 000000000000..dd6d7f90c2e0 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorRequestReplicator.java @@ -0,0 +1,30 @@ +/* + * 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.components.connector; + +import org.apache.nifi.controller.flow.FlowManager; + +import java.io.IOException; + +public interface ConnectorRequestReplicator { + + ConnectorState getState(String connectorId) throws IOException; + + void setFlowManager(FlowManager flowManager); + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorState.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorState.java new file mode 100644 index 000000000000..c13999c4fa05 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorState.java @@ -0,0 +1,31 @@ +/* + * 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.components.connector; + +public enum ConnectorState { + STARTING, + RUNNING, + STOPPING, + STOPPED, + DRAINING, + PURGING, + PREPARING_FOR_UPDATE, + UPDATING, + UPDATE_FAILED, + UPDATED; +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorStateTransition.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorStateTransition.java new file mode 100644 index 000000000000..a9693f8eafc3 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorStateTransition.java @@ -0,0 +1,82 @@ +/* + * 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.components.connector; + +import java.util.concurrent.CompletableFuture; + +/** + * Manages the state transitions and lifecycle of a ConnectorNode, including tracking current and desired states, + * handling asynchronous state transition completion, and coordinating update resume states. + */ +public interface ConnectorStateTransition { + + /** + * Returns the current state of the connector. + * + * @return the current ConnectorState + */ + ConnectorState getCurrentState(); + + /** + * Returns the desired state of the connector. + * + * @return the desired ConnectorState + */ + ConnectorState getDesiredState(); + + /** + * Sets the desired state for the connector. This operation is used to indicate what state the connector + * should be in, and the framework will work to transition to that state. + * + * @param desiredState the target state for the connector + */ + void setDesiredState(ConnectorState desiredState); + + /** + * Attempts to atomically update the current state from the expected state to the new state. + * + * @param expectedState the expected current state + * @param newState the new state to transition to + * @return true if the state was successfully updated, false if the current state did not match the expected state + */ + boolean trySetCurrentState(ConnectorState expectedState, ConnectorState newState); + + /** + * Sets the current state to the specified state, regardless of the previous state. + * + * @param newState the new current state + */ + void setCurrentState(ConnectorState newState); + + /** + * Registers a future to be completed when the connector transitions to the RUNNING state. + * This method is thread-safe and handles internal synchronization. + * + * @param future the CompletableFuture to complete when the connector starts + */ + void addPendingStartFuture(CompletableFuture future); + + /** + * Registers a future to be completed when the connector transitions to the STOPPED state. + * This method is thread-safe and handles internal synchronization. + * + * @param future the CompletableFuture to complete when the connector stops + */ + void addPendingStopFuture(CompletableFuture future); +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorUpdateContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorUpdateContext.java new file mode 100644 index 000000000000..c5a00bb61b8d --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorUpdateContext.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.components.connector; + +import java.io.IOException; + +public interface ConnectorUpdateContext { + + void saveFlow() throws IOException; + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorValidationTrigger.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorValidationTrigger.java new file mode 100644 index 000000000000..f1e617be3252 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorValidationTrigger.java @@ -0,0 +1,44 @@ +/* + * 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.components.connector; + +/** + * Trigger for initiating validation of a ConnectorNode. + */ +public interface ConnectorValidationTrigger { + + /** + * Triggers validation of the given connector to occur asynchronously. + * If the Connector's validation is already in progress by another thread, this method will + * return without triggering another validation. If the Connector's validation is paused, this + * will schedule the validation to occur once unpaused. + * + * @param connector the connector to validate + */ + void triggerAsync(ConnectorNode connector); + + /** + * Triggers validation of the given connector immediately in the current thread. This will + * trigger validation even if other validation is currently in progress or if the Connector's + * validation is paused. + * + * @param connector the connector to validate + */ + void trigger(ConnectorNode connector); +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FlowContextFactory.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FlowContextFactory.java new file mode 100644 index 000000000000..c4a2cca21772 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FlowContextFactory.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.components.connector; + +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.logging.ComponentLog; + +public interface FlowContextFactory { + FrameworkFlowContext createActiveFlowContext(String connectorId, ComponentLog connectorLogger, Bundle bundle); + + FrameworkFlowContext createWorkingFlowContext(String connectorId, ComponentLog connectorLogger, MutableConnectorConfigurationContext currentConfiguration, Bundle bundle); +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FrameworkConnectorInitializationContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FrameworkConnectorInitializationContext.java new file mode 100644 index 000000000000..8749e48192b6 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FrameworkConnectorInitializationContext.java @@ -0,0 +1,29 @@ +/* + * 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.components.connector; + +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.connector.secrets.SecretsManager; + +public interface FrameworkConnectorInitializationContext extends ConnectorInitializationContext { + + SecretsManager getSecretsManager(); + + AssetManager getAssetManager(); + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FrameworkConnectorInitializationContextBuilder.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FrameworkConnectorInitializationContextBuilder.java new file mode 100644 index 000000000000..9ef92941a8f6 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FrameworkConnectorInitializationContextBuilder.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.components.connector; + +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.connector.secrets.SecretsManager; +import org.apache.nifi.logging.ComponentLog; + +public interface FrameworkConnectorInitializationContextBuilder { + + FrameworkConnectorInitializationContextBuilder identifier(String identifier); + + FrameworkConnectorInitializationContextBuilder name(String name); + + FrameworkConnectorInitializationContextBuilder componentLog(ComponentLog componentLog); + + FrameworkConnectorInitializationContextBuilder assetManager(AssetManager assetManager); + + FrameworkConnectorInitializationContextBuilder secretsManager(SecretsManager secretsManager); + + FrameworkConnectorInitializationContextBuilder componentBundleLookup(ComponentBundleLookup bundleLookup); + + FrameworkConnectorInitializationContext build(); +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FrameworkFlowContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FrameworkFlowContext.java new file mode 100644 index 000000000000..2c05207bf87b --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/FrameworkFlowContext.java @@ -0,0 +1,32 @@ +/* + * 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.components.connector; + +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.groups.ProcessGroup; + +public interface FrameworkFlowContext extends FlowContext { + ProcessGroup getManagedProcessGroup(); + + @Override + MutableConnectorConfigurationContext getConfigurationContext(); + + void updateFlow(VersionedExternalFlow versionedExternalFlow, AssetManager assetManager) throws FlowUpdateException; +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/MutableConnectorConfigurationContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/MutableConnectorConfigurationContext.java new file mode 100644 index 000000000000..22191633af01 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/MutableConnectorConfigurationContext.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.components.connector; + +import java.util.Map; + +public interface MutableConnectorConfigurationContext extends ConnectorConfigurationContext { + + /** + * Sets the properties for the given step to the provided properties. Any existing properties + * for the step that are not included in the provided configuration will remain unchanged. + * + * @param stepName the name of the configuration step + * @param configuration the configuration to set + * @return the result of the configuration update + */ + ConfigurationUpdateResult setProperties(String stepName, StepConfiguration configuration); + + /** + * Replaces all of the properties for the given step with the provided properties. Any existing properties + * for the step that are not included in the provided configuration will be removed. + * + * @param stepName the name of the configuration step + * @param configuration the configuration to set + * @return the result of the configuration update + */ + ConfigurationUpdateResult replaceProperties(String stepName, StepConfiguration configuration); + + /** + * Resolves all existing property values. + */ + void resolvePropertyValues(); + + /** + * Converts this mutable configuration context to an immutable ConnectorConfiguration. + * @return the ConnectorConfiguration + */ + ConnectorConfiguration toConnectorConfiguration(); + + @Override + MutableConnectorConfigurationContext createWithOverrides(String stepName, Map propertyValues); + + /** + * Creates a clone of this MutableConnectorConfigurationContext. + * @return the cloned context + */ + @Override + MutableConnectorConfigurationContext clone(); +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/NamedStepConfiguration.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/NamedStepConfiguration.java new file mode 100644 index 000000000000..7c17798ac81a --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/NamedStepConfiguration.java @@ -0,0 +1,21 @@ +/* + * 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.components.connector; + +public record NamedStepConfiguration(String stepName, StepConfiguration configuration) { +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ParameterContextFacadeFactory.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ParameterContextFacadeFactory.java new file mode 100644 index 000000000000..7ac6b66a7a93 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ParameterContextFacadeFactory.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.components.connector; + +import org.apache.nifi.components.connector.components.ParameterContextFacade; +import org.apache.nifi.groups.ProcessGroup; + +public interface ParameterContextFacadeFactory { + ParameterContextFacade create(ProcessGroup processGroup); +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ProcessGroupFacadeFactory.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ProcessGroupFacadeFactory.java new file mode 100644 index 000000000000..e8d2189c7219 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ProcessGroupFacadeFactory.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.components.connector; + +import org.apache.nifi.components.connector.components.ProcessGroupFacade; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.logging.ComponentLog; + +public interface ProcessGroupFacadeFactory { + ProcessGroupFacade create(ProcessGroup processGroup, ComponentLog connectorLogger); +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ProcessGroupFactory.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ProcessGroupFactory.java new file mode 100644 index 000000000000..5e02a794bbae --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ProcessGroupFactory.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.components.connector; + +import org.apache.nifi.groups.ProcessGroup; + +public interface ProcessGroupFactory { + ProcessGroup create(String id); +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/secrets/SecretProvider.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/secrets/SecretProvider.java new file mode 100644 index 000000000000..3c0a2f4e5f16 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/secrets/SecretProvider.java @@ -0,0 +1,34 @@ +/* + * 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.components.connector.secrets; + +import org.apache.nifi.components.connector.Secret; + +import java.util.List; + +public interface SecretProvider { + + String getProviderId(); + + String getProviderName(); + + List getAllSecrets(); + + List getSecrets(List fullyQualifiedSecretNames); + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/secrets/SecretsManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/secrets/SecretsManager.java new file mode 100644 index 000000000000..62365ed940e7 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/secrets/SecretsManager.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.components.connector.secrets; + +import org.apache.nifi.components.connector.Secret; +import org.apache.nifi.components.connector.SecretReference; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +public interface SecretsManager { + + void initialize(SecretsManagerInitializationContext initializationContext); + + List getAllSecrets(); + + Set getSecretProviders(); + + Optional getSecret(SecretReference secretReference); + + Map getSecrets(Set secretReferences); + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/secrets/SecretsManagerInitializationContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/secrets/SecretsManagerInitializationContext.java new file mode 100644 index 000000000000..4f4cf8e9a861 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/secrets/SecretsManagerInitializationContext.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.components.connector.secrets; + +import org.apache.nifi.controller.flow.FlowManager; + +public interface SecretsManagerInitializationContext { + + FlowManager getFlowManager(); + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/validation/DisabledServiceValidationResult.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/validation/DisabledServiceValidationResult.java index 7cf7c31909a4..a1895b27b8c9 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/validation/DisabledServiceValidationResult.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/validation/DisabledServiceValidationResult.java @@ -19,7 +19,10 @@ import org.apache.nifi.components.ValidationResult; +import java.util.regex.Pattern; + public class DisabledServiceValidationResult extends ValidationResult { + private static final Pattern MESSAGE_PATTERN = Pattern.compile("Controller Service with ID (.+) is disabled"); private String serviceId; public DisabledServiceValidationResult(final String subject, final String serviceId) { @@ -39,4 +42,15 @@ public DisabledServiceValidationResult(final String subject, final String servic public String getControllerServiceIdentifier() { return serviceId; } + + public static boolean isMatch(final ValidationResult result) { + if (result instanceof DisabledServiceValidationResult) { + return true; + } + if (result.isValid()) { + return false; + } + + return MESSAGE_PATTERN.matcher(result.getExplanation()).matches(); + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/validation/ValidationState.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/validation/ValidationState.java index 184f891d455c..d1efd43ab9c3 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/validation/ValidationState.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/validation/ValidationState.java @@ -20,14 +20,26 @@ import org.apache.nifi.components.ValidationResult; import java.util.Collection; +import java.util.Collections; public class ValidationState { private final ValidationStatus status; private final Collection validationErrors; - public ValidationState(final ValidationStatus status, final Collection validationErrors) { + public ValidationState(final ValidationStatus status, final Collection validationResults) { this.status = status; - this.validationErrors = validationErrors; + // Ensure that if we are provided any valid results, they are filtered out because we only want to store validation failures. + this.validationErrors = removeValidResults(validationResults); + } + + private Collection removeValidResults(final Collection validationResults) { + if (validationResults.isEmpty()) { + return Collections.emptyList(); + } + + return validationResults.stream() + .filter(vr -> !vr.isValid()) + .toList(); } public ValidationStatus getStatus() { diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Connectable.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Connectable.java index 669c6f3c837a..f320d9ed550e 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Connectable.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/Connectable.java @@ -314,4 +314,6 @@ default boolean isSessionBatchingSupported() { void setMaxBackoffPeriod(String maxBackoffPeriod); String evaluateParameters(String value); + + FlowFileActivity getFlowFileActivity(); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/ConnectableFlowFileActivity.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/ConnectableFlowFileActivity.java new file mode 100644 index 000000000000..bb4373da5915 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/ConnectableFlowFileActivity.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.connectable; + +import java.util.OptionalLong; +import java.util.concurrent.atomic.AtomicReference; + +public class ConnectableFlowFileActivity implements FlowFileActivity { + private volatile long latestActivityTime = -1L; + private final AtomicReference transferCounts = new AtomicReference<>(new FlowFileTransferCounts()); + + @Override + public void updateLatestActivityTime() { + latestActivityTime = System.currentTimeMillis(); + } + + @Override + public void updateTransferCounts(final int receivedCount, final long receivedBytes, final int sentCount, final long sentBytes) { + if (receivedCount < 0) { + throw new IllegalArgumentException("Received count cannot be negative: " + receivedCount); + } + if (receivedBytes < 0L) { + throw new IllegalArgumentException("Received bytes cannot be negative: " + receivedBytes); + } + if (sentCount < 0) { + throw new IllegalArgumentException("Sent count cannot be negative: " + sentCount); + } + if (sentBytes < 0L) { + throw new IllegalArgumentException("Sent bytes cannot be negative: " + sentBytes); + } + + boolean updated = false; + while (!updated) { + final FlowFileTransferCounts currentCounts = transferCounts.get(); + final FlowFileTransferCounts updatedCounts = currentCounts.plus(receivedCount, receivedBytes, sentCount, sentBytes); + updated = transferCounts.compareAndSet(currentCounts, updatedCounts); + } + } + + @Override + public void reset() { + latestActivityTime = -1L; + transferCounts.set(new FlowFileTransferCounts()); + } + + @Override + public OptionalLong getLatestActivityTime() { + return latestActivityTime == -1L ? OptionalLong.empty() : OptionalLong.of(latestActivityTime); + } + + @Override + public FlowFileTransferCounts getTransferCounts() { + return transferCounts.get(); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/FlowFileActivity.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/FlowFileActivity.java new file mode 100644 index 000000000000..50610c138770 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/FlowFileActivity.java @@ -0,0 +1,34 @@ +/* + * 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.connectable; + +import java.util.OptionalLong; + +public interface FlowFileActivity { + + void updateLatestActivityTime(); + + void updateTransferCounts(int receivedCount, long receivedBytes, int sentCount, long sentBytes); + + void reset(); + + OptionalLong getLatestActivityTime(); + + FlowFileTransferCounts getTransferCounts(); + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/FlowFileTransferCounts.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/FlowFileTransferCounts.java new file mode 100644 index 000000000000..74b604d5bc33 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/FlowFileTransferCounts.java @@ -0,0 +1,88 @@ +/* + * 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.connectable; + +public class FlowFileTransferCounts { + private final long receivedCount; + private final long receivedBytes; + private final long sentCount; + private final long sentBytes; + + public FlowFileTransferCounts() { + this(0, 0L, 0, 0L); + } + + public FlowFileTransferCounts(final long receivedCount, final long receivedBytes, final long sentCount, final long sentBytes) { + this.receivedCount = receivedCount; + this.receivedBytes = receivedBytes; + this.sentCount = sentCount; + this.sentBytes = sentBytes; + } + + public long getReceivedCount() { + return receivedCount; + } + + public long getReceivedBytes() { + return receivedBytes; + } + + public long getSentCount() { + return sentCount; + } + + public long getSentBytes() { + return sentBytes; + } + + public FlowFileTransferCounts plus(final long additionalReceivedCount, final long additionalReceivedBytes, final long additionalSentCount, final long additionalSentBytes) { + return new FlowFileTransferCounts( + this.receivedCount + additionalReceivedCount, + this.receivedBytes + additionalReceivedBytes, + this.sentCount + additionalSentCount, + this.sentBytes + additionalSentBytes + ); + } + + public FlowFileTransferCounts plus(final FlowFileTransferCounts other) { + return new FlowFileTransferCounts( + this.receivedCount + other.getReceivedCount(), + this.receivedBytes + other.getReceivedBytes(), + this.sentCount + other.getSentCount(), + this.sentBytes + other.getSentBytes() + ); + } + + public FlowFileTransferCounts minus(final FlowFileTransferCounts other) { + return new FlowFileTransferCounts( + this.receivedCount - other.getReceivedCount(), + this.receivedBytes - other.getReceivedBytes(), + this.sentCount - other.getSentCount(), + this.sentBytes - other.getSentBytes() + ); + } + + public FlowFileTransferCounts minus(final long additionalReceivedCount, final long additionalReceivedBytes, final long additionalSentCount, final long additionalSentBytes) { + return new FlowFileTransferCounts( + this.receivedCount - additionalReceivedCount, + this.receivedBytes - additionalReceivedBytes, + this.sentCount - additionalSentCount, + this.sentBytes - additionalSentBytes + ); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/ProcessGroupFlowFileActivity.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/ProcessGroupFlowFileActivity.java new file mode 100644 index 000000000000..1d8449ef47a2 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/connectable/ProcessGroupFlowFileActivity.java @@ -0,0 +1,170 @@ +/* + * 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.connectable; + +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.groups.StatelessGroupNode; + +import java.util.Optional; +import java.util.OptionalLong; +import java.util.concurrent.atomic.AtomicLong; + +public class ProcessGroupFlowFileActivity implements FlowFileActivity { + private final ProcessGroup group; + + public ProcessGroupFlowFileActivity(final ProcessGroup group) { + this.group = group; + } + + @Override + public void updateLatestActivityTime() { + throw new UnsupportedOperationException(); + } + + @Override + public void updateTransferCounts(final int receivedCount, final long receivedBytes, final int sentCount, final long sentBytes) { + throw new UnsupportedOperationException(); + } + + @Override + public void reset() { + } + + @Override + public OptionalLong getLatestActivityTime() { + final AtomicLong latestActivityTime = new AtomicLong(-1L); + + // Check all processors + for (final Connectable connectable : group.getProcessors()) { + final OptionalLong activityTime = connectable.getFlowFileActivity().getLatestActivityTime(); + activityTime.ifPresent(time -> { + if (time > latestActivityTime.get()) { + latestActivityTime.set(time); + } + }); + } + + // Check all input ports + for (final Connectable connectable : group.getInputPorts()) { + final OptionalLong activityTime = connectable.getFlowFileActivity().getLatestActivityTime(); + activityTime.ifPresent(time -> { + if (time > latestActivityTime.get()) { + latestActivityTime.set(time); + } + }); + } + + // Check all output ports + for (final Connectable connectable : group.getOutputPorts()) { + final OptionalLong activityTime = connectable.getFlowFileActivity().getLatestActivityTime(); + activityTime.ifPresent(time -> { + if (time > latestActivityTime.get()) { + latestActivityTime.set(time); + } + }); + } + + // Check all funnels + for (final Connectable connectable : group.getFunnels()) { + final OptionalLong activityTime = connectable.getFlowFileActivity().getLatestActivityTime(); + activityTime.ifPresent(time -> { + if (time > latestActivityTime.get()) { + latestActivityTime.set(time); + } + }); + } + + // Check stateless group node if present + final Optional statelessGroupNode = group.getStatelessGroupNode(); + if (statelessGroupNode.isPresent()) { + final OptionalLong activityTime = statelessGroupNode.get().getFlowFileActivity().getLatestActivityTime(); + activityTime.ifPresent(time -> { + if (time > latestActivityTime.get()) { + latestActivityTime.set(time); + } + }); + } + + for (final ProcessGroup childGroup : group.getProcessGroups()) { + final OptionalLong activityTime = childGroup.getFlowFileActivity().getLatestActivityTime(); + activityTime.ifPresent(time -> { + if (time > latestActivityTime.get()) { + latestActivityTime.set(time); + } + }); + } + + final long result = latestActivityTime.get(); + return result == -1L ? OptionalLong.empty() : OptionalLong.of(result); + } + + @Override + public FlowFileTransferCounts getTransferCounts() { + long totalReceivedCount = 0L; + long totalReceivedBytes = 0L; + long totalSentCount = 0L; + long totalSentBytes = 0L; + + // Aggregate transfer counts from all processors + for (final Connectable connectable : group.getProcessors()) { + final FlowFileTransferCounts counts = connectable.getFlowFileActivity().getTransferCounts(); + totalReceivedCount += counts.getReceivedCount(); + totalReceivedBytes += counts.getReceivedBytes(); + totalSentCount += counts.getSentCount(); + totalSentBytes += counts.getSentBytes(); + } + + // Aggregate transfer counts from all input ports + for (final Connectable connectable : group.getInputPorts()) { + final FlowFileTransferCounts counts = connectable.getFlowFileActivity().getTransferCounts(); + totalReceivedCount += counts.getReceivedCount(); + totalReceivedBytes += counts.getReceivedBytes(); + totalSentCount += counts.getSentCount(); + totalSentBytes += counts.getSentBytes(); + } + + // Aggregate transfer counts from all output ports + for (final Connectable connectable : group.getOutputPorts()) { + final FlowFileTransferCounts counts = connectable.getFlowFileActivity().getTransferCounts(); + totalReceivedCount += counts.getReceivedCount(); + totalReceivedBytes += counts.getReceivedBytes(); + totalSentCount += counts.getSentCount(); + totalSentBytes += counts.getSentBytes(); + } + + for (final ProcessGroup childGroup : group.getProcessGroups()) { + final FlowFileTransferCounts counts = childGroup.getFlowFileActivity().getTransferCounts(); + totalReceivedCount += counts.getReceivedCount(); + totalReceivedBytes += counts.getReceivedBytes(); + totalSentCount += counts.getSentCount(); + totalSentBytes += counts.getSentBytes(); + } + + // Aggregate transfer counts from stateless group node if present + final Optional statelessGroupNode = group.getStatelessGroupNode(); + if (statelessGroupNode.isPresent()) { + final FlowFileTransferCounts counts = statelessGroupNode.get().getFlowFileActivity().getTransferCounts(); + totalReceivedCount += counts.getReceivedCount(); + totalReceivedBytes += counts.getReceivedBytes(); + totalSentCount += counts.getSentCount(); + totalSentBytes += counts.getSentBytes(); + } + + return new FlowFileTransferCounts(totalReceivedCount, totalReceivedBytes, totalSentCount, totalSentBytes); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java index 490f8f61e295..c506cb0ba727 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractComponentNode.java @@ -32,6 +32,7 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.components.ConnectorMethod; import org.apache.nifi.components.resource.ResourceContext; import org.apache.nifi.components.resource.ResourceReferenceFactory; import org.apache.nifi.components.resource.ResourceReferences; @@ -68,6 +69,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.lang.reflect.Method; import java.net.URL; import java.util.ArrayList; import java.util.Collection; @@ -414,7 +416,7 @@ public void verifyCanUpdateProperties(final Map properties) { } } - protected List verifyConfig(final Map propertyValues, final String annotationData, final ParameterContext parameterContext) { + protected List verifyConfig(final Map propertyValues, final String annotationData, final ParameterLookup parameterLookup) { final List results = new ArrayList<>(); try { @@ -422,7 +424,7 @@ protected List verifyConfig(final Map descriptorToConfigMap = createDescriptorToConfigMap(propertyValues); final ValidationContext validationContext = getValidationContextFactory().newValidationContext(descriptorToConfigMap, annotationData, - getProcessGroupIdentifier(), getIdentifier(), parameterContext, false); + getProcessGroupIdentifier(), getIdentifier(), parameterLookup, false); final ValidationState validationState = performValidation(validationContext); final ValidationStatus validationStatus = validationState.getStatus(); @@ -471,6 +473,22 @@ protected List verifyConfig(final Map propertyValues, final String annotationData, + final ParameterLookup parameterLookup, final boolean validateConnections) { + + final PropertyConfigurationMapper configurationMapper = new PropertyConfigurationMapper(); + final Map descriptorToRawValueMap = new LinkedHashMap<>(); + for (final Map.Entry entry : propertyValues.entrySet()) { + final PropertyDescriptor descriptor = getPropertyDescriptor(entry.getKey()); + final PropertyConfiguration propertyConfiguration = configurationMapper.mapRawPropertyValuesToPropertyConfiguration(descriptor, entry.getValue()); + descriptorToRawValueMap.put(descriptor, propertyConfiguration); + } + + return getValidationContextFactory().newValidationContext(descriptorToRawValueMap, annotationData, getProcessGroupIdentifier(), getIdentifier(), + parameterLookup, validateConnections); + } + private static Map createDescriptorToConfigMap(final Map propertyValues) { final Map descriptorToConfigMap = new LinkedHashMap<>(); for (final Map.Entry entry : propertyValues.entrySet()) { @@ -590,7 +608,9 @@ private void setProperty(final PropertyDescriptor descriptor, final PropertyConf if (!propertyConfiguration.equals(propertyModComparisonValue)) { try { final String oldValue = propertyModComparisonValue == null ? null : propertyModComparisonValue.getEffectiveValue(getParameterContext()); - onPropertyModified(descriptor, oldValue, resolvedValue); + if (!Objects.equals(oldValue, resolvedValue)) { + onPropertyModified(descriptor, oldValue, resolvedValue); + } } catch (final Exception e) { // nothing really to do here... logger.error("Failed to notify {} that property {} changed", this, descriptor, e); @@ -851,13 +871,15 @@ public ValidationState performValidation(final Map results; + final Collection allResults; try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, getComponent().getClass(), getIdentifier())) { - results = computeValidationErrors(validationContext); + allResults = computeValidationErrors(validationContext); } - final ValidationStatus status = results.isEmpty() ? ValidationStatus.VALID : ValidationStatus.INVALID; - final ValidationState validationState = new ValidationState(status, results); + final Collection invalidResults = allResults.isEmpty() ? Collections.emptyList() + : allResults.stream().filter(result -> !result.isValid()).toList(); + final ValidationStatus status = invalidResults.isEmpty() ? ValidationStatus.VALID : ValidationStatus.INVALID; + final ValidationState validationState = new ValidationState(status, invalidResults); return validationState; } @@ -876,7 +898,7 @@ public final ValidationStatus performValidation() { } protected Collection computeValidationErrors(final ValidationContext validationContext) { - Throwable failureCause = null; + Throwable failureCause; try { if (!sensitiveDynamicPropertyNames.get().isEmpty() && !isSupportsSensitiveDynamicProperties()) { @@ -910,7 +932,7 @@ protected Collection computeValidationErrors(final ValidationC } final List invalidParameterResults = validateParameterReferences(validationContext); - invalidParameterResults.addAll(validateConfig()); + invalidParameterResults.addAll(validateConfig(validationContext)); if (!invalidParameterResults.isEmpty()) { // At this point, we are not able to properly resolve all property values, so we will not attempt to perform @@ -971,11 +993,12 @@ protected Collection computeValidationErrors(final ValidationC * Validates the current configuration, returning ValidationResults for any * invalid configuration parameter. * + * @param validationContext the ValidationContext to use for parameter lookup * @return Collection of validation result objects for any invalid findings * only. If the collection is empty then the component is valid. Should guarantee * non-null */ - protected abstract List validateConfig(); + protected abstract List validateConfig(ValidationContext validationContext); private List validateParameterReferences(final ValidationContext validationContext) { final List results = new ArrayList<>(); @@ -1539,6 +1562,32 @@ public void verifyCanUpdateBundle(final BundleCoordinate incomingCoordinate) thr } } + protected List getConnectorMethods(final Class componentClass) { + final List connectorMethods = new ArrayList<>(); + for (final Method method : componentClass.getDeclaredMethods()) { + final ConnectorMethod annotation = method.getAnnotation(ConnectorMethod.class); + connectorMethods.add(annotation); + } + + return connectorMethods; + } + + protected Method discoverConnectorMethod(final Class componentClass, final String connectorMethodName) { + for (final Method method : componentClass.getDeclaredMethods()) { + final ConnectorMethod annotation = method.getAnnotation(ConnectorMethod.class); + if (annotation != null && annotation.name().equals(connectorMethodName)) { + return method; + } + } + + final Class superClass = componentClass.getSuperclass(); + if (superClass != null && !Object.class.equals(superClass)) { + return discoverConnectorMethod(superClass, connectorMethodName); + } + + return null; + } + protected void setAdditionalResourcesFingerprint(String additionalResourcesFingerprint) { this.additionalResourcesFingerprint = additionalResourcesFingerprint; } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java index 4edb4b8e2ce4..07463946c6a4 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java @@ -25,8 +25,10 @@ import org.apache.nifi.components.PortFunction; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.connectable.ConnectableFlowFileActivity; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; +import org.apache.nifi.connectable.FlowFileActivity; import org.apache.nifi.connectable.Port; import org.apache.nifi.connectable.Position; import org.apache.nifi.groups.ProcessGroup; @@ -96,6 +98,8 @@ public abstract class AbstractPort implements Port { private final Lock readLock = rwLock.readLock(); private final Lock writeLock = rwLock.writeLock(); + private final ConnectableFlowFileActivity flowFileActivity = new ConnectableFlowFileActivity(); + public AbstractPort(final String id, final String name, final ConnectableType type, final ProcessScheduler scheduler) { this.id = requireNonNull(id); this.name = new AtomicReference<>(requireNonNull(name)); @@ -438,6 +442,7 @@ public void shutdown() { @Override public void onSchedulingStart() { + flowFileActivity.reset(); scheduledState.set(ScheduledState.RUNNING); logger.info("{} started", this); } @@ -737,4 +742,9 @@ public PortFunction getPortFunction() { public void setPortFunction(final PortFunction portFunction) { this.portFunction.set(requireNonNull(portFunction)); } + + @Override + public FlowFileActivity getFlowFileActivity() { + return flowFileActivity; + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ComponentNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ComponentNode.java index db9618129971..9461fba3aa9d 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ComponentNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ComponentNode.java @@ -69,6 +69,9 @@ default void setProperties(Map properties) { void verifyCanUpdateProperties(final Map properties); + ValidationContext createValidationContext(Map propertyValues, String annotationData, + ParameterLookup parameterLookup, boolean validateConnections); + /** * @return the Set of names of all Parameters that are referenced by this component */ diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ParameterProviderNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ParameterProviderNode.java index 04f14b09bb79..a0ce84607c87 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ParameterProviderNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ParameterProviderNode.java @@ -51,6 +51,19 @@ public interface ParameterProviderNode extends ComponentNode { */ void fetchParameters(); + /** + * Fetches all parameter values from the Parameter Provider and returns them. This method does not cache the results for later retrieval. + * @return all Parameter Groups with Parameter Names and Values + */ + List fetchParameterValues(); + + /** + * Fetches parameters that match the provided fully qualified parameter names. This method does not cache the results for later retrieval. + * @param fullyQualifiedParameterNames fully qualified names of parameters to fetch + * @return Parameter Groups with Parameter Names and Values that match the provided names + */ + List fetchParameterValues(List fullyQualifiedParameterNames); + /** * Find a named Parameter Group cached from previous request to fetch Parameters from the configured Parameter Provider * diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java index 2df3cb477a39..ebef0a18c25e 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java @@ -17,6 +17,7 @@ package org.apache.nifi.controller; import org.apache.nifi.annotation.notification.PrimaryNodeState; +import org.apache.nifi.components.connector.ConnectorNode; import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.Port; import org.apache.nifi.controller.service.ControllerServiceNode; @@ -60,7 +61,7 @@ public interface ProcessScheduler { * will throw an {@link IllegalStateException}. * @throws IllegalStateException if the Processor is disabled */ - Future startProcessor(ProcessorNode procNode, boolean failIfStopping); + CompletableFuture startProcessor(ProcessorNode procNode, boolean failIfStopping); /** * Starts scheduling the given processor to run once, after invoking all methods @@ -117,7 +118,7 @@ public interface ProcessScheduler { * @param groupNode the group to start * @return a Future that will be completed whenever the group has started */ - Future startStatelessGroup(StatelessGroupNode groupNode); + CompletableFuture startStatelessGroup(StatelessGroupNode groupNode); /** * Stops scheduling the given Stateless Group to run. Returns a Future that will be completed whenever all components within the group have been stopped @@ -251,6 +252,17 @@ public interface ProcessScheduler { */ CompletableFuture enableControllerService(ControllerServiceNode service); + /** + * Enables the Controller Service using the provided ConfigurationContext instead of deriving + * the context from the service's current configuration. This allows enabling a service with + * temporary/override property values. + * + * @param service the controller service to enable + * @param configurationContext the configuration context to use when enabling the service + * @return a CompletableFuture that completes when the service has been enabled + */ + CompletableFuture enableControllerService(ControllerServiceNode service, ConfigurationContext configurationContext); + /** * Disables all of the given Controller Services in the order provided by the List * @param services the controller services to disable @@ -264,6 +276,12 @@ public interface ProcessScheduler { */ CompletableFuture disableControllerService(ControllerServiceNode service); + CompletableFuture startConnector(ConnectorNode connectorNode); + + CompletableFuture stopConnector(ConnectorNode connectorNode); + + void onConnectorRemoved(ConnectorNode connectorNode); + /** * Submits the given task to be executed exactly once in a background thread * diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java index 61a9cc5b5f60..25ce747b7365 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java @@ -20,6 +20,8 @@ import org.apache.nifi.annotation.lifecycle.OnUnscheduled; import org.apache.nifi.annotation.notification.PrimaryNodeState; import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.connector.InvocationFailedException; +import org.apache.nifi.components.connector.components.ConnectorMethod; import org.apache.nifi.components.validation.ValidationStatus; import org.apache.nifi.components.validation.ValidationTrigger; import org.apache.nifi.connectable.Connectable; @@ -31,6 +33,7 @@ import org.apache.nifi.logging.LogLevel; import org.apache.nifi.migration.ControllerServiceFactory; import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.parameter.ParameterLookup; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.Relationship; @@ -136,7 +139,8 @@ public void verifyCanPerformVerification() { } } - public abstract List verifyConfiguration(ProcessContext processContext, ComponentLog logger, Map attributes, ExtensionManager extensionManager); + public abstract List verifyConfiguration(ProcessContext processContext, ComponentLog logger, Map attributes, ExtensionManager extensionManager, + ParameterLookup parameterLookup); public abstract void verifyCanTerminate(); @@ -212,12 +216,12 @@ public abstract void start(ScheduledExecutorService scheduler, long administrati * @param administrativeYieldMillis * the amount of milliseconds to wait for administrative yield * @param timeoutMillis the number of milliseconds to wait after triggering the Processor's @OnScheduled methods before timing out and considering -* the startup a failure. This will result in the thread being interrupted and trying again. + * the startup a failure. This will result in the thread being interrupted and trying again. * @param processContextFactory -* a factory for creating instances of {@link ProcessContext} + * a factory for creating instances of {@link ProcessContext} * @param schedulingAgentCallback -* the callback provided by the {@link ProcessScheduler} to -* execute upon successful start of the Processor + * the callback provided by the {@link ProcessScheduler} to + * execute upon successful start of the Processor */ public abstract void runOnce(ScheduledExecutorService scheduler, long administrativeYieldMillis, long timeoutMillis, Supplier processContextFactory, SchedulingAgentCallback schedulingAgentCallback); @@ -307,4 +311,7 @@ protected void performFlowAnalysisOnThis() { public abstract void migrateConfiguration(Map originalPropertyValues, ControllerServiceFactory serviceFactory); + public abstract String invokeConnectorMethod(String methodName, Map jsonArguments, ProcessContext processContext) throws InvocationFailedException; + + public abstract List getConnectorMethods(); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java index b9a0008e5af2..091a3e36433d 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java @@ -24,8 +24,10 @@ import org.apache.nifi.authorization.resource.ResourceType; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.connectable.ConnectableFlowFileActivity; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; +import org.apache.nifi.connectable.FlowFileActivity; import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.Position; import org.apache.nifi.flowfile.FlowFile; @@ -84,6 +86,8 @@ public class StandardFunnel implements Funnel { final int maxIterations; private final int maxConcurrentTasks; + private final ConnectableFlowFileActivity flowFileActivity = new ConnectableFlowFileActivity(); + public StandardFunnel(final String identifier, final int maxConcurrentTasks, final int maxBatchSize) { this.identifier = identifier; this.processGroupRef = new AtomicReference<>(); @@ -646,4 +650,9 @@ public void setVersionedComponentId(final String versionedComponentId) { } } } + + @Override + public FlowFileActivity getFlowFileActivity() { + return flowFileActivity; + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java index 96806fca1d70..019875c31259 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java @@ -19,7 +19,7 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.controller.flowanalysis.FlowAnalyzer; -import org.apache.nifi.parameter.ParameterContext; +import org.apache.nifi.parameter.ParameterLookup; import org.apache.nifi.validation.RuleViolationsManager; import java.util.Map; @@ -27,7 +27,7 @@ public interface ValidationContextFactory { - ValidationContext newValidationContext(Map properties, String annotationData, String groupId, String componentId, ParameterContext parameterContext, + ValidationContext newValidationContext(Map properties, String annotationData, String groupId, String componentId, ParameterLookup parameterLookup, boolean validateConnections); Optional getRuleViolationsManager(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/FlowManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/FlowManager.java index 70c27cd9d8a8..c29788069e50 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/FlowManager.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/FlowManager.java @@ -17,6 +17,7 @@ package org.apache.nifi.controller.flow; import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.components.connector.ConnectorNode; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.Funnel; @@ -164,6 +165,15 @@ public interface FlowManager extends ParameterProviderLookup { */ ProcessGroup getGroup(String id); + /** + * Returns the ProcessGroup with the given ID that is managed by the Connector with the given ID, + * or null if no such ProcessGroup exists. + * @param groupId id of the group + * @param connectorId id of the connector + * @return the ProcessGroup with the given ID or null if none can be found + */ + ProcessGroup getGroup(String groupId, String connectorId); + void onProcessGroupAdded(ProcessGroup group); void onProcessGroupRemoved(ProcessGroup group); @@ -247,7 +257,11 @@ ProcessorNode createProcessor(String type, String id, BundleCoordinate coordinat Port createLocalOutputPort(String id, String name); - ProcessGroup createProcessGroup(String id); + default ProcessGroup createProcessGroup(String id) { + return createProcessGroup(id, null); + } + + ProcessGroup createProcessGroup(String id, String connectorId); void onConnectionAdded(Connection connection); @@ -345,20 +359,32 @@ ControllerServiceNode createControllerService(String type, String id, BundleCoor * IllegalStateException is thrown. See {@link FlowManager#withParameterContextResolution(Runnable)} * for example usage. * - * @param id The unique id - * @param name The ParameterContext name - * @param description The ParameterContext description - * @param parameters The Parameters - * @param inheritedContextIds The identifiers of any Parameter Contexts that the newly created Parameter Context should inherit from. The order of the identifiers in the List determines the + * @param id the unique id + * @param name the ParameterContext name + * @param description the ParameterContext description + * @param parameters the Parameters + * @param inheritedContextIds the identifiers of any Parameter Contexts that the newly created Parameter Context should inherit from. The order of the identifiers in the List determines the * order in which parameters with conflicting names are resolved. I.e., the Parameter Context whose ID comes first in the List is preferred. - * @param parameterProviderConfiguration Optional configuration for a ParameterProvider - * @return The created ParameterContext - * @throws IllegalStateException If parameterContexts is not empty and this method is called without being wrapped + * @param parameterProviderConfiguration optional configuration for a ParameterProvider + * @return the created ParameterContext + * @throws IllegalStateException if parameterContexts is not empty and this method is called without being wrapped * by {@link FlowManager#withParameterContextResolution(Runnable)} */ ParameterContext createParameterContext(String id, String name, String description, Map parameters, List inheritedContextIds, ParameterProviderConfiguration parameterProviderConfiguration); + /** + * Creates an empty Parameter Context with the given ID for the provided root process group. This Parameter Context is not + * registered with the ParameterContextManager. + * + * @param id the id of the new ParameterContext + * @param name the name of the new ParameterContext + * @param description the description of the new ParameterContext + * @param rootGroup the root process group + * @return the duplicated ParameterContext + */ + ParameterContext createEmptyParameterContext(String id, String name, String description, ProcessGroup rootGroup); + /** * Performs the given ParameterContext-related action, and then resolves all inherited ParameterContext references. * Example usage:

@@ -379,6 +405,22 @@ ParameterContext createParameterContext(String id, String name, String descripti ParameterContextManager getParameterContextManager(); + /** + * Creates a connector using hte provided information. + * @param type the fully qualified class name of the connector + * @param id the unique ID of the connector + * @param coordinate the bundle coordinate for this connector + * @param firstTimeAdded whether or not this is the first time this connector is added to the graph. If {@code true}, will invoke methods + * annotated with the {@link org.apache.nifi.annotation.lifecycle.OnAdded} annotation. + * @param registerLogObserver whether or not to register a log observer for this connector + * @return the created connector + */ + ConnectorNode createConnector(String type, String id, BundleCoordinate coordinate, boolean firstTimeAdded, boolean registerLogObserver); + + List getAllConnectors(); + + ConnectorNode getConnector(String id); + /** * @return the number of each type of component (Processor, Controller Service, Process Group, Funnel, Input Port, Output Port, * Parameter Provider, Reporting Task, Flow Analysis Rule, Remote Process Group) diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedDataflow.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedDataflow.java index e8b92339a533..83b9464cd1ad 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedDataflow.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/flow/VersionedDataflow.java @@ -17,6 +17,7 @@ package org.apache.nifi.controller.flow; +import org.apache.nifi.flow.VersionedConnector; import org.apache.nifi.flow.VersionedControllerService; import org.apache.nifi.flow.VersionedFlowAnalysisRule; import org.apache.nifi.flow.VersionedFlowRegistryClient; @@ -36,6 +37,7 @@ public class VersionedDataflow { private List controllerServices; private List reportingTasks; private List flowAnalysisRules; + private List connectors; private VersionedProcessGroup rootGroup; public VersionedFlowEncodingVersion getEncodingVersion() { @@ -102,6 +104,14 @@ public void setParameterProviders(final List paramet this.parameterProviders = parameterProviders; } + public List getConnectors() { + return connectors; + } + + public void setConnectors(final List connectors) { + this.connectors = connectors; + } + public VersionedProcessGroup getRootGroup() { return rootGroup; } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/queue/DropFlowFileRequest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/queue/DropFlowFileRequest.java index 0aec6c750d5a..9400a79b5457 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/queue/DropFlowFileRequest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/queue/DropFlowFileRequest.java @@ -17,9 +17,12 @@ package org.apache.nifi.controller.queue; +import java.util.concurrent.CompletableFuture; + public class DropFlowFileRequest implements DropFlowFileStatus { private final String identifier; private final long submissionTime = System.currentTimeMillis(); + private final CompletableFuture completionFuture = new CompletableFuture<>(); private volatile QueueSize originalSize; private volatile QueueSize currentSize; @@ -85,6 +88,11 @@ public String getFailureReason() { return failureReason; } + @Override + public CompletableFuture getCompletionFuture() { + return completionFuture; + } + public synchronized void setState(final DropFlowFileState state) { setState(state, null); } @@ -93,6 +101,12 @@ public synchronized void setState(final DropFlowFileState state, final String ex this.state = state; this.failureReason = explanation; this.lastUpdated = System.currentTimeMillis(); + + if (state == DropFlowFileState.CANCELED || state == DropFlowFileState.COMPLETE) { + completionFuture.complete(null); + } else if (state == DropFlowFileState.FAILURE) { + completionFuture.completeExceptionally(new RuntimeException(explanation)); + } } public synchronized boolean cancel() { @@ -101,6 +115,7 @@ public synchronized boolean cancel() { } this.state = DropFlowFileState.CANCELED; + completionFuture.complete(null); return true; } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java index 52a4f0db477d..0f1bd09535f9 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java @@ -20,6 +20,8 @@ import org.apache.nifi.components.ConfigVerificationResult; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.VersionedComponent; +import org.apache.nifi.components.connector.InvocationFailedException; +import org.apache.nifi.components.connector.components.ConnectorMethod; import org.apache.nifi.controller.ComponentNode; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.controller.ControllerService; @@ -30,6 +32,7 @@ import org.apache.nifi.logging.LogLevel; import org.apache.nifi.migration.ControllerServiceFactory; import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.parameter.ParameterLookup; import java.util.List; import java.util.Map; @@ -111,6 +114,20 @@ public interface ControllerServiceNode extends ComponentNode, VersionedComponent */ CompletableFuture enable(ScheduledExecutorService scheduler, long administrativeYieldMillis, boolean completeExceptionallyOnFailure); + /** + * Enables this service using the provided ConfigurationContext, calling any method annotated with @OnEnabled and updating the state of the service. + * This allows enabling a service with temporary/override property values without modifying the service's configuration. + * + * @param scheduler implementation of {@link ScheduledExecutorService} used to initiate service enabling task as well as its retries + * @param administrativeYieldMillis the amount of milliseconds to wait for administrative yield + * @param completeExceptionallyOnFailure if the Controller Service cannot be enabled because it is invalid or throws an Exception from the @OnEnabled lifecycle method, + * dictates whether the CompletableFuture should be completed exceptionally or not + * @param configurationContext the ConfigurationContext to use when enabling the service, or null to use the default + * + * @return a CompletableFuture that can be used to wait for the service to finish enabling + */ + CompletableFuture enable(ScheduledExecutorService scheduler, long administrativeYieldMillis, boolean completeExceptionallyOnFailure, ConfigurationContext configurationContext); + /** * Will disable this service. Disabling of the service typically means * invoking it's operation that is annotated with @OnDisabled. @@ -238,9 +255,11 @@ public interface ControllerServiceNode extends ComponentNode, VersionedComponent * @param logger a logger that can be used when performing verification * @param variables variables that can be used to resolve property values via Expression Language * @param extensionManager extension manager that is used for obtaining appropriate NAR ClassLoaders + * @param parameterLookup the parameter lookup used to resolve parameter references * @return a list of results indicating whether or not the given configuration is valid */ - List verifyConfiguration(ConfigurationContext context, ComponentLog logger, Map variables, ExtensionManager extensionManager); + List verifyConfiguration(ConfigurationContext context, ComponentLog logger, Map variables, ExtensionManager extensionManager, + ParameterLookup parameterLookup); /** * Sets a new proxy and implementation for this node. @@ -257,4 +276,7 @@ void setControllerServiceAndProxy(final LoggableComponent imp void migrateConfiguration(Map originalPropertyValues, ControllerServiceFactory serviceFactory); + String invokeConnectorMethod(String methodName, Map jsonArguments, ConfigurationContext configurationContext) throws InvocationFailedException; + + List getConnectorMethods(); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java index c836e93063af..6e3755427964 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java @@ -84,14 +84,14 @@ public interface ControllerServiceProvider extends ControllerServiceLookup { * @param serviceNodes the nodes * @return a Future that can be used to cancel the task or wait until it is completed */ - Future enableControllerServicesAsync(Collection serviceNodes); + CompletableFuture enableControllerServicesAsync(Collection serviceNodes); /** * Enables the given Controller Service and any dependencies that it has * @param serviceNode the Controller Service to enable * @return a Future that can be used to cancel the task or wait until it is completed */ - Future enableControllerServiceAndDependencies(ControllerServiceNode serviceNode); + CompletableFuture enableControllerServiceAndDependencies(ControllerServiceNode serviceNode); /** * Disables the given controller service so that it cannot be used by other diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java index 443ac18e8e73..86c80ce986bf 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java @@ -16,6 +16,7 @@ */ package org.apache.nifi.events; +import org.apache.nifi.components.connector.ConnectorNode; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.reporting.Bulletin; @@ -68,6 +69,11 @@ public static Bulletin createBulletin(final Connectable connectable, final Strin return bulletin; } + public static Bulletin createBulletin(final ConnectorNode connector, final String category, final String severity, final String message) { + final String connectorName = connector.getName(); + return createBulletin(null, connectorName, connector.getIdentifier(), ComponentType.CONNECTOR, connector.getName(), category, severity, message, null, null); + } + private static String buildGroupPath(ProcessGroup group) { if (group == null) { return null; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java index cc53fe961dea..91a0a295aa4f 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java @@ -16,10 +16,12 @@ */ package org.apache.nifi.groups; +import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.authorization.resource.ComponentAuthorizable; import org.apache.nifi.components.VersionedComponent; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connection; +import org.apache.nifi.connectable.FlowFileActivity; import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.Port; import org.apache.nifi.connectable.Positionable; @@ -136,6 +138,11 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable, Versi */ void setName(String name); + /** + * @return the ID of the Connector that is responsible for this Process Group, or an empty optional if no Connector is associated + */ + Optional getConnectorIdentifier(); + /** * @return the user-set comments about this ProcessGroup, or * null if no comments have been set @@ -255,7 +262,7 @@ default CompletableFuture stopComponents() { * @throws IllegalStateException if the processor is not valid, or is * already running */ - Future startProcessor(ProcessorNode processor, boolean failIfStopping); + CompletableFuture startProcessor(ProcessorNode processor, boolean failIfStopping); /** * Runs the given Processor once and the stops it by calling the provided callback. @@ -1268,4 +1275,26 @@ default CompletableFuture stopComponents() { * @return the configured maximum amount of time that a Stateless Flow can run before it times out and is considered a failure */ String getStatelessFlowTimeout(); + + /** + * @return the FlowFileActivity for this Process Group + */ + FlowFileActivity getFlowFileActivity(); + + /** + * Sets an explicit Authorizable that is to be used as the Process Group's parent for the purposes of authorization. + * If not set, the Process Group's parent group will be used. + * @param parent the parent Authorizable to set, or null to clear any explicit parent + */ + void setExplicitParentAuthorizable(Authorizable parent); + + /** + * Deletes any data in any FlowFile Queues in this Process Group and any child Process Groups + * and removes any components (Controller Services, Processors, Connections, etc.) from this Process Group + * and any child Process Groups. Because the emptying of FlowFile Queues may be a long-running operation, + * this method returns a CompletableFuture that will be completed once the purge operation has finished. + * + * @return a CompletableFuture that will be completed once the purge operation has finished + */ + CompletableFuture purge(); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java index 87676aae9372..201200b4ed96 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/RemoteProcessGroup.java @@ -30,7 +30,7 @@ import java.util.Collection; import java.util.Date; import java.util.Set; -import java.util.concurrent.Future; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; public interface RemoteProcessGroup extends ComponentAuthorizable, Positionable, VersionedComponent { @@ -127,7 +127,7 @@ public interface RemoteProcessGroup extends ComponentAuthorizable, Positionable, * Immediately terminates communications between this instance and the * remote instance. */ - Future stopTransmitting(); + CompletableFuture stopTransmitting(); /** * Initiates communications between this instance and the remote instance diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/manifest/RuntimeManifestService.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/manifest/RuntimeManifestService.java index 3acbfda5b4d9..60c9463cbd06 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/manifest/RuntimeManifestService.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/manifest/RuntimeManifestService.java @@ -43,12 +43,24 @@ public interface RuntimeManifestService { RuntimeManifest getManifestForBundle(String group, String artifact, String version); /** - * Returns a mapping of additionalDetails for the speicfied bundle. + * Returns a mapping of additionalDetails for the specified bundle. * * @param group The bundle group * @param artifact The bundle artifact * @param version The bundle version - * @return The additionaDetails mapping + * @return The additionalDetails mapping */ Map discoverAdditionalDetails(String group, String artifact, String version); + + /** + * Returns a mapping of step documentation files for the specified Connector type. + * The keys are the step names and the values are the corresponding documentation files. + * + * @param group The bundle group + * @param artifact The bundle artifact + * @param version The bundle version + * @param connectorType The fully qualified class name of the Connector + * @return The step documentation mapping + */ + Map discoverStepDocumentation(String group, String artifact, String version, String connectorType); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/connectable/ProcessGroupFlowFileActivityTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/connectable/ProcessGroupFlowFileActivityTest.java new file mode 100644 index 000000000000..754b955dad03 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/connectable/ProcessGroupFlowFileActivityTest.java @@ -0,0 +1,162 @@ +/* + * 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.connectable; + +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.groups.StatelessGroupNode; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class ProcessGroupFlowFileActivityTest { + + @Mock + private ProcessGroup processGroup; + + private ProcessGroupFlowFileActivity activity; + + @BeforeEach + void setUp() { + activity = new ProcessGroupFlowFileActivity(processGroup); + + lenient().when(processGroup.getProcessors()).thenReturn(Collections.emptyList()); + lenient().when(processGroup.getInputPorts()).thenReturn(Collections.emptySet()); + lenient().when(processGroup.getOutputPorts()).thenReturn(Collections.emptySet()); + lenient().when(processGroup.getFunnels()).thenReturn(Collections.emptySet()); + lenient().when(processGroup.getProcessGroups()).thenReturn(Collections.emptySet()); + lenient().when(processGroup.getStatelessGroupNode()).thenReturn(Optional.empty()); + } + + @Test + void testGetTransferCountsWithNoComponents() { + final FlowFileTransferCounts counts = activity.getTransferCounts(); + + assertEquals(0L, counts.getReceivedCount()); + assertEquals(0L, counts.getReceivedBytes()); + assertEquals(0L, counts.getSentCount()); + assertEquals(0L, counts.getSentBytes()); + } + + @Test + void testGetTransferCountsAggregatesProcessors() { + final ProcessorNode processor = createMockConnectable(ProcessorNode.class, 10, 100L, 5, 50L); + when(processGroup.getProcessors()).thenReturn(List.of(processor)); + + final FlowFileTransferCounts counts = activity.getTransferCounts(); + + assertEquals(10L, counts.getReceivedCount()); + assertEquals(100L, counts.getReceivedBytes()); + assertEquals(5L, counts.getSentCount()); + assertEquals(50L, counts.getSentBytes()); + } + + @Test + void testGetTransferCountsIncludesStatelessGroupNode() { + final StatelessGroupNode statelessGroupNode = mock(StatelessGroupNode.class); + final FlowFileActivity statelessActivity = new ConnectableFlowFileActivity(); + statelessActivity.updateTransferCounts(20, 2000L, 15, 1500L); + + when(statelessGroupNode.getFlowFileActivity()).thenReturn(statelessActivity); + when(processGroup.getStatelessGroupNode()).thenReturn(Optional.of(statelessGroupNode)); + + final FlowFileTransferCounts counts = activity.getTransferCounts(); + + assertEquals(20L, counts.getReceivedCount()); + assertEquals(2000L, counts.getReceivedBytes()); + assertEquals(15L, counts.getSentCount()); + assertEquals(1500L, counts.getSentBytes()); + } + + @Test + void testGetTransferCountsAggregatesProcessorsAndStatelessGroupNode() { + final ProcessorNode processor = createMockConnectable(ProcessorNode.class, 10, 100L, 5, 50L); + when(processGroup.getProcessors()).thenReturn(List.of(processor)); + + final StatelessGroupNode statelessGroupNode = mock(StatelessGroupNode.class); + final FlowFileActivity statelessActivity = new ConnectableFlowFileActivity(); + statelessActivity.updateTransferCounts(20, 2000L, 15, 1500L); + when(statelessGroupNode.getFlowFileActivity()).thenReturn(statelessActivity); + when(processGroup.getStatelessGroupNode()).thenReturn(Optional.of(statelessGroupNode)); + + final FlowFileTransferCounts counts = activity.getTransferCounts(); + + assertEquals(30L, counts.getReceivedCount()); + assertEquals(2100L, counts.getReceivedBytes()); + assertEquals(20L, counts.getSentCount()); + assertEquals(1550L, counts.getSentBytes()); + } + + @Test + void testGetTransferCountsAggregatesChildGroups() { + final ProcessGroup childGroup = mock(ProcessGroup.class); + final FlowFileActivity childActivity = mock(FlowFileActivity.class); + when(childActivity.getTransferCounts()).thenReturn(new FlowFileTransferCounts(8, 800L, 3, 300L)); + when(childGroup.getFlowFileActivity()).thenReturn(childActivity); + when(processGroup.getProcessGroups()).thenReturn(Set.of(childGroup)); + + final FlowFileTransferCounts counts = activity.getTransferCounts(); + + assertEquals(8L, counts.getReceivedCount()); + assertEquals(800L, counts.getReceivedBytes()); + assertEquals(3L, counts.getSentCount()); + assertEquals(300L, counts.getSentBytes()); + } + + @Test + void testGetLatestActivityTimeEmpty() { + final OptionalLong result = activity.getLatestActivityTime(); + assertTrue(result.isEmpty()); + } + + @Test + void testGetLatestActivityTimeIncludesStatelessGroupNode() { + final StatelessGroupNode statelessGroupNode = mock(StatelessGroupNode.class); + final ConnectableFlowFileActivity statelessActivity = new ConnectableFlowFileActivity(); + statelessActivity.updateLatestActivityTime(); + when(statelessGroupNode.getFlowFileActivity()).thenReturn(statelessActivity); + when(processGroup.getStatelessGroupNode()).thenReturn(Optional.of(statelessGroupNode)); + + final OptionalLong result = activity.getLatestActivityTime(); + assertTrue(result.isPresent()); + } + + private T createMockConnectable(final Class type, + final int receivedCount, final long receivedBytes, final int sentCount, final long sentBytes) { + final T connectable = mock(type); + final FlowFileActivity connectableActivity = new ConnectableFlowFileActivity(); + connectableActivity.updateTransferCounts(receivedCount, receivedBytes, sentCount, sentBytes); + when(connectable.getFlowFileActivity()).thenReturn(connectableActivity); + return connectable; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java index e9d383c60df2..8f8edd4724b1 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/test/java/org/apache/nifi/controller/TestAbstractComponentNode.java @@ -536,7 +536,7 @@ public Resource getResource() { } @Override - protected List validateConfig() { + protected List validateConfig(final ValidationContext validationContext) { return Collections.emptyList(); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml index 0a3a60621e05..351d3c33fecd 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml @@ -128,7 +128,6 @@ org.apache.nifi nifi-expression-language 2.9.0-SNAPSHOT - test org.apache.nifi @@ -290,6 +289,12 @@ netty-transport test + + org.apache.nifi + nifi-connector-utils + 2.9.0-SNAPSHOT + test + com.github.ben-manes.caffeine caffeine @@ -319,6 +324,9 @@ src/test/resources/old-swap-file.swap src/test/resources/xxe_template.xml src/test/resources/swap/444-old-swap-file.swap + src/test/resources/colors.txt + src/test/resources/TestRuntimeManifest/nifi-test-components-nar/META-INF/docs/steps/org.example.TestConnector/Another_Test_Step.md + src/test/resources/TestRuntimeManifest/nifi-test-components-nar/META-INF/docs/steps/org.example.TestConnector/Test_Step.md diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/AssetsRestApiClient.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/AssetsRestApiClient.java index b8f2052dd514..29cecc02a0e4 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/AssetsRestApiClient.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/AssetsRestApiClient.java @@ -40,13 +40,14 @@ class AssetsRestApiClient extends NiFiRestApiClient { private static final String NIFI_API_PATH_PART = "nifi-api"; private static final String PARAMETER_CONTEXTS_PATH_PART = "parameter-contexts"; + private static final String CONNECTORS_PATH_PART = "connectors"; private static final String ASSETS_PATH_PART = "assets"; public AssetsRestApiClient(final WebClientService webClientService, final String host, final int port, final boolean secure) { super(webClientService, host, port, secure); } - public AssetsEntity getAssets(final String parameterContextId) { + public AssetsEntity getParameterContextAssets(final String parameterContextId) { final URI requestUri = new StandardHttpUriBuilder() .scheme(baseUri.getScheme()) .host(baseUri.getHost()) @@ -56,7 +57,7 @@ public AssetsEntity getAssets(final String parameterContextId) { .addPathSegment(parameterContextId) .addPathSegment(ASSETS_PATH_PART) .build(); - logger.debug("Requesting Asset listing from {}", requestUri); + logger.debug("Requesting Parameter Context Asset listing from {}", requestUri); // Send the replicated header so that the cluster coordinator does not replicate the request, otherwise this call can happen when no nodes // are considered connected and result in a 500 exception that can't easily be differentiated from other unknown errors @@ -68,7 +69,7 @@ public AssetsEntity getAssets(final String parameterContextId) { return executeEntityRequest(requestUri, requestBodySpec, AssetsEntity.class); } - public InputStream getAssetContent(final String parameterContextId, final String assetId) { + public InputStream getParameterContextAssetContent(final String parameterContextId, final String assetId) { final URI requestUri = new StandardHttpUriBuilder() .scheme(baseUri.getScheme()) .host(baseUri.getHost()) @@ -79,7 +80,53 @@ public InputStream getAssetContent(final String parameterContextId, final String .addPathSegment(ASSETS_PATH_PART) .addPathSegment(assetId) .build(); - logger.debug("Getting asset content from {}", requestUri); + logger.debug("Getting Parameter Context asset content from {}", requestUri); + + try { + final HttpResponseEntity response = webClientService.get() + .uri(requestUri) + .header(ACCEPT_HEADER, APPLICATION_OCTET_STREAM) + .retrieve(); + return getResponseBody(requestUri, response); + } catch (final WebClientServiceException e) { + throw new NiFiRestApiRetryableException(e.getMessage(), e); + } + } + + public AssetsEntity getConnectorAssets(final String connectorId) { + final URI requestUri = new StandardHttpUriBuilder() + .scheme(baseUri.getScheme()) + .host(baseUri.getHost()) + .port(baseUri.getPort()) + .addPathSegment(NIFI_API_PATH_PART) + .addPathSegment(CONNECTORS_PATH_PART) + .addPathSegment(connectorId) + .addPathSegment(ASSETS_PATH_PART) + .build(); + logger.debug("Requesting Connector Asset listing from {}", requestUri); + + // Send the replicated header so that the cluster coordinator does not replicate the request, otherwise this call can happen when no nodes + // are considered connected and result in a 500 exception that can't easily be differentiated from other unknown errors + final HttpRequestBodySpec requestBodySpec = webClientService.get() + .uri(requestUri) + .header(ACCEPT_HEADER, APPLICATION_JSON) + .header(REQUEST_REPLICATED_HEADER, Boolean.TRUE.toString()); + + return executeEntityRequest(requestUri, requestBodySpec, AssetsEntity.class); + } + + public InputStream getConnectorAssetContent(final String connectorId, final String assetId) { + final URI requestUri = new StandardHttpUriBuilder() + .scheme(baseUri.getScheme()) + .host(baseUri.getHost()) + .port(baseUri.getPort()) + .addPathSegment(NIFI_API_PATH_PART) + .addPathSegment(CONNECTORS_PATH_PART) + .addPathSegment(connectorId) + .addPathSegment(ASSETS_PATH_PART) + .addPathSegment(assetId) + .build(); + logger.debug("Getting Connector asset content from {}", requestUri); try { final HttpResponseEntity response = webClientService.get() diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/StandardAssetComponentManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/StandardAssetComponentManager.java index 9954c947c154..9449ce8ec6d4 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/StandardAssetComponentManager.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/StandardAssetComponentManager.java @@ -58,7 +58,7 @@ public StandardAssetComponentManager(final FlowController flowController) { @Override public void restartReferencingComponentsAsync(final Asset asset) { final String assetName = asset.getName(); - final String paramContextId = asset.getParameterContextIdentifier(); + final String paramContextId = asset.getOwnerIdentifier(); Thread.ofVirtual().name("Restart Components for Asset [%s] in ParameterContext [%s]".formatted(assetName, paramContextId)).start(() -> { try { restartReferencingComponents(asset); @@ -71,7 +71,7 @@ public void restartReferencingComponentsAsync(final Asset asset) { @Override public void restartReferencingComponents(final Asset asset) { final ParameterContextManager parameterContextManager = flowManager.getParameterContextManager(); - final ParameterContext parameterContext = parameterContextManager.getParameterContext(asset.getParameterContextIdentifier()); + final ParameterContext parameterContext = parameterContextManager.getParameterContext(asset.getOwnerIdentifier()); // Determine which parameters reference the given asset final Set parametersReferencingAsset = getParametersReferencingAsset(parameterContext, asset); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/StandardAssetSynchronizer.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/StandardAssetSynchronizer.java index 96b4f027cf01..c4bd08fbd372 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/StandardAssetSynchronizer.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/StandardAssetSynchronizer.java @@ -212,7 +212,7 @@ private AssetsEntity listAssetsWithRetry(final AssetsRestApiClient assetsRestApi private AssetsEntity listAssets(final AssetsRestApiClient assetsRestApiClient, final String parameterContextId) { try { - return assetsRestApiClient.getAssets(parameterContextId); + return assetsRestApiClient.getParameterContextAssets(parameterContextId); } catch (final NiFiRestApiRetryableException e) { final Throwable rootCause = ExceptionUtils.getRootCause(e); logger.warn("{}, root cause [{}]: retrying", e.getMessage(), rootCause.getMessage()); @@ -242,7 +242,7 @@ private Asset synchronizeAsset(final AssetsRestApiClient assetsRestApiClient, fi if (Boolean.TRUE == coordinatorAsset.getMissingContent()) { return assetManager.createMissingAsset(parameterContextId, assetName); } else { - try (final InputStream assetInputStream = assetsRestApiClient.getAssetContent(parameterContextId, assetId)) { + try (final InputStream assetInputStream = assetsRestApiClient.getParameterContextAssetContent(parameterContextId, assetId)) { return assetManager.createAsset(parameterContextId, assetName, assetInputStream); } catch (final NiFiRestApiRetryableException e) { final Throwable rootCause = ExceptionUtils.getRootCause(e); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/StandardConnectorAssetSynchronizer.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/StandardConnectorAssetSynchronizer.java new file mode 100644 index 000000000000..ad70446ce5fc --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/asset/StandardConnectorAssetSynchronizer.java @@ -0,0 +1,280 @@ +/* + * 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.asset; + +import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.nifi.client.NiFiRestApiRetryableException; +import org.apache.nifi.cluster.coordination.ClusterCoordinator; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorRepository; +import org.apache.nifi.components.connector.ConnectorState; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.flow.FlowManager; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.api.dto.AssetDTO; +import org.apache.nifi.web.api.entity.AssetEntity; +import org.apache.nifi.web.api.entity.AssetsEntity; +import org.apache.nifi.web.client.api.WebClientService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.time.Duration; +import java.time.Instant; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Synchronizes connector assets from the cluster coordinator using the NiFi REST API. + */ +public class StandardConnectorAssetSynchronizer implements AssetSynchronizer { + + private static final Logger logger = LoggerFactory.getLogger(StandardConnectorAssetSynchronizer.class); + + private static final Duration CLUSTER_COORDINATOR_RETRY_DURATION = Duration.ofSeconds(60); + private static final Duration LIST_ASSETS_RETRY_DURATION = Duration.ofMinutes(5); + private static final Duration SYNC_ASSET_RETRY_DURATION = Duration.ofSeconds(30); + + private final AssetManager assetManager; + private final FlowManager flowManager; + private final ConnectorRepository connectorRepository; + private final ClusterCoordinator clusterCoordinator; + private final WebClientService webClientService; + private final NiFiProperties properties; + + public StandardConnectorAssetSynchronizer(final FlowController flowController, + final ClusterCoordinator clusterCoordinator, + final WebClientService webClientService, + final NiFiProperties properties) { + this.assetManager = flowController.getConnectorAssetManager(); + this.flowManager = flowController.getFlowManager(); + this.connectorRepository = flowController.getConnectorRepository(); + this.clusterCoordinator = clusterCoordinator; + this.webClientService = webClientService; + this.properties = properties; + } + + @Override + public void synchronize() { + if (clusterCoordinator == null) { + logger.info("Clustering is not configured: Connector asset synchronization disabled"); + return; + } + + final NodeIdentifier coordinatorNodeId = getElectedActiveCoordinatorNode(); + if (coordinatorNodeId == null) { + logger.warn("Unable to obtain the node identifier for the cluster coordinator: Connector asset synchronization disabled"); + return; + } + + if (clusterCoordinator.isActiveClusterCoordinator()) { + logger.info("Current node is the cluster coordinator: Connector asset synchronization disabled"); + return; + } + + final String coordinatorAddress = coordinatorNodeId.getApiAddress(); + final int coordinatorPort = coordinatorNodeId.getApiPort(); + final AssetsRestApiClient assetsRestApiClient = new AssetsRestApiClient(webClientService, coordinatorAddress, coordinatorPort, properties.isHTTPSConfigured()); + logger.info("Synchronizing connector assets with cluster coordinator at {}:{}", coordinatorAddress, coordinatorPort); + + final List connectors = flowManager.getAllConnectors(); + logger.info("Found {} connectors for synchronizing assets", connectors.size()); + + final Set connectorsWithSynchronizedAssets = new HashSet<>(); + for (final ConnectorNode connector : connectors) { + try { + final boolean assetSynchronized = synchronize(assetsRestApiClient, connector); + if (assetSynchronized) { + connectorsWithSynchronizedAssets.add(connector); + } + } catch (final Exception e) { + logger.error("Failed to synchronize assets for connector [{}]", connector.getIdentifier(), e); + } + } + + restartConnectorsWithSynchronizedAssets(connectorsWithSynchronizedAssets); + } + + private void restartConnectorsWithSynchronizedAssets(final Set connectorsWithSynchronizedAssets) { + for (final ConnectorNode connector : connectorsWithSynchronizedAssets) { + final ConnectorState currentState = connector.getDesiredState(); + if (currentState == ConnectorState.RUNNING) { + logger.info("Restarting connector [{}] after asset synchronization", connector.getIdentifier()); + try { + final Future restartFuture = connectorRepository.restartConnector(connector); + restartFuture.get(); + logger.info("Successfully restarted connector [{}] after asset synchronization", connector.getIdentifier()); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + logger.error("Interrupted while restarting connector [{}] after asset synchronization", connector.getIdentifier(), e); + } catch (final ExecutionException e) { + logger.error("Failed to restart connector [{}] after asset synchronization", connector.getIdentifier(), e.getCause()); + } + } else { + logger.info("Connector [{}] is not running (state={}): skipping restart after asset synchronization", connector.getIdentifier(), currentState); + } + } + } + + private boolean synchronize(final AssetsRestApiClient assetsRestApiClient, final ConnectorNode connector) { + final String connectorId = connector.getIdentifier(); + final Map existingAssets = assetManager.getAssets(connectorId).stream() + .collect(Collectors.toMap(Asset::getIdentifier, Function.identity())); + + final AssetsEntity coordinatorAssetsEntity = listConnectorAssetsWithRetry(assetsRestApiClient, connectorId); + if (coordinatorAssetsEntity == null) { + logger.error("Timeout listing assets from cluster coordinator for connector [{}]", connectorId); + return false; + } + + final Collection coordinatorAssets = coordinatorAssetsEntity.getAssets(); + if (coordinatorAssets == null || coordinatorAssets.isEmpty()) { + logger.info("Connector [{}] did not return any assets from the cluster coordinator", connectorId); + return false; + } + + logger.info("Connector [{}] returned {} assets from the cluster coordinator", connectorId, coordinatorAssets.size()); + + boolean assetSynchronized = false; + for (final AssetEntity coordinatorAssetEntity : coordinatorAssets) { + final AssetDTO coordinatorAsset = coordinatorAssetEntity.getAsset(); + final Asset matchingAsset = existingAssets.get(coordinatorAsset.getId()); + try { + final boolean assetWasSynchronized = synchronize(assetsRestApiClient, connectorId, coordinatorAsset, matchingAsset); + if (assetWasSynchronized) { + assetSynchronized = true; + } + } catch (final Exception e) { + logger.error("Failed to synchronize asset [id={},name={}] for connector [{}]", + coordinatorAsset.getId(), coordinatorAsset.getName(), connectorId, e); + } + } + return assetSynchronized; + } + + private boolean synchronize(final AssetsRestApiClient assetsRestApiClient, final String connectorId, final AssetDTO coordinatorAsset, final Asset matchingAsset) { + final String assetId = coordinatorAsset.getId(); + final String assetName = coordinatorAsset.getName(); + if (matchingAsset == null || !matchingAsset.getFile().exists()) { + logger.info("Synchronizing missing asset [id={},name={}] for connector [{}]", assetId, assetName, connectorId); + synchronizeConnectorAssetWithRetry(assetsRestApiClient, connectorId, coordinatorAsset); + return true; + } else { + final String coordinatorAssetDigest = coordinatorAsset.getDigest(); + final String matchingAssetDigest = matchingAsset.getDigest().orElse(null); + if (!coordinatorAssetDigest.equals(matchingAssetDigest)) { + logger.info("Synchronizing asset [id={},name={}] with updated digest [{}] for connector [{}]", + assetId, assetName, coordinatorAssetDigest, connectorId); + synchronizeConnectorAssetWithRetry(assetsRestApiClient, connectorId, coordinatorAsset); + return true; + } else { + logger.info("Coordinator asset [id={},name={}] found for connector [{}]: retrieval not required", assetId, assetName, connectorId); + return false; + } + } + } + + private AssetsEntity listConnectorAssetsWithRetry(final AssetsRestApiClient assetsRestApiClient, final String connectorId) { + final Instant expirationTime = Instant.ofEpochMilli(System.currentTimeMillis() + LIST_ASSETS_RETRY_DURATION.toMillis()); + while (System.currentTimeMillis() < expirationTime.toEpochMilli()) { + final AssetsEntity assetsEntity = listConnectorAssets(assetsRestApiClient, connectorId); + if (assetsEntity != null) { + return assetsEntity; + } + logger.info("Unable to list assets from cluster coordinator for connector [{}]: retrying until [{}]", connectorId, expirationTime); + sleep(Duration.ofSeconds(5)); + } + return null; + } + + private AssetsEntity listConnectorAssets(final AssetsRestApiClient assetsRestApiClient, final String connectorId) { + try { + return assetsRestApiClient.getConnectorAssets(connectorId); + } catch (final NiFiRestApiRetryableException e) { + final Throwable rootCause = ExceptionUtils.getRootCause(e); + logger.warn("{}, root cause [{}]: retrying", e.getMessage(), rootCause.getMessage()); + return null; + } + } + + private void synchronizeConnectorAssetWithRetry(final AssetsRestApiClient assetsRestApiClient, final String connectorId, final AssetDTO coordinatorAsset) { + final Instant expirationTime = Instant.ofEpochMilli(System.currentTimeMillis() + SYNC_ASSET_RETRY_DURATION.toMillis()); + while (System.currentTimeMillis() < expirationTime.toEpochMilli()) { + final Asset syncedAsset = synchronizeConnectorAsset(assetsRestApiClient, connectorId, coordinatorAsset); + if (syncedAsset != null) { + return; + } + logger.info("Unable to synchronize asset [id={},name={}] for connector [{}]: retrying until [{}]", + coordinatorAsset.getId(), coordinatorAsset.getName(), connectorId, expirationTime); + sleep(Duration.ofSeconds(5)); + } + } + + private Asset synchronizeConnectorAsset(final AssetsRestApiClient assetsRestApiClient, final String connectorId, final AssetDTO coordinatorAsset) { + final String assetId = coordinatorAsset.getId(); + final String assetName = coordinatorAsset.getName(); + if (Boolean.TRUE == coordinatorAsset.getMissingContent()) { + return assetManager.createMissingAsset(connectorId, assetName); + } else { + try (final InputStream assetInputStream = assetsRestApiClient.getConnectorAssetContent(connectorId, assetId)) { + return assetManager.saveAsset(connectorId, assetId, assetName, assetInputStream); + } catch (final NiFiRestApiRetryableException e) { + final Throwable rootCause = ExceptionUtils.getRootCause(e); + logger.warn("{}, root cause [{}]: retrying", e.getMessage(), rootCause.getMessage()); + return null; + } catch (final IOException e) { + final Throwable rootCause = ExceptionUtils.getRootCause(e); + logger.warn("Asset Manager failed to create asset [id={},name={}], root cause [{}]: retrying", assetId, assetId, rootCause.getMessage()); + return null; + } + } + } + + private NodeIdentifier getElectedActiveCoordinatorNode() { + final Instant expirationTime = Instant.ofEpochMilli(System.currentTimeMillis() + CLUSTER_COORDINATOR_RETRY_DURATION.toMillis()); + while (System.currentTimeMillis() < expirationTime.toEpochMilli()) { + final NodeIdentifier coordinatorNodeId = clusterCoordinator.getElectedActiveCoordinatorNode(); + if (coordinatorNodeId != null) { + return coordinatorNodeId; + } + logger.info("Node identifier for the active cluster coordinator is not known yet: retrying until [{}]", expirationTime); + sleep(Duration.ofSeconds(2)); + } + return null; + } + + private void sleep(final Duration duration) { + try { + Thread.sleep(duration); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } +} + + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/ConnectorDetails.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/ConnectorDetails.java new file mode 100644 index 000000000000..4ffafd865238 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/ConnectorDetails.java @@ -0,0 +1,45 @@ +/* + * 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.components.connector; + +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.logging.ComponentLog; + +public class ConnectorDetails { + private final Connector connector; + private final ComponentLog componentLog; + private final BundleCoordinate bundleCoordinate; + + public ConnectorDetails(final Connector connector, final BundleCoordinate bundleCoordinate, final ComponentLog logger) { + this.connector = connector; + this.bundleCoordinate = bundleCoordinate; + this.componentLog = logger; + } + + public Connector getConnector() { + return connector; + } + + public ComponentLog getComponentLog() { + return componentLog; + } + + public BundleCoordinate getBundleCoordinate() { + return bundleCoordinate; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/ConnectorParameterLookup.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/ConnectorParameterLookup.java new file mode 100644 index 000000000000..ada91a1d418f --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/ConnectorParameterLookup.java @@ -0,0 +1,193 @@ +/* + * 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.components.connector; + +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.connector.components.ParameterValue; +import org.apache.nifi.flow.VersionedAsset; +import org.apache.nifi.flow.VersionedParameter; +import org.apache.nifi.flow.VersionedParameterContext; +import org.apache.nifi.parameter.Parameter; +import org.apache.nifi.parameter.ParameterLookup; + +import java.util.ArrayList; +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.Optional; +import java.util.Set; + +public class ConnectorParameterLookup implements ParameterLookup { + private final Map parameters; + private final List parameterValues; + private final AssetManager assetManager; + + // TODO: Refactor how parameters work + // It is possible that a VersionedProcessGroup has a child where different parameter contexts are used or are used in a different inheritance order. + public ConnectorParameterLookup(final Collection parameterContexts, final AssetManager assetManager) { + this.assetManager = assetManager; + + parameterValues = Collections.unmodifiableList(createParameterValues(parameterContexts)); + parameters = new HashMap<>(); + for (final ParameterValue parameterValue : parameterValues) { + parameters.put(parameterValue.getName(), createParameter(parameterValue)); + } + } + + private Parameter createParameter(final ParameterValue parameterValue) { + return new Parameter.Builder() + .name(parameterValue.getName()) + .value(parameterValue.getValue()) + .sensitive(parameterValue.isSensitive()) + .referencedAssets(parameterValue.getAssets()) + .build(); + } + + public VersionedParameterContext createVersionedParameterContext(final String name) { + final VersionedParameterContext context = new VersionedParameterContext(); + context.setName(name); + + final Set versionedParameters = new HashSet<>(); + for (final ParameterValue parameterValue : parameterValues) { + final VersionedParameter versionedParameter = new VersionedParameter(); + versionedParameter.setName(parameterValue.getName()); + versionedParameter.setValue(parameterValue.getValue()); + versionedParameter.setSensitive(parameterValue.isSensitive()); + versionedParameters.add(versionedParameter); + } + + context.setParameters(versionedParameters); + return context; + } + + @Override + public Optional getParameter(final String parameterName) { + return Optional.ofNullable(parameters.get(parameterName)); + } + + @Override + public boolean isEmpty() { + return parameters.isEmpty(); + } + + @Override + public long getVersion() { + return 0; + } + + public List getParameterValues() { + return parameterValues; + } + + /** + * Converts a {@code List} found in a VersionedExternalFlow to a + * {@code List} that can be used to update a ParameterContext from a Connector, + * respecting parameter context inheritance and precedence. + * + * @param parameterContexts the list of parameter contexts from a VersionedExternalFlow + * @return the list of ParameterValues + */ + public List createParameterValues(final Collection parameterContexts) { + final List parameterValues = new ArrayList<>(); + + if (parameterContexts == null || parameterContexts.isEmpty()) { + return parameterValues; + } + + // Create a map for easy lookup of parameter contexts by name + final Map contextMap = new HashMap<>(); + for (final VersionedParameterContext context : parameterContexts) { + contextMap.put(context.getName(), context); + } + + // Process each parameter context, including inherited contexts + final Set processedContexts = new HashSet<>(); + for (final VersionedParameterContext context : parameterContexts) { + collectParameterValues(context, contextMap, processedContexts, parameterValues); + } + + return parameterValues; + } + + private void collectParameterValues(final VersionedParameterContext context, final Map contextMap, + final Set processedContexts, final List parameterValues) { + + if (context == null || processedContexts.contains(context.getName())) { + return; + } + + processedContexts.add(context.getName()); + + // Create a map to track existing parameters for efficient lookup + final Map existingParametersByName = new HashMap<>(); + for (final ParameterValue existing : parameterValues) { + existingParametersByName.put(existing.getName(), existing); + } + + // First, process inherited parameter contexts in reverse order (lowest precedence first) + // This ensures that the first inherited context (highest precedence) will override later ones + if (context.getInheritedParameterContexts() != null && !context.getInheritedParameterContexts().isEmpty()) { + final List inheritedContextNames = context.getInheritedParameterContexts(); + // Process in reverse order so that the first (highest precedence) inherited context processes last + for (int i = inheritedContextNames.size() - 1; i >= 0; i--) { + final String inheritedContextName = inheritedContextNames.get(i); + final VersionedParameterContext inheritedContext = contextMap.get(inheritedContextName); + if (inheritedContext != null) { + collectParameterValues(inheritedContext, contextMap, processedContexts, parameterValues); + } + } + } + + // Then, process this context's own parameters (they have the highest precedence and override all inherited ones) + if (context.getParameters() != null) { + // Rebuild the existing parameters map since inherited contexts may have added parameters + existingParametersByName.clear(); + for (final ParameterValue existing : parameterValues) { + existingParametersByName.put(existing.getName(), existing); + } + + for (final VersionedParameter versionedParameter : context.getParameters()) { + final String parameterName = versionedParameter.getName(); + + // Remove existing parameter if present, then add the new one (current context overrides) + if (existingParametersByName.containsKey(parameterName)) { + parameterValues.removeIf(param -> param.getName().equals(parameterName)); + } + + final ParameterValue.Builder builder = new ParameterValue.Builder() + .name(parameterName) + .value(versionedParameter.getValue()) + .sensitive(versionedParameter.isSensitive()); + + if (assetManager != null && versionedParameter.getReferencedAssets() != null) { + for (final VersionedAsset versionedAsset : versionedParameter.getReferencedAssets()) { + assetManager.getAsset(versionedAsset.getIdentifier()).ifPresent(builder::addReferencedAsset); + } + } + + final ParameterValue parameterValue = builder.build(); + + parameterValues.add(parameterValue); + } + } + } + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/ConnectorValidationContextBridge.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/ConnectorValidationContextBridge.java new file mode 100644 index 000000000000..6f867925be7c --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/ConnectorValidationContextBridge.java @@ -0,0 +1,146 @@ +/* + * 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.components.connector; + +import org.apache.nifi.attribute.expression.language.StandardPropertyValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.documentation.init.EmptyControllerServiceLookup; +import org.apache.nifi.expression.ExpressionLanguageCompiler; +import org.apache.nifi.parameter.ExpressionLanguageAgnosticParameterParser; +import org.apache.nifi.parameter.Parameter; +import org.apache.nifi.parameter.ParameterLookup; +import org.apache.nifi.parameter.ParameterParser; +import org.apache.nifi.parameter.ParameterTokenList; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +public class ConnectorValidationContextBridge implements ValidationContext { + private final Map rawValues; + private final ParameterLookup parameterLookup; + + public ConnectorValidationContextBridge(final Map rawValues, final ParameterLookup parameterLookup) { + this.rawValues = rawValues; + this.parameterLookup = parameterLookup; + } + + @Override + public ControllerServiceLookup getControllerServiceLookup() { + return new EmptyControllerServiceLookup(); + } + + @Override + public ValidationContext getControllerServiceValidationContext(final ControllerService controllerService) { + return null; + } + + @Override + public ExpressionLanguageCompiler newExpressionLanguageCompiler() { + return null; + } + + @Override + public PropertyValue newPropertyValue(final String rawValue) { + return new StandardPropertyValue(rawValue, getControllerServiceLookup(), parameterLookup); + } + + @Override + public Map getProperties() { + final Map properties = new HashMap<>(); + for (final Map.Entry entry : rawValues.entrySet()) { + final PropertyDescriptor descriptor = new PropertyDescriptor.Builder() + .name(entry.getKey()) + .build(); + properties.put(descriptor, entry.getValue()); + } + + return properties; + } + + @Override + public String getAnnotationData() { + return null; + } + + @Override + public boolean isValidationRequired(final ControllerService controllerService) { + return false; + } + + @Override + public boolean isExpressionLanguagePresent(final String value) { + return false; + } + + @Override + public boolean isExpressionLanguageSupported(final String value) { + return false; + } + + @Override + public String getProcessGroupIdentifier() { + return null; + } + + @Override + public Collection getReferencedParameters(final String propertyName) { + return List.of(); + } + + @Override + public boolean isParameterDefined(final String parameterName) { + return parameterLookup.getParameter(parameterName).isPresent(); + } + + @Override + public boolean isParameterSet(final String parameterName) { + final Optional optionalParameter = parameterLookup.getParameter(parameterName); + if (optionalParameter.isEmpty()) { + return false; + } + + return optionalParameter.get().getValue() != null; + } + + @Override + public PropertyValue getProperty(final PropertyDescriptor propertyDescriptor) { + final String rawValue = rawValues.get(propertyDescriptor.getName()); + final String effectiveValue = rawValue == null ? propertyDescriptor.getDefaultValue() : rawValue; + return newPropertyValue(effectiveValue); + } + + @Override + public Map getAllProperties() { + return Collections.unmodifiableMap(rawValues); + } + + @Override + public String evaluateParameters(final String value) { + final ParameterParser parameterParser = new ExpressionLanguageAgnosticParameterParser(); + final ParameterTokenList parameterTokenList = parameterParser.parseTokens(value); + return parameterTokenList.substitute(parameterLookup); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/DescribedValueProvider.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/DescribedValueProvider.java new file mode 100644 index 000000000000..7b5004fac7d2 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/DescribedValueProvider.java @@ -0,0 +1,28 @@ +/* + * 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.components.connector; + +import org.apache.nifi.components.DescribedValue; + +import java.util.List; + +public interface DescribedValueProvider { + + List fetchAllowableValues(String stepName, String propertyName); + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/GhostConnector.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/GhostConnector.java new file mode 100644 index 000000000000..0508821e9d1f --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/GhostConnector.java @@ -0,0 +1,138 @@ +/* + * 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.components.connector; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.ConfigVerificationResult.Outcome; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.flow.VersionedExternalFlow; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +public class GhostConnector implements Connector { + private final String identifier; + private final String canonicalClassName; + private final List validationResults; + private final List configVerificationResults; + private final Exception causeOfGhost; + + public GhostConnector(final String identifier, final String canonicalClassName, final Exception causeOfGhost) { + this.identifier = identifier; + this.canonicalClassName = canonicalClassName; + this.causeOfGhost = causeOfGhost; + + validationResults = List.of(new ValidationResult.Builder() + .subject("Missing Connector") + .valid(false) + .explanation("Could not create Connector of type " + canonicalClassName) + .build()); + + configVerificationResults = List.of(new ConfigVerificationResult.Builder() + .verificationStepName("Create Connector") + .outcome(Outcome.FAILED) + .explanation("Could not create Connector of type " + canonicalClassName) + .build()); + } + + public Exception getCauseOfGhost() { + return causeOfGhost; + } + + @Override + public void initialize(final ConnectorInitializationContext initContext) { + } + + @Override + public VersionedExternalFlow getInitialFlow() { + return null; + } + + @Override + public void start(final FlowContext activeContext) throws FlowUpdateException { + } + + @Override + public void stop(final FlowContext activeContext) throws FlowUpdateException { + } + + @Override + public List validate(final FlowContext flowContext, final ConnectorValidationContext connectorValidationContext) { + return validationResults; + } + + @Override + public List validateConfigurationStep(final ConfigurationStep configurationStep, final ConnectorConfigurationContext connectorConfigurationContext, + final ConnectorValidationContext connectorValidationContext) { + return validationResults; + } + + @Override + public List getConfigurationSteps() { + return List.of(); + } + + @Override + public void onConfigurationStepConfigured(final String stepName, final FlowContext workingContext) { + } + + @Override + public void prepareForUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + } + + @Override + public void abortUpdate(final FlowContext workingContext, final Throwable throwable) { + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext workingContext) { + return configVerificationResults; + } + + @Override + public List verify(final FlowContext flowContext) { + return configVerificationResults; + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName, final FlowContext workingContext, final String filter) { + return List.of(); + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName, final FlowContext workingContext) { + return List.of(); + } + + @Override + public CompletableFuture drainFlowFiles(final FlowContext flowContext) { + return CompletableFuture.completedFuture(null); + } + + @Override + public String toString() { + return "GhostConnector[id=" + identifier + ", type=" + canonicalClassName + "]"; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandaloneConnectorRequestReplicator.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandaloneConnectorRequestReplicator.java new file mode 100644 index 000000000000..5880530745c6 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandaloneConnectorRequestReplicator.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.components.connector; + +import org.apache.nifi.controller.flow.FlowManager; + +public class StandaloneConnectorRequestReplicator implements ConnectorRequestReplicator { + + private FlowManager flowManager; + + @Override + public ConnectorState getState(final String connectorId) { + final ConnectorNode connectorNode = flowManager.getConnector(connectorId); + if (connectorNode == null) { + throw new IllegalArgumentException("No connector found with id: " + connectorId); + } + + return connectorNode.getCurrentState(); + } + + @Override + public void setFlowManager(final FlowManager flowManager) { + this.flowManager = flowManager; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardComponentBundleLookup.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardComponentBundleLookup.java new file mode 100644 index 000000000000..cc91698827f0 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardComponentBundleLookup.java @@ -0,0 +1,172 @@ +/* + * 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.components.connector; + +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.nar.ExtensionManager; + +import java.util.List; +import java.util.Optional; + +public class StandardComponentBundleLookup implements ComponentBundleLookup { + private final ExtensionManager extensionManager; + + public StandardComponentBundleLookup(final ExtensionManager extensionManager) { + this.extensionManager = extensionManager; + } + + @Override + public List getAvailableBundles(final String componentType) { + final List bundles = extensionManager.getBundles(componentType); + return bundles.stream() + .map(this::convertBundle) + .toList(); + } + + @Override + public Optional getLatestBundle(final String componentType) { + final List availableBundles = getAvailableBundles(componentType); + if (availableBundles.isEmpty()) { + return Optional.empty(); + } + + Bundle newest = null; + for (final Bundle bundle : availableBundles) { + if (newest == null || compareVersion(bundle.getVersion(), newest.getVersion()) > 0) { + newest = bundle; + } + } + + return Optional.ofNullable(newest); + } + + private int compareVersion(final String v1, final String v2) { + final String baseVersion1 = getBaseVersion(v1); + final String baseVersion2 = getBaseVersion(v2); + + final String[] parts1 = baseVersion1.split("\\."); + final String[] parts2 = baseVersion2.split("\\."); + + final int length = Math.max(parts1.length, parts2.length); + for (int i = 0; i < length; i++) { + final String part1Str = i < parts1.length ? parts1[i] : "0"; + final String part2Str = i < parts2.length ? parts2[i] : "0"; + + final int comparison = compareVersionPart(part1Str, part2Str); + if (comparison != 0) { + return comparison; + } + } + + // Base versions are equal; compare qualifiers + final String qualifier1 = getQualifier(v1); + final String qualifier2 = getQualifier(v2); + return compareQualifiers(qualifier1, qualifier2); + } + + private int compareQualifiers(final String qualifier1, final String qualifier2) { + final int rank1 = getQualifierRank(qualifier1); + final int rank2 = getQualifierRank(qualifier2); + + if (rank1 != rank2) { + return Integer.compare(rank1, rank2); + } + + // Same qualifier type; compare numeric suffixes (e.g., RC2 > RC1, M4 > M3) + final int num1 = getQualifierNumber(qualifier1); + final int num2 = getQualifierNumber(qualifier2); + return Integer.compare(num1, num2); + } + + private int getQualifierRank(final String qualifier) { + if (qualifier == null || qualifier.isEmpty()) { + return 4; + } else if (qualifier.startsWith("RC")) { + return 3; + } else if (qualifier.startsWith("M")) { + return 2; + } else if (qualifier.equals("SNAPSHOT")) { + return 0; + } else { + return 1; + } + } + + private int getQualifierNumber(final String qualifier) { + if (qualifier == null || qualifier.isEmpty()) { + return 0; + } + + final StringBuilder digits = new StringBuilder(); + for (int i = 0; i < qualifier.length(); i++) { + final char c = qualifier.charAt(i); + if (Character.isDigit(c)) { + digits.append(c); + } + } + + if (digits.isEmpty()) { + return 0; + } + + try { + return Integer.parseInt(digits.toString()); + } catch (final NumberFormatException e) { + return 0; + } + } + + private String getQualifier(final String version) { + final int qualifierIndex = version.indexOf('-'); + return qualifierIndex > 0 ? version.substring(qualifierIndex + 1) : null; + } + + private int compareVersionPart(final String part1, final String part2) { + final Integer num1 = parseVersionPart(part1); + final Integer num2 = parseVersionPart(part2); + + if (num1 != null && num2 != null) { + return Integer.compare(num1, num2); + } else if (num1 != null) { + return 1; + } else if (num2 != null) { + return -1; + } else { + return part1.compareTo(part2); + } + } + + private Integer parseVersionPart(final String part) { + try { + return Integer.parseInt(part); + } catch (final NumberFormatException e) { + return null; + } + } + + private String getBaseVersion(final String version) { + final int qualifierIndex = version.indexOf('-'); + return qualifierIndex > 0 ? version.substring(0, qualifierIndex) : version; + } + + private Bundle convertBundle(final org.apache.nifi.bundle.Bundle bundle) { + final BundleCoordinate coordinate = bundle.getBundleDetails().getCoordinate(); + return new Bundle(coordinate.getGroup(), coordinate.getId(), coordinate.getVersion()); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorAction.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorAction.java new file mode 100644 index 000000000000..3989bbe02c3e --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorAction.java @@ -0,0 +1,81 @@ +/* + * 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.components.connector; + +import java.util.Objects; + +public class StandardConnectorAction implements ConnectorAction { + + private final String name; + private final String description; + private final boolean allowed; + private final String reasonNotAllowed; + + public StandardConnectorAction(final String name, final String description, final boolean allowed, final String reasonNotAllowed) { + this.name = Objects.requireNonNull(name, "name is required"); + this.description = Objects.requireNonNull(description, "description is required"); + this.allowed = allowed; + this.reasonNotAllowed = reasonNotAllowed; + } + + @Override + public String getName() { + return name; + } + + @Override + public String getDescription() { + return description; + } + + @Override + public boolean isAllowed() { + return allowed; + } + + @Override + public String getReasonNotAllowed() { + return reasonNotAllowed; + } + + @Override + public boolean equals(final Object other) { + if (other == null || getClass() != other.getClass()) { + return false; + } + final StandardConnectorAction that = (StandardConnectorAction) other; + return allowed == that.allowed + && Objects.equals(name, that.name) + && Objects.equals(description, that.description) + && Objects.equals(reasonNotAllowed, that.reasonNotAllowed); + } + + @Override + public int hashCode() { + return Objects.hash(name, description, allowed, reasonNotAllowed); + } + + @Override + public String toString() { + if (reasonNotAllowed == null) { + return "StandardConnectorAction[name=" + name + ", allowed=" + allowed + "]"; + } + return "StandardConnectorAction[name=" + name + ", allowed=" + allowed + ", reason=" + reasonNotAllowed + "]"; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorConfigurationContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorConfigurationContext.java new file mode 100644 index 000000000000..c37a246933ea --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorConfigurationContext.java @@ -0,0 +1,323 @@ +/* + * 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.components.connector; + +import org.apache.nifi.asset.Asset; +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.connector.secrets.SecretProvider; +import org.apache.nifi.components.connector.secrets.SecretsManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +public class StandardConnectorConfigurationContext implements MutableConnectorConfigurationContext, Cloneable { + private static final Logger logger = LoggerFactory.getLogger(StandardConnectorConfigurationContext.class); + + private final ReadWriteLock rwLock = new ReentrantReadWriteLock(); + private final Lock readLock = rwLock.readLock(); + private final Lock writeLock = rwLock.writeLock(); + private final AssetManager assetManager; + private final SecretsManager secretsManager; + + final Map propertyConfigurations = new HashMap<>(); + final Map resolvedPropertyConfigurations = new HashMap<>(); + + public StandardConnectorConfigurationContext(final AssetManager assetManager, final SecretsManager secretsManager) { + this.assetManager = assetManager; + this.secretsManager = secretsManager; + } + + @Override + public ConnectorPropertyValue getProperty(final String stepName, final String propertyName) { + return getProperty(stepName, propertyName, null); + } + + private ConnectorPropertyValue getProperty(final String stepName, final String propertyName, final String defaultValue) { + readLock.lock(); + try { + final StepConfiguration resolvedConfig = resolvedPropertyConfigurations.get(stepName); + if (resolvedConfig == null) { + return new StandardConnectorPropertyValue(defaultValue); + } + + final ConnectorValueReference valueReference = resolvedConfig.getPropertyValue(propertyName); + if (valueReference != null) { + // The resolvedPropertyConfigurations contains only StringLiteralValue references. + return new StandardConnectorPropertyValue(((StringLiteralValue) valueReference).getValue()); + } + + // Property not found + return new StandardConnectorPropertyValue(defaultValue); + } finally { + readLock.unlock(); + } + } + + @Override + public ConnectorPropertyValue getProperty(final ConfigurationStep configurationStep, final ConnectorPropertyDescriptor connectorPropertyDescriptor) { + return getProperty(configurationStep.getName(), connectorPropertyDescriptor.getName(), connectorPropertyDescriptor.getDefaultValue()); + } + + @Override + public Set getPropertyNames(final String configurationStepName) { + readLock.lock(); + try { + final StepConfiguration config = propertyConfigurations.get(configurationStepName); + if (config == null) { + return Collections.emptySet(); + } + + return new HashSet<>(config.getPropertyValues().keySet()); + } finally { + readLock.unlock(); + } + } + + @Override + public Set getPropertyNames(final ConfigurationStep configurationStep) { + return getPropertyNames(configurationStep.getName()); + } + + @Override + public StepConfigurationContext scopedToStep(final String stepName) { + return new StandardStepConfigurationContext(stepName, this); + } + + @Override + public StepConfigurationContext scopedToStep(final ConfigurationStep configurationStep) { + return scopedToStep(configurationStep.getName()); + } + + @Override + public StandardConnectorConfigurationContext createWithOverrides(final String stepName, final Map propertyOverrides) { + final StandardConnectorConfigurationContext created = new StandardConnectorConfigurationContext(assetManager, secretsManager); + readLock.lock(); + try { + for (final Map.Entry stepEntry : propertyConfigurations.entrySet()) { + final String existingStepName = stepEntry.getKey(); + final StepConfiguration existingConfig = stepEntry.getValue(); + + if (!existingStepName.equals(stepName)) { + created.setProperties(existingStepName, new StepConfiguration(new HashMap<>(existingConfig.getPropertyValues()))); + continue; + } + + final Map mergedProperties = new HashMap<>(existingConfig.getPropertyValues()); + for (final Map.Entry override : propertyOverrides.entrySet()) { + final String propertyValue = override.getValue(); + if (propertyValue == null) { + mergedProperties.remove(override.getKey()); + continue; + } + + mergedProperties.put(override.getKey(), new StringLiteralValue(propertyValue)); + } + created.setProperties(stepName, new StepConfiguration(mergedProperties)); + } + + return created; + } finally { + readLock.unlock(); + } + } + + + @Override + public ConfigurationUpdateResult setProperties(final String stepName, final StepConfiguration configuration) { + writeLock.lock(); + try { + final StepConfiguration existingConfig = propertyConfigurations.get(stepName); + final Map existingProperties = existingConfig != null ? existingConfig.getPropertyValues() : new HashMap<>(); + final Map mergedProperties = new HashMap<>(existingProperties); + mergedProperties.putAll(configuration.getPropertyValues()); + + final StepConfiguration resolvedConfig = resolvePropertyValues(mergedProperties); + + final StepConfiguration updatedStepConfig = new StepConfiguration(new HashMap<>(mergedProperties)); + final StepConfiguration existingStepConfig = this.propertyConfigurations.put(stepName, updatedStepConfig); + final StepConfiguration existingResolvedStepConfig = this.resolvedPropertyConfigurations.put(stepName, resolvedConfig); + + if (Objects.equals(existingStepConfig, updatedStepConfig) && Objects.equals(existingResolvedStepConfig, resolvedConfig)) { + return ConfigurationUpdateResult.NO_CHANGES; + } + + return ConfigurationUpdateResult.CHANGES_MADE; + } finally { + writeLock.unlock(); + } + } + + private StepConfiguration resolvePropertyValues(final Map propertyValues) { + final Map resolvedProperties = new HashMap<>(); + for (final Map.Entry entry : propertyValues.entrySet()) { + final ConnectorValueReference resolved = resolve(entry.getValue()); + resolvedProperties.put(entry.getKey(), resolved); + } + return new StepConfiguration(resolvedProperties); + } + + private ConnectorValueReference resolve(final ConnectorValueReference reference) { + if (reference == null) { + return null; + } + + try { + return switch (reference) { + case StringLiteralValue stringLiteral -> stringLiteral; + case AssetReference assetReference -> resolveAssetReferences(assetReference); + case SecretReference secretReference -> new StringLiteralValue(getSecretValue(secretReference)); + }; + } catch (final IOException ioe) { + throw new UncheckedIOException("Unable to obtain Secrets from Secret Manager", ioe); + } + } + + private StringLiteralValue resolveAssetReferences(final AssetReference assetReference) { + final Set resolvedAssetValues = new HashSet<>(); + for (final String assetId : assetReference.getAssetIdentifiers()) { + assetManager.getAsset(assetId) + .map(Asset::getFile) + .map(File::getAbsolutePath) + .ifPresent(resolvedAssetValues::add); + } + + logger.debug("Resolved {} to {}", assetReference, resolvedAssetValues); + return new StringLiteralValue(String.join(",", resolvedAssetValues)); + } + + private String getSecretValue(final SecretReference secretReference) throws IOException { + final SecretProvider provider = getSecretProvider(secretReference); + if (provider == null) { + return null; + } + + final List secrets = provider.getSecrets(List.of(secretReference.getFullyQualifiedName())); + return secrets.isEmpty() ? null : secrets.getFirst().getValue(); + } + + private SecretProvider getSecretProvider(final SecretReference secretReference) { + final Set providers = secretsManager.getSecretProviders(); + for (final SecretProvider provider : providers) { + if (Objects.equals(provider.getProviderId(), secretReference.getProviderId())) { + return provider; + } + } + + for (final SecretProvider provider : providers) { + if (Objects.equals(provider.getProviderName(), secretReference.getProviderName())) { + return provider; + } + } + + // Try to find by Fully Qualified Name prefix + final String fqn = secretReference.getFullyQualifiedName(); + if (fqn != null) { + for (final SecretProvider provider : providers) { + if (fqn.startsWith(provider.getProviderName() + ".")) { + return provider; + } + } + } + + return null; + } + + @Override + public ConfigurationUpdateResult replaceProperties(final String stepName, final StepConfiguration configuration) { + writeLock.lock(); + try { + final StepConfiguration resolvedConfig = resolvePropertyValues(configuration.getPropertyValues()); + + final StepConfiguration updatedStepConfig = new StepConfiguration(new HashMap<>(configuration.getPropertyValues())); + final StepConfiguration existingStepConfig = this.propertyConfigurations.put(stepName, updatedStepConfig); + final StepConfiguration existingResolvedStepConfig = this.resolvedPropertyConfigurations.put(stepName, resolvedConfig); + + if (Objects.equals(existingStepConfig, updatedStepConfig) && Objects.equals(existingResolvedStepConfig, resolvedConfig)) { + return ConfigurationUpdateResult.NO_CHANGES; + } + + return ConfigurationUpdateResult.CHANGES_MADE; + } finally { + writeLock.unlock(); + } + } + + @Override + public void resolvePropertyValues() { + writeLock.lock(); + try { + for (final Map.Entry entry : propertyConfigurations.entrySet()) { + final String stepName = entry.getKey(); + final StepConfiguration stepConfig = entry.getValue(); + final Map stepProperties = stepConfig.getPropertyValues(); + + final StepConfiguration resolvedConfig = resolvePropertyValues(stepProperties); + this.resolvedPropertyConfigurations.put(stepName, resolvedConfig); + } + } finally { + writeLock.unlock(); + } + } + + @Override + public ConnectorConfiguration toConnectorConfiguration() { + readLock.lock(); + try { + final Set stepConfigs = new HashSet<>(); + for (final Map.Entry entry : propertyConfigurations.entrySet()) { + final String stepName = entry.getKey(); + final StepConfiguration config = entry.getValue(); + final StepConfiguration configCopy = new StepConfiguration(new HashMap<>(config.getPropertyValues())); + + stepConfigs.add(new NamedStepConfiguration(stepName, configCopy)); + } + + return new ConnectorConfiguration(stepConfigs); + } finally { + readLock.unlock(); + } + } + + @Override + public MutableConnectorConfigurationContext clone() { + readLock.lock(); + try { + final StandardConnectorConfigurationContext cloned = new StandardConnectorConfigurationContext(assetManager, secretsManager); + for (final Map.Entry entry : this.propertyConfigurations.entrySet()) { + cloned.setProperties(entry.getKey(), new StepConfiguration(new HashMap<>(entry.getValue().getPropertyValues()))); + } + return cloned; + } finally { + readLock.unlock(); + } + } + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorConfigurationProviderInitializationContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorConfigurationProviderInitializationContext.java new file mode 100644 index 000000000000..99f3bee1207e --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorConfigurationProviderInitializationContext.java @@ -0,0 +1,44 @@ +/* + * 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.components.connector; + +import org.apache.nifi.asset.AssetManager; + +import java.util.Map; +import java.util.Objects; + +public class StandardConnectorConfigurationProviderInitializationContext implements ConnectorConfigurationProviderInitializationContext { + + private final Map properties; + private final AssetManager assetManager; + + public StandardConnectorConfigurationProviderInitializationContext(final Map properties, final AssetManager assetManager) { + this.properties = Map.copyOf(Objects.requireNonNull(properties, "Properties is required")); + this.assetManager = Objects.requireNonNull(assetManager, "AssetManager is required"); + } + + @Override + public Map getProperties() { + return properties; + } + + @Override + public AssetManager getAssetManager() { + return assetManager; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorInitializationContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorInitializationContext.java new file mode 100644 index 000000000000..002f2a8aa2f8 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorInitializationContext.java @@ -0,0 +1,205 @@ +/* + * 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.components.connector; + +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.secrets.SecretsManager; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; +import org.apache.nifi.logging.ComponentLog; + +import java.util.List; +import java.util.function.Consumer; + +public class StandardConnectorInitializationContext implements FrameworkConnectorInitializationContext { + private final String identifier; + private final String name; + private final ComponentLog componentLog; + private final SecretsManager secretsManager; + private final AssetManager assetManager; + private final ComponentBundleLookup componentBundleLookup; + + + protected StandardConnectorInitializationContext(final Builder builder) { + this.identifier = builder.identifier; + this.name = builder.name; + this.componentLog = builder.componentLog; + this.secretsManager = builder.secretsManager; + this.assetManager = builder.assetManager; + this.componentBundleLookup = builder.componentBundleLookup; + } + + @Override + public String getIdentifier() { + return identifier; + } + + @Override + public String getName() { + return name; + } + + @Override + public ComponentLog getLogger() { + return componentLog; + } + + @Override + public ComponentBundleLookup getComponentBundleLookup() { + return componentBundleLookup; + } + + @Override + public SecretsManager getSecretsManager() { + return secretsManager; + } + + @Override + public AssetManager getAssetManager() { + return assetManager; + } + + @Override + public void updateFlow(final FlowContext flowContext, final VersionedExternalFlow versionedExternalFlow, + final BundleCompatibility bundleCompatability) throws FlowUpdateException { + if (!(flowContext instanceof final FrameworkFlowContext frameworkFlowContext)) { + throw new IllegalArgumentException("FlowContext is not an instance provided by the framework"); + } + + resolveBundles(versionedExternalFlow.getFlowContents(), bundleCompatability); + frameworkFlowContext.updateFlow(versionedExternalFlow, assetManager); + } + + protected void resolveBundles(final VersionedProcessGroup group, final BundleCompatibility bundleCompatability) { + if (bundleCompatability == BundleCompatibility.REQUIRE_EXACT_BUNDLE) { + return; + } + + if (group.getProcessors() != null) { + for (final VersionedProcessor processor : group.getProcessors()) { + resolveBundle(processor.getType(), processor.getBundle(), bundleCompatability, processor::setBundle); + } + } + + if (group.getControllerServices() != null) { + for (final VersionedControllerService service : group.getControllerServices()) { + resolveBundle(service.getType(), service.getBundle(), bundleCompatability, service::setBundle); + } + } + + if (group.getProcessGroups() != null) { + for (final VersionedProcessGroup childGroup : group.getProcessGroups()) { + resolveBundles(childGroup, bundleCompatability); + } + } + } + + private void resolveBundle(final String componentType, final Bundle currentBundle, + final BundleCompatibility bundleCompatability, final Consumer bundleSetter) { + final List availableBundles = componentBundleLookup.getAvailableBundles(componentType); + + if (availableBundles.contains(currentBundle)) { + return; + } + + if (availableBundles.isEmpty()) { + return; + } + + switch (bundleCompatability) { + case RESOLVE_BUNDLE: + if (availableBundles.size() == 1) { + final Bundle resolvedBundle = availableBundles.getFirst(); + componentLog.debug("Resolved bundle for {} from {} to {}", componentType, currentBundle, resolvedBundle); + bundleSetter.accept(resolvedBundle); + } + break; + case RESOLVE_NEWEST_BUNDLE: + componentBundleLookup.getLatestBundle(componentType).ifPresent(latestBundle -> { + componentLog.debug("Resolved bundle for {} from {} to {}", componentType, currentBundle, latestBundle); + bundleSetter.accept(latestBundle); + }); + break; + default: + break; + } + } + + private void updateParameterContext(final VersionedProcessGroup group, final String parameterContextName) { + group.setParameterContextName(parameterContextName); + if (group.getProcessGroups() != null) { + for (final VersionedProcessGroup childGroup : group.getProcessGroups()) { + updateParameterContext(childGroup, parameterContextName); + } + } + } + + public static class Builder implements FrameworkConnectorInitializationContextBuilder { + private String identifier; + private String name; + private ComponentLog componentLog; + private SecretsManager secretsManager; + private AssetManager assetManager; + private ComponentBundleLookup componentBundleLookup; + + @Override + public Builder identifier(final String identifier) { + this.identifier = identifier; + return this; + } + + @Override + public Builder name(final String name) { + this.name = name; + return this; + } + + @Override + public Builder componentLog(final ComponentLog componentLog) { + this.componentLog = componentLog; + return this; + } + + @Override + public Builder secretsManager(final SecretsManager secretsManager) { + this.secretsManager = secretsManager; + return this; + } + + @Override + public Builder assetManager(final AssetManager assetManager) { + this.assetManager = assetManager; + return this; + } + + @Override + public Builder componentBundleLookup(final ComponentBundleLookup bundleLookup) { + this.componentBundleLookup = bundleLookup; + return this; + } + + @Override + public StandardConnectorInitializationContext build() { + return new StandardConnectorInitializationContext(this); + } + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorNode.java new file mode 100644 index 000000000000..d819554f3775 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorNode.java @@ -0,0 +1,1592 @@ +/* + * 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.components.connector; + +import org.apache.nifi.asset.Asset; +import org.apache.nifi.authorization.Resource; +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.resource.ResourceFactory; +import org.apache.nifi.authorization.resource.ResourceType; +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.ConfigVerificationResult.Outcome; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.components.ParameterContextFacade; +import org.apache.nifi.components.validation.DisabledServiceValidationResult; +import org.apache.nifi.components.validation.ValidationState; +import org.apache.nifi.components.validation.ValidationStatus; +import org.apache.nifi.connectable.FlowFileActivity; +import org.apache.nifi.connectable.FlowFileTransferCounts; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.flow.FlowManager; +import org.apache.nifi.controller.queue.DropFlowFileStatus; +import org.apache.nifi.controller.queue.QueueSize; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.engine.FlowEngine; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.ScheduledState; +import org.apache.nifi.flow.VersionedConfigurationStep; +import org.apache.nifi.flow.VersionedConnectorValueReference; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.nar.NarCloseable; +import org.apache.nifi.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.ArrayList; +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.Objects; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +public class StandardConnectorNode implements ConnectorNode { + private static final Logger logger = LoggerFactory.getLogger(StandardConnectorNode.class); + + private final String identifier; + private final FlowManager flowManager; + private final ExtensionManager extensionManager; + private final Authorizable parentAuthorizable; + private final ConnectorDetails connectorDetails; + private final String componentType; + private final String componentCanonicalClass; + private final BundleCoordinate bundleCoordinate; + private final ConnectorStateTransition stateTransition; + private final AtomicReference versionedComponentId = new AtomicReference<>(); + private final FlowContextFactory flowContextFactory; + private final FrameworkFlowContext activeFlowContext; + + private final AtomicReference validationState = new AtomicReference<>(new ValidationState(ValidationStatus.VALIDATING, Collections.emptyList())); + private final ConnectorValidationTrigger validationTrigger; + private final boolean extensionMissing; + private volatile boolean triggerValidation = true; + private final AtomicReference> drainFutureRef = new AtomicReference<>(); + private volatile ValidationResult unresolvedBundleValidationResult = null; + + private volatile FrameworkFlowContext workingFlowContext; + + private volatile String name; + private volatile FrameworkConnectorInitializationContext initializationContext; + + + public StandardConnectorNode(final String identifier, final FlowManager flowManager, final ExtensionManager extensionManager, + final Authorizable parentAuthorizable, final ConnectorDetails connectorDetails, final String componentType, final String componentCanonicalClass, + final MutableConnectorConfigurationContext configurationContext, + final ConnectorStateTransition stateTransition, final FlowContextFactory flowContextFactory, + final ConnectorValidationTrigger validationTrigger, final boolean extensionMissing) { + + this.identifier = identifier; + this.flowManager = flowManager; + this.extensionManager = extensionManager; + this.parentAuthorizable = parentAuthorizable; + this.connectorDetails = connectorDetails; + this.componentType = componentType; + this.componentCanonicalClass = componentCanonicalClass; + this.bundleCoordinate = connectorDetails.getBundleCoordinate(); + this.stateTransition = stateTransition; + this.flowContextFactory = flowContextFactory; + this.validationTrigger = validationTrigger; + this.extensionMissing = extensionMissing; + + this.name = connectorDetails.getConnector().getClass().getSimpleName(); + + final Bundle activeFlowBundle = new Bundle(bundleCoordinate.getGroup(), bundleCoordinate.getId(), bundleCoordinate.getVersion()); + this.activeFlowContext = flowContextFactory.createActiveFlowContext(identifier, connectorDetails.getComponentLog(), activeFlowBundle); + } + + @Override + public String getName() { + return name; + } + + @Override + public void setName(final String name) { + this.name = name; + } + + @Override + public void transitionStateForUpdating() { + final ConnectorState initialState = getCurrentState(); + if (initialState == ConnectorState.UPDATING || initialState == ConnectorState.PREPARING_FOR_UPDATE) { + return; + } + + stateTransition.setDesiredState(ConnectorState.UPDATING); + stateTransition.setCurrentState(ConnectorState.PREPARING_FOR_UPDATE); + } + + @Override + public void prepareForUpdate() throws FlowUpdateException { + if (getCurrentState() != ConnectorState.PREPARING_FOR_UPDATE) { + throw new IllegalStateException("Cannot prepare update for " + this + " because its state is currently " + getCurrentState() + + "; it must be PREPARING_FOR_UPDATE."); + } + if (getDesiredState() != ConnectorState.UPDATING) { + throw new IllegalStateException("Cannot prepare update for " + this + " because its desired state is currently " + getDesiredState() + + "; it must be UPDATING."); + } + + logger.debug("Preparing {} for update", this); + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, getConnector().getClass(), getIdentifier())) { + getConnector().prepareForUpdate(workingFlowContext, activeFlowContext); + stateTransition.setCurrentState(ConnectorState.UPDATING); + logger.debug("Successfully prepared {} for update", this); + } catch (final Throwable t) { + logger.error("Failed to prepare update for {}", this, t); + + try { + abortUpdate(t); + } catch (final Throwable abortFailure) { + logger.error("Failed to abort update preparation for {}", this, abortFailure); + } + + throw t; + } + } + + @Override + public void inheritConfiguration(final List activeConfig, final List workingConfig, + final Bundle flowContextBundle) throws FlowUpdateException { + + logger.debug("Inheriting configuration for {}", this); + final MutableConnectorConfigurationContext configurationContext = createConfigurationContext(activeConfig); + final FrameworkFlowContext inheritContext = flowContextFactory.createWorkingFlowContext(identifier, + connectorDetails.getComponentLog(), configurationContext, flowContextBundle); + + // Apply the update for the active config + applyUpdate(inheritContext); + + // Configure the working config but do not apply + for (final VersionedConfigurationStep step : workingConfig) { + final StepConfiguration stepConfig = createStepConfiguration(step); + setConfiguration(step.getName(), stepConfig, true); + } + + logger.debug("Successfully inherited configuration for {}", this); + } + + private StepConfiguration createStepConfiguration(final VersionedConfigurationStep step) { + final Map convertedProperties = new HashMap<>(); + if (step.getProperties() != null) { + for (final Map.Entry entry : step.getProperties().entrySet()) { + final ConnectorValueReference valueReference = createValueReference(entry.getValue()); + convertedProperties.put(entry.getKey(), valueReference); + } + } + + return new StepConfiguration(convertedProperties); + } + + private MutableConnectorConfigurationContext createConfigurationContext(final List flowConfiguration) { + final StandardConnectorConfigurationContext configurationContext = new StandardConnectorConfigurationContext( + initializationContext.getAssetManager(), initializationContext.getSecretsManager()); + + for (final VersionedConfigurationStep versionedConfigStep : flowConfiguration) { + final StepConfiguration stepConfig = createStepConfiguration(versionedConfigStep); + configurationContext.setProperties(versionedConfigStep.getName(), stepConfig); + } + + return configurationContext; + } + + private ConnectorValueReference createValueReference(final VersionedConnectorValueReference versionedReference) { + final ConnectorValueType valueType = ConnectorValueType.valueOf(versionedReference.getValueType()); + return switch (valueType) { + case STRING_LITERAL -> new StringLiteralValue(versionedReference.getValue()); + case ASSET_REFERENCE -> new AssetReference(versionedReference.getAssetIds()); + case SECRET_REFERENCE -> new SecretReference(versionedReference.getProviderId(), versionedReference.getProviderName(), + versionedReference.getSecretName(), versionedReference.getFullyQualifiedSecretName()); + }; + } + + @Override + public void applyUpdate() throws FlowUpdateException { + try { + applyUpdate(workingFlowContext); + } catch (final FlowUpdateException e) { + // Since we failed to update, make sure that we stop the Connector. Note that we do not do this for all + // throwables because IllegalStateException for example indicates that we did not even attempt to perform the update. + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, connectorDetails.getConnector().getClass(), getIdentifier())) { + connectorDetails.getConnector().stop(activeFlowContext); + } catch (final Throwable stopThrowable) { + e.addSuppressed(stopThrowable); + } + + throw e; + } + } + + private void applyUpdate(final FrameworkFlowContext contextToInherit) throws FlowUpdateException { + final ConnectorState currentState = getCurrentState(); + if (currentState != ConnectorState.UPDATING) { + throw new IllegalStateException("Cannot finish update for " + this + " because its state is currently " + currentState + + "; it must be UPDATING."); + } + + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, getConnector().getClass(), getIdentifier())) { + getConnector().applyUpdate(contextToInherit, activeFlowContext); + + // Update the active flow context based on the properties of the provided context, as the connector has now been updated. + final ConnectorConfiguration workingConfig = contextToInherit.getConfigurationContext().toConnectorConfiguration(); + for (final NamedStepConfiguration stepConfig : workingConfig.getNamedStepConfigurations()) { + activeFlowContext.getConfigurationContext().replaceProperties(stepConfig.stepName(), stepConfig.configuration()); + } + + getComponentLog().info("Working Context has been applied to Active Context"); + + // The update has been completed. Tear down and recreate the working flow context to ensure it is in a clean state. + resetValidationState(); + recreateWorkingFlowContext(); + } catch (final Throwable t) { + logger.error("Failed to finish update for {}", this, t); + stateTransition.setCurrentState(ConnectorState.UPDATE_FAILED); + stateTransition.setDesiredState(ConnectorState.UPDATE_FAILED); + + throw new FlowUpdateException("Failed to finish update for " + this, t); + } + + stateTransition.setCurrentState(ConnectorState.UPDATED); + stateTransition.setDesiredState(ConnectorState.UPDATED); + logger.info("Successfully applied update for {}", this); + } + + private void destroyWorkingContext() { + if (this.workingFlowContext == null) { + return; + } + + try { + workingFlowContext.getManagedProcessGroup().purge().get(1, TimeUnit.MINUTES); + } catch (final Exception e) { + logger.warn("Failed to purge working flow context for {}", this, e); + } + + flowManager.onProcessGroupRemoved(workingFlowContext.getManagedProcessGroup()); + + this.workingFlowContext = null; + } + + @Override + public void abortUpdate(final Throwable cause) { + stateTransition.setCurrentState(ConnectorState.UPDATE_FAILED); + stateTransition.setDesiredState(ConnectorState.UPDATE_FAILED); + + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, getConnector().getClass(), getIdentifier())) { + getConnector().abortUpdate(workingFlowContext, cause); + } + logger.debug("Aborted update for {}", this); + } + + @Override + public void markInvalid(final String subject, final String explanation) { + validationState.set(new ValidationState(ValidationStatus.INVALID, List.of(new ValidationResult.Builder() + .subject(subject) + .valid(false) + .explanation(explanation) + .build()))); + } + + @Override + public void setConfiguration(final String stepName, final StepConfiguration configuration) throws FlowUpdateException { + setConfiguration(stepName, configuration, false); + } + + private void setConfiguration(final String stepName, final StepConfiguration configuration, final boolean forceOnConfigurationStepConfigured) throws FlowUpdateException { + // Update properties and check if the configuration changed. + final ConfigurationUpdateResult updateResult = workingFlowContext.getConfigurationContext().setProperties(stepName, configuration); + if (updateResult == ConfigurationUpdateResult.NO_CHANGES && !forceOnConfigurationStepConfigured) { + return; + } + + // If there were changes, trigger Processor to be notified of the change. + final Connector connector = connectorDetails.getConnector(); + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, connector.getClass(), getIdentifier())) { + logger.debug("Notifying {} of configuration change for configuration step {}", this, stepName); + connector.onConfigurationStepConfigured(stepName, workingFlowContext); + logger.debug("Successfully set configuration for step {} on {}", stepName, this); + } catch (final FlowUpdateException e) { + throw e; + } catch (final Exception e) { + logger.error("Failed to invoke onConfigured for {}", this, e); + throw new RuntimeException("Failed to invoke onConfigured for " + this, e); + } + } + + @Override + public ConnectorState getCurrentState() { + return stateTransition.getCurrentState(); + } + + @Override + public ConnectorState getDesiredState() { + return stateTransition.getDesiredState(); + } + + + @Override + public Optional getIdleDuration() { + final ProcessGroup processGroup = getActiveFlowContext().getManagedProcessGroup(); + final FlowFileActivity activity = processGroup.getFlowFileActivity(); + final OptionalLong lastActivityTimestamp = activity.getLatestActivityTime(); + if (lastActivityTimestamp.isEmpty()) { + return Optional.empty(); + } + + if (processGroup.isDataQueued()) { + return Optional.empty(); + } + + final Duration idleDuration = Duration.ofMillis(System.currentTimeMillis() - lastActivityTimestamp.getAsLong()); + return Optional.of(idleDuration); + } + + @Override + public FlowFileTransferCounts getFlowFileTransferCounts() { + return getActiveFlowContext().getManagedProcessGroup().getFlowFileActivity().getTransferCounts(); + } + + @Override + public Future start(final FlowEngine scheduler) { + final CompletableFuture startCompleteFuture = new CompletableFuture<>(); + start(scheduler, startCompleteFuture); + return startCompleteFuture; + } + + private void start(final FlowEngine scheduler, final CompletableFuture startCompleteFuture) { + try { + stateTransition.setDesiredState(ConnectorState.RUNNING); + activeFlowContext.getConfigurationContext().resolvePropertyValues(); + + verifyCanStart(); + + final ConnectorState currentState = getCurrentState(); + switch (currentState) { + case STARTING -> { + logger.debug("{} is already starting; adding future to pending start futures", this); + stateTransition.addPendingStartFuture(startCompleteFuture); + } + case RUNNING -> { + logger.debug("{} is already {}; will not attempt to start", this, currentState); + startCompleteFuture.complete(null); + } + case STOPPING -> { + // We have set the Desired State to RUNNING so when the Connector fully stops, it will be started again automatically + logger.info("{} is currently stopping so will not trigger Connector to start until it has fully stopped", this); + stateTransition.addPendingStartFuture(startCompleteFuture); + } + case STOPPED, PREPARING_FOR_UPDATE, UPDATED -> { + stateTransition.setCurrentState(ConnectorState.STARTING); + scheduler.schedule(() -> startComponent(scheduler, startCompleteFuture), 0, TimeUnit.SECONDS); + } + default -> { + logger.warn("{} is in state {} and cannot be started", this, currentState); + stateTransition.addPendingStartFuture(startCompleteFuture); + } + } + } catch (final Exception e) { + logger.error("Failed to start {}", this, e); + startCompleteFuture.completeExceptionally(e); + } + } + + @Override + public Future stop(final FlowEngine scheduler) { + logger.info("Stopping {}", this); + final CompletableFuture stopCompleteFuture = new CompletableFuture<>(); + + stateTransition.setDesiredState(ConnectorState.STOPPED); + + boolean stateUpdated = false; + while (!stateUpdated) { + final ConnectorState currentState = getCurrentState(); + if (currentState == ConnectorState.STOPPED) { + logger.info("{} is already stopped.", this); + stopCompleteFuture.complete(null); + return stopCompleteFuture; + } + + if (currentState == ConnectorState.STOPPING) { + logger.debug("{} is already stopping; adding future to pending stop futures", this); + stateTransition.addPendingStopFuture(stopCompleteFuture); + return stopCompleteFuture; + } + + stateUpdated = stateTransition.trySetCurrentState(currentState, ConnectorState.STOPPING); + } + + scheduler.schedule(() -> stopComponent(scheduler, stopCompleteFuture), 0, TimeUnit.SECONDS); + + return stopCompleteFuture; + } + + @Override + public Future drainFlowFiles() { + logger.debug("Draining FlowFiles for {}", this); + requireStopped("drain FlowFiles", ConnectorState.DRAINING); + + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, connectorDetails.getConnector().getClass(), getIdentifier())) { + getComponentLog().info("Draining FlowFiles from {}", this); + final CompletableFuture drainFuture = connectorDetails.getConnector().drainFlowFiles(activeFlowContext); + drainFutureRef.set(drainFuture); + + final CompletableFuture stateUpdateFuture = drainFuture.whenComplete((result, failureCause) -> { + drainFutureRef.set(null); + + if (failureCause == null) { + logger.info("Successfully drained FlowFiles for {}", this); + } else { + logger.error("Failed to drain FlowFiles for {}", this, failureCause); + } + + try { + connectorDetails.getConnector().stop(activeFlowContext); + } catch (final Exception e) { + logger.warn("Failed to stop {} after draining FlowFiles", this, e); + } + + stateTransition.setCurrentState(ConnectorState.STOPPED); + logger.info("All components of {} are now stopped after draining FlowFiles.", this); + }); + + return stateUpdateFuture; + } catch (final Throwable t) { + logger.error("Failed to drain FlowFiles for {}", this, t); + stateTransition.setCurrentState(ConnectorState.STOPPED); + throw t; + } + } + + @Override + public void cancelDrainFlowFiles() { + final Future future = this.drainFutureRef.getAndSet(null); + if (future == null) { + logger.debug("No active drain to cancel for {}; drain may have already completed", this); + return; + } + + future.cancel(true); + logger.info("Cancelled draining of FlowFiles for {}", this); + } + + @Override + public void verifyCancelDrainFlowFiles() throws IllegalStateException { + final ConnectorState state = getCurrentState(); + + // Allow if we're currently draining or if we're stopped; if stopped the cancel drain action will be a no-op + // but we don't want to throw an IllegalStateException in that case because doing so would mean that if one + // node in the cluster is stopped while another is draining we cannot cancel the drain. + if (state != ConnectorState.DRAINING && state != ConnectorState.STOPPED) { + throw new IllegalStateException("Cannot cancel draining of FlowFiles for " + this + " because its current state is " + state + "; it must be DRAINING."); + } + } + + @Override + public void verifyCanPurgeFlowFiles() throws IllegalStateException { + final ConnectorState desiredState = getDesiredState(); + if (desiredState != ConnectorState.STOPPED) { + throw new IllegalStateException("Cannot purge FlowFiles for " + this + " because its desired state is currently " + desiredState + "; it must be STOPPED."); + } + + final ConnectorState currentState = getCurrentState(); + if (currentState != ConnectorState.STOPPED) { + throw new IllegalStateException("Cannot purge FlowFiles for " + this + " because its current state is " + currentState + "; it must be STOPPED."); + } + } + + @Override + public Future purgeFlowFiles(final String requestor) { + logger.debug("Purging FlowFiles for {}", this); + requireStopped("purge FlowFiles", ConnectorState.PURGING); + + try { + final String dropRequestId = UUID.randomUUID().toString(); + final DropFlowFileStatus status = activeFlowContext.getManagedProcessGroup().dropAllFlowFiles(dropRequestId, requestor); + final CompletableFuture future = status.getCompletionFuture(); + final CompletableFuture stateUpdateFuture = future.whenComplete((result, failureCause) -> { + stateTransition.setCurrentState(ConnectorState.STOPPED); + + if (failureCause == null) { + logger.info("Successfully purged FlowFiles for {}", this); + } else { + logger.error("Failed to purge FlowFiles for {}", this, failureCause); + } + }); + return stateUpdateFuture; + } catch (final Throwable t) { + logger.error("Failed to purge FlowFiles for {}", this, t); + stateTransition.setCurrentState(ConnectorState.STOPPED); + throw t; + } + } + + private void requireStopped(final String action, final ConnectorState newState) { + final ConnectorState desiredState = getDesiredState(); + if (desiredState != ConnectorState.STOPPED) { + throw new IllegalStateException("Cannot " + action + " for " + this + " because its desired state is currently " + desiredState + "; it must be STOPPED."); + } + + boolean stateUpdated = false; + while (!stateUpdated) { + final ConnectorState currentState = getCurrentState(); + if (currentState != ConnectorState.STOPPED) { + throw new IllegalStateException("Cannot " + action + " for " + this + " because its current state is " + currentState + "; it must be STOPPED."); + } + + stateUpdated = stateTransition.trySetCurrentState(ConnectorState.STOPPED, newState); + } + } + + private void stopComponent(final FlowEngine scheduler, final CompletableFuture stopCompleteFuture) { + logger.debug("Stopping component for {}", this); + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, connectorDetails.getConnector().getClass(), getIdentifier())) { + connectorDetails.getConnector().stop(activeFlowContext); + } catch (final Exception e) { + logger.error("Failed to stop {}. Will try again in 10 seconds", this, e); + scheduler.schedule(() -> stopComponent(scheduler, stopCompleteFuture), 10, TimeUnit.SECONDS); + return; + } + + stateTransition.setCurrentState(ConnectorState.STOPPED); + stopCompleteFuture.complete(null); + logger.info("Successfully stopped {}", this); + + final ConnectorState desiredState = getDesiredState(); + if (desiredState == ConnectorState.RUNNING) { + logger.info("{} was requested to be RUNNING while it was stopping so will attempt to start again", this); + start(scheduler, new CompletableFuture<>()); + } + } + + private void startComponent(final ScheduledExecutorService scheduler, final CompletableFuture startCompleteFuture) { + logger.debug("Starting component for {}", this); + final ConnectorState desiredState = getDesiredState(); + if (desiredState != ConnectorState.RUNNING) { + logger.info("Will not start {} because the desired state is no longer RUNNING but is now {}", this, desiredState); + return; + } + + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, connectorDetails.getConnector().getClass(), getIdentifier())) { + connectorDetails.getConnector().start(activeFlowContext); + } catch (final Exception e) { + logger.error("Failed to start {}. Will try again in 10 seconds", this, e); + scheduler.schedule(() -> startComponent(scheduler, startCompleteFuture), 10, TimeUnit.SECONDS); + return; + } + + stateTransition.setCurrentState(ConnectorState.RUNNING); + startCompleteFuture.complete(null); + logger.info("Successfully started {}", this); + } + + + @Override + public void verifyCanDelete() { + final QueueSize queueSize = getActiveFlowContext().getManagedProcessGroup().getQueueSize(); + if (queueSize.getObjectCount() > 0) { + throw new IllegalStateException("Cannot delete " + this + " because its Process Group has " + queueSize.getObjectCount() + + " FlowFiles queued; all FlowFiles must be removed before it can be deleted."); + } + + final ConnectorState currentState = getCurrentState(); + if (currentState == ConnectorState.STOPPED || currentState == ConnectorState.UPDATE_FAILED || currentState == ConnectorState.UPDATED) { + return; + } + + throw new IllegalStateException("Cannot delete " + this + " because its state is currently " + currentState + "; it must be stopped before it can be deleted."); + } + + @Override + public void verifyCanStart() { + final ValidationState state = performValidation(); + if (state.getStatus() != ValidationStatus.VALID) { + throw new IllegalStateException("Cannot start " + this + " because it is not valid: " + state.getValidationErrors()); + } + } + + @Override + public Connector getConnector() { + return connectorDetails.getConnector(); + } + + @Override + public String getComponentType() { + return componentType; + } + + @Override + public String getCanonicalClassName() { + return componentCanonicalClass; + } + + @Override + public BundleCoordinate getBundleCoordinate() { + return bundleCoordinate; + } + + @Override + public boolean isExtensionMissing() { + return extensionMissing; + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName) { + if (workingFlowContext == null) { + throw new IllegalStateException("Cannot fetch Allowable Values for %s.%s because %s is not being updated.".formatted( + stepName, propertyName, this)); + } + + workingFlowContext.getConfigurationContext().resolvePropertyValues(); + + try (NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, getConnector().getClass(), getIdentifier())) { + return getConnector().fetchAllowableValues(stepName, propertyName, workingFlowContext); + } + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName, final String filter) { + if (workingFlowContext == null) { + throw new IllegalStateException("Cannot fetch Allowable Values for %s.%s because %s is not being updated.".formatted( + stepName, propertyName, this)); + } + + workingFlowContext.getConfigurationContext().resolvePropertyValues(); + + try (NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, getConnector().getClass(), getIdentifier())) { + return getConnector().fetchAllowableValues(stepName, propertyName, workingFlowContext, filter); + } + } + + @Override + public void initializeConnector(final FrameworkConnectorInitializationContext initializationContext) { + logger.debug("Initializing {}", this); + this.initializationContext = initializationContext; + + try (NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, getConnector().getClass(), getIdentifier())) { + getConnector().initialize(initializationContext); + } + + recreateWorkingFlowContext(); + logger.info("Successfully initialized {}", this); + } + + @Override + public void loadInitialFlow() throws FlowUpdateException { + logger.debug("Loading initial flow for {}", this); + if (initializationContext == null) { + throw new IllegalStateException("Cannot load initial flow because " + this + " has not been initialized yet."); + } + + final VersionedExternalFlow initialFlow; + try (NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, getConnector().getClass(), getIdentifier())) { + initialFlow = getConnector().getInitialFlow(); + } + + if (initialFlow == null) { + logger.info("{} has no initial flow to load", this); + } else { + final ValidationResult unresolvedBundleResult = validateBundlesCanBeResolved(initialFlow.getFlowContents(), initializationContext.getComponentBundleLookup()); + + if (unresolvedBundleResult != null) { + logger.error("Cannot load initial flow for {} because some component bundles cannot be resolved: {}", this, unresolvedBundleResult.getExplanation()); + unresolvedBundleValidationResult = unresolvedBundleResult; + } else { + logger.info("Loading initial flow for {}", this); + // Update all RUNNING components to ENABLED before applying the initial flow so that components + // are not started before being configured. + stopComponents(initialFlow.getFlowContents()); + initializationContext.updateFlow(activeFlowContext, initialFlow, BundleCompatibility.RESOLVE_BUNDLE); + } + } + + resetValidationState(); + recreateWorkingFlowContext(); + } + + private void stopComponents(final VersionedProcessGroup group) { + for (final VersionedProcessor processor : group.getProcessors()) { + if (processor.getScheduledState() == ScheduledState.RUNNING) { + processor.setScheduledState(ScheduledState.ENABLED); + } + } + + for (final VersionedControllerService service : group.getControllerServices()) { + if (service.getScheduledState() == ScheduledState.RUNNING) { + service.setScheduledState(ScheduledState.ENABLED); + } + } + + for (final VersionedProcessGroup childGroup : group.getProcessGroups()) { + stopComponents(childGroup); + } + } + + /** + * Ensures that all bundles required by the given Process Group can be resolved. We do this in order to make the Connector + * invalid if any Processor or Controller Service cannot be properly instantiated due to missing bundles. We intentionally + * differentiate between making the Connector invalid versus Ghosting the Connector for a few reasons: + *
    + *
  • + * Ghosting the Connector would prevent us from even getting the Configuration Steps, and it results in all Properties becoming sensitive. This can lead to confusion. + *
  • + *
  • + * The flow may change dynamically and so it's possible for a Connector to be valid given its initial flow and then become invalid + * based on configuration because the new configuration requires a new component that is unavailable. We would not suddenly change from + * a valid Connector to a Ghosted Connector, we could only become invalid. We do not want a missing component in the Initial Flow to be + * treated differently than a missing component from a subsequent flow update. + *
  • + *
  • + * Ghosting should be reserved for situations where the extension itself is missing. + *
  • + *
+ * + * @param group the process group to validate + * @param bundleLookup the bundle lookup + * @return a ValidationResult describing the missing bundles if any are missing; null if all bundles can be resolved + */ + private ValidationResult validateBundlesCanBeResolved(final VersionedProcessGroup group, final ComponentBundleLookup bundleLookup) { + final Set missingBundles = new HashSet<>(); + final Set missingProcessorTypes = new HashSet<>(); + final Set missingControllerServiceTypes = new HashSet<>(); + + collectUnresolvedBundles(group, bundleLookup, missingBundles, missingProcessorTypes, missingControllerServiceTypes); + + if (missingBundles.isEmpty()) { + return null; + } + + final StringBuilder explanation = new StringBuilder(); + explanation.append("%d Processors and %d Controller Services unavailable from %d missing bundles".formatted( + missingProcessorTypes.size(), missingControllerServiceTypes.size(), missingBundles.size())); + explanation.append("\nMissing Bundles: %s".formatted(missingBundles)); + if (!missingProcessorTypes.isEmpty()) { + explanation.append("\nMissing Processors: %s".formatted(missingProcessorTypes)); + } + if (!missingControllerServiceTypes.isEmpty()) { + explanation.append("\nMissing Controller Services: %s".formatted(missingControllerServiceTypes)); + } + + return new ValidationResult.Builder() + .subject("Missing Bundles") + .valid(false) + .explanation(explanation.toString()) + .build(); + } + + private void collectUnresolvedBundles(final VersionedProcessGroup group, final ComponentBundleLookup bundleLookup, + final Set missingBundles, final Set missingProcessorTypes, + final Set missingControllerServiceTypes) { + if (group.getProcessors() != null) { + for (final VersionedProcessor processor : group.getProcessors()) { + if (!isBundleResolvable(processor.getType(), processor.getBundle(), bundleLookup)) { + missingBundles.add(formatBundle(processor.getBundle())); + missingProcessorTypes.add(processor.getType()); + } + } + } + + if (group.getControllerServices() != null) { + for (final VersionedControllerService service : group.getControllerServices()) { + if (!isBundleResolvable(service.getType(), service.getBundle(), bundleLookup)) { + missingBundles.add(formatBundle(service.getBundle())); + missingControllerServiceTypes.add(service.getType()); + } + } + } + + if (group.getProcessGroups() != null) { + for (final VersionedProcessGroup childGroup : group.getProcessGroups()) { + collectUnresolvedBundles(childGroup, bundleLookup, missingBundles, missingProcessorTypes, missingControllerServiceTypes); + } + } + } + + private String formatBundle(final Bundle bundle) { + return "%s:%s:%s".formatted(bundle.getGroup(), bundle.getArtifact(), bundle.getVersion()); + } + + private boolean isBundleResolvable(final String componentType, final Bundle currentBundle, final ComponentBundleLookup bundleLookup) { + final List availableBundles = bundleLookup.getAvailableBundles(componentType); + + if (availableBundles.contains(currentBundle)) { + return true; + } + + // With RESOLVE_BUNDLE, a bundle can be resolved only if exactly one alternative bundle is available + return availableBundles.size() == 1; + } + + private void recreateWorkingFlowContext() { + destroyWorkingContext(); + workingFlowContext = flowContextFactory.createWorkingFlowContext(identifier, + connectorDetails.getComponentLog(), activeFlowContext.getConfigurationContext(), activeFlowContext.getBundle()); + + getComponentLog().info("Working Flow Context has been recreated"); + } + + @Override + public void pauseValidationTrigger() { + triggerValidation = false; + } + + @Override + public void resumeValidationTrigger() { + triggerValidation = true; + logger.debug("Resuming Triggering of Validation State for {}; Resetting validation state", this); + resetValidationState(); + } + + @Override + public boolean isValidationPaused() { + return !triggerValidation; + } + + @Override + public List verifyConfigurationStep(final String stepName, final StepConfiguration configurationOverrides) { + logger.debug("Verifying configuration step {} for {}", stepName, this); + final List invalidSecretRefs = new ArrayList<>(); + final List invalidAssetRefs = new ArrayList<>(); + final Map resolvedPropertyOverrides = resolvePropertyReferences(configurationOverrides, invalidSecretRefs, invalidAssetRefs); + + final List results = new ArrayList<>(); + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, getConnector().getClass(), getIdentifier())) { + + final DescribedValueProvider allowableValueProvider = (step, propertyName) -> fetchAllowableValues(step, propertyName, workingFlowContext); + + final MutableConnectorConfigurationContext configContext = workingFlowContext.getConfigurationContext().createWithOverrides(stepName, resolvedPropertyOverrides); + final ConnectorConfiguration connectorConfig = configContext.toConnectorConfiguration(); + final ParameterContextFacade paramContext = workingFlowContext.getParameterContext(); + final ConnectorValidationContext validationContext = new StandardConnectorValidationContext(connectorConfig, allowableValueProvider, paramContext); + + final Optional optionalStep = getConfigurationStep(stepName); + if (optionalStep.isEmpty()) { + results.add(new ConfigVerificationResult.Builder() + .verificationStepName("Property Validation") + .outcome(Outcome.FAILED) + .explanation("Configuration step with name '" + stepName + "' does not exist.") + .build()); + return results; + } + + final ConfigurationStep configurationStep = optionalStep.get(); + + final List validationResults = new ArrayList<>(); + validatePropertyReferences(configurationStep, configurationOverrides, validationResults); + + // If there are any invalid secrets or assets referenced, add Validation Results for them. + addInvalidReferenceResults(validationResults, invalidSecretRefs, invalidAssetRefs); + + // If there are any framework-level validation failures, we do not run the Connector-specific validation because + // doing so would mean that we must provide weak guarantees about the state of the configuration when the Connector's + // validation is invoked. But if there are no framework-level validation failures, we can proceed to invoke the + // Connector's validation logic. + if (validationResults.isEmpty()) { + final List implValidationResults = getConnector().validateConfigurationStep(configurationStep, configContext, validationContext); + validationResults.addAll(implValidationResults); + } + + final List invalidConfigResults = validationResults.stream() + .filter(result -> !result.isValid()) + .map(this::createConfigVerificationResult) + .toList(); + + if (invalidConfigResults.isEmpty()) { + results.add(new ConfigVerificationResult.Builder() + .verificationStepName("Property Validation") + .outcome(Outcome.SUCCESSFUL) + .build()); + + results.addAll(getConnector().verifyConfigurationStep(stepName, resolvedPropertyOverrides, workingFlowContext)); + } else { + results.addAll(invalidConfigResults); + } + + logger.debug("Completed verification of configuration step {} for {}", stepName, this); + return results; + } + } + + private ConfigVerificationResult createConfigVerificationResult(final ValidationResult validationResult) { + return new ConfigVerificationResult.Builder() + .verificationStepName("Property Validation - " + validationResult.getSubject()) + .outcome(validationResult.isValid() ? Outcome.SUCCESSFUL : Outcome.FAILED) + .subject(validationResult.getSubject()) + .explanation(validationResult.getExplanation()) + .build(); + } + + private Map resolvePropertyReferences(final StepConfiguration configurationOverrides, final List invalidSecretRefs, + final List invalidAssetRefs) { + + final Map resolvedProperties = new HashMap<>(); + + try { + // Secret References can be expensive to lookup so we don't want to call getSecret() for each one. Instead, we + // want to find all Secrets by Provider and then call fetchSecrets() once per Provider. + final Set secretReferences = configurationOverrides.getPropertyValues().values().stream() + .filter(Objects::nonNull) + .filter(ref -> ref.getValueType() == ConnectorValueType.SECRET_REFERENCE) + .map(ref -> (SecretReference) ref) + .collect(Collectors.toSet()); + + final Map secretsByReference = initializationContext.getSecretsManager().getSecrets(secretReferences); + secretsByReference.forEach((ref, secret) -> { + if (secret == null) { + invalidSecretRefs.add(ref); + } + }); + + for (final Map.Entry entry : configurationOverrides.getPropertyValues().entrySet()) { + final String propertyName = entry.getKey(); + final ConnectorValueReference valueReference = entry.getValue(); + + if (valueReference == null) { + continue; + } + + // We've already looked up secrets above, so use the cached value here. + if (valueReference.getValueType() == ConnectorValueType.SECRET_REFERENCE) { + final SecretReference secretReference = (SecretReference) valueReference; + final Secret secret = secretsByReference.get(secretReference); + final String resolvedValue = (secret == null) ? null : secret.getValue(); + resolvedProperties.put(propertyName, resolvedValue); + continue; + } + + final String resolvedValue = resolvePropertyReference(valueReference); + resolvedProperties.put(propertyName, resolvedValue); + + if (resolvedValue == null && valueReference.getValueType() == ConnectorValueType.ASSET_REFERENCE) { + invalidAssetRefs.add((AssetReference) valueReference); + } + } + } catch (final IOException ioe) { + throw new UncheckedIOException("Failed to resolve Secret references for " + this, ioe); + } + + return resolvedProperties; + } + + private String resolvePropertyReference(final ConnectorValueReference valueReference) throws IOException { + if (valueReference == null) { + return null; + } + + return switch (valueReference) { + case StringLiteralValue stringLiteralValue -> stringLiteralValue.getValue(); + case AssetReference assetReference -> resolveAssetReferences(assetReference); + case SecretReference secretReference -> initializationContext.getSecretsManager() + .getSecret(secretReference) + .map(Secret::getValue) + .orElse(null); + }; + } + + private String resolveAssetReferences(final AssetReference assetReference) { + final Set resolvedAssetValues = new HashSet<>(); + for (final String assetId : assetReference.getAssetIdentifiers()) { + initializationContext.getAssetManager().getAsset(assetId) + .map(Asset::getFile) + .map(File::getAbsolutePath) + .ifPresent(resolvedAssetValues::add); + } + + logger.debug("Resolved {} to {} for {}", assetReference, resolvedAssetValues, this); + return String.join(",", resolvedAssetValues); + } + + private Optional getConfigurationStep(final String stepName) { + for (final ConfigurationStep step : getConfigurationSteps()) { + if (Objects.equals(step.getName(), stepName)) { + return Optional.of(step); + } + } + + return Optional.empty(); + } + + @Override + public List verify() { + logger.debug("Verifying {}", this); + final List results = new ArrayList<>(); + + final ValidationState state = performValidation(); + if (state.getStatus() == ValidationStatus.INVALID) { + final List validationFailureExplanations = state.getValidationErrors().stream() + .map(ValidationResult::getExplanation) + .toList(); + + results.add(new ConfigVerificationResult.Builder() + .verificationStepName("Property Validation") + .outcome(Outcome.FAILED) + .explanation("There are " + validationFailureExplanations.size() + " validation failures: " + validationFailureExplanations) + .build()); + + logger.debug("Completed verification for {} with validation failures", this); + return results; + } + + workingFlowContext.getConfigurationContext().resolvePropertyValues(); + + try (NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, getConnector().getClass(), getIdentifier())) { + results.addAll(getConnector().verify(workingFlowContext)); + } + + logger.debug("Completed verification for {}", this); + return results; + } + + + @Override + public String getIdentifier() { + return identifier; + } + + @Override + public String getProcessGroupIdentifier() { + return null; + } + + @Override + public ComponentLog getComponentLog() { + return connectorDetails.getComponentLog(); + } + + @Override + public List getConfigurationSteps() { + try (NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, getConnector().getClass(), getIdentifier())) { + return getConnector().getConfigurationSteps(); + } + } + + @Override + public FrameworkFlowContext getActiveFlowContext() { + return activeFlowContext; + } + + @Override + public FrameworkFlowContext getWorkingFlowContext() { + return workingFlowContext; + } + + @Override + public void discardWorkingConfiguration() { + recreateWorkingFlowContext(); + logger.debug("Discarded working configuration for {}", this); + } + + @Override + public List getAvailableActions() { + final List actions = new ArrayList<>(); + final ConnectorState currentState = getCurrentState(); + final boolean dataQueued = activeFlowContext.getManagedProcessGroup().isDataQueued(); + final boolean stopped = isStopped(); + + actions.add(createStartAction(stopped)); + actions.add(createStopAction(currentState)); + actions.add(createConfigureAction()); + actions.add(createDiscardWorkingConfigAction()); + actions.add(createPurgeFlowFilesAction(stopped, dataQueued)); + actions.add(createDrainFlowFilesAction(stopped, dataQueued)); + actions.add(createCancelDrainFlowFilesAction(currentState == ConnectorState.DRAINING)); + actions.add(createApplyUpdatesAction(currentState)); + actions.add(createDeleteAction(stopped, dataQueued)); + + return actions; + } + + private boolean isStopped() { + final ConnectorState currentState = getCurrentState(); + if (currentState == ConnectorState.STOPPED) { + return true; + } + if (currentState == ConnectorState.UPDATED || currentState == ConnectorState.UPDATE_FAILED) { + return !hasActiveThread(getActiveFlowContext().getManagedProcessGroup()); + } + + return false; + } + + private ConnectorAction createStartAction(final boolean stopped) { + final boolean allowed; + final String reason; + + if (!stopped) { + allowed = false; + reason = "Connector is not stopped"; + } else { + final Collection validationResults = getValidationErrors(); + if (validationResults.isEmpty()) { + allowed = true; + reason = null; + } else { + allowed = false; + reason = "Connector is not valid: " + validationResults.stream() + .map(ValidationResult::getExplanation) + .collect(Collectors.joining("; ")); + } + } + + return new StandardConnectorAction("START", "Start the connector", allowed, reason); + } + + private ConnectorAction createStopAction(final ConnectorState currentState) { + final boolean allowed; + if (currentState == ConnectorState.RUNNING || currentState == ConnectorState.STARTING) { + allowed = true; + } else if (currentState == ConnectorState.UPDATED || currentState == ConnectorState.UPDATE_FAILED) { + allowed = hasActiveThread(activeFlowContext.getManagedProcessGroup()); + } else { + allowed = false; + } + + final String reason = allowed ? null : "Connector is not running"; + return new StandardConnectorAction("STOP", "Stop the connector", allowed, reason); + } + + private ConnectorAction createConfigureAction() { + return new StandardConnectorAction("CONFIGURE", "Configure the connector", true, null); + } + + private ConnectorAction createDiscardWorkingConfigAction() { + final boolean allowed = hasWorkingConfigurationChanges(); + final String reason = allowed ? null : "No pending changes to discard"; + + return new StandardConnectorAction("DISCARD_WORKING_CONFIGURATION", "Discard any changes made to the working configuration", allowed, reason); + } + + private boolean hasActiveThread(final ProcessGroup group) { + for (final ProcessorNode processor : group.getProcessors()) { + if (processor.getActiveThreadCount() > 0) { + return true; + } + } + + for (final ProcessGroup childGroup : group.getProcessGroups()) { + if (hasActiveThread(childGroup)) { + return true; + } + } + + return false; + } + + private ConnectorAction createPurgeFlowFilesAction(final boolean stopped, final boolean dataQueued) { + return createDataQueuedAction(stopped, dataQueued, "PURGE_FLOWFILES", "Purge all FlowFiles from the connector, dropping all data without processing it"); + } + + private ConnectorAction createDrainFlowFilesAction(final boolean stopped, final boolean dataQueued) { + return createDataQueuedAction(stopped, dataQueued, "DRAIN_FLOWFILES", "Process data that is currently in the flow but do not ingest any additional data"); + } + + private static ConnectorAction createDataQueuedAction(final boolean stopped, final boolean dataQueued, final String actionName, final String description) { + final boolean allowed; + final String reason; + + if (!stopped) { + allowed = false; + reason = "Connector must be stopped"; + } else if (!dataQueued) { + allowed = false; + reason = "No data is queued"; + } else { + allowed = true; + reason = null; + } + + return new StandardConnectorAction(actionName, description, allowed, reason); + } + + private ConnectorAction createCancelDrainFlowFilesAction(final boolean draining) { + if (draining) { + return new StandardConnectorAction("CANCEL_DRAIN_FLOWFILES", "Cancel the ongoing drain of FlowFiles", true, null); + } + + return new StandardConnectorAction("CANCEL_DRAIN_FLOWFILES", "Cancel the ongoing drain of FlowFiles", false, + "Connector is not currently draining FlowFiles"); + } + + private ConnectorAction createApplyUpdatesAction(final ConnectorState currentState) { + final boolean allowed; + final String reason; + + if (currentState == ConnectorState.PREPARING_FOR_UPDATE || currentState == ConnectorState.UPDATING) { + allowed = false; + reason = "Connector is updating"; + } else if (!hasWorkingConfigurationChanges()) { + allowed = false; + reason = "No pending changes"; + } else { + allowed = true; + reason = null; + } + + return new StandardConnectorAction("APPLY_UPDATES", "Apply the working configuration to the active configuration", allowed, reason); + } + + private ConnectorAction createDeleteAction(final boolean stopped, final boolean dataQueued) { + final boolean allowed; + final String reason; + + if (!stopped) { + allowed = false; + reason = "Connector must be stopped"; + } else if (dataQueued) { + allowed = false; + reason = "Data is queued"; + } else { + allowed = true; + reason = null; + } + + return new StandardConnectorAction("DELETE", "Delete the connector", allowed, reason); + } + + private boolean hasWorkingConfigurationChanges() { + final FrameworkFlowContext workingContext = this.workingFlowContext; + if (workingContext == null) { + return false; + } + + final ConnectorConfiguration activeConfig = activeFlowContext.getConfigurationContext().toConnectorConfiguration(); + final ConnectorConfiguration workingConfig = workingContext.getConfigurationContext().toConnectorConfiguration(); + return !Objects.equals(activeConfig, workingConfig); + } + + @Override + public Authorizable getParentAuthorizable() { + return parentAuthorizable; + } + + @Override + public Resource getResource() { + return ResourceFactory.getComponentResource(ResourceType.Connector, getIdentifier(), getName()); + } + + @Override + public Optional getVersionedComponentId() { + return Optional.ofNullable(versionedComponentId.get()); + } + + @Override + public void setVersionedComponentId(final String versionedComponentId) { + boolean updated = false; + while (!updated) { + final String currentId = this.versionedComponentId.get(); + + if (currentId == null) { + updated = this.versionedComponentId.compareAndSet(null, versionedComponentId); + } else if (currentId.equals(versionedComponentId)) { + return; + } else if (versionedComponentId == null) { + updated = this.versionedComponentId.compareAndSet(currentId, null); + } else { + throw new IllegalStateException(this + " is already under version control"); + } + } + } + + private void resetValidationState() { + validationState.set(new ValidationState(ValidationStatus.VALIDATING, Collections.emptyList())); + validationTrigger.triggerAsync(this); + logger.debug("Validation state has been reset for {}", this); + } + + @Override + public ValidationStatus getValidationStatus() { + return validationState.get().getStatus(); + } + + @Override + public ValidationState getValidationState() { + return validationState.get(); + } + + @Override + public Collection getValidationErrors() { + final ValidationState state = validationState.get(); + return state.getValidationErrors(); + } + + @Override + public ValidationState performValidation() { + logger.debug("Performing validation for {}", this); + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, getConnector().getClass(), getIdentifier())) { + + final List allResults = new ArrayList<>(); + + if (unresolvedBundleValidationResult != null) { + allResults.add(unresolvedBundleValidationResult); + } else { + final ConnectorValidationContext validationContext = createValidationContext(activeFlowContext); + + validateManagedFlowComponents(allResults); + validatePropertyReferences(allResults); + + if (allResults.isEmpty()) { + try { + final List implValidationResults = getConnector().validate(activeFlowContext, validationContext); + allResults.addAll(implValidationResults); + } catch (final Exception e) { + allResults.add(new ValidationResult.Builder() + .subject("Validation Failure") + .valid(false) + .explanation("Encountered a failure while attempting to perform validation: " + e.getMessage()) + .build()); + } + } + } + + final ValidationState resultState; + if (allResults.isEmpty()) { + resultState = new ValidationState(ValidationStatus.VALID, Collections.emptyList()); + } else { + // Filter out any results that are 'valid' and any results that are invalid due to the fact that a Controller Service is disabled, + // since these will not be relevant when started. + final List relevantResults = allResults.stream() + .filter(result -> !result.isValid()) + .filter(result -> !DisabledServiceValidationResult.isMatch(result)) + .toList(); + + if (relevantResults.isEmpty()) { + resultState = new ValidationState(ValidationStatus.VALID, Collections.emptyList()); + } else { + resultState = new ValidationState(ValidationStatus.INVALID, relevantResults); + } + } + + validationState.set(resultState); + + if (resultState.getStatus() == ValidationStatus.VALID) { + logger.info("Validation completed for {}. Connector is valid.", this); + } else { + logger.info("Validation completed for {}. Connector is invalid: {}", this, + resultState.getValidationErrors().stream() + .map(ValidationResult::getExplanation) + .collect(Collectors.joining("; "))); + } + + return resultState; + } + } + + + private void validateManagedFlowComponents(final List results) { + final ProcessGroup managedProcessGroup = activeFlowContext.getManagedProcessGroup(); + + // Check for any missing / ghosted Processors + final Set missingProcessors = new HashSet<>(); + for (final ProcessorNode processor : managedProcessGroup.findAllProcessors()) { + if (processor.isExtensionMissing()) { + missingProcessors.add(getSimpleClassName(processor.getCanonicalClassName()) + " from NAR " + processor.getBundleCoordinate()); + } + } + + if (!missingProcessors.isEmpty()) { + results.add(new ValidationResult.Builder() + .subject("Missing Processors") + .valid(false) + .explanation("The following processors are missing: " + missingProcessors) + .build()); + } + + // Check for any missing / ghosted Controller Services + final Set missingControllerServices = new HashSet<>(); + for (final ControllerServiceNode controllerService : managedProcessGroup.findAllControllerServices()) { + if (controllerService.isExtensionMissing()) { + missingControllerServices.add(getSimpleClassName(controllerService.getCanonicalClassName()) + " from NAR " + controllerService.getBundleCoordinate()); + } + } + + if (!missingControllerServices.isEmpty()) { + results.add(new ValidationResult.Builder() + .subject("Missing Controller Services") + .valid(false) + .explanation("The following controller services are missing: " + missingControllerServices) + .build()); + } + } + + private static String getSimpleClassName(final String canonicalClassName) { + final int lastDot = canonicalClassName.lastIndexOf('.'); + return lastDot < 0 ? canonicalClassName : canonicalClassName.substring(lastDot + 1); + } + + private void validatePropertyReferences(final List allResults) { + final List configurationSteps = getConnector().getConfigurationSteps(); + final ConnectorConfiguration connectorConfiguration = activeFlowContext.getConfigurationContext().toConnectorConfiguration(); + + for (final ConfigurationStep step : configurationSteps) { + final NamedStepConfiguration namedStepConfig = connectorConfiguration.getNamedStepConfiguration(step.getName()); + if (namedStepConfig == null) { + continue; + } + + final StepConfiguration stepConfiguration = namedStepConfig.configuration(); + validatePropertyReferences(step, namedStepConfig.configuration(), allResults); + + // Check for invalid Secret and Asset references + final List invalidSecrets = new ArrayList<>(); + final List invalidAssets = new ArrayList<>(); + resolvePropertyReferences(stepConfiguration, invalidSecrets, invalidAssets); + addInvalidReferenceResults(allResults, invalidSecrets, invalidAssets); + } + } + + private void addInvalidReferenceResults(final List results, final List invalidSecretRefs, final List invalidAssetRefs) { + for (final SecretReference invalidSecretRef : invalidSecretRefs) { + results.add(new ValidationResult.Builder() + .subject("Secret Reference") + .valid(false) + .explanation("The referenced secret [" + invalidSecretRef.getFullyQualifiedName() + "] could not be found") + .build()); + } + + for (final AssetReference invalidAssetRef : invalidAssetRefs) { + results.add(new ValidationResult.Builder() + .subject("Asset Reference") + .valid(false) + .explanation("The referenced assets [" + StringUtils.join(invalidAssetRef.getAssetIdentifiers(), ",") + "] could not be found") + .build()); + } + } + + private void validatePropertyReferences(final ConfigurationStep step, final StepConfiguration stepConfig, final List allResults) { + for (final ConnectorPropertyGroup propertyGroup : step.getPropertyGroups()) { + for (final ConnectorPropertyDescriptor descriptor : propertyGroup.getProperties()) { + final PropertyType propertyType = descriptor.getType(); + final ConnectorValueReference reference = stepConfig.getPropertyValue(descriptor.getName()); + + final String subject = step.getName() + " / " + descriptor.getName(); + + if (!isReferenceAllowed(reference, propertyType)) { + final String providedReferenceType = switch (reference.getValueType()) { + case ASSET_REFERENCE -> ""; + case SECRET_REFERENCE -> ""; + case STRING_LITERAL -> ""; + }; + + final String expectedReferenceType = propertyType == PropertyType.SECRET ? "a Secret reference" : "an Explicit value"; + + allResults.add(new ValidationResult.Builder() + .subject(subject) + .input(providedReferenceType) + .explanation("This property must be configured with " + expectedReferenceType) + .build()); + } + } + } + } + + private boolean isReferenceAllowed(final ConnectorValueReference reference, final PropertyType propertyType) { + // If the reference is null or its value is unset, then it is allowed + if (reference == null) { + return true; + } + + switch (reference) { + case StringLiteralValue stringLiteralValue -> { + if (stringLiteralValue.getValue() == null) { + return true; + } + } + case AssetReference assetReference -> { + if (assetReference.getAssetIdentifiers() == null || assetReference.getAssetIdentifiers().isEmpty()) { + return true; + } + } + case SecretReference secretReference -> { + if (secretReference.getSecretName() == null) { + return true; + } + } + } + + if (propertyType == PropertyType.SECRET) { + return reference.getValueType() == ConnectorValueType.SECRET_REFERENCE; + } + + if (propertyType == PropertyType.ASSET || propertyType == PropertyType.ASSET_LIST) { + return reference.getValueType() == ConnectorValueType.ASSET_REFERENCE; + } + + return reference.getValueType() != ConnectorValueType.SECRET_REFERENCE && reference.getValueType() != ConnectorValueType.ASSET_REFERENCE; + } + + private ConnectorValidationContext createValidationContext(final FrameworkFlowContext context) { + final DescribedValueProvider allowableValueProvider = (stepName, propertyName) -> + fetchAllowableValues(stepName, propertyName, context); + final ConnectorConfiguration connectorConfiguration = context.getConfigurationContext().toConnectorConfiguration(); + return new StandardConnectorValidationContext(connectorConfiguration, allowableValueProvider, context.getParameterContext()); + } + + private List fetchAllowableValues(final String stepName, final String propertyName, final FlowContext context) { + final List allowableValues; + try (NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, getConnector().getClass(), getIdentifier())) { + allowableValues = getConnector().fetchAllowableValues(stepName, propertyName, context); + } + + if (allowableValues == null || allowableValues.isEmpty()) { + return Collections.emptyList(); + } + + return allowableValues; + } + + @Override + public boolean equals(final Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + final StandardConnectorNode that = (StandardConnectorNode) o; + return Objects.equals(identifier, that.identifier); + } + + @Override + public int hashCode() { + return Objects.hashCode(identifier); + } + + @Override + public String toString() { + return "StandardConnectorNode[id=" + identifier + ", name=" + name + ", state=" + stateTransition.getCurrentState() + "]"; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorPropertyValue.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorPropertyValue.java new file mode 100644 index 000000000000..82603da0b8cb --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorPropertyValue.java @@ -0,0 +1,101 @@ +/* + * 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.components.connector; + +import org.apache.nifi.processor.DataUnit; +import org.apache.nifi.util.FormatUtils; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; + +public class StandardConnectorPropertyValue implements ConnectorPropertyValue { + + private final String rawValue; + + public StandardConnectorPropertyValue(final String rawValue) { + this.rawValue = rawValue; + } + + @Override + public String getValue() { + return rawValue; + } + + @Override + public Integer asInteger() { + return rawValue == null ? null : Integer.parseInt(rawValue); + } + + @Override + public Long asLong() { + return rawValue == null ? null : Long.parseLong(rawValue); + } + + @Override + public Boolean asBoolean() { + return rawValue == null ? null : Boolean.parseBoolean(rawValue); + } + + @Override + public Float asFloat() { + return rawValue == null ? null : Float.parseFloat(rawValue); + } + + @Override + public Double asDouble() { + return rawValue == null ? null : Double.parseDouble(rawValue); + } + + @Override + public Long asTimePeriod(final TimeUnit timeUnit) { + return rawValue == null ? null : FormatUtils.getTimeDuration(rawValue, timeUnit); + } + + @Override + public Duration asDuration() { + return rawValue == null ? null : Duration.ofNanos(asTimePeriod(TimeUnit.NANOSECONDS)); + } + + @Override + public Double asDataSize(final DataUnit dataUnit) { + return rawValue == null ? null : DataUnit.parseDataSize(rawValue.trim(), dataUnit); + } + + @Override + public List asList() { + if (rawValue == null) { + return Collections.emptyList(); + } + + final String[] splits = rawValue.split(","); + final List values = new ArrayList<>(splits.length); + for (final String split : splits) { + values.add(split.trim()); + } + + return values; + } + + @Override + public boolean isSet() { + return rawValue != null; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorRepoInitializationContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorRepoInitializationContext.java new file mode 100644 index 000000000000..a0661fa58f92 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorRepoInitializationContext.java @@ -0,0 +1,85 @@ +/* + * 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.components.connector; + +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.connector.secrets.SecretsManager; +import org.apache.nifi.controller.NodeTypeProvider; +import org.apache.nifi.controller.flow.FlowManager; +import org.apache.nifi.nar.ExtensionManager; + +public class StandardConnectorRepoInitializationContext implements ConnectorRepositoryInitializationContext { + private final FlowManager flowManager; + private final ExtensionManager extensionManager; + private final SecretsManager secretsManager; + private final AssetManager assetManager; + private final NodeTypeProvider nodeTypeProvider; + private final ConnectorRequestReplicator requestReplicator; + private final ConnectorConfigurationProvider connectorConfigurationProvider; + + public StandardConnectorRepoInitializationContext(final FlowManager flowManager, + final ExtensionManager extensionManager, + final SecretsManager secretsManager, + final AssetManager assetManager, + final NodeTypeProvider nodeTypeProvider, + final ConnectorRequestReplicator requestReplicator, + final ConnectorConfigurationProvider connectorConfigurationProvider) { + this.flowManager = flowManager; + this.extensionManager = extensionManager; + this.secretsManager = secretsManager; + this.assetManager = assetManager; + this.nodeTypeProvider = nodeTypeProvider; + this.requestReplicator = requestReplicator; + this.connectorConfigurationProvider = connectorConfigurationProvider; + } + + @Override + public FlowManager getFlowManager() { + return flowManager; + } + + @Override + public ExtensionManager getExtensionManager() { + return extensionManager; + } + + @Override + public SecretsManager getSecretsManager() { + return secretsManager; + } + + @Override + public AssetManager getAssetManager() { + return assetManager; + } + + @Override + public NodeTypeProvider getNodeTypeProvider() { + return nodeTypeProvider; + } + + @Override + public ConnectorRequestReplicator getRequestReplicator() { + return requestReplicator; + } + + @Override + public ConnectorConfigurationProvider getConnectorConfigurationProvider() { + return connectorConfigurationProvider; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorRepository.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorRepository.java new file mode 100644 index 000000000000..2df018c63825 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorRepository.java @@ -0,0 +1,594 @@ +/* + * 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.components.connector; + +import org.apache.nifi.annotation.lifecycle.OnRemoved; +import org.apache.nifi.asset.Asset; +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.connector.secrets.SecretsManager; +import org.apache.nifi.engine.FlowEngine; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.VersionedConfigurationStep; +import org.apache.nifi.flow.VersionedConnectorValueReference; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.nar.NarCloseable; +import org.apache.nifi.util.ReflectionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +public class StandardConnectorRepository implements ConnectorRepository { + private static final Logger logger = LoggerFactory.getLogger(StandardConnectorRepository.class); + + private final Map connectors = new ConcurrentHashMap<>(); + private final FlowEngine lifecycleExecutor = new FlowEngine(8, "NiFi Connector Lifecycle"); + + private volatile ExtensionManager extensionManager; + private volatile ConnectorRequestReplicator requestReplicator; + private volatile SecretsManager secretsManager; + private volatile AssetManager assetManager; + private volatile ConnectorConfigurationProvider configurationProvider; + + @Override + public void initialize(final ConnectorRepositoryInitializationContext context) { + logger.debug("Initializing ConnectorRepository"); + this.extensionManager = context.getExtensionManager(); + this.requestReplicator = context.getRequestReplicator(); + this.secretsManager = context.getSecretsManager(); + this.assetManager = context.getAssetManager(); + this.configurationProvider = context.getConnectorConfigurationProvider(); + logger.debug("Successfully initialized ConnectorRepository with configurationProvider={}", configurationProvider != null ? configurationProvider.getClass().getSimpleName() : "null"); + } + + @Override + public void verifyCreate(final String connectorId) { + if (connectors.containsKey(connectorId)) { + throw new IllegalStateException("A Connector already exists with ID %s".formatted(connectorId)); + } + if (configurationProvider != null) { + configurationProvider.verifyCreate(connectorId); + } + } + + @Override + public void addConnector(final ConnectorNode connector) { + syncFromProvider(connector); + connectors.put(connector.getIdentifier(), connector); + } + + @Override + public void restoreConnector(final ConnectorNode connector) { + connectors.put(connector.getIdentifier(), connector); + logger.debug("Successfully restored {}", connector); + } + + @Override + public void removeConnector(final String connectorId) { + logger.debug("Removing {}", connectorId); + final ConnectorNode connectorNode = connectors.get(connectorId); + if (connectorNode == null) { + throw new IllegalStateException("No connector found with ID " + connectorId); + } + + connectorNode.verifyCanDelete(); + if (configurationProvider != null) { + configurationProvider.delete(connectorId); + } + connectors.remove(connectorId); + + final Class taskClass = connectorNode.getConnector().getClass(); + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, taskClass, connectorId)) { + ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, connectorNode.getConnector()); + } + + extensionManager.removeInstanceClassLoader(connectorId); + } + + @Override + public ConnectorNode getConnector(final String identifier) { + final ConnectorNode connector = connectors.get(identifier); + if (connector != null) { + syncFromProvider(connector); + } + return connector; + } + + @Override + public List getConnectors() { + final List connectorList = List.copyOf(connectors.values()); + for (final ConnectorNode connector : connectorList) { + syncFromProvider(connector); + } + return connectorList; + } + + @Override + public Future startConnector(final ConnectorNode connector) { + return connector.start(lifecycleExecutor); + } + + @Override + public Future stopConnector(final ConnectorNode connector) { + return connector.stop(lifecycleExecutor); + } + + @Override + public Future restartConnector(final ConnectorNode connector) { + final CompletableFuture restartCompleteFuture = new CompletableFuture<>(); + restartConnector(connector, restartCompleteFuture); + return restartCompleteFuture; + } + + private void restartConnector(final ConnectorNode connector, final CompletableFuture restartCompleteFuture) { + try { + final Future stopFuture = connector.stop(lifecycleExecutor); + stopFuture.get(); + + final Future startFuture = connector.start(lifecycleExecutor); + startFuture.get(); + + logger.info("Successfully restarted connector [{}]", connector.getIdentifier()); + restartCompleteFuture.complete(null); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + logger.error("Interrupted while restarting connector [{}]", connector.getIdentifier(), e); + restartCompleteFuture.completeExceptionally(e); + } catch (final ExecutionException e) { + logger.error("Failed to restart connector [{}]", connector.getIdentifier(), e.getCause()); + restartCompleteFuture.completeExceptionally(e); + } + } + + @Override + public void applyUpdate(final ConnectorNode connector, final ConnectorUpdateContext context) throws FlowUpdateException { + logger.debug("Applying update to {}", connector); + + if (configurationProvider != null && !configurationProvider.shouldApplyUpdate(connector.getIdentifier())) { + logger.info("ConnectorConfigurationProvider indicated framework should not apply update for {}; skipping framework update process", connector); + return; + } + + final ConnectorState initialDesiredState = connector.getDesiredState(); + logger.info("Applying update to Connector {}", connector); + + // Sync asset binaries first (provider downloads changed assets, assigns new UUIDs), + // then load the updated configuration (which will reflect any new NiFi UUIDs). + syncAssetsFromProvider(connector); + + // Transition the connector's state to PREPARING_FOR_UPDATE before starting the background process. + // This allows us to ensure that if we poll and see the state in the same state it was in before that + // we know the update has already completed (successfully or otherwise). + logger.debug("Transitioning {} to PREPARING_FOR_UPDATE state before applying update", connector); + connector.transitionStateForUpdating(); + logger.debug("{} is now in PREPARING_FOR_UPDATE state", connector); + + // Update connector in a background thread. This will handle transitioning the Connector state appropriately + // so that it's clear when the update has completed. + lifecycleExecutor.submit(() -> { + updateConnector(connector, initialDesiredState, context); + cleanUpAssets(connector); + }); + } + + private void updateConnector(final ConnectorNode connector, final ConnectorState initialDesiredState, final ConnectorUpdateContext context) { + logger.debug("Updating {}", connector); + try { + // Perform whatever preparation is necessary for the update. Default implementation is to stop the connector. + logger.debug("Preparing {} for update", connector); + connector.prepareForUpdate(); + + // Wait for Connector State to become UPDATING + logger.debug("Waiting for {} to transition to UPDATING state", connector); + waitForState(connector, Set.of(ConnectorState.UPDATING), Set.of(ConnectorState.PREPARING_FOR_UPDATE)); + + // Apply the update to the connector. + logger.info("{} has now completed preparations for update; applying update now", connector); + connector.applyUpdate(); + logger.info("{} has successfully applied update", connector); + + // Now that the update has been applied, save the flow so that the updated configuration is persisted. + context.saveFlow(); + + // Wait for all nodes to complete the update. + waitForState(connector, Set.of(ConnectorState.UPDATED), Set.of(ConnectorState.UPDATING)); + logger.info("{} has successfully completed update on all nodes", connector); + + // If the initial desired state was RUNNING, start the connector again. Otherwise, stop it. + // We don't simply leave it be as the prepareForUpdate / update may have changed the state of some components. + if (initialDesiredState == ConnectorState.RUNNING) { + logger.info("{} has been successfully updated; starting to resume initial state", connector); + connector.start(lifecycleExecutor); + } else { + logger.info("{} has been successfully updated; stopping to resume initial state", connector); + connector.stop(lifecycleExecutor); + } + + // We've updated the state of the connector so save flow again + context.saveFlow(); + } catch (final Exception e) { + logger.error("Failed to apply update for {}", connector, e); + connector.abortUpdate(e); + } + } + + private void waitForState(final ConnectorNode connector, final Set desiredStates, final Set allowableStates) + throws FlowUpdateException, IOException, InterruptedException { + + // Wait for Connector State to become the desired state + int iterations = 0; + final long startNanos = System.nanoTime(); + while (true) { + final ConnectorState clusterState = requestReplicator.getState(connector.getIdentifier()); + if (desiredStates.contains(clusterState)) { + logger.info("State for {} is now {}", connector, clusterState); + break; + } else if (allowableStates.contains(clusterState)) { + final long elapsedSeconds = Duration.ofNanos(System.nanoTime() - startNanos).toSeconds(); + if (++iterations % 10 == 0) { + logger.info("Waiting for {} to transition to one of {}. Current state is {}; elapsed time = {} secs", connector, desiredStates, clusterState, elapsedSeconds); + } else { + logger.debug("Waiting for {} to transition to one of {}. Current state is {}; elapsed time = {} secs", connector, desiredStates, clusterState, elapsedSeconds); + } + + Thread.sleep(Duration.ofSeconds(1)); + continue; + } else if (clusterState == ConnectorState.UPDATE_FAILED) { + throw new FlowUpdateException("State of " + connector + " transitioned to UPDATE_FAILED while waiting for state to become one of " + desiredStates); + } + + throw new FlowUpdateException("While waiting for %s to transition to state in set %s, connector transitioned to unexpected state: %s".formatted(connector, desiredStates, clusterState)); + } + } + + private void cleanUpAssets(final ConnectorNode connector) { + final String connectorId = connector.getIdentifier(); + final Set referencedAssetIds = new HashSet<>(); + collectReferencedAssetIds(connector.getActiveFlowContext(), referencedAssetIds); + collectReferencedAssetIds(connector.getWorkingFlowContext(), referencedAssetIds); + + logger.debug("Found {} assets referenced for Connector [{}]", referencedAssetIds.size(), connectorId); + + final List allConnectorAssets = assetManager.getAssets(connectorId); + for (final Asset asset : allConnectorAssets) { + final String assetId = asset.getIdentifier(); + if (!referencedAssetIds.contains(assetId)) { + try { + logger.info("Deleting unreferenced asset [id={},name={}] for connector [{}]", assetId, asset.getName(), connectorId); + + if (configurationProvider != null) { + // Provider deletes from external store and local AssetManager + configurationProvider.deleteAsset(connectorId, assetId); + } else { + assetManager.deleteAsset(assetId); + } + } catch (final Exception e) { + logger.warn("Unable to delete unreferenced asset [id={},name={}] for connector [{}]", assetId, asset.getName(), connectorId, e); + } + } + } + } + + private void collectReferencedAssetIds(final FrameworkFlowContext flowContext, final Set referencedAssetIds) { + if (flowContext == null) { + return; + } + + final ConnectorConfiguration configuration = flowContext.getConfigurationContext().toConnectorConfiguration(); + for (final NamedStepConfiguration namedStepConfiguration : configuration.getNamedStepConfigurations()) { + final StepConfiguration stepConfiguration = namedStepConfiguration.configuration(); + final Map stepPropertyValues = stepConfiguration.getPropertyValues(); + if (stepPropertyValues == null) { + continue; + } + for (final ConnectorValueReference valueReference : stepPropertyValues.values()) { + if (valueReference instanceof AssetReference assetReference) { + referencedAssetIds.addAll(assetReference.getAssetIdentifiers()); + } + } + } + } + + @Override + public void updateConnector(final ConnectorNode connector, final String name) { + if (configurationProvider != null) { + final ConnectorWorkingConfiguration workingConfiguration = buildWorkingConfiguration(connector); + workingConfiguration.setName(name); + configurationProvider.save(connector.getIdentifier(), workingConfiguration); + } + connector.setName(name); + } + + @Override + public void configureConnector(final ConnectorNode connector, final String stepName, final StepConfiguration configuration) throws FlowUpdateException { + if (configurationProvider != null) { + final ConnectorWorkingConfiguration mergedConfiguration = buildMergedWorkingConfiguration(connector, stepName, configuration); + configurationProvider.save(connector.getIdentifier(), mergedConfiguration); + } + + connector.setConfiguration(stepName, configuration); + logger.info("Successfully configured {} for step {}", connector, stepName); + } + + @Override + public void inheritConfiguration(final ConnectorNode connector, final List activeFlowConfiguration, + final List workingFlowConfiguration, final Bundle flowContextBundle) throws FlowUpdateException { + + logger.debug("Inheriting configuration for {}", connector); + connector.transitionStateForUpdating(); + connector.prepareForUpdate(); + + try { + connector.inheritConfiguration(activeFlowConfiguration, workingFlowConfiguration, flowContextBundle); + logger.debug("Successfully inherited configuration for {}", connector); + } catch (final Exception e) { + logger.error("Failed to inherit configuration for {}", connector, e); + connector.abortUpdate(e); + connector.markInvalid("Flow Update Failure", "The flow could not be updated: " + e.getMessage()); + throw e; + } + } + + @Override + public void discardWorkingConfiguration(final ConnectorNode connector) { + if (configurationProvider != null) { + configurationProvider.discard(connector.getIdentifier()); + } + connector.discardWorkingConfiguration(); + cleanUpAssets(connector); + } + + @Override + public SecretsManager getSecretsManager() { + return secretsManager; + } + + @Override + public ConnectorStateTransition createStateTransition(final String type, final String id) { + final String componentDescription = "StandardConnectorNode[id=" + id + ", type=" + type + "]"; + return new StandardConnectorStateTransition(componentDescription); + } + + @Override + public FrameworkConnectorInitializationContextBuilder createInitializationContextBuilder() { + return new StandardConnectorInitializationContext.Builder(); + } + + @Override + public Asset storeAsset(final String connectorId, final String assetId, final String assetName, final InputStream content) throws IOException { + if (configurationProvider == null) { + return assetManager.saveAsset(connectorId, assetId, assetName, content); + } + + // When a provider is configured, delegate entirely to the provider, which should write to the AssetManager and sync to the external store. + try { + configurationProvider.storeAsset(connectorId, assetId, assetName, content); + } catch (final IOException e) { + throw e; + } catch (final Exception e) { + throw new IOException("Failed to store asset [" + assetName + "] to provider for connector [" + connectorId + "]", e); + } + logger.debug("Stored asset [nifiId={}, name={}] for connector [{}]", assetId, assetName, connectorId); + + return assetManager.getAsset(assetId) + .orElseThrow(() -> new IOException("Asset [" + assetId + "] not found after storing for connector [" + connectorId + "]")); + } + + @Override + public Optional getAsset(final String assetId) { + return assetManager.getAsset(assetId); + } + + @Override + public List getAssets(final String connectorId) { + return assetManager.getAssets(connectorId); + } + + @Override + public void deleteAssets(final String connectorId) { + final List assets = assetManager.getAssets(connectorId); + for (final Asset asset : assets) { + try { + if (configurationProvider != null) { + // When a provider is configured, delegate entirely to the provider, which should delete from the AssetManager and sync to the external store. + configurationProvider.deleteAsset(connectorId, asset.getIdentifier()); + } else { + assetManager.deleteAsset(asset.getIdentifier()); + } + } catch (final Exception e) { + logger.warn("Failed to delete asset [nifiUuid={}] for connector [{}]", asset.getIdentifier(), connectorId, e); + } + } + } + + @Override + public void syncAssetsFromProvider(final ConnectorNode connector) { + if (configurationProvider == null) { + return; + } + + final String connectorId = connector.getIdentifier(); + + // Step 1: Sync Connector Assets from Provider + logger.debug("Syncing assets from provider for connector [{}]", connectorId); + configurationProvider.syncAssets(connectorId); + + // Step 2: Sync ConnectorNode Configuration from Provider + syncFromProvider(connector); + } + + private void syncFromProvider(final ConnectorNode connector) { + if (configurationProvider == null) { + return; + } + + final String connectorId = connector.getIdentifier(); + final Optional externalConfig = configurationProvider.load(connectorId); + if (externalConfig.isEmpty()) { + return; + } + + final ConnectorWorkingConfiguration config = externalConfig.get(); + if (config.getName() != null) { + connector.setName(config.getName()); + } + + final List workingFlowConfiguration = config.getWorkingFlowConfiguration(); + if (workingFlowConfiguration != null) { + final MutableConnectorConfigurationContext workingConfigContext = connector.getWorkingFlowContext().getConfigurationContext(); + for (final VersionedConfigurationStep step : workingFlowConfiguration) { + final StepConfiguration stepConfiguration = toStepConfiguration(step); + workingConfigContext.replaceProperties(step.getName(), stepConfiguration); + } + } + } + + private ConnectorWorkingConfiguration buildWorkingConfiguration(final ConnectorNode connector) { + final ConnectorWorkingConfiguration config = new ConnectorWorkingConfiguration(); + config.setName(connector.getName()); + config.setWorkingFlowConfiguration(buildVersionedConfigurationSteps(connector.getWorkingFlowContext())); + return config; + } + + private List buildVersionedConfigurationSteps(final FrameworkFlowContext flowContext) { + if (flowContext == null) { + return List.of(); + } + + final ConnectorConfiguration configuration = flowContext.getConfigurationContext().toConnectorConfiguration(); + final List configurationSteps = new ArrayList<>(); + + for (final NamedStepConfiguration namedStepConfiguration : configuration.getNamedStepConfigurations()) { + final VersionedConfigurationStep versionedConfigurationStep = new VersionedConfigurationStep(); + versionedConfigurationStep.setName(namedStepConfiguration.stepName()); + versionedConfigurationStep.setProperties(toVersionedProperties(namedStepConfiguration.configuration())); + configurationSteps.add(versionedConfigurationStep); + } + + return configurationSteps; + } + + private ConnectorWorkingConfiguration buildMergedWorkingConfiguration(final ConnectorNode connector, final String stepName, final StepConfiguration incomingConfiguration) { + final ConnectorWorkingConfiguration existingConfig; + if (configurationProvider != null) { + final Optional externalConfig = configurationProvider.load(connector.getIdentifier()); + existingConfig = externalConfig.orElseGet(() -> buildWorkingConfiguration(connector)); + } else { + existingConfig = buildWorkingConfiguration(connector); + } + + final List existingSteps = existingConfig.getWorkingFlowConfiguration() != null + ? new ArrayList<>(existingConfig.getWorkingFlowConfiguration()) + : new ArrayList<>(); + + VersionedConfigurationStep targetStep = null; + for (final VersionedConfigurationStep step : existingSteps) { + if (stepName.equals(step.getName())) { + targetStep = step; + break; + } + } + + if (targetStep == null) { + targetStep = new VersionedConfigurationStep(); + targetStep.setName(stepName); + targetStep.setProperties(new HashMap<>()); + existingSteps.add(targetStep); + } + + final Map mergedProperties = targetStep.getProperties() != null + ? new HashMap<>(targetStep.getProperties()) + : new HashMap<>(); + + for (final Map.Entry entry : incomingConfiguration.getPropertyValues().entrySet()) { + if (entry.getValue() != null) { + mergedProperties.put(entry.getKey(), toVersionedValueReference(entry.getValue())); + } + } + targetStep.setProperties(mergedProperties); + + existingConfig.setWorkingFlowConfiguration(existingSteps); + return existingConfig; + } + + private Map toVersionedProperties(final StepConfiguration configuration) { + final Map versionedProperties = new HashMap<>(); + for (final Map.Entry entry : configuration.getPropertyValues().entrySet()) { + final ConnectorValueReference valueReference = entry.getValue(); + if (valueReference != null) { + versionedProperties.put(entry.getKey(), toVersionedValueReference(valueReference)); + } + } + return versionedProperties; + } + + private VersionedConnectorValueReference toVersionedValueReference(final ConnectorValueReference valueReference) { + final VersionedConnectorValueReference versionedReference = new VersionedConnectorValueReference(); + versionedReference.setValueType(valueReference.getValueType().name()); + + switch (valueReference) { + case StringLiteralValue stringLiteral -> versionedReference.setValue(stringLiteral.getValue()); + case AssetReference assetRef -> versionedReference.setAssetIds(assetRef.getAssetIdentifiers()); + case SecretReference secretRef -> { + versionedReference.setProviderId(secretRef.getProviderId()); + versionedReference.setProviderName(secretRef.getProviderName()); + versionedReference.setSecretName(secretRef.getSecretName()); + versionedReference.setFullyQualifiedSecretName(secretRef.getFullyQualifiedName()); + } + } + + return versionedReference; + } + + private StepConfiguration toStepConfiguration(final VersionedConfigurationStep step) { + final Map propertyValues = new HashMap<>(); + final Map versionedProperties = step.getProperties(); + if (versionedProperties != null) { + for (final Map.Entry entry : versionedProperties.entrySet()) { + final VersionedConnectorValueReference versionedRef = entry.getValue(); + if (versionedRef != null) { + propertyValues.put(entry.getKey(), toConnectorValueReference(versionedRef)); + } + } + } + return new StepConfiguration(propertyValues); + } + + private ConnectorValueReference toConnectorValueReference(final VersionedConnectorValueReference versionedReference) { + final ConnectorValueType valueType = ConnectorValueType.valueOf(versionedReference.getValueType()); + return switch (valueType) { + case STRING_LITERAL -> new StringLiteralValue(versionedReference.getValue()); + case ASSET_REFERENCE -> new AssetReference(versionedReference.getAssetIds()); + case SECRET_REFERENCE -> new SecretReference(versionedReference.getProviderId(), versionedReference.getProviderName(), + versionedReference.getSecretName(), versionedReference.getFullyQualifiedSecretName()); + }; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorStateTransition.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorStateTransition.java new file mode 100644 index 000000000000..9cfd4cccb382 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorStateTransition.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.components.connector; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; + +public class StandardConnectorStateTransition implements ConnectorStateTransition { + private static final Logger logger = LoggerFactory.getLogger(StandardConnectorStateTransition.class); + + private final String componentDescription; + private final AtomicReference currentState = new AtomicReference<>(ConnectorState.STOPPED); + private final AtomicReference desiredState = new AtomicReference<>(ConnectorState.STOPPED); + private final List> pendingStartFutures = new ArrayList<>(); + private final List> pendingStopFutures = new ArrayList<>(); + + public StandardConnectorStateTransition(final String componentDescription) { + this.componentDescription = componentDescription; + } + + @Override + public ConnectorState getCurrentState() { + return currentState.get(); + } + + @Override + public ConnectorState getDesiredState() { + return desiredState.get(); + } + + @Override + public void setDesiredState(final ConnectorState desiredState) { + this.desiredState.set(desiredState); + logger.info("Desired State for {} set to {}", componentDescription, desiredState); + } + + @Override + public boolean trySetCurrentState(final ConnectorState expectedState, final ConnectorState newState) { + final boolean changed = currentState.compareAndSet(expectedState, newState); + if (changed) { + logger.info("Transitioned current state for {} from {} to {}", componentDescription, expectedState, newState); + completeFuturesForStateTransition(newState); + } + + return changed; + } + + @Override + public void setCurrentState(final ConnectorState newState) { + final ConnectorState oldState = currentState.getAndSet(newState); + logger.info("Transitioned current state for {} from {} to {}", componentDescription, oldState, newState); + completeFuturesForStateTransition(newState); + } + + @Override + public synchronized void addPendingStartFuture(final CompletableFuture future) { + pendingStartFutures.add(future); + } + + @Override + public synchronized void addPendingStopFuture(final CompletableFuture future) { + pendingStopFutures.add(future); + } + + private synchronized void completeFuturesForStateTransition(final ConnectorState newState) { + if (newState == ConnectorState.RUNNING) { + final List> futuresToComplete = new ArrayList<>(pendingStartFutures); + pendingStartFutures.clear(); + + for (final CompletableFuture future : futuresToComplete) { + future.complete(null); + } + + if (!futuresToComplete.isEmpty()) { + logger.debug("Completed {} pending start futures for {}", futuresToComplete.size(), componentDescription); + } + } + + if (newState == ConnectorState.STOPPED) { + final List> futuresToComplete = new ArrayList<>(pendingStopFutures); + pendingStopFutures.clear(); + + for (final CompletableFuture future : futuresToComplete) { + future.complete(null); + } + + if (!futuresToComplete.isEmpty()) { + logger.debug("Completed {} pending stop futures for {}", futuresToComplete.size(), componentDescription); + } + } + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorValidationContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorValidationContext.java new file mode 100644 index 000000000000..61d36dfe8c13 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorValidationContext.java @@ -0,0 +1,107 @@ +/* + * 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.components.connector; + +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.connector.components.ParameterContextFacade; +import org.apache.nifi.parameter.Parameter; +import org.apache.nifi.parameter.ParameterLookup; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +public class StandardConnectorValidationContext implements ConnectorValidationContext { + private final DescribedValueProvider allowableValueProvider; + private final Map stepConfigurations; + private final ParameterLookup parameterLookup; + + public StandardConnectorValidationContext(final ConnectorConfiguration connectorConfiguration, final DescribedValueProvider allowableValueProvider, + final ParameterContextFacade parameterContextFacade) { + + this.allowableValueProvider = allowableValueProvider; + + stepConfigurations = new HashMap<>(); + for (final NamedStepConfiguration stepConfiguration : connectorConfiguration.getNamedStepConfigurations()) { + stepConfigurations.put(stepConfiguration.stepName(), stepConfiguration.configuration()); + } + + parameterLookup = new ParameterLookupBridge(parameterContextFacade); + } + + @Override + public ValidationContext createValidationContext(final String stepName, final String propertyGroupName) { + final StepConfiguration stepConfig = stepConfigurations.get(stepName); + final Map stepProperties = stepConfig != null ? stepConfig.getPropertyValues() : Collections.emptyMap(); + + final Map stringValues = new HashMap<>(); + for (final Map.Entry entry : stepProperties.entrySet()) { + final ConnectorValueReference valueRef = entry.getValue(); + + if (valueRef instanceof StringLiteralValue stringLiteral) { + stringValues.put(entry.getKey(), stringLiteral.getValue()); + } else { + stringValues.put(entry.getKey(), null); + } + } + + return new ConnectorValidationContextBridge(stringValues, parameterLookup); + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName) { + return allowableValueProvider.fetchAllowableValues(stepName, propertyName); + } + + private static class ParameterLookupBridge implements ParameterLookup { + private final ParameterContextFacade parameterContextFacade; + + public ParameterLookupBridge(final ParameterContextFacade parameterContextFacade) { + this.parameterContextFacade = parameterContextFacade; + } + + @Override + public Optional getParameter(final String parameterName) { + final String parameterValue = parameterContextFacade.getValue(parameterName); + if (parameterValue == null) { + return Optional.empty(); + } + + final Parameter parameter = new Parameter.Builder() + .name(parameterName) + .provided(false) + .sensitive(parameterContextFacade.isSensitive(parameterName)) + .value(parameterValue) + .build(); + return Optional.of(parameter); + } + + @Override + public boolean isEmpty() { + return parameterContextFacade.getDefinedParameterNames().isEmpty(); + } + + @Override + public long getVersion() { + return 0; + } + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardFlowContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardFlowContext.java new file mode 100644 index 000000000000..3d56a999af81 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardFlowContext.java @@ -0,0 +1,130 @@ +/* + * 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.components.connector; + +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.connector.components.FlowContextType; +import org.apache.nifi.components.connector.components.ParameterContextFacade; +import org.apache.nifi.components.connector.components.ProcessGroupFacade; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.parameter.ParameterContext; + +import java.util.HashMap; +import java.util.Map; + +public class StandardFlowContext implements FrameworkFlowContext { + private final ProcessGroup managedProcessGroup; + private final MutableConnectorConfigurationContext configurationContext; + private final ProcessGroupFacadeFactory groupFacadeFactory; + private final ParameterContextFacadeFactory parameterContextFacadeFactory; + private final ComponentLog connectorLog; + private final FlowContextType flowContextType; + private final Bundle bundle; + + private volatile ProcessGroupFacade rootGroup; + private volatile ParameterContextFacade parameterContext; + + + public StandardFlowContext(final ProcessGroup managedProcessGroup, final MutableConnectorConfigurationContext configurationContext, + final ProcessGroupFacadeFactory groupFacadeFactory, final ParameterContextFacadeFactory parameterContextFacadeFactory, + final ComponentLog connectorLog, final FlowContextType flowContextType, + final Bundle bundle) { + + this.managedProcessGroup = managedProcessGroup; + this.groupFacadeFactory = groupFacadeFactory; + this.parameterContextFacadeFactory = parameterContextFacadeFactory; + this.connectorLog = connectorLog; + this.configurationContext = configurationContext; + this.flowContextType = flowContextType; + this.bundle = bundle; + + this.rootGroup = groupFacadeFactory.create(managedProcessGroup, connectorLog); + this.parameterContext = parameterContextFacadeFactory.create(managedProcessGroup); + } + + @Override + public ProcessGroupFacade getRootGroup() { + return rootGroup; + } + + @Override + public ParameterContextFacade getParameterContext() { + return parameterContext; + } + + @Override + public MutableConnectorConfigurationContext getConfigurationContext() { + return configurationContext; + } + + @Override + public void updateFlow(final VersionedExternalFlow versionedExternalFlow, final AssetManager assetManager) throws FlowUpdateException { + if (versionedExternalFlow.getParameterContexts() == null) { + versionedExternalFlow.setParameterContexts(new HashMap<>()); + } + + try { + managedProcessGroup.verifyCanUpdate(versionedExternalFlow, true, false); + } catch (final IllegalStateException e) { + throw new FlowUpdateException("Flow is not in a state that allows the requested updated", e); + } + + final ParameterContext managedGroupParameterContext = managedProcessGroup.getParameterContext(); + updateParameterContextNames(versionedExternalFlow.getFlowContents(), managedGroupParameterContext.getName()); + + final VersionedExternalFlow externalFlowWithResolvedParameters = new VersionedExternalFlow(); + externalFlowWithResolvedParameters.setFlowContents(versionedExternalFlow.getFlowContents()); + externalFlowWithResolvedParameters.setParameterContexts(Map.of()); + + managedProcessGroup.updateFlow(externalFlowWithResolvedParameters, managedProcessGroup.getIdentifier(), false, true, true); + + rootGroup = groupFacadeFactory.create(managedProcessGroup, connectorLog); + + final ConnectorParameterLookup parameterLookup = new ConnectorParameterLookup(versionedExternalFlow.getParameterContexts().values(), assetManager); + getParameterContext().updateParameters(parameterLookup.getParameterValues()); + parameterContext = parameterContextFacadeFactory.create(managedProcessGroup); + } + + private void updateParameterContextNames(final VersionedProcessGroup group, final String parameterContextName) { + group.setParameterContextName(parameterContextName); + if (group.getProcessGroups() != null) { + for (final VersionedProcessGroup childGroup : group.getProcessGroups()) { + updateParameterContextNames(childGroup, parameterContextName); + } + } + } + + @Override + public Bundle getBundle() { + return bundle; + } + + @Override + public FlowContextType getType() { + return flowContextType; + } + + @Override + public ProcessGroup getManagedProcessGroup() { + return managedProcessGroup; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardStepConfigurationContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardStepConfigurationContext.java new file mode 100644 index 000000000000..b728033d899d --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardStepConfigurationContext.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.components.connector; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class StandardStepConfigurationContext implements StepConfigurationContext { + private final String stepName; + private final ConnectorConfigurationContext parentContext; + + public StandardStepConfigurationContext(final String stepName, final ConnectorConfigurationContext parentContext) { + this.stepName = stepName; + this.parentContext = parentContext; + } + + @Override + public ConnectorPropertyValue getProperty(final String propertyName) { + return parentContext.getProperty(stepName, propertyName); + } + + @Override + public ConnectorPropertyValue getProperty(final ConnectorPropertyDescriptor propertyDescriptor) { + return parentContext.getProperty(stepName, propertyDescriptor.getName()); + } + + @Override + public StepConfigurationContext createWithOverrides(final Map propertyValues) { + final ConnectorConfigurationContext updatedContext = parentContext.createWithOverrides(stepName, propertyValues); + return updatedContext.scopedToStep(stepName); + } + + @Override + public Map getProperties() { + final Map properties = new HashMap<>(); + final Set propertyNames = parentContext.getPropertyNames(stepName); + for (final String propertyName : propertyNames) { + properties.put(propertyName, getProperty(propertyName)); + } + + return properties; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/ComponentContextProvider.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/ComponentContextProvider.java new file mode 100644 index 000000000000..ed03fc21b4aa --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/ComponentContextProvider.java @@ -0,0 +1,41 @@ +/* + * 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.components.connector.facades.standalone; + +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.parameter.ParameterLookup; +import org.apache.nifi.processor.ProcessContext; + +import java.util.Map; + +public interface ComponentContextProvider { + ProcessContext createProcessContext(ProcessorNode processorNode, ParameterLookup parameterLookup); + + ProcessContext createProcessContext(ProcessorNode processorNode, Map propertiesOverride, ParameterLookup parameterLookup); + + ValidationContext createValidationContext(ProcessorNode processorNode, Map properties, ParameterLookup parameterLookup); + + ConfigurationContext createConfigurationContext(ControllerServiceNode serviceNode, ParameterLookup parameterLookup); + + ConfigurationContext createConfigurationContext(ControllerServiceNode serviceNode, Map propertiesOverride, ParameterLookup parameterLookup); + + ValidationContext createValidationContext(ControllerServiceNode serviceNode, Map properties, ParameterLookup parameterLookup); +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/IllegalExecutionEngineStatelessGroupLifecycle.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/IllegalExecutionEngineStatelessGroupLifecycle.java new file mode 100644 index 000000000000..79e25e0d1a93 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/IllegalExecutionEngineStatelessGroupLifecycle.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.components.connector.facades.standalone; + +import org.apache.nifi.components.connector.components.StatelessGroupLifecycle; +import org.apache.nifi.groups.ProcessGroup; + +import java.util.concurrent.CompletableFuture; + +public class IllegalExecutionEngineStatelessGroupLifecycle implements StatelessGroupLifecycle { + private final ProcessGroup processGroup; + + public IllegalExecutionEngineStatelessGroupLifecycle(final ProcessGroup processGroup) { + this.processGroup = processGroup; + } + + @Override + public CompletableFuture start() { + throw new IllegalStateException("Cannot start " + processGroup + " as a Stateless Group because the Process Group is not configured to run using the Stateless Execution Engine"); + } + + @Override + public CompletableFuture stop() { + throw new IllegalStateException("Cannot stop " + processGroup + " as a Stateless Group because the Process Group is not configured to run using the Stateless Execution Engine"); + } + + @Override + public CompletableFuture terminate() { + throw new IllegalStateException("Cannot terminate " + processGroup + " as a Stateless Group because the Process Group is not configured to run using the Stateless Execution Engine"); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneConnectionFacade.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneConnectionFacade.java new file mode 100644 index 000000000000..cc21353731d5 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneConnectionFacade.java @@ -0,0 +1,82 @@ +/* + * 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.components.connector.facades.standalone; + +import org.apache.nifi.components.connector.DropFlowFileSummary; +import org.apache.nifi.components.connector.components.ConnectionFacade; +import org.apache.nifi.connectable.Connection; +import org.apache.nifi.controller.queue.QueueSize; +import org.apache.nifi.flow.VersionedConnection; +import org.apache.nifi.flowfile.FlowFile; + +import java.io.IOException; +import java.util.Objects; +import java.util.function.Predicate; + +public class StandaloneConnectionFacade implements ConnectionFacade { + private final Connection connection; + private final VersionedConnection versionedConnection; + + public StandaloneConnectionFacade(final Connection connection, final VersionedConnection versionedConnection) { + this.connection = connection; + this.versionedConnection = versionedConnection; + } + + @Override + public VersionedConnection getDefinition() { + return versionedConnection; + } + + @Override + public QueueSize getQueueSize() { + return connection.getFlowFileQueue().size(); + } + + @Override + public void purge() { + // TODO: Require arguments here + connection.getFlowFileQueue().dropFlowFiles("User requested purge", "User"); + } + + @Override + public DropFlowFileSummary dropFlowFiles(final Predicate predicate) throws IOException { + return connection.getFlowFileQueue().dropFlowFiles(predicate); + } + + @Override + public String toString() { + return "StandaloneConnectionFacade[id=" + versionedConnection.getIdentifier() + ", name=" + versionedConnection.getName() + "]"; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final StandaloneConnectionFacade that = (StandaloneConnectionFacade) o; + return Objects.equals(versionedConnection.getIdentifier(), that.versionedConnection.getIdentifier()); + } + + @Override + public int hashCode() { + return Objects.hashCode(versionedConnection.getIdentifier()); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneControllerServiceFacade.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneControllerServiceFacade.java new file mode 100644 index 000000000000..808b5a4747dc --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneControllerServiceFacade.java @@ -0,0 +1,243 @@ +/* + * 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.components.connector.facades.standalone; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.ConnectorParameterLookup; +import org.apache.nifi.components.connector.InvocationFailedException; +import org.apache.nifi.components.connector.components.ControllerServiceFacade; +import org.apache.nifi.components.connector.components.ControllerServiceLifecycle; +import org.apache.nifi.components.validation.DisabledServiceValidationResult; +import org.apache.nifi.components.validation.ValidationState; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.ProcessScheduler; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedParameterContext; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.parameter.ParameterContext; +import org.apache.nifi.parameter.ParameterLookup; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class StandaloneControllerServiceFacade implements ControllerServiceFacade { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + .setDefaultPropertyInclusion(JsonInclude.Include.NON_NULL); + + private final ControllerServiceNode controllerServiceNode; + private final VersionedControllerService versionedControllerService; + private final ParameterContext parameterContext; + private final ControllerServiceLifecycle lifecycle; + private final ComponentContextProvider componentContextProvider; + private final ComponentLog connectorLogger; + private final ExtensionManager extensionManager; + private final AssetManager assetManager; + + public StandaloneControllerServiceFacade(final ControllerServiceNode controllerServiceNode, final VersionedControllerService versionedControllerService, + final ParameterContext parameterContext, final ProcessScheduler processScheduler, final ComponentContextProvider componentContextProvider, final ComponentLog connectorLogger, + final ExtensionManager extensionManager, final AssetManager assetManager) { + + this.controllerServiceNode = controllerServiceNode; + this.versionedControllerService = versionedControllerService; + this.parameterContext = parameterContext; + this.componentContextProvider = componentContextProvider; + this.connectorLogger = connectorLogger; + this.extensionManager = extensionManager; + this.assetManager = assetManager; + + this.lifecycle = new StandaloneControllerServiceLifecycle(controllerServiceNode, processScheduler, componentContextProvider, parameterContext); + } + + @Override + public VersionedControllerService getDefinition() { + return versionedControllerService; + } + + @Override + public ControllerServiceLifecycle getLifecycle() { + return lifecycle; + } + + @Override + public List validate() { + return validate(getDefinition().getProperties()); + } + + // TODO: Refactor to avoid duplicate code with StandaloneProcessorFacade + @Override + public List validate(final Map propertyValues) { + final ValidationContext validationContext = componentContextProvider.createValidationContext(controllerServiceNode, propertyValues, parameterContext); + final ValidationState validationState = controllerServiceNode.performValidation(validationContext); + final List validationErrors = validationState.getValidationErrors().stream() + .filter(result -> !result.isValid()) + .filter(result -> !DisabledServiceValidationResult.isMatch(result)) + .toList(); + + return validationErrors; + } + + @Override + public List verify(final Map propertyValues, final Map variables) { + final ConfigurationContext configurationContext = componentContextProvider.createConfigurationContext(controllerServiceNode, propertyValues, parameterContext); + return controllerServiceNode.verifyConfiguration(configurationContext, connectorLogger, variables, extensionManager, parameterContext); + } + + @Override + public List verify(final Map propertyValues, final VersionedParameterContext parameterContext, final Map variables) { + final ParameterLookup parameterLookup = new ConnectorParameterLookup(List.of(parameterContext), assetManager); + final ConfigurationContext configurationContext = componentContextProvider.createConfigurationContext(controllerServiceNode, propertyValues, parameterLookup); + return controllerServiceNode.verifyConfiguration(configurationContext, connectorLogger, variables, extensionManager, parameterLookup); + } + + @Override + public List verify(final VersionedExternalFlow versionedExternalFlow, final Map variables) { + final VersionedProcessGroup parentGroup = findVersionedServiceGroup(versionedExternalFlow.getFlowContents()); + final VersionedControllerService versionedService = findVersionedService(parentGroup); + if (versionedService == null) { + throw new IllegalArgumentException("Cannot verify Controller Service with ID " + versionedControllerService.getIdentifier() + " because it does not exist in the provided flow"); + } + + final ConnectorParameterLookup connectorParameterLookup = new ConnectorParameterLookup(versionedExternalFlow.getParameterContexts().values(), assetManager); + + final Map propertyValues = versionedService.getProperties(); + final ConfigurationContext configurationContext = componentContextProvider.createConfigurationContext(controllerServiceNode, propertyValues, connectorParameterLookup); + return controllerServiceNode.verifyConfiguration(configurationContext, connectorLogger, variables, extensionManager, connectorParameterLookup); + } + + private VersionedProcessGroup findVersionedServiceGroup(final VersionedProcessGroup group) { + for (final VersionedControllerService service : group.getControllerServices()) { + if (service.getIdentifier().equals(versionedControllerService.getIdentifier())) { + return group; + } + } + + for (final VersionedProcessGroup childGroup : group.getProcessGroups()) { + final VersionedProcessGroup serviceGroup = findVersionedServiceGroup(childGroup); + if (serviceGroup != null) { + return serviceGroup; + } + } + + return null; + } + + private VersionedControllerService findVersionedService(final VersionedProcessGroup group) { + if (group == null) { + return null; + } + + final VersionedProcessGroup parentGroup = findVersionedServiceGroup(group); + if (parentGroup == null) { + return null; + } + + for (final VersionedControllerService service : parentGroup.getControllerServices()) { + if (service.getIdentifier().equals(controllerServiceNode.getVersionedComponentId().orElseThrow())) { + return service; + } + } + + return null; + } + + @Override + public Object invokeConnectorMethod(final String methodName, final Map arguments) throws InvocationFailedException { + final Map jsonArguments = serializeArgumentsToJson(arguments); + final ConfigurationContext configurationContext = componentContextProvider.createConfigurationContext(controllerServiceNode, parameterContext); + final String jsonResult = controllerServiceNode.invokeConnectorMethod(methodName, jsonArguments, configurationContext); + if (jsonResult == null) { + return null; + } + + try { + return OBJECT_MAPPER.readValue(jsonResult, Object.class); + } catch (final JsonProcessingException e) { + throw new InvocationFailedException("Failed to deserialize return value from Connector Method '" + methodName + "'", e); + } + } + + @Override + public T invokeConnectorMethod(final String methodName, final Map arguments, final Class returnType) throws InvocationFailedException { + final Map jsonArguments = serializeArgumentsToJson(arguments); + final ConfigurationContext configurationContext = componentContextProvider.createConfigurationContext(controllerServiceNode, parameterContext); + final String jsonResult = controllerServiceNode.invokeConnectorMethod(methodName, jsonArguments, configurationContext); + if (jsonResult == null) { + return null; + } + + try { + return OBJECT_MAPPER.readValue(jsonResult, returnType); + } catch (final JsonProcessingException e) { + throw new InvocationFailedException("Failed to deserialize return value from Connector Method '" + methodName + "'", e); + } + } + + @Override + public String toString() { + final String type = versionedControllerService.getType(); + final String simpleType = type == null ? null : type.substring(type.lastIndexOf('.') + 1); + return "StandaloneControllerServiceFacade[id=" + versionedControllerService.getIdentifier() + ", name=" + versionedControllerService.getName() + ", type=" + simpleType + "]"; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final StandaloneControllerServiceFacade that = (StandaloneControllerServiceFacade) o; + return Objects.equals(versionedControllerService.getIdentifier(), that.versionedControllerService.getIdentifier()); + } + + @Override + public int hashCode() { + return Objects.hashCode(versionedControllerService.getIdentifier()); + } + + private Map serializeArgumentsToJson(final Map arguments) throws InvocationFailedException { + final Map jsonArguments = new HashMap<>(); + for (final Map.Entry entry : arguments.entrySet()) { + if (entry.getValue() == null) { + jsonArguments.put(entry.getKey(), null); + } else { + try { + jsonArguments.put(entry.getKey(), OBJECT_MAPPER.writeValueAsString(entry.getValue())); + } catch (final JsonProcessingException e) { + throw new InvocationFailedException("Failed to serialize argument '" + entry.getKey() + "' to JSON", e); + } + } + } + return jsonArguments; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneControllerServiceLifecycle.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneControllerServiceLifecycle.java new file mode 100644 index 000000000000..c8016ce67a4d --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneControllerServiceLifecycle.java @@ -0,0 +1,77 @@ +/* + * 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.components.connector.facades.standalone; + +import org.apache.nifi.components.connector.components.ControllerServiceLifecycle; +import org.apache.nifi.components.connector.components.ControllerServiceState; +import org.apache.nifi.components.validation.ValidationStatus; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.ProcessScheduler; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.parameter.ParameterContext; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +public class StandaloneControllerServiceLifecycle implements ControllerServiceLifecycle { + private final ControllerServiceNode controllerServiceNode; + private final ProcessScheduler processScheduler; + private final ComponentContextProvider componentContextProvider; + private final ParameterContext parameterContext; + + public StandaloneControllerServiceLifecycle(final ControllerServiceNode controllerServiceNode, final ProcessScheduler scheduler, + final ComponentContextProvider componentContextProvider, final ParameterContext parameterContext) { + + this.controllerServiceNode = controllerServiceNode; + this.processScheduler = scheduler; + this.componentContextProvider = componentContextProvider; + this.parameterContext = parameterContext; + } + + @Override + public ControllerServiceState getState() { + return switch (controllerServiceNode.getState()) { + case DISABLED -> ControllerServiceState.DISABLED; + case ENABLING -> ControllerServiceState.ENABLING; + case ENABLED -> ControllerServiceState.ENABLED; + case DISABLING -> ControllerServiceState.DISABLING; + }; + } + + @Override + public CompletableFuture enable() { + // If validating, perform validation to ensure it's complete before enabling + final ValidationStatus currentStatus = controllerServiceNode.getValidationStatus(); + if (currentStatus != ValidationStatus.VALID) { + controllerServiceNode.performValidation(); + } + + return processScheduler.enableControllerService(controllerServiceNode); + } + + @Override + public CompletableFuture enable(final Map propertyValueOverrides) { + final ConfigurationContext configurationContext = componentContextProvider.createConfigurationContext(controllerServiceNode, propertyValueOverrides, parameterContext); + return processScheduler.enableControllerService(controllerServiceNode, configurationContext); + } + + @Override + public CompletableFuture disable() { + return processScheduler.disableControllerService(controllerServiceNode); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneParameterContextFacade.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneParameterContextFacade.java new file mode 100644 index 000000000000..2599269d3554 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneParameterContextFacade.java @@ -0,0 +1,229 @@ +/* + * 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.components.connector.facades.standalone; + +import org.apache.nifi.asset.Asset; +import org.apache.nifi.components.connector.components.ParameterContextFacade; +import org.apache.nifi.components.connector.components.ParameterValue; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.serialization.AffectedComponentSet; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.parameter.Parameter; +import org.apache.nifi.parameter.ParameterContext; +import org.apache.nifi.parameter.ParameterDescriptor; +import org.apache.nifi.parameter.ParameterReferenceManager; +import org.apache.nifi.parameter.StandardParameterReferenceManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +public class StandaloneParameterContextFacade implements ParameterContextFacade { + private static final Logger logger = LoggerFactory.getLogger(StandaloneParameterContextFacade.class); + + private final FlowController flowController; + private final ProcessGroup managedProcessGroup; + private final ParameterContext parameterContext; + + public StandaloneParameterContextFacade(final FlowController flowController, final ProcessGroup managedProcessGroup) { + this.flowController = flowController; + this.managedProcessGroup = managedProcessGroup; + this.parameterContext = managedProcessGroup.getParameterContext(); + } + + @Override + public void updateParameters(final Collection updatedValues) { + // Check if the parameter already exists and if the sensitivity matches + validateParameterSensitivities(updatedValues); + + final AffectedComponentSet affectedComponentSet = new AffectedComponentSet(flowController); + final ParameterReferenceManager parameterReferenceManager = new StandardParameterReferenceManager(() -> managedProcessGroup); + + final Set allReferencingProcessors = new HashSet<>(); + final Set allReferencingServices = new HashSet<>(); + for (final ParameterValue parameterValue : updatedValues) { + if (!isParameterChanged(parameterValue)) { + continue; + } + + final String parameterName = parameterValue.getName(); + final Set referencingProcessors = parameterReferenceManager.getProcessorsReferencing(managedProcessGroup.getParameterContext(), parameterName); + final Set referencingServices = parameterReferenceManager.getControllerServicesReferencing(managedProcessGroup.getParameterContext(), parameterName); + referencingProcessors.forEach(affectedComponentSet::addProcessor); + referencingServices.forEach(affectedComponentSet::addControllerService); + + allReferencingProcessors.addAll(referencingProcessors); + allReferencingServices.addAll(referencingServices); + } + + if (affectedComponentSet.isEmpty()) { + logger.info("No components are affected by the parameter update; no need to restart anything."); + return; + } + + final AffectedComponentSet activeSet = affectedComponentSet.toActiveSet(); + logger.info("Stopping {} affected components due to parameter update", activeSet.getComponentCount()); + activeSet.stop(); + + // Update the parameter value in our map + final Map updatedParameters = createParameterMap(updatedValues); + managedProcessGroup.getParameterContext().setParameters(updatedParameters); + + allReferencingProcessors.forEach(ProcessorNode::resetValidationState); + allReferencingServices.forEach(ControllerServiceNode::resetValidationState); + + logger.info("Parameter Context updated {} parameter. Restarting {} affected components.", updatedValues.size(), activeSet.getComponentCount()); + activeSet.start(); + } + + private Map createParameterMap(final Collection parameterValues) { + final Map paramMap = new HashMap<>(); + for (final ParameterValue parameterValue : parameterValues) { + final Parameter parameter = new Parameter.Builder() + .name(parameterValue.getName()) + .value(parameterValue.getValue()) + .sensitive(parameterValue.isSensitive()) + .build(); + + paramMap.put(parameterValue.getName(), parameter); + } + + return paramMap; + } + + private boolean isParameterChanged(final ParameterValue parameterValue) { + final String parameterName = parameterValue.getName(); + final Optional optionalParameter = parameterContext.getParameter(parameterName); + if (optionalParameter.isEmpty()) { + return true; // New parameter + } + + final Parameter existingParameter = optionalParameter.get(); + if (!Objects.equals(existingParameter.getValue(), parameterValue.getValue())) { + return true; // Value changed + } + + if (!Objects.equals(existingParameter.getReferencedAssets(), parameterValue.getAssets())) { + return true; // Assets changed + } + + return false; + } + + private void validateParameterSensitivities(final Collection updatedValues) { + for (final ParameterValue parameterValue : updatedValues) { + final String parameterName = parameterValue.getName(); + final boolean sensitive = parameterValue.isSensitive(); + + final Optional optionalParameter = parameterContext.getParameter(parameterName); + if (optionalParameter.isEmpty()) { + continue; // New parameter, no existing sensitivity to check + } + + final boolean existingSensitive = optionalParameter.get().getDescriptor().isSensitive(); + if (existingSensitive != sensitive) { + throw new IllegalStateException("Cannot update parameter " + parameterName + " because the Parameter is already defined as being " + + (existingSensitive ? "sensitive" : "non-sensitive") + " but the update is defined as being " + + (sensitive ? "sensitive" : "non-sensitive")); + } + } + } + + @Override + public synchronized String getValue(final String parameterName) { + final Optional optionalParameter = parameterContext.getParameter(parameterName); + if (optionalParameter.isEmpty()) { + return null; + } + + final Parameter existing = optionalParameter.get(); + final String explicitValue = existing.getValue(); + if (explicitValue != null) { + return explicitValue; + } + + final List assets = existing.getReferencedAssets(); + if (assets.isEmpty()) { + return null; + } + + final StringBuilder sb = new StringBuilder(); + for (final Asset asset : assets) { + sb.append(asset.getFile().getAbsolutePath()); + sb.append(","); + } + sb.setLength(sb.length() - 1); // Remove trailing comma + return sb.toString(); + } + + @Override + public synchronized Set getDefinedParameterNames() { + final Set parameterNames = new HashSet<>(); + for (final ParameterDescriptor descriptor : parameterContext.getEffectiveParameters().keySet()) { + parameterNames.add(descriptor.getName()); + } + return parameterNames; + } + + @Override + public synchronized boolean isSensitive(final String parameterName) { + return parameterContext.getParameter(parameterName) + .map(Parameter::getDescriptor) + .map(ParameterDescriptor::isSensitive) + .orElse(false); + } + + @Override + public Asset createAsset(final InputStream inputStream) throws IOException { + // TODO: Implement + return null; + } + + @Override + public String toString() { + return "StandaloneParameterContextFacade[id=" + parameterContext.getIdentifier() + ", name=" + parameterContext.getName() + "]"; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final StandaloneParameterContextFacade that = (StandaloneParameterContextFacade) o; + return Objects.equals(parameterContext.getIdentifier(), that.parameterContext.getIdentifier()); + } + + @Override + public int hashCode() { + return Objects.hashCode(parameterContext.getIdentifier()); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessGroupFacade.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessGroupFacade.java new file mode 100644 index 000000000000..83cd0a471917 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessGroupFacade.java @@ -0,0 +1,424 @@ +/* + * 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.components.connector.facades.standalone; + +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.connector.DropFlowFileSummary; +import org.apache.nifi.components.connector.components.ComponentHierarchyScope; +import org.apache.nifi.components.connector.components.ConnectionFacade; +import org.apache.nifi.components.connector.components.ControllerServiceFacade; +import org.apache.nifi.components.connector.components.ControllerServiceReferenceScope; +import org.apache.nifi.components.connector.components.ProcessGroupFacade; +import org.apache.nifi.components.connector.components.ProcessGroupLifecycle; +import org.apache.nifi.components.connector.components.ProcessorFacade; +import org.apache.nifi.components.connector.components.StatelessGroupLifecycle; +import org.apache.nifi.connectable.Connection; +import org.apache.nifi.controller.ProcessScheduler; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.queue.QueueSize; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.flow.ExecutionEngine; +import org.apache.nifi.flow.VersionedConnection; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.parameter.ParameterContext; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +/** + * An implementation of ProcessGroupFacade that implements the functionality for a standalone + * (non-clustered) NiFi instance. + */ +public class StandaloneProcessGroupFacade implements ProcessGroupFacade { + private final ProcessGroup processGroup; + private final VersionedProcessGroup flowDefinition; + private final ProcessScheduler processScheduler; + private final ParameterContext parameterContext; + private final Map processorMap; + private final Map controllerServiceMap; + private final Map connectionMap; + private final Map processGroupMap; + private final ControllerServiceProvider controllerServiceProvider; + private final StandaloneProcessGroupLifecycle lifecycle; + private final StatelessGroupLifecycle statelessGroupLifecycle; + private final ComponentContextProvider componentContextProvider; + private final ComponentLog connectorLogger; + private final ExtensionManager extensionManager; + private final AssetManager assetManager; + + public StandaloneProcessGroupFacade(final ProcessGroup processGroup, final VersionedProcessGroup flowDefinition, final ProcessScheduler processScheduler, + final ParameterContext parameterContext, final ControllerServiceProvider controllerServiceProvider, final ComponentContextProvider componentContextProvider, + final ComponentLog connectorLogger, final ExtensionManager extensionManager, final AssetManager assetManager) { + this.processGroup = processGroup; + this.flowDefinition = flowDefinition; + this.processScheduler = processScheduler; + this.parameterContext = parameterContext; + this.controllerServiceProvider = controllerServiceProvider; + this.componentContextProvider = componentContextProvider; + this.connectorLogger = connectorLogger; + this.extensionManager = extensionManager; + this.assetManager = assetManager; + + this.processorMap = mapProcessors(flowDefinition); + this.controllerServiceMap = mapControllerServices(flowDefinition); + this.connectionMap = mapConnections(flowDefinition); + this.processGroupMap = mapProcessGroups(flowDefinition); + + final ExecutionEngine executionEngine = processGroup.resolveExecutionEngine(); + if (executionEngine == ExecutionEngine.STATELESS) { + this.statelessGroupLifecycle = new StandaloneStatelessGroupLifecycle(processGroup, processScheduler); + } else { + this.statelessGroupLifecycle = new IllegalExecutionEngineStatelessGroupLifecycle(processGroup); + } + + this.lifecycle = new StandaloneProcessGroupLifecycle(processGroup, controllerServiceProvider, statelessGroupLifecycle, + id -> getProcessGroup(id).getLifecycle()); + } + + private Map mapProcessors(final VersionedProcessGroup flowDefinition) { + final Map processors = new HashMap<>(); + flowDefinition.getProcessors().forEach(processor -> processors.put(processor.getIdentifier(), processor)); + return processors; + } + + private Map mapControllerServices(final VersionedProcessGroup flowDefinition) { + final Map controllerServices = new HashMap<>(); + flowDefinition.getControllerServices().forEach(controllerService -> controllerServices.put(controllerService.getIdentifier(), controllerService)); + return controllerServices; + } + + private Map mapConnections(final VersionedProcessGroup flowDefinition) { + final Map connections = new HashMap<>(); + flowDefinition.getConnections().forEach(connection -> connections.put(connection.getIdentifier(), connection)); + return connections; + } + + private Map mapProcessGroups(final VersionedProcessGroup flowDefinition) { + final Map processGroups = new HashMap<>(); + flowDefinition.getProcessGroups().forEach(processGroup -> processGroups.put(processGroup.getIdentifier(), processGroup)); + return processGroups; + } + + @Override + public VersionedProcessGroup getDefinition() { + return flowDefinition; + } + + @Override + public ProcessorFacade getProcessor(final String id) { + final ProcessorNode processorNode = lookupProcessorNode(id); + if (processorNode == null) { + return null; + } + + final VersionedProcessor processor = processorMap.get(id); + if (processor == null) { + return null; + } + + return new StandaloneProcessorFacade(processorNode, processor, processScheduler, parameterContext, + componentContextProvider, connectorLogger, extensionManager, assetManager); + } + + private ProcessorNode lookupProcessorNode(final String versionedComponentId) { + for (final ProcessorNode processorNode : processGroup.getProcessors()) { + final Optional versionedId = processorNode.getVersionedComponentId(); + if (versionedId.isPresent() && versionedId.get().equals(versionedComponentId)) { + return processorNode; + } + } + + return null; + } + + @Override + public Set getProcessors() { + if (processorMap.isEmpty()) { + return Collections.emptySet(); + } + + final Set processors = new HashSet<>(); + for (final VersionedProcessor versionedProcessor : processorMap.values()) { + final ProcessorNode processorNode = processGroup.getProcessor(versionedProcessor.getInstanceIdentifier()); + if (processorNode != null) { + final ProcessorFacade processorFacade = new StandaloneProcessorFacade(processorNode, versionedProcessor, processScheduler, + parameterContext, componentContextProvider, connectorLogger, extensionManager, assetManager); + processors.add(processorFacade); + } + } + + return processors; + } + + @Override + public ControllerServiceFacade getControllerService(final String id) { + final ControllerServiceNode controllerServiceNode = lookupControllerServiceNode(id); + if (controllerServiceNode == null) { + return null; + } + + final VersionedControllerService controllerService = controllerServiceMap.get(id); + if (controllerService == null) { + return null; + } + + return new StandaloneControllerServiceFacade(controllerServiceNode, controllerService, parameterContext, processScheduler, + componentContextProvider, connectorLogger, extensionManager, assetManager); + } + + private ControllerServiceNode lookupControllerServiceNode(final String versionedComponentId) { + for (final ControllerServiceNode controllerServiceNode : processGroup.getControllerServices(false)) { + final Optional versionedId = controllerServiceNode.getVersionedComponentId(); + if (versionedId.isPresent() && versionedId.get().equals(versionedComponentId)) { + return controllerServiceNode; + } + } + + return null; + } + + @Override + public Set getControllerServices() { + if (controllerServiceMap.isEmpty()) { + return Collections.emptySet(); + } + + final Set controllerServices = new HashSet<>(); + for (final VersionedControllerService versionedControllerService : controllerServiceMap.values()) { + final ControllerServiceNode controllerServiceNode = processGroup.getControllerService(versionedControllerService.getInstanceIdentifier()); + if (controllerServiceNode != null) { + final ControllerServiceFacade serviceFacade = new StandaloneControllerServiceFacade(controllerServiceNode, versionedControllerService, parameterContext, + processScheduler, componentContextProvider, connectorLogger, extensionManager, assetManager); + controllerServices.add(serviceFacade); + } + } + + return controllerServices; + } + + @Override + public Set getControllerServices(final ControllerServiceReferenceScope referenceScope, final ComponentHierarchyScope hierarchyScope) { + + final boolean recursive = (hierarchyScope == ComponentHierarchyScope.INCLUDE_CHILD_GROUPS); + if (referenceScope == ControllerServiceReferenceScope.INCLUDE_ALL) { + final Set facades = new HashSet<>(); + collectControllerServiceFacades(this, facades, facade -> true, recursive); + return facades; + } else { + final Set serviceNodes = lifecycle.findReferencedServices(recursive); + final Set versionedComponentIds = serviceNodes.stream() + .map(ControllerServiceNode::getVersionedComponentId) + .filter(Optional::isPresent) + .map(Optional::get) + .collect(Collectors.toSet()); + + final Set facades = new HashSet<>(); + final Predicate serviceFacadeFilter = facade -> versionedComponentIds.contains(facade.getDefinition().getIdentifier()); + collectControllerServiceFacades(this, facades, serviceFacadeFilter, recursive); + return facades; + } + } + + private void collectControllerServiceFacades(final ProcessGroupFacade group, final Set facades, final Predicate filter, + final boolean recursive) { + + for (final ControllerServiceFacade serviceFacade : group.getControllerServices()) { + if (filter.test(serviceFacade)) { + facades.add(serviceFacade); + } + } + + if (recursive) { + for (final ProcessGroupFacade childGroup : group.getProcessGroups()) { + collectControllerServiceFacades(childGroup, facades, filter, true); + } + } + } + + @Override + public ConnectionFacade getConnection(final String id) { + final Connection connection = lookupConnection(id); + if (connection == null) { + return null; + } + + final VersionedConnection versionedConnection = connectionMap.get(id); + if (versionedConnection == null) { + return null; + } + + return new StandaloneConnectionFacade(connection, versionedConnection); + } + + private Connection lookupConnection(final String versionedComponentId) { + for (final Connection connection : processGroup.getConnections()) { + final Optional versionedId = connection.getVersionedComponentId(); + if (versionedId.isPresent() && versionedId.get().equals(versionedComponentId)) { + return connection; + } + } + + return null; + } + + @Override + public Set getConnections() { + if (connectionMap.isEmpty()) { + return Collections.emptySet(); + } + + final Set connections = new HashSet<>(); + for (final VersionedConnection versionedConnection : connectionMap.values()) { + final Connection connection = processGroup.getConnection(versionedConnection.getInstanceIdentifier()); + if (connection != null) { + connections.add(new StandaloneConnectionFacade(connection, versionedConnection)); + } + } + + return connections; + } + + @Override + public ProcessGroupFacade getProcessGroup(final String id) { + final ProcessGroup childProcessGroup = lookupProcessGroup(processGroup, id); + if (childProcessGroup == null) { + return null; + } + + final VersionedProcessGroup versionedProcessGroup = processGroupMap.get(id); + if (versionedProcessGroup == null) { + return null; + } + + return new StandaloneProcessGroupFacade(childProcessGroup, versionedProcessGroup, processScheduler, parameterContext, + controllerServiceProvider, componentContextProvider, connectorLogger, extensionManager, assetManager); + } + + private ProcessGroup lookupProcessGroup(final ProcessGroup start, final String versionedComponentId) { + for (final ProcessGroup childProcessGroup : start.getProcessGroups()) { + final Optional versionedId = childProcessGroup.getVersionedComponentId(); + if (versionedId.isPresent() && versionedId.get().equals(versionedComponentId)) { + return childProcessGroup; + } + } + + for (final ProcessGroup childProcessGroup : start.getProcessGroups()) { + final ProcessGroup found = lookupProcessGroup(childProcessGroup, versionedComponentId); + if (found != null) { + return found; + } + } + + return null; + } + + @Override + public Set getProcessGroups() { + if (processGroupMap.isEmpty()) { + return Collections.emptySet(); + } + + final Set processGroups = new HashSet<>(); + for (final VersionedProcessGroup versionedProcessGroup : processGroupMap.values()) { + final ProcessGroup childProcessGroup = processGroup.findProcessGroup(versionedProcessGroup.getInstanceIdentifier()); + if (childProcessGroup != null) { + final ProcessGroupFacade groupFacade = new StandaloneProcessGroupFacade(childProcessGroup, versionedProcessGroup, processScheduler, parameterContext, + controllerServiceProvider, componentContextProvider, connectorLogger, extensionManager, assetManager); + processGroups.add(groupFacade); + } + } + + return processGroups; + } + + @Override + public QueueSize getQueueSize() { + return processGroup.getQueueSize(); + } + + @Override + public boolean isFlowEmpty() { + return processGroup.isEmpty(); + } + + @Override + public StatelessGroupLifecycle getStatelessLifecycle() { + return statelessGroupLifecycle; + } + + @Override + public ProcessGroupLifecycle getLifecycle() { + return lifecycle; + } + + @Override + public String toString() { + return "StandaloneProcessGroupFacade[id=" + flowDefinition.getIdentifier() + ", name=" + flowDefinition.getName() + "]"; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final StandaloneProcessGroupFacade that = (StandaloneProcessGroupFacade) o; + return Objects.equals(flowDefinition.getIdentifier(), that.flowDefinition.getIdentifier()); + } + + @Override + public int hashCode() { + return Objects.hashCode(flowDefinition.getIdentifier()); + } + + @Override + public DropFlowFileSummary dropFlowFiles(final Predicate predicate) throws IOException { + DropFlowFileSummary summary = new DropFlowFileSummary(0, 0); + + // Drop from all connections in this group + for (final ConnectionFacade connection : getConnections()) { + final DropFlowFileSummary connectionSummary = connection.dropFlowFiles(predicate); + summary = summary.add(connectionSummary); + } + + // Recursively drop from all child process groups + for (final ProcessGroupFacade childGroup : getProcessGroups()) { + final DropFlowFileSummary childSummary = childGroup.dropFlowFiles(predicate); + summary = summary.add(childSummary); + } + + return summary; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessGroupLifecycle.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessGroupLifecycle.java new file mode 100644 index 000000000000..3ee3d4a641e7 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessGroupLifecycle.java @@ -0,0 +1,463 @@ +/* + * 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.components.connector.facades.standalone; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.connector.components.ComponentHierarchyScope; +import org.apache.nifi.components.connector.components.ControllerServiceReferenceScope; +import org.apache.nifi.components.connector.components.ProcessGroupLifecycle; +import org.apache.nifi.components.connector.components.StatelessGroupLifecycle; +import org.apache.nifi.components.validation.ValidationStatus; +import org.apache.nifi.connectable.Port; +import org.apache.nifi.controller.ComponentNode; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.flow.ExecutionEngine; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.groups.RemoteProcessGroup; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class StandaloneProcessGroupLifecycle implements ProcessGroupLifecycle { + private static final Logger logger = LoggerFactory.getLogger(StandaloneProcessGroupLifecycle.class); + + private final ProcessGroup processGroup; + private final ControllerServiceProvider controllerServiceProvider; + private final StatelessGroupLifecycle statelessGroupLifecycle; + private final Function childGroupLifecycleFactory; + + public StandaloneProcessGroupLifecycle(final ProcessGroup processGroup, final ControllerServiceProvider controllerServiceProvider, + final StatelessGroupLifecycle statelessGroupLifecycle, final Function childGroupLifecycleFactory) { + + this.processGroup = processGroup; + this.controllerServiceProvider = controllerServiceProvider; + this.statelessGroupLifecycle = statelessGroupLifecycle; + this.childGroupLifecycleFactory = childGroupLifecycleFactory; + } + + @Override + public CompletableFuture enableControllerServices(final ControllerServiceReferenceScope referenceScope, final ComponentHierarchyScope hierarchyScope) { + final boolean recursive = (hierarchyScope == ComponentHierarchyScope.INCLUDE_CHILD_GROUPS); + final Set controllerServices = (referenceScope == ControllerServiceReferenceScope.INCLUDE_ALL) + ? processGroup.findAllControllerServices() : findReferencedServices(recursive); + return enableControllerServices(controllerServices); + } + + public Set findReferencedServices(final boolean recursive) { + final Set referencedServices = new HashSet<>(); + collectReferencedServices(processGroup, referencedServices, recursive); + return referencedServices; + } + + private void collectReferencedServices(final ProcessGroup group, final Set referencedServices, final boolean recursive) { + for (final ProcessorNode processor : group.getProcessors()) { + final ValidationContext validationContext = createValidationContext(processor); + + for (final PropertyDescriptor descriptor : processor.getPropertyDescriptors()) { + if (descriptor.getControllerServiceDefinition() == null) { + continue; + } + + final String serviceId = processor.getEffectivePropertyValue(descriptor); + if (serviceId == null) { + continue; + } + + // Skip properties whose dependencies are not satisfied, as the property is not relevant to the component's functionality + if (!validationContext.isDependencySatisfied(descriptor, processor::getPropertyDescriptor)) { + logger.debug("Not enabling Controller Service {} because it is referenced by {} property of {} whose dependencies are not satisfied", + serviceId, descriptor.getName(), processor); + continue; + } + + final ControllerServiceNode serviceNode = controllerServiceProvider.getControllerServiceNode(serviceId); + if (serviceNode == null) { + continue; + } + + logger.debug("Marking {} as a Referenced Controller Service because it is referenced by {} property of {}", + serviceNode, descriptor.getName(), processor); + referencedServices.add(serviceNode); + } + } + + // Transitively collect services required by the already-referenced services. + // ControllerServiceNode.getRequiredControllerServices() already filters based on satisfied property dependencies. + while (true) { + final Set newlyAddedServices = new HashSet<>(); + for (final ControllerServiceNode service : referencedServices) { + for (final ControllerServiceNode requiredService : service.getRequiredControllerServices()) { + if (!referencedServices.contains(requiredService)) { + logger.debug("Marking {} as a Referenced Controller Service because it is required by {}", + requiredService, service); + newlyAddedServices.add(requiredService); + } + } + } + + referencedServices.addAll(newlyAddedServices); + if (newlyAddedServices.isEmpty()) { + break; + } + } + + if (recursive) { + for (final ProcessGroup childGroup : group.getProcessGroups()) { + collectReferencedServices(childGroup, referencedServices, true); + } + } + } + + private ValidationContext createValidationContext(final ComponentNode component) { + final Map effectivePropertyValues = new LinkedHashMap<>(); + for (final Map.Entry entry : component.getEffectivePropertyValues().entrySet()) { + effectivePropertyValues.put(entry.getKey().getName(), entry.getValue()); + } + + return component.createValidationContext(effectivePropertyValues, component.getAnnotationData(), component.getParameterLookup(), false); + } + + @Override + public CompletableFuture enableControllerServices(final Collection collection) { + final Set serviceNodes = findControllerServices(collection); + return enableControllerServices(serviceNodes); + } + + private CompletableFuture enableControllerServices(final Set serviceNodes) { + if (serviceNodes.isEmpty()) { + return CompletableFuture.completedFuture(null); + } + + // If any service is not currently valid, perform validation again to ensure that the status is up to date. + for (final ControllerServiceNode serviceNode : serviceNodes) { + final ValidationStatus validationStatus = serviceNode.getValidationStatus(); + if (validationStatus != ValidationStatus.VALID) { + serviceNode.performValidation(); + } + } + + return controllerServiceProvider.enableControllerServicesAsync(serviceNodes); + } + + private Set findControllerServices(final Collection serviceIds) { + return processGroup.findAllControllerServices().stream() + .filter(service -> service.getVersionedComponentId().isPresent()) + .filter(service -> serviceIds.contains(service.getVersionedComponentId().get())) + .collect(Collectors.toSet()); + } + + @Override + public CompletableFuture disableControllerServices(final ComponentHierarchyScope scope) { + final boolean recursive = (scope == ComponentHierarchyScope.INCLUDE_CHILD_GROUPS); + final Set controllerServices = recursive ? processGroup.findAllControllerServices() : processGroup.getControllerServices(false); + return disableControllerServices(controllerServices); + } + + private CompletableFuture disableControllerServices(final Set serviceNodes) { + if (serviceNodes.isEmpty()) { + return CompletableFuture.completedFuture(null); + } + + return controllerServiceProvider.disableControllerServicesAsync(serviceNodes); + } + + @Override + public CompletableFuture disableControllerServices(final Collection collection) { + final Set serviceNodes = findControllerServices(collection); + return disableControllerServices(serviceNodes); + } + + @Override + public CompletableFuture startProcessors(final ComponentHierarchyScope scope) { + final boolean recursive = (scope == ComponentHierarchyScope.INCLUDE_CHILD_GROUPS); + final Collection processors = recursive ? processGroup.findAllProcessors() : processGroup.getProcessors(); + final List> startFutures = new ArrayList<>(); + for (final ProcessorNode processor : processors) { + // If Processor is not valid, perform validation again to ensure that the status is up to date. + final ValidationStatus validationStatus = processor.getValidationStatus(); + if (validationStatus != ValidationStatus.VALID) { + processor.performValidation(); + } + + if (processor.getScheduledState() == ScheduledState.DISABLED) { + logger.debug("Not starting Processor {} because it is disabled", processor); + continue; + } + + startFutures.add(processor.getProcessGroup().startProcessor(processor, true)); + } + + return CompletableFuture.allOf(startFutures.toArray(new CompletableFuture[0])); + } + + @Override + public CompletableFuture start(final ControllerServiceReferenceScope serviceReferenceScope) { + if (processGroup.resolveExecutionEngine() == ExecutionEngine.STATELESS) { + return statelessGroupLifecycle.start(); + } + + final CompletableFuture result = new CompletableFuture<>(); + + Thread.startVirtualThread(() -> { + try { + enableControllerServices(serviceReferenceScope, ComponentHierarchyScope.IMMEDIATE_GROUP_ONLY).get(); + startPorts(ComponentHierarchyScope.IMMEDIATE_GROUP_ONLY).get(); + startRemoteProcessGroups(ComponentHierarchyScope.IMMEDIATE_GROUP_ONLY).get(); + startProcessors(ComponentHierarchyScope.IMMEDIATE_GROUP_ONLY).get(); + + final List> childGroupFutures = new ArrayList<>(); + for (final ProcessGroup childGroup : processGroup.getProcessGroups()) { + final String childGroupId = childGroup.getVersionedComponentId().orElse(null); + if (childGroupId == null) { + logger.warn("Encountered child Process Group {} without a Versioned Component ID. Skipping start of child group.", childGroup.getIdentifier()); + continue; + } + + final ProcessGroupLifecycle childLifecycle = childGroupLifecycleFactory.apply(childGroupId); + childGroupFutures.add(childLifecycle.start(serviceReferenceScope)); + } + + CompletableFuture.allOf(childGroupFutures.toArray(new CompletableFuture[0])).get(); + result.complete(null); + } catch (final Exception e) { + result.completeExceptionally(e); + } + }); + + return result; + } + + @Override + public CompletableFuture startPorts(final ComponentHierarchyScope scope) { + final boolean recursive = (scope == ComponentHierarchyScope.INCLUDE_CHILD_GROUPS); + logger.debug("{} starting all ports", this); + final Collection inputPorts = recursive ? processGroup.findAllInputPorts() : processGroup.getInputPorts(); + for (final Port inputPort : inputPorts) { + inputPort.getProcessGroup().startInputPort(inputPort); + } + + final Collection outputPorts = recursive ? processGroup.findAllOutputPorts() : processGroup.getOutputPorts(); + for (final Port outputPort : outputPorts) { + outputPort.getProcessGroup().startOutputPort(outputPort); + } + + logger.info("{} started all ports", this); + return CompletableFuture.completedFuture(null); + } + + @Override + public CompletableFuture stopPorts(final ComponentHierarchyScope scope) { + final boolean recursive = (scope == ComponentHierarchyScope.INCLUDE_CHILD_GROUPS); + logger.debug("{} stopping all ports", this); + + final Collection inputPorts = recursive ? processGroup.findAllInputPorts() : processGroup.getInputPorts(); + for (final Port inputPort : inputPorts) { + inputPort.getProcessGroup().stopInputPort(inputPort); + } + + final Collection outputPorts = recursive ? processGroup.findAllOutputPorts() : processGroup.getOutputPorts(); + for (final Port outputPort : outputPorts) { + outputPort.getProcessGroup().stopOutputPort(outputPort); + } + + logger.info("{} stopped all ports", this); + return CompletableFuture.completedFuture(null); + } + + @Override + public CompletableFuture startRemoteProcessGroups(final ComponentHierarchyScope scope) { + final boolean recursive = (scope == ComponentHierarchyScope.INCLUDE_CHILD_GROUPS); + logger.debug("{} starting all Remote Process Groups", this); + + final Collection rpgs = recursive ? processGroup.findAllRemoteProcessGroups() : processGroup.getRemoteProcessGroups(); + for (final RemoteProcessGroup rpg : rpgs) { + rpg.startTransmitting(); + } + + logger.info("{} started all Remote Process Groups", this); + return CompletableFuture.completedFuture(null); + } + + @Override + public CompletableFuture stopRemoteProcessGroups(final ComponentHierarchyScope scope) { + final boolean recursive = (scope == ComponentHierarchyScope.INCLUDE_CHILD_GROUPS); + logger.debug("{} stopping all Remote Process Groups", this); + final List> stopFutures = new ArrayList<>(); + + final Collection rpgs = recursive ? processGroup.findAllRemoteProcessGroups() : processGroup.getRemoteProcessGroups(); + for (final RemoteProcessGroup rpg : rpgs) { + stopFutures.add(rpg.stopTransmitting()); + } + + logger.info("{} stopped all Remote Process Groups", this); + return CompletableFuture.allOf(stopFutures.toArray(new CompletableFuture[0])); + } + + @Override + public CompletableFuture startStatelessGroups(final ComponentHierarchyScope scope) { + final boolean recursive = (scope == ComponentHierarchyScope.INCLUDE_CHILD_GROUPS); + logger.debug("{} starting all Stateless Process Groups", this); + final List> startFutures = new ArrayList<>(); + + final Collection processGroups = processGroup.getProcessGroups(); + for (final ProcessGroup childGroup : processGroups) { + final String childGroupId = childGroup.getVersionedComponentId().orElse(null); + if (childGroupId == null) { + logger.warn("Encountered stateless child Process Group {} without a Versioned Component ID. Skipping start.", childGroup.getIdentifier()); + continue; + } + + final ProcessGroupLifecycle childLifecycle = childGroupLifecycleFactory.apply(childGroupId); + + if (childGroup.resolveExecutionEngine() == ExecutionEngine.STATELESS) { + startFutures.add(childLifecycle.start(ControllerServiceReferenceScope.INCLUDE_REFERENCED_SERVICES_ONLY)); + } else if (recursive) { + startFutures.add(childLifecycle.startStatelessGroups(ComponentHierarchyScope.INCLUDE_CHILD_GROUPS)); + } + } + + logger.info("{} started all Stateless Process Groups", this); + return CompletableFuture.allOf(startFutures.toArray(new CompletableFuture[0])); + } + + @Override + public CompletableFuture stopStatelessGroups(final ComponentHierarchyScope scope) { + final boolean recursive = (scope == ComponentHierarchyScope.INCLUDE_CHILD_GROUPS); + logger.debug("{} stopping all Stateless Process Groups", this); + final List> stopFutures = new ArrayList<>(); + + for (final ProcessGroup childGroup : processGroup.getProcessGroups()) { + final String childGroupId = childGroup.getVersionedComponentId().orElse(null); + if (childGroupId == null) { + logger.warn("Encountered stateless child Process Group {} without a Versioned Component ID. Skipping stop.", childGroup.getIdentifier()); + continue; + } + + final ProcessGroupLifecycle childLifecycle = childGroupLifecycleFactory.apply(childGroupId); + + if (childGroup.resolveExecutionEngine() == ExecutionEngine.STATELESS) { + stopFutures.add(childLifecycle.stop()); + } else if (recursive) { + stopFutures.add(childLifecycle.stopStatelessGroups(ComponentHierarchyScope.INCLUDE_CHILD_GROUPS)); + } + } + + logger.info("{} stopped all Stateless Process Groups", this); + return CompletableFuture.allOf(stopFutures.toArray(new CompletableFuture[0])); + } + + @Override + public CompletableFuture stop() { + logger.debug("Stopping Process Group {}", processGroup.getIdentifier()); + if (processGroup.resolveExecutionEngine() == ExecutionEngine.STATELESS) { + return statelessGroupLifecycle.stop(); + } + + final CompletableFuture result = new CompletableFuture<>(); + + Thread.startVirtualThread(() -> { + try { + stopProcessors(ComponentHierarchyScope.IMMEDIATE_GROUP_ONLY).get(); + stopChildren().get(); + stopPorts(ComponentHierarchyScope.IMMEDIATE_GROUP_ONLY).get(); + stopRemoteProcessGroups(ComponentHierarchyScope.IMMEDIATE_GROUP_ONLY).get(); + disableControllerServices(ComponentHierarchyScope.INCLUDE_CHILD_GROUPS).get(); + + logger.info("Stopped Process Group {}", processGroup.getIdentifier()); + result.complete(null); + } catch (final Exception e) { + result.completeExceptionally(e); + } + }); + + return result; + } + + private CompletableFuture stopChildren() { + final List> childGroupFutures = new ArrayList<>(); + for (final ProcessGroup childGroup : processGroup.getProcessGroups()) { + final String childGroupId = childGroup.getVersionedComponentId().orElse(null); + if (childGroupId == null) { + logger.warn("Encountered child Process Group {} without a Versioned Component ID. Skipping stop of child group.", childGroup.getIdentifier()); + continue; + } + + final ProcessGroupLifecycle childLifecycle = childGroupLifecycleFactory.apply(childGroupId); + final CompletableFuture childFuture = childLifecycle.stop(); + childGroupFutures.add(childFuture); + } + + return CompletableFuture.allOf(childGroupFutures.toArray(new CompletableFuture[0])); + } + + @Override + public CompletableFuture stopProcessors(final ComponentHierarchyScope scope) { + final boolean recursive = (scope == ComponentHierarchyScope.INCLUDE_CHILD_GROUPS); + final Collection processors = recursive ? processGroup.findAllProcessors() : processGroup.getProcessors(); + final List> stopFutures = new ArrayList<>(); + for (final ProcessorNode processor : processors) { + final ScheduledState processorState = processor.getScheduledState(); + if (processorState == ScheduledState.DISABLED || processorState == ScheduledState.STOPPED) { + logger.debug("Not stopping Processor {} because its state is {}", processor, processorState); + continue; + } + + final CompletableFuture stopFuture = processor.getProcessGroup().stopProcessor(processor); + stopFutures.add(stopFuture); + } + + return CompletableFuture.allOf(stopFutures.toArray(new CompletableFuture[0])); + } + + @Override + public int getActiveThreadCount() { + return getActiveThreadCount(processGroup); + } + + private int getActiveThreadCount(final ProcessGroup group) { + int total = 0; + for (final ProcessorNode processor : group.getProcessors()) { + total += processor.getActiveThreadCount(); + } + for (final ProcessGroup childGroup : group.getProcessGroups()) { + total += getActiveThreadCount(childGroup); + } + return total; + } + + @Override + public String toString() { + return "StandaloneProcessGroupLifecycle[" + + "processGroup=" + processGroup.getIdentifier() + + "]"; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessorFacade.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessorFacade.java new file mode 100644 index 000000000000..3de26ad4431a --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessorFacade.java @@ -0,0 +1,253 @@ +/* + * 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.components.connector.facades.standalone; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.ConnectorParameterLookup; +import org.apache.nifi.components.connector.InvocationFailedException; +import org.apache.nifi.components.connector.components.ProcessorFacade; +import org.apache.nifi.components.connector.components.ProcessorLifecycle; +import org.apache.nifi.components.validation.DisabledServiceValidationResult; +import org.apache.nifi.components.validation.ValidationState; +import org.apache.nifi.components.validation.ValidationStatus; +import org.apache.nifi.controller.ProcessScheduler; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedParameterContext; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.parameter.ParameterContext; +import org.apache.nifi.parameter.ParameterLookup; +import org.apache.nifi.processor.ProcessContext; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class StandaloneProcessorFacade implements ProcessorFacade { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + .setDefaultPropertyInclusion(JsonInclude.Include.NON_NULL); + + private final ProcessorNode processorNode; + private final VersionedProcessor versionedProcessor; + private final ParameterContext parameterContext; + private final ProcessorLifecycle lifecycle; + private final ComponentContextProvider componentContextProvider; + private final ComponentLog connectorLogger; + private final ExtensionManager extensionManager; + private final AssetManager assetManager; + + public StandaloneProcessorFacade(final ProcessorNode processorNode, final VersionedProcessor versionedProcessor, final ProcessScheduler scheduler, + final ParameterContext parameterContext, final ComponentContextProvider componentContextProvider, final ComponentLog connectorLogger, + final ExtensionManager extensionManager, final AssetManager assetManager) { + + this.processorNode = processorNode; + this.versionedProcessor = versionedProcessor; + this.parameterContext = parameterContext; + this.componentContextProvider = componentContextProvider; + this.connectorLogger = connectorLogger; + this.extensionManager = extensionManager; + this.assetManager = assetManager; + + this.lifecycle = new StandaloneProcessorLifecycle(processorNode, scheduler); + } + + @Override + public VersionedProcessor getDefinition() { + return versionedProcessor; + } + + @Override + public ProcessorLifecycle getLifecycle() { + return lifecycle; + } + + @Override + public List validate() { + final ValidationStatus status = processorNode.performValidation(); + if (status == ValidationStatus.VALID) { + return Collections.emptyList(); + } + + return processorNode.getValidationErrors().stream() + .filter(result -> !result.isValid()) + .filter(result -> !DisabledServiceValidationResult.isMatch(result)) + .toList(); + } + + @Override + public List validate(final Map propertyValues) { + final ValidationContext validationContext = componentContextProvider.createValidationContext(processorNode, propertyValues, parameterContext); + final ValidationState validationState = processorNode.performValidation(validationContext); + final List validationErrors = validationState.getValidationErrors().stream() + .filter(result -> !result.isValid()) + .filter(result -> !DisabledServiceValidationResult.isMatch(result)) + .toList(); + + return validationErrors; + } + + @Override + public List verify(final Map propertyValues, final Map attributes) { + final ProcessContext processContext = componentContextProvider.createProcessContext(processorNode, propertyValues, parameterContext); + return processorNode.verifyConfiguration(processContext, connectorLogger, attributes, extensionManager, parameterContext); + } + + @Override + public List verify(final Map propertyValues, final VersionedParameterContext versionedParameterContext, final Map attributes) { + final ParameterLookup parameterLookup = new ConnectorParameterLookup(List.of(versionedParameterContext), assetManager); + final ProcessContext processContext = componentContextProvider.createProcessContext(processorNode, propertyValues, parameterLookup); + return processorNode.verifyConfiguration(processContext, connectorLogger, attributes, extensionManager, parameterContext); + } + + @Override + public List verify(final VersionedExternalFlow versionedExternalFlow, final Map attributes) { + final VersionedProcessGroup parentGroup = findParentGroup(versionedExternalFlow.getFlowContents()); + final VersionedProcessor providedProcessor = findVersionedProcessor(parentGroup); + if (providedProcessor == null) { + throw new IllegalArgumentException("Cannot verify Processor with ID " + this.versionedProcessor.getIdentifier() + " because it does not exist in the provided flow"); + } + final ParameterLookup parameterLookup = new ConnectorParameterLookup(versionedExternalFlow.getParameterContexts().values(), assetManager); + + final ProcessContext processContext = componentContextProvider.createProcessContext(processorNode, providedProcessor.getProperties(), parameterLookup); + return processorNode.verifyConfiguration(processContext, connectorLogger, attributes, extensionManager, parameterLookup); + } + + private VersionedProcessGroup findParentGroup(final VersionedProcessGroup group) { + for (final VersionedProcessor processor : group.getProcessors()) { + if (processor.getIdentifier().equals(versionedProcessor.getIdentifier())) { + return group; + } + } + + for (final VersionedProcessGroup childGroup : group.getProcessGroups()) { + final VersionedProcessGroup serviceGroup = findParentGroup(childGroup); + if (serviceGroup != null) { + return serviceGroup; + } + } + + return null; + } + + private VersionedProcessor findVersionedProcessor(final VersionedProcessGroup group) { + if (group == null) { + return null; + } + + final VersionedProcessGroup parentGroup = findParentGroup(group); + if (parentGroup == null) { + return null; + } + + for (final VersionedProcessor processor : group.getProcessors()) { + if (processor.getIdentifier().equals(processorNode.getVersionedComponentId().orElseThrow())) { + return processor; + } + } + + return null; + } + + + @Override + public Object invokeConnectorMethod(final String methodName, final Map arguments) throws InvocationFailedException { + final Map jsonArguments = serializeArgumentsToJson(arguments); + final ProcessContext processContext = componentContextProvider.createProcessContext(processorNode, parameterContext); + final String jsonResult = processorNode.invokeConnectorMethod(methodName, jsonArguments, processContext); + if (jsonResult == null) { + return null; + } + + try { + return OBJECT_MAPPER.readValue(jsonResult, Object.class); + } catch (final JsonProcessingException e) { + throw new InvocationFailedException("Failed to deserialize return value from Connector Method '" + methodName + "'", e); + } + } + + @Override + public T invokeConnectorMethod(final String methodName, final Map arguments, final Class returnType) throws InvocationFailedException { + final Map jsonArguments = serializeArgumentsToJson(arguments); + final ProcessContext processContext = componentContextProvider.createProcessContext(processorNode, parameterContext); + final String jsonResult = processorNode.invokeConnectorMethod(methodName, jsonArguments, processContext); + if (jsonResult == null) { + return null; + } + + try { + return OBJECT_MAPPER.readValue(jsonResult, returnType); + } catch (final JsonProcessingException e) { + throw new InvocationFailedException("Failed to deserialize return value from Connector Method '" + methodName + "'", e); + } + } + + @Override + public String toString() { + final String type = versionedProcessor.getType(); + final String simpleType = type == null ? null : type.substring(type.lastIndexOf('.') + 1); + return "StandaloneProcessorFacade[id=" + versionedProcessor.getIdentifier() + ", name=" + versionedProcessor.getName() + ", type=" + simpleType + "]"; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final StandaloneProcessorFacade that = (StandaloneProcessorFacade) o; + return Objects.equals(versionedProcessor.getIdentifier(), that.versionedProcessor.getIdentifier()); + } + + @Override + public int hashCode() { + return Objects.hashCode(versionedProcessor.getIdentifier()); + } + + private Map serializeArgumentsToJson(final Map arguments) throws InvocationFailedException { + final Map jsonArguments = new HashMap<>(); + + for (final Map.Entry entry : arguments.entrySet()) { + if (entry.getValue() == null) { + jsonArguments.put(entry.getKey(), null); + } else { + try { + jsonArguments.put(entry.getKey(), OBJECT_MAPPER.writeValueAsString(entry.getValue())); + } catch (final JsonProcessingException e) { + throw new InvocationFailedException("Failed to serialize argument '" + entry.getKey() + "' to JSON", e); + } + } + } + + return jsonArguments; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessorLifecycle.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessorLifecycle.java new file mode 100644 index 000000000000..bd83cbe3e70c --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneProcessorLifecycle.java @@ -0,0 +1,93 @@ +/* + * 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.components.connector.facades.standalone; + +import org.apache.nifi.components.connector.components.ProcessorLifecycle; +import org.apache.nifi.components.connector.components.ProcessorState; +import org.apache.nifi.components.validation.ValidationStatus; +import org.apache.nifi.controller.ProcessScheduler; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.lifecycle.ProcessorStopLifecycleMethods; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.CompletableFuture; + +public class StandaloneProcessorLifecycle implements ProcessorLifecycle { + private static final Logger logger = LoggerFactory.getLogger(StandaloneProcessorLifecycle.class); + private final ProcessorNode processorNode; + private final ProcessScheduler processScheduler; + + public StandaloneProcessorLifecycle(final ProcessorNode processorNode, final ProcessScheduler scheduler) { + this.processorNode = processorNode; + this.processScheduler = scheduler; + } + + @Override + public ProcessorState getState() { + final ScheduledState scheduledState = processorNode.getScheduledState(); + return switch (scheduledState) { + case DISABLED -> ProcessorState.DISABLED; + case RUNNING, RUN_ONCE -> ProcessorState.RUNNING; + case STOPPED -> processorNode.getActiveThreadCount() == 0 ? ProcessorState.STOPPED : ProcessorState.STOPPING; + case STOPPING -> ProcessorState.STOPPING; + case STARTING -> ProcessorState.STARTING; + }; + } + + @Override + public int getActiveThreadCount() { + return processorNode.getActiveThreadCount(); + } + + @Override + public void terminate() { + try { + processScheduler.terminateProcessor(processorNode); + } catch (final Exception e) { + logger.warn("Failed to terminate processor {}", processorNode, e); + } + } + + @Override + public CompletableFuture stop() { + return processScheduler.stopProcessor(processorNode, ProcessorStopLifecycleMethods.TRIGGER_ALL); + } + + @Override + public CompletableFuture start() { + // Perform validation if necessary before starting. + final ValidationStatus validationStatus = processorNode.getValidationStatus(); + if (validationStatus != ValidationStatus.VALID) { + processorNode.performValidation(); + } + + return processScheduler.startProcessor(processorNode, false); + } + + @Override + public void disable() { + processScheduler.disableProcessor(processorNode); + } + + @Override + public void enable() { + processScheduler.enableProcessor(processorNode); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneStatelessGroupLifecycle.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneStatelessGroupLifecycle.java new file mode 100644 index 000000000000..f29287b32390 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/facades/standalone/StandaloneStatelessGroupLifecycle.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.components.connector.facades.standalone; + +import org.apache.nifi.components.connector.components.StatelessGroupLifecycle; +import org.apache.nifi.controller.ProcessScheduler; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.groups.StatelessGroupNode; + +import java.util.concurrent.CompletableFuture; + +public class StandaloneStatelessGroupLifecycle implements StatelessGroupLifecycle { + private final StatelessGroupNode statelessGroupNode; + private final ProcessScheduler processScheduler; + + public StandaloneStatelessGroupLifecycle(final ProcessGroup processGroup, final ProcessScheduler processScheduler) { + this.statelessGroupNode = processGroup.getStatelessGroupNode().orElseThrow(() -> new IllegalStateException("Process Group is not configured to run using the Stateless Execution Engine")); + this.processScheduler = processScheduler; + } + + @Override + public CompletableFuture start() { + statelessGroupNode.setDesiredState(ScheduledState.RUNNING); + return processScheduler.startStatelessGroup(statelessGroupNode); + } + + @Override + public CompletableFuture stop() { + statelessGroupNode.setDesiredState(ScheduledState.STOPPED); + return processScheduler.stopStatelessGroup(statelessGroupNode); + } + + // TODO: Stateless Group does not currently support termination. + @Override + public CompletableFuture terminate() { + return CompletableFuture.completedFuture(null); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/ExtensionBuilder.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/ExtensionBuilder.java index 82cf84190cd4..45ef9980d4b2 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/ExtensionBuilder.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/ExtensionBuilder.java @@ -21,10 +21,32 @@ import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.configuration.DefaultSettings; +import org.apache.nifi.authorization.Resource; +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.resource.ResourceFactory; import org.apache.nifi.bundle.Bundle; import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.connector.ComponentBundleLookup; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.Connector; +import org.apache.nifi.components.connector.ConnectorDetails; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorPropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorPropertyGroup; +import org.apache.nifi.components.connector.ConnectorStateTransition; +import org.apache.nifi.components.connector.ConnectorValidationTrigger; +import org.apache.nifi.components.connector.ConnectorValueReference; +import org.apache.nifi.components.connector.FlowContextFactory; +import org.apache.nifi.components.connector.FrameworkConnectorInitializationContext; +import org.apache.nifi.components.connector.FrameworkConnectorInitializationContextBuilder; +import org.apache.nifi.components.connector.FrameworkFlowContext; +import org.apache.nifi.components.connector.GhostConnector; +import org.apache.nifi.components.connector.MutableConnectorConfigurationContext; +import org.apache.nifi.components.connector.StandardConnectorNode; +import org.apache.nifi.components.connector.StepConfiguration; +import org.apache.nifi.components.connector.StringLiteralValue; import org.apache.nifi.components.state.StateManager; import org.apache.nifi.components.state.StateManagerProvider; import org.apache.nifi.components.validation.ValidationTrigger; @@ -52,6 +74,7 @@ import org.apache.nifi.flowanalysis.FlowAnalysisRule; import org.apache.nifi.flowanalysis.FlowAnalysisRuleInitializationContext; import org.apache.nifi.flowanalysis.GhostFlowAnalysisRule; +import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.LoggingContext; import org.apache.nifi.logging.StandardLoggingContext; @@ -89,8 +112,10 @@ import java.lang.reflect.Proxy; import java.net.URL; 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.TimeUnit; import java.util.stream.Collectors; @@ -119,6 +144,13 @@ public class ExtensionBuilder { private SSLContext systemSslContext; private PythonBridge pythonBridge; private VerifiableComponentFactory verifiableComponentFactory; + private ProcessGroup managedProcessGroup; + private FlowContextFactory flowContextFactory; + private MutableConnectorConfigurationContext activeConfigurationContext; + private ConnectorStateTransition connectorStateTransition; + private FrameworkConnectorInitializationContextBuilder connectorInitializationContextBuilder; + private ConnectorValidationTrigger connectorValidationTrigger; + private ComponentBundleLookup componentBundleLookup; public ExtensionBuilder type(final String type) { this.type = type; @@ -193,12 +225,12 @@ public ExtensionBuilder stateManagerProvider(final StateManagerProvider stateMan return this; } - public ExtensionBuilder ruleViolationsManager(RuleViolationsManager ruleViolationsManager) { + public ExtensionBuilder ruleViolationsManager(final RuleViolationsManager ruleViolationsManager) { this.ruleViolationsManager = ruleViolationsManager; return this; } - public ExtensionBuilder flowAnalyzer(FlowAnalyzer flowAnalyzer) { + public ExtensionBuilder flowAnalyzer(final FlowAnalyzer flowAnalyzer) { this.flowAnalyzer = flowAnalyzer; return this; } @@ -228,31 +260,49 @@ public ExtensionBuilder verifiableComponentFactory(final VerifiableComponentFact return this; } + public ExtensionBuilder managedProcessGroup(final ProcessGroup managedProcessGroup) { + this.managedProcessGroup = managedProcessGroup; + return this; + } + + public ExtensionBuilder activeConfigurationContext(final MutableConnectorConfigurationContext configurationContext) { + this.activeConfigurationContext = configurationContext; + return this; + } + + public ExtensionBuilder flowContextFactory(final FlowContextFactory flowContextFactory) { + this.flowContextFactory = flowContextFactory; + return this; + } + + public ExtensionBuilder connectorStateTransition(final ConnectorStateTransition connectorStateTransition) { + this.connectorStateTransition = connectorStateTransition; + return this; + } + + public ExtensionBuilder connectorInitializationContextBuilder(final FrameworkConnectorInitializationContextBuilder contextBuilder) { + this.connectorInitializationContextBuilder = contextBuilder; + return this; + } + + public ExtensionBuilder connectorValidationTrigger(final ConnectorValidationTrigger connectorValidationTrigger) { + this.connectorValidationTrigger = connectorValidationTrigger; + return this; + } + + public ExtensionBuilder componentBundleLookup(final ComponentBundleLookup componentBundleLookup) { + this.componentBundleLookup = componentBundleLookup; + return this; + } + public ProcessorNode buildProcessor() { - if (identifier == null) { - throw new IllegalStateException("Processor ID must be specified"); - } - if (type == null) { - throw new IllegalStateException("Processor Type must be specified"); - } - if (bundleCoordinate == null) { - throw new IllegalStateException("Bundle Coordinate must be specified"); - } - if (extensionManager == null) { - throw new IllegalStateException("Extension Manager must be specified"); - } - if (serviceProvider == null) { - throw new IllegalStateException("Controller Service Provider must be specified"); - } - if (nodeTypeProvider == null) { - throw new IllegalStateException("Node Type Provider must be specified"); - } - if (reloadComponent == null) { - throw new IllegalStateException("Reload Component must be specified"); - } - if (verifiableComponentFactory == null) { - throw new IllegalStateException("Verifiable Component Factory must be specified"); - } + requireNonNull(identifier, "Processor ID"); + requireNonNull(type, "Processor Type"); + requireNonNull(bundleCoordinate, "Bundle Coordinate"); + requireNonNull(extensionManager, "Extension Manager"); + requireNonNull(serviceProvider, "Controller Service Provider"); + requireNonNull(nodeTypeProvider, "Node Type Provider"); + requireNonNull(reloadComponent, "Reload Component"); boolean creationSuccessful = true; final StandardLoggingContext loggingContext = new StandardLoggingContext(); @@ -260,10 +310,7 @@ public ProcessorNode buildProcessor() { try { loggableComponent = createLoggableProcessor(loggingContext); } catch (final ProcessorInstantiationException pie) { - logger.error("Could not create Processor of type {} from {} for ID {} due to: {}; creating \"Ghost\" implementation", type, bundleCoordinate, identifier, pie.getMessage()); - if (logger.isDebugEnabled()) { - logger.debug(pie.getMessage(), pie); - } + logger.error("Could not create Processor of type {} from {} for ID {} due to: {}; creating \"Ghost\" implementation", type, bundleCoordinate, identifier, pie.getMessage(), pie); final GhostProcessor ghostProc = new GhostProcessor(); ghostProc.setIdentifier(identifier); ghostProc.setCanonicalClassName(type); @@ -287,30 +334,14 @@ public ProcessorNode buildProcessor() { } public FlowRegistryClientNode buildFlowRegistryClient() { - if (identifier == null) { - throw new IllegalStateException("ReportingTask ID must be specified"); - } - if (type == null) { - throw new IllegalStateException("ReportingTask Type must be specified"); - } - if (bundleCoordinate == null) { - throw new IllegalStateException("Bundle Coordinate must be specified"); - } - if (serviceProvider == null) { - throw new IllegalStateException("Controller Service Provider must be specified"); - } - if (extensionManager == null) { - throw new IllegalStateException("Extension Manager must be specified"); - } - if (nodeTypeProvider == null) { - throw new IllegalStateException("Node Type Provider must be specified"); - } - if (reloadComponent == null) { - throw new IllegalStateException("Reload Component must be specified"); - } - if (flowController == null) { - throw new IllegalStateException("FlowController must be specified"); - } + requireNonNull(identifier, "Flow Registry Client ID"); + requireNonNull(type, "Flow Registry Client Type"); + requireNonNull(bundleCoordinate, "Bundle Coordinate"); + requireNonNull(serviceProvider, "Controller Service Provider"); + requireNonNull(extensionManager, "Extension Manager"); + requireNonNull(nodeTypeProvider, "Node Type Provider"); + requireNonNull(reloadComponent, "Reload Component"); + requireNonNull(flowController, "FlowController"); boolean creationSuccessful = true; LoggableComponent loggableComponent; @@ -331,30 +362,14 @@ public FlowRegistryClientNode buildFlowRegistryClient() { } public ReportingTaskNode buildReportingTask() { - if (identifier == null) { - throw new IllegalStateException("ReportingTask ID must be specified"); - } - if (type == null) { - throw new IllegalStateException("ReportingTask Type must be specified"); - } - if (bundleCoordinate == null) { - throw new IllegalStateException("Bundle Coordinate must be specified"); - } - if (extensionManager == null) { - throw new IllegalStateException("Extension Manager must be specified"); - } - if (serviceProvider == null) { - throw new IllegalStateException("Controller Service Provider must be specified"); - } - if (nodeTypeProvider == null) { - throw new IllegalStateException("Node Type Provider must be specified"); - } - if (reloadComponent == null) { - throw new IllegalStateException("Reload Component must be specified"); - } - if (flowController == null) { - throw new IllegalStateException("FlowController must be specified"); - } + requireNonNull(identifier, "ReportingTask ID"); + requireNonNull(type, "ReportingTask Type"); + requireNonNull(bundleCoordinate, "Bundle Coordinate"); + requireNonNull(extensionManager, "Extension Manager"); + requireNonNull(serviceProvider, "Controller Service Provider"); + requireNonNull(nodeTypeProvider, "Node Type Provider"); + requireNonNull(reloadComponent, "Reload Component"); + requireNonNull(flowController, "FlowController"); boolean creationSuccessful = true; LoggableComponent loggableComponent; @@ -377,30 +392,14 @@ public ReportingTaskNode buildReportingTask() { } public ParameterProviderNode buildParameterProvider() { - if (identifier == null) { - throw new IllegalStateException("ParameterProvider ID must be specified"); - } - if (type == null) { - throw new IllegalStateException("ParameterProvider Type must be specified"); - } - if (bundleCoordinate == null) { - throw new IllegalStateException("Bundle Coordinate must be specified"); - } - if (extensionManager == null) { - throw new IllegalStateException("Extension Manager must be specified"); - } - if (serviceProvider == null) { - throw new IllegalStateException("Controller Service Provider must be specified"); - } - if (nodeTypeProvider == null) { - throw new IllegalStateException("Node Type Provider must be specified"); - } - if (reloadComponent == null) { - throw new IllegalStateException("Reload Component must be specified"); - } - if (flowController == null) { - throw new IllegalStateException("FlowController must be specified"); - } + requireNonNull(identifier, "ParameterProvider ID"); + requireNonNull(type, "ParameterProvider Type"); + requireNonNull(bundleCoordinate, "Bundle Coordinate"); + requireNonNull(extensionManager, "Extension Manager"); + requireNonNull(serviceProvider, "Controller Service Provider"); + requireNonNull(nodeTypeProvider, "Node Type Provider"); + requireNonNull(reloadComponent, "Reload Component"); + requireNonNull(flowController, "FlowController"); boolean creationSuccessful = true; LoggableComponent loggableComponent; @@ -464,30 +463,14 @@ public ControllerServiceNode buildControllerService() { } public FlowAnalysisRuleNode buildFlowAnalysisRuleNode() { - if (identifier == null) { - throw new IllegalStateException("FlowAnalysisRule ID must be specified"); - } - if (type == null) { - throw new IllegalStateException("FlowAnalysisRule Type must be specified"); - } - if (bundleCoordinate == null) { - throw new IllegalStateException("Bundle Coordinate must be specified"); - } - if (extensionManager == null) { - throw new IllegalStateException("Extension Manager must be specified"); - } - if (serviceProvider == null) { - throw new IllegalStateException("Controller Service Provider must be specified"); - } - if (nodeTypeProvider == null) { - throw new IllegalStateException("Node Type Provider must be specified"); - } - if (reloadComponent == null) { - throw new IllegalStateException("Reload Component must be specified"); - } - if (flowController == null) { - throw new IllegalStateException("FlowController must be specified"); - } + requireNonNull(identifier, "FlowAnalysisRule ID"); + requireNonNull(type, "FlowAnalysisRule Type"); + requireNonNull(bundleCoordinate, "Bundle Coordinate"); + requireNonNull(extensionManager, "Extension Manager"); + requireNonNull(serviceProvider, "Controller Service Provider"); + requireNonNull(nodeTypeProvider, "Node Type Provider"); + requireNonNull(reloadComponent, "Reload Component"); + requireNonNull(flowController, "FlowController"); boolean creationSuccessful = true; LoggableComponent loggableComponent; @@ -510,6 +493,145 @@ public FlowAnalysisRuleNode buildFlowAnalysisRuleNode() { return flowAnalysisRuleNode; } + public ConnectorNode buildConnector(final boolean loadInitialFlow) { + requireNonNull(identifier, "Connector ID"); + requireNonNull(type, "Connector Type"); + requireNonNull(bundleCoordinate, "Bundle Coordinate"); + requireNonNull(extensionManager, "Extension Manager"); + requireNonNull(managedProcessGroup, "Managed Process Group"); + + final Authorizable connectorsAuthorizable = new Authorizable() { + @Override + public Authorizable getParentAuthorizable() { + return null; + } + + @Override + public Resource getResource() { + return ResourceFactory.getConnectorsResource(); + } + }; + + final Connector connector; + try { + connector = createConnector(); + } catch (final Exception e) { + logger.error("Could not create Connector of type {} from {} for ID {} due to: {}; creating \"Ghost\" implementation", type, bundleCoordinate, identifier, e.getMessage(), e); + return createGhostConnectorNode(connectorsAuthorizable, e); + } + + final String componentType = connector.getClass().getSimpleName(); + final ComponentLog componentLog = new SimpleProcessLogger(identifier, connector, new StandardLoggingContext()); + final ConnectorDetails connectorDetails = new ConnectorDetails(connector, bundleCoordinate, componentLog); + + final ConnectorNode connectorNode = new StandardConnectorNode( + identifier, + flowController.getFlowManager(), + extensionManager, + connectorsAuthorizable, + connectorDetails, + componentType, + type, + activeConfigurationContext, + connectorStateTransition, + flowContextFactory, + connectorValidationTrigger, + false + ); + + try { + initializeDefaultValues(connector, connectorNode.getActiveFlowContext()); + + final FrameworkConnectorInitializationContext initContext = createConnectorInitializationContext(managedProcessGroup, componentLog); + connectorNode.initializeConnector(initContext); + } catch (final Exception e) { + logger.error("Could not initialize Connector of type {} from {} for ID {}; creating \"Ghost\" implementation", type, bundleCoordinate, identifier, e); + return createGhostConnectorNode(connectorsAuthorizable, e); + } + + if (loadInitialFlow) { + try { + connectorNode.loadInitialFlow(); + } catch (final Exception e) { + logger.error("Failed to load initial flow for {}; creating \"Ghost\" implementation", connectorNode, e); + return createGhostConnectorNode(connectorsAuthorizable, e); + } + } + + return connectorNode; + } + + private ConnectorNode createGhostConnectorNode(final Authorizable connectorsAuthorizable, final Exception cause) { + final GhostConnector ghostConnector = new GhostConnector(identifier, type, cause); + final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type; + final String componentType = "(Missing) " + simpleClassName; + final ComponentLog componentLog = new SimpleProcessLogger(identifier, ghostConnector, new StandardLoggingContext()); + final ConnectorDetails connectorDetails = new ConnectorDetails(ghostConnector, bundleCoordinate, componentLog); + + // If an instance class loader has been created for this connector, remove it because it's no longer necessary. + extensionManager.removeInstanceClassLoader(identifier); + + final ConnectorNode connectorNode = new StandardConnectorNode( + identifier, + flowController.getFlowManager(), + extensionManager, + connectorsAuthorizable, + connectorDetails, + componentType, + type, + activeConfigurationContext, + connectorStateTransition, + flowContextFactory, + connectorValidationTrigger, + true + ); + + // Initialize the ghost connector so that it can be properly configured during flow synchronization + final FrameworkConnectorInitializationContext initContext = createConnectorInitializationContext(managedProcessGroup, componentLog); + connectorNode.initializeConnector(initContext); + + return connectorNode; + } + + private void initializeDefaultValues(final Connector connector, final FrameworkFlowContext flowContext) { + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, connector.getClass(), identifier)) { + final List configSteps = connector.getConfigurationSteps(); + + for (final ConfigurationStep step : configSteps) { + final Map defaultValues = new HashMap<>(); + + for (final ConnectorPropertyGroup propertyGroup : step.getPropertyGroups()) { + for (final ConnectorPropertyDescriptor descriptor : propertyGroup.getProperties()) { + final String name = descriptor.getName(); + final String defaultValue = descriptor.getDefaultValue(); + defaultValues.put(name, defaultValue == null ? null : new StringLiteralValue(defaultValue)); + } + } + + flowContext.getConfigurationContext().setProperties(step.getName(), new StepConfiguration(defaultValues)); + } + } + } + + private FrameworkConnectorInitializationContext createConnectorInitializationContext(final ProcessGroup managedProcessGroup, final ComponentLog componentLog) { + final String name = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type; + + return connectorInitializationContextBuilder + .identifier(identifier) + .name(name) + .componentLog(componentLog) + .secretsManager(flowController.getConnectorRepository().getSecretsManager()) + .assetManager(flowController.getConnectorAssetManager()) + .componentBundleLookup(componentBundleLookup) + .build(); + } + + private void requireNonNull(final Object value, final String fieldName) { + if (value == null) { + throw new IllegalStateException(fieldName + " must be specified"); + } + } + private ProcessorNode createProcessorNode(final LoggableComponent processor, final String componentType, final boolean extensionMissing) { final ValidationContextFactory validationContextFactory = createValidationContextFactory(serviceProvider); @@ -542,7 +664,7 @@ private ReportingTaskNode createReportingTaskNode(final LoggableComponent createLoggablePythonProcessor(final Logging } } - private LoggableComponent createLoggableComponent(Class nodeType, LoggingContext loggingContext) + private Connector createConnector() throws Exception { + final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader(); + try { + final Bundle bundle = extensionManager.getBundle(bundleCoordinate); + if (bundle == null) { + throw new IllegalStateException("Unable to find bundle for coordinate " + bundleCoordinate.getCoordinate()); + } + + final ClassLoader detectedClassLoader = extensionManager.createInstanceClassLoader(type, identifier, bundle, + classpathUrls == null ? Collections.emptySet() : classpathUrls, true, classloaderIsolationKey); + final Class rawClass = Class.forName(type, true, detectedClassLoader); + Thread.currentThread().setContextClassLoader(detectedClassLoader); + + final Object extensionInstance = rawClass.getDeclaredConstructor().newInstance(); + final Connector connector = (Connector) extensionInstance; + + // TODO: Initialize the Connector - this will require creating a ConnectorInitializationContext + // For now, we'll skip initialization as it requires additional infrastructure + // that may not be available at this point. The initialization should happen + // when the connector is actually started/configured. + + return connector; + } catch (final Throwable t) { + throw new Exception("Failed to create Connector of type " + type, t); + } finally { + if (ctxClassLoader != null) { + Thread.currentThread().setContextClassLoader(ctxClassLoader); + } + } + } + + private LoggableComponent createLoggableComponent(final Class nodeType, final LoggingContext loggingContext) throws ClassNotFoundException, IllegalAccessException, InstantiationException, NoSuchMethodException, InvocationTargetException { final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java index ea40ea135bbd..14cd4f505d0c 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java @@ -215,6 +215,15 @@ public void purge() { } } + @Override + public void deleteSwapFile(final String swapLocation) throws IOException { + final File swapFile = new File(swapLocation); + if (swapFile.exists() && !swapFile.delete()) { + throw new IOException("Failed to delete swap file " + swapLocation); + } + logger.debug("Deleted swap file {}", swapLocation); + } + @Override public String getQueueIdentifier(final String swapLocation) { final String filename = swapLocation.contains("/") ? StringUtils.substringAfterLast(swapLocation, "/") : swapLocation; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index cdbbc6e2c9be..e83b60bde9d6 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -27,6 +27,7 @@ import org.apache.nifi.asset.StandardAssetManager; import org.apache.nifi.asset.StandardAssetManagerInitializationContext; import org.apache.nifi.asset.StandardAssetReferenceLookup; +import org.apache.nifi.asset.StandardConnectorAssetManager; import org.apache.nifi.authorization.Authorizer; import org.apache.nifi.authorization.Resource; import org.apache.nifi.authorization.resource.Authorizable; @@ -48,6 +49,20 @@ import org.apache.nifi.cluster.protocol.UnknownServiceAddressException; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; import org.apache.nifi.components.ClassLoaderAwarePythonBridge; +import org.apache.nifi.components.connector.ConnectorConfigurationProvider; +import org.apache.nifi.components.connector.ConnectorConfigurationProviderInitializationContext; +import org.apache.nifi.components.connector.ConnectorRepository; +import org.apache.nifi.components.connector.ConnectorRepositoryInitializationContext; +import org.apache.nifi.components.connector.ConnectorRequestReplicator; +import org.apache.nifi.components.connector.ConnectorValidationTrigger; +import org.apache.nifi.components.connector.StandardConnectorConfigurationProviderInitializationContext; +import org.apache.nifi.components.connector.StandardConnectorRepoInitializationContext; +import org.apache.nifi.components.connector.StandardConnectorRepository; +import org.apache.nifi.components.connector.StandardConnectorValidationTrigger; +import org.apache.nifi.components.connector.secrets.ParameterProviderSecretsManager; +import org.apache.nifi.components.connector.secrets.SecretsManager; +import org.apache.nifi.components.connector.secrets.SecretsManagerInitializationContext; +import org.apache.nifi.components.connector.secrets.StandardSecretsManagerInitializationContext; import org.apache.nifi.components.monitor.LongRunningTaskMonitor; import org.apache.nifi.components.state.StateManagerProvider; import org.apache.nifi.components.state.StateProvider; @@ -190,7 +205,7 @@ import org.apache.nifi.python.PythonBridgeInitializationContext; import org.apache.nifi.python.PythonProcessConfig; import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedProcessGroup; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup; import org.apache.nifi.remote.HttpRemoteSiteListener; import org.apache.nifi.remote.RemoteGroupPort; @@ -266,6 +281,9 @@ public class FlowController implements ReportingTaskProvider, FlowAnalysisRulePr public static final String DEFAULT_PROVENANCE_REPO_IMPLEMENTATION = "org.apache.nifi.provenance.VolatileProvenanceRepository"; public static final String DEFAULT_SWAP_MANAGER_IMPLEMENTATION = "org.apache.nifi.controller.FileSystemSwapManager"; public static final String DEFAULT_ASSET_MANAGER_IMPLEMENTATION = StandardAssetManager.class.getName(); + public static final String DEFAULT_CONNECTOR_ASSET_MANAGER_IMPLEMENTATION = StandardConnectorAssetManager.class.getName(); + public static final String DEFAULT_CONNECTOR_REPOSITORY_IMPLEMENTATION = StandardConnectorRepository.class.getName(); + public static final String DEFAULT_SECRETS_MANAGER_IMPLEMENTATION = ParameterProviderSecretsManager.class.getName(); public static final String GRACEFUL_SHUTDOWN_PERIOD = "nifi.flowcontroller.graceful.shutdown.seconds"; public static final long DEFAULT_GRACEFUL_SHUTDOWN_SECONDS = 10; @@ -281,6 +299,7 @@ public class FlowController implements ReportingTaskProvider, FlowAnalysisRulePr private final ProvenanceRepository provenanceRepository; private final BulletinRepository bulletinRepository; private final AssetManager assetManager; + private final AssetManager connectorAssetManager; private final LifecycleStateManager lifecycleStateManager; private final StandardProcessScheduler processScheduler; private final SnippetManager snippetManager; @@ -300,6 +319,7 @@ public class FlowController implements ReportingTaskProvider, FlowAnalysisRulePr private final StateManagerProvider stateManagerProvider; private final long systemStartTime = System.currentTimeMillis(); // time at which the node was started private final RevisionManager revisionManager; + private final ConnectorRepository connectorRepository; private final ConnectionLoadBalanceServer loadBalanceServer; private final NioAsyncLoadBalanceClientRegistry loadBalanceClientRegistry; @@ -328,6 +348,7 @@ public class FlowController implements ReportingTaskProvider, FlowAnalysisRulePr private final FlowEngine validationThreadPool; private final FlowEngine flowAnalysisThreadPool; private final ValidationTrigger validationTrigger; + private final ConnectorValidationTrigger connectorValidationTrigger; private final ReloadComponent reloadComponent; private final VerifiableComponentFactory verifiableComponentFactory; private final ProvenanceAuthorizableFactory provenanceAuthorizableFactory; @@ -339,6 +360,7 @@ public class FlowController implements ReportingTaskProvider, FlowAnalysisRulePr private final RingBufferGarbageCollectionLog gcLog; private final Optional longRunningTaskMonitorThreadPool; + /** * true if controller is configured to operate in a clustered environment */ @@ -413,7 +435,8 @@ public static FlowController createStandaloneInstance( final ExtensionDiscoveringManager extensionManager, final StatusHistoryRepository statusHistoryRepository, final RuleViolationsManager ruleViolationsManager, - final StateManagerProvider stateManagerProvider + final StateManagerProvider stateManagerProvider, + final ConnectorRequestReplicator connectorRequestReplicator ) { return new FlowController( @@ -434,7 +457,8 @@ public static FlowController createStandaloneInstance( null, statusHistoryRepository, ruleViolationsManager, - stateManagerProvider + stateManagerProvider, + connectorRequestReplicator ); } @@ -455,7 +479,8 @@ public static FlowController createClusteredInstance( final RevisionManager revisionManager, final StatusHistoryRepository statusHistoryRepository, final RuleViolationsManager ruleViolationsManager, - final StateManagerProvider stateManagerProvider + final StateManagerProvider stateManagerProvider, + final ConnectorRequestReplicator connectorRequestReplicator ) { return new FlowController( @@ -476,7 +501,8 @@ public static FlowController createClusteredInstance( revisionManager, statusHistoryRepository, ruleViolationsManager, - stateManagerProvider + stateManagerProvider, + connectorRequestReplicator ); } @@ -498,7 +524,8 @@ private FlowController( final RevisionManager revisionManager, final StatusHistoryRepository statusHistoryRepository, final RuleViolationsManager ruleViolationsManager, - final StateManagerProvider stateManagerProvider + final StateManagerProvider stateManagerProvider, + final ConnectorRequestReplicator connectorRequestReplicator ) { maxTimerDrivenThreads = new AtomicInteger(10); @@ -567,21 +594,26 @@ private FlowController( } lifecycleStateManager = new StandardLifecycleStateManager(); + reloadComponent = new StandardReloadComponent(this); processScheduler = new StandardProcessScheduler(timerDrivenEngineRef.get(), this, stateManagerProvider, this.nifiProperties, lifecycleStateManager); parameterContextManager = new StandardParameterContextManager(); final long maxAppendableBytes = getMaxAppendableBytes(); - repositoryContextFactory = new RepositoryContextFactory( - contentRepository, - flowFileRepository, - flowFileEventRepository, - counterRepositoryRef.get(), - getComponentMetricReporter(), - provenanceRepository, - stateManagerProvider, - maxAppendableBytes + repositoryContextFactory = new RepositoryContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, + counterRepositoryRef.get(), componentMetricReporter, provenanceRepository, stateManagerProvider, maxAppendableBytes); + + assetManager = createAssetManager( + nifiProperties, + NiFiProperties.ASSET_MANAGER_IMPLEMENTATION, + NiFiProperties.ASSET_MANAGER_PREFIX, + DEFAULT_ASSET_MANAGER_IMPLEMENTATION + ); + connectorAssetManager = createAssetManager( + nifiProperties, + NiFiProperties.CONNECTOR_ASSET_MANAGER_IMPLEMENTATION, + NiFiProperties.CONNECTOR_ASSET_MANAGER_PREFIX, + DEFAULT_CONNECTOR_ASSET_MANAGER_IMPLEMENTATION ); - assetManager = createAssetManager(nifiProperties); this.flowAnalysisThreadPool = new FlowEngine(1, "Background Flow Analysis", true); if (ruleViolationsManager != null) { @@ -602,10 +634,17 @@ private FlowController( parameterContextManager ); + connectorRequestReplicator.setFlowManager(flowManager); + controllerServiceProvider = new StandardControllerServiceProvider(processScheduler, bulletinRepository, flowManager, extensionManager); - controllerServiceResolver = new StandardControllerServiceResolver(authorizer, flowManager, new NiFiRegistryFlowMapper(extensionManager), + controllerServiceResolver = new StandardControllerServiceResolver(authorizer, flowManager, new VersionedComponentFlowMapper(extensionManager), controllerServiceProvider, new StandardControllerServiceApiLookup(extensionManager)); + final SecretsManager secretsManager = createSecretsManager(nifiProperties, extensionManager, flowManager); + final ConnectorConfigurationProvider connectorConfigurationProvider = createConnectorConfigurationProvider(nifiProperties, extensionManager, connectorAssetManager); + connectorRepository = createConnectorRepository(nifiProperties, extensionManager, flowManager, connectorAssetManager, secretsManager, this, connectorRequestReplicator, + connectorConfigurationProvider); + final PythonBridge rawPythonBridge = createPythonBridge(nifiProperties, controllerServiceProvider); final ClassLoader pythonBridgeClassLoader = rawPythonBridge.getClass().getClassLoader(); final PythonBridge classloaderAwareBridge = new ClassLoaderAwarePythonBridge(rawPythonBridge, pythonBridgeClassLoader); @@ -658,7 +697,6 @@ private FlowController( this.heartbeatDelaySeconds = (int) FormatUtils.getTimeDuration(nifiProperties.getNodeHeartbeatInterval(), TimeUnit.SECONDS); this.snippetManager = new SnippetManager(); - this.reloadComponent = new StandardReloadComponent(this); this.verifiableComponentFactory = new StandardVerifiableComponentFactory(this, this.nifiProperties); final ProcessGroup rootGroup = flowManager.createProcessGroup(ComponentIdGenerator.generateId().toString()); @@ -668,6 +706,7 @@ private FlowController( this.validationThreadPool = new FlowEngine(5, "Validate Components", true); this.validationTrigger = new StandardValidationTrigger(validationThreadPool, this::isInitialized); + this.connectorValidationTrigger = new StandardConnectorValidationTrigger(validationThreadPool, this::isInitialized); if (remoteInputSocketPort == null) { LOG.info("Not enabling RAW Socket Site-to-Site functionality because nifi.remote.input.socket.port is not set"); @@ -858,7 +897,7 @@ public Resource getResource() { return ResourceFactory.getControllerResource(); } - private static FlowFileRepository createFlowFileRepository(final NiFiProperties properties, final ExtensionManager extensionManager, final ResourceClaimManager contentClaimManager) { + private static FlowFileRepository createFlowFileRepository(final NiFiProperties properties, final ExtensionManager extensionManager, final ResourceClaimManager resourceClaimManager) { final String implementationClassName = properties.getProperty(NiFiProperties.FLOWFILE_REPOSITORY_IMPLEMENTATION, DEFAULT_FLOWFILE_REPO_IMPLEMENTATION); if (implementationClassName == null) { throw new RuntimeException("Cannot create FlowFile Repository because the NiFi Properties is missing the following property: " @@ -868,7 +907,7 @@ private static FlowFileRepository createFlowFileRepository(final NiFiProperties try { final FlowFileRepository created = NarThreadContextClassLoader.createInstance(extensionManager, implementationClassName, FlowFileRepository.class, properties); synchronized (created) { - created.initialize(contentClaimManager); + created.initialize(resourceClaimManager); } return created; @@ -877,6 +916,113 @@ private static FlowFileRepository createFlowFileRepository(final NiFiProperties } } + private static ConnectorRepository createConnectorRepository(final NiFiProperties properties, final ExtensionDiscoveringManager extensionManager, final FlowManager flowManager, + final AssetManager assetManager, final SecretsManager secretsManager, final NodeTypeProvider nodeTypeProvider, final ConnectorRequestReplicator requestReplicator, + final ConnectorConfigurationProvider connectorConfigurationProvider) { + + final String implementationClassName = properties.getProperty(NiFiProperties.CONNECTOR_REPOSITORY_IMPLEMENTATION, DEFAULT_CONNECTOR_REPOSITORY_IMPLEMENTATION); + + try { + // Discover implementations of Connector Repository. This is not done at startup because the ConnectorRepository class is not + // provided in the list of standard extension points. This is due to the fact that ConnectorRepository lives in the nifi-framework-core-api, and + // does not make sense to refactor it into some other module due to its dependencies, simply to allow it to be discovered at startup. + final Set> additionalExtensionTypes = Set.of(ConnectorRepository.class, SecretsManager.class, ConnectorConfigurationProvider.class); + extensionManager.discoverExtensions(extensionManager.getAllBundles(), additionalExtensionTypes, true); + final ConnectorRepository created = NarThreadContextClassLoader.createInstance(extensionManager, implementationClassName, ConnectorRepository.class, properties); + + final ConnectorRepositoryInitializationContext initializationContext = new StandardConnectorRepoInitializationContext( + flowManager, + extensionManager, + secretsManager, + assetManager, + nodeTypeProvider, + requestReplicator, + connectorConfigurationProvider + ); + + synchronized (created) { + // Ensure that any NAR dependencies are available when we initialize the ConnectorRepository + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, created.getClass(), "connector-repository")) { + created.initialize(initializationContext); + } + } + + LOG.info("Created Connector Repository of type {}", created.getClass().getSimpleName()); + + return created; + } catch (final Exception e) { + throw new RuntimeException(e); + } + } + + private static SecretsManager createSecretsManager(final NiFiProperties properties, final ExtensionDiscoveringManager extensionManager, final FlowManager flowManager) { + final String implementationClassName = properties.getProperty(NiFiProperties.SECRETS_MANAGER_IMPLEMENTATION, DEFAULT_SECRETS_MANAGER_IMPLEMENTATION); + + try { + // Discover implementations of Secrets Manager. This is not done at startup because the SecretsManager class is not + // provided in the list of standard extension points. This is due to the fact that SecretsManager lives in the nifi-framework-core-api, and + // does not make sense to refactor it into some other module due to its dependencies, simply to allow it to be discovered at startup. + extensionManager.discoverExtensions(extensionManager.getAllBundles(), Set.of(SecretsManager.class), true); + final SecretsManager created = NarThreadContextClassLoader.createInstance(extensionManager, implementationClassName, SecretsManager.class, properties); + + final SecretsManagerInitializationContext initializationContext = new StandardSecretsManagerInitializationContext(flowManager); + + synchronized (created) { + // Ensure that any NAR dependencies are available when we initialize the ConnectorRepository + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, created.getClass(), "secrets-manager")) { + created.initialize(initializationContext); + } + } + + LOG.info("Created Secrets Manager of type {}", created.getClass().getSimpleName()); + + return created; + } catch (final Exception e) { + throw new RuntimeException(e); + } + } + + private static ConnectorConfigurationProvider createConnectorConfigurationProvider(final NiFiProperties properties, final ExtensionDiscoveringManager extensionManager, + final AssetManager connectorAssetManager) { + final String implementationClassName = properties.getProperty(NiFiProperties.CONNECTOR_CONFIGURATION_PROVIDER_IMPLEMENTATION); + if (implementationClassName == null || implementationClassName.isBlank()) { + LOG.info("No Connector Configuration Provider implementation configured; external connector configuration management is disabled"); + return null; + } + + try { + extensionManager.discoverExtensions(extensionManager.getAllBundles(), Set.of(ConnectorConfigurationProvider.class), true); + final ConnectorConfigurationProvider created = NarThreadContextClassLoader.createInstance( + extensionManager, implementationClassName, ConnectorConfigurationProvider.class, properties); + + final Map initializationProperties = properties.getPropertiesWithPrefix(NiFiProperties.CONNECTOR_CONFIGURATION_PROVIDER_PROPERTIES_PREFIX) + .entrySet().stream() + .collect(Collectors.toMap( + entry -> entry.getKey().substring(NiFiProperties.CONNECTOR_CONFIGURATION_PROVIDER_PROPERTIES_PREFIX.length()), + Map.Entry::getValue + )); + + final ConnectorConfigurationProviderInitializationContext initializationContext = + new StandardConnectorConfigurationProviderInitializationContext(initializationProperties, connectorAssetManager); + + synchronized (created) { + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, created.getClass(), "connector-configuration-provider")) { + created.initialize(initializationContext); + } + } + + LOG.info("Created Connector Configuration Provider of type {}", created.getClass().getSimpleName()); + + return created; + } catch (final Exception e) { + throw new RuntimeException("Failed to create Connector Configuration Provider", e); + } + } + + public ConnectorRepository getConnectorRepository() { + return connectorRepository; + } + private PythonBridge createPythonBridge(final NiFiProperties nifiProperties, final ControllerServiceProvider serviceProvider) { final String pythonCommand = nifiProperties.getProperty(NiFiProperties.PYTHON_COMMAND); if (pythonCommand == null) { @@ -1192,7 +1338,7 @@ public void onFlowInitialized(final boolean startDelayedComponents) { Supplier rootProcessGroupSupplier = () -> { ProcessGroup rootProcessGroup = getFlowManager().getRootGroup(); - NiFiRegistryFlowMapper mapper = FlowAnalysisUtil.createMapper(getExtensionManager()); + VersionedComponentFlowMapper mapper = FlowAnalysisUtil.createMapper(getExtensionManager()); InstantiatedVersionedProcessGroup versionedRootProcessGroup = mapper.mapNonVersionedProcessGroup( rootProcessGroup, @@ -1390,14 +1536,14 @@ private ContentRepository createContentRepository(final NiFiProperties propertie } } - private AssetManager createAssetManager(final NiFiProperties properties) { - final String implementationClassName = properties.getProperty(NiFiProperties.ASSET_MANAGER_IMPLEMENTATION, DEFAULT_ASSET_MANAGER_IMPLEMENTATION); + private AssetManager createAssetManager(final NiFiProperties properties, final String implementationClassProperty, final String propertyPrefix, final String defaultImplementationClass) { + final String implementationClassName = properties.getProperty(implementationClassProperty, defaultImplementationClass); try { final AssetManager assetManager = NarThreadContextClassLoader.createInstance(extensionManager, implementationClassName, AssetManager.class, properties); final AssetReferenceLookup assetReferenceLookup = new StandardAssetReferenceLookup(parameterContextManager); - final Map relevantNiFiProperties = properties.getPropertiesWithPrefix(NiFiProperties.ASSET_MANAGER_PREFIX); - final int prefixLength = NiFiProperties.ASSET_MANAGER_PREFIX.length(); + final Map relevantNiFiProperties = properties.getPropertiesWithPrefix(propertyPrefix); + final int prefixLength = propertyPrefix.length(); final Map assetManagerProperties = relevantNiFiProperties.entrySet().stream() .collect(Collectors.toMap(entry -> entry.getKey().substring(prefixLength), Map.Entry::getValue)); @@ -1415,9 +1561,16 @@ public void initialize(final AssetManagerInitializationContext context) { } @Override - public Asset createAsset(final String parameterContextId, final String assetName, final InputStream contents) throws IOException { + public Asset createAsset(final String ownerId, final String assetName, final InputStream contents) throws IOException { try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(assetManagerClassLoader)) { - return assetManager.createAsset(parameterContextId, assetName, contents); + return assetManager.createAsset(ownerId, assetName, contents); + } + } + + @Override + public Asset saveAsset(final String ownerId, final String assetId, final String assetName, final InputStream contents) throws IOException { + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(assetManagerClassLoader)) { + return assetManager.saveAsset(ownerId, assetId, assetName, contents); } } @@ -1429,16 +1582,16 @@ public Optional getAsset(final String id) { } @Override - public List getAssets(final String parameterContextId) { + public List getAssets(final String ownerId) { try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(assetManagerClassLoader)) { - return assetManager.getAssets(parameterContextId); + return assetManager.getAssets(ownerId); } } @Override - public Asset createMissingAsset(final String parameterContextId, final String assetName) { + public Asset createMissingAsset(final String ownerId, final String assetName) { try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(assetManagerClassLoader)) { - return assetManager.createMissingAsset(parameterContextId, assetName); + return assetManager.createMissingAsset(ownerId, assetName); } } @@ -1462,6 +1615,10 @@ public AssetManager getAssetManager() { return assetManager; } + public AssetManager getConnectorAssetManager() { + return connectorAssetManager; + } + private ProvenanceRepository createProvenanceRepository(final NiFiProperties properties) { final String implementationClassName = properties.getProperty(NiFiProperties.PROVENANCE_REPO_IMPLEMENTATION_CLASS, DEFAULT_PROVENANCE_REPO_IMPLEMENTATION); if (StringUtils.isBlank(implementationClassName)) { @@ -1494,6 +1651,10 @@ public ValidationTrigger getValidationTrigger() { return validationTrigger; } + public ConnectorValidationTrigger getConnectorValidationTrigger() { + return connectorValidationTrigger; + } + public PropertyEncryptor getEncryptor() { return encryptor; } @@ -1530,6 +1691,7 @@ public BulletinRepository getBulletinRepository() { public LifecycleStateManager getLifecycleStateManager() { return lifecycleStateManager; } + public SnippetManager getSnippetManager() { return snippetManager; } @@ -2075,7 +2237,7 @@ private Set findAllConnections(final VersionedProcessGroup * @return the process group or null if not group is found */ private ProcessGroup lookupGroup(final String id) { - final ProcessGroup group = flowManager.getGroup(id); + final ProcessGroup group = flowManager.getGroup(id, null); if (group == null) { throw new IllegalStateException("No Group with ID " + id + " exists"); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java index 9cdab7879a4a..fc212ad88e41 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java @@ -122,7 +122,8 @@ public class StandardFlowService implements FlowService, ProtocolHandler { private final ClusterCoordinator clusterCoordinator; private final RevisionManager revisionManager; private final NarManager narManager; - private final AssetSynchronizer assetSynchronizer; + private final AssetSynchronizer parameterContextAssetSynchronizer; + private final AssetSynchronizer connectorAssetSynchronizer; private volatile SaveReportingTask saveReportingTask; /** @@ -153,10 +154,12 @@ public static StandardFlowService createStandaloneInstance( final NiFiProperties nifiProperties, final RevisionManager revisionManager, final NarManager narManager, - final AssetSynchronizer assetSynchronizer, + final AssetSynchronizer parameterContextAssetSynchronizer, + final AssetSynchronizer connectorAssetSynchronizer, final Authorizer authorizer) throws IOException { - return new StandardFlowService(controller, nifiProperties, null, false, null, revisionManager, narManager, assetSynchronizer, authorizer); + return new StandardFlowService(controller, nifiProperties, null, false, null, revisionManager, narManager, + parameterContextAssetSynchronizer, connectorAssetSynchronizer, authorizer); } public static StandardFlowService createClusteredInstance( @@ -166,10 +169,12 @@ public static StandardFlowService createClusteredInstance( final ClusterCoordinator coordinator, final RevisionManager revisionManager, final NarManager narManager, - final AssetSynchronizer assetSynchronizer, + final AssetSynchronizer parameterContextAssetSynchronizer, + final AssetSynchronizer connectorAssetSynchronizer, final Authorizer authorizer) throws IOException { - return new StandardFlowService(controller, nifiProperties, senderListener, true, coordinator, revisionManager, narManager, assetSynchronizer, authorizer); + return new StandardFlowService(controller, nifiProperties, senderListener, true, coordinator, revisionManager, + narManager, parameterContextAssetSynchronizer, connectorAssetSynchronizer, authorizer); } private StandardFlowService( @@ -180,7 +185,8 @@ private StandardFlowService( final ClusterCoordinator clusterCoordinator, final RevisionManager revisionManager, final NarManager narManager, - final AssetSynchronizer assetSynchronizer, + final AssetSynchronizer parameterContextAssetSynchronizer, + final AssetSynchronizer connectorAssetSynchronizer, final Authorizer authorizer) throws IOException { this.nifiProperties = nifiProperties; @@ -196,7 +202,8 @@ private StandardFlowService( } this.revisionManager = revisionManager; this.narManager = narManager; - this.assetSynchronizer = assetSynchronizer; + this.parameterContextAssetSynchronizer = parameterContextAssetSynchronizer; + this.connectorAssetSynchronizer = connectorAssetSynchronizer; this.authorizer = authorizer; if (configuredForClustering) { @@ -926,8 +933,13 @@ private void loadFromConnectionResponse(final ConnectionResponse response) throw // load new controller state loadFromBytes(dataFlow, true, BundleUpdateStrategy.USE_SPECIFIED_OR_COMPATIBLE_OR_GHOST); - // sync assets after loading the flow so that parameter contexts exist first - assetSynchronizer.synchronize(); + // sync assets after loading the flow so that parameter contexts and connectors exist first + if (parameterContextAssetSynchronizer != null) { + parameterContextAssetSynchronizer.synchronize(); + } + if (connectorAssetSynchronizer != null) { + connectorAssetSynchronizer.synchronize(); + } // set node ID on controller before we start heartbeating because heartbeat needs node ID clusterCoordinator.setLocalNodeIdentifier(nodeId); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/FlowControllerFlowContextFactory.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/FlowControllerFlowContextFactory.java new file mode 100644 index 000000000000..dbca31b7e327 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/FlowControllerFlowContextFactory.java @@ -0,0 +1,140 @@ +/* + * 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.flow; + +import org.apache.nifi.components.connector.FlowContextFactory; +import org.apache.nifi.components.connector.FrameworkFlowContext; +import org.apache.nifi.components.connector.MutableConnectorConfigurationContext; +import org.apache.nifi.components.connector.ParameterContextFacadeFactory; +import org.apache.nifi.components.connector.ProcessGroupFacadeFactory; +import org.apache.nifi.components.connector.ProcessGroupFactory; +import org.apache.nifi.components.connector.StandardFlowContext; +import org.apache.nifi.components.connector.components.FlowContextType; +import org.apache.nifi.components.connector.components.ParameterContextFacade; +import org.apache.nifi.components.connector.components.ParameterValue; +import org.apache.nifi.components.connector.facades.standalone.StandaloneParameterContextFacade; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.parameter.Parameter; +import org.apache.nifi.parameter.ParameterContext; +import org.apache.nifi.registry.flow.mapping.ComponentIdLookup; +import org.apache.nifi.registry.flow.mapping.FlowMappingOptions; +import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedProcessGroup; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +public class FlowControllerFlowContextFactory implements FlowContextFactory { + private final FlowController flowController; + private final ProcessGroup activeManagedProcessGroup; + private final MutableConnectorConfigurationContext activeConfigurationContext; + private final ParameterContextFacadeFactory parameterContextFacadeFactory; + private final ProcessGroupFacadeFactory processGroupFacadeFactory; + private final ProcessGroupFactory processGroupFactory; + + public FlowControllerFlowContextFactory(final FlowController flowController, final ProcessGroup activeManagedProcessGroup, + final MutableConnectorConfigurationContext activeConfigurationContext, final ProcessGroupFactory processGroupFactory) { + + this.flowController = flowController; + this.activeManagedProcessGroup = activeManagedProcessGroup; + this.activeConfigurationContext = activeConfigurationContext; + this.processGroupFactory = processGroupFactory; + + this.processGroupFacadeFactory = new FlowControllerProcessGroupFacadeFactory(flowController); + this.parameterContextFacadeFactory = processGroup -> new StandaloneParameterContextFacade(flowController, processGroup); + } + + @Override + public FrameworkFlowContext createActiveFlowContext(final String connectorId, final ComponentLog connectorLogger, final Bundle bundle) { + return new StandardFlowContext(activeManagedProcessGroup, activeConfigurationContext, processGroupFacadeFactory, + parameterContextFacadeFactory, connectorLogger, FlowContextType.ACTIVE, bundle); + } + + @Override + public FrameworkFlowContext createWorkingFlowContext(final String connectorId, final ComponentLog connectorLogger, + final MutableConnectorConfigurationContext activeConfigurationContext, final Bundle bundle) { + + final String workingGroupId = UUID.nameUUIDFromBytes((connectorId + "-working").getBytes(StandardCharsets.UTF_8)).toString(); + final ProcessGroup processGroup = processGroupFactory.create(workingGroupId); + copyGroupContents(activeManagedProcessGroup, processGroup, connectorId + "-working-context"); + + final MutableConnectorConfigurationContext workingConfigurationContext = activeConfigurationContext.clone(); + + return new StandardFlowContext(processGroup, workingConfigurationContext, processGroupFacadeFactory, + parameterContextFacadeFactory, connectorLogger, FlowContextType.WORKING, bundle); + } + + private void copyGroupContents(final ProcessGroup sourceGroup, final ProcessGroup destinationGroup, final String componentIdSeed) { + final FlowMappingOptions flowMappingOptions = new FlowMappingOptions.Builder() + .mapSensitiveConfiguration(true) + .mapPropertyDescriptors(true) + .stateLookup(VersionedComponentStateLookup.ENABLED_OR_DISABLED) + .sensitiveValueEncryptor(value -> value) + .componentIdLookup(ComponentIdLookup.VERSIONED_OR_GENERATE) + .mapInstanceIdentifiers(true) + .mapControllerServiceReferencesToVersionedId(true) + .mapFlowRegistryClientId(true) + .mapAssetReferences(true) + .build(); + + final VersionedComponentFlowMapper flowMapper = new VersionedComponentFlowMapper(flowController.getExtensionManager(), flowMappingOptions); + final InstantiatedVersionedProcessGroup versionedGroup = flowMapper.mapProcessGroup(sourceGroup, flowController.getControllerServiceProvider(), + flowController.getFlowManager(), true); + + final String contextName = sourceGroup.getParameterContext().getName(); + + final VersionedExternalFlow externalFlowWithoutParameterContext = new VersionedExternalFlow(); + externalFlowWithoutParameterContext.setFlowContents(versionedGroup); + externalFlowWithoutParameterContext.setParameterContexts(Map.of()); + + final String duplicateContextId = UUID.nameUUIDFromBytes((destinationGroup.getIdentifier() + "-param-context").getBytes(StandardCharsets.UTF_8)).toString(); + final ParameterContext sourceContext = sourceGroup.getParameterContext(); + final ParameterContext duplicateParameterContext = flowController.getFlowManager().createEmptyParameterContext( + duplicateContextId, contextName, sourceContext.getDescription(), destinationGroup); + + destinationGroup.setParameterContext(duplicateParameterContext); + destinationGroup.updateFlow(externalFlowWithoutParameterContext, componentIdSeed, false, true, true); + + final ParameterContextFacade contextFacade = new StandaloneParameterContextFacade(flowController, destinationGroup); + final List parameterValues = createParameterValues(sourceContext); + contextFacade.updateParameters(parameterValues); + } + + private List createParameterValues(final ParameterContext context) { + final List parameterValues = new ArrayList<>(); + for (final Parameter parameter : context.getParameters().values()) { + final ParameterValue.Builder parameterValueBuilder = new ParameterValue.Builder() + .name(parameter.getDescriptor().getName()) + .sensitive(parameter.getDescriptor().isSensitive()) + .value(parameter.getValue()); + + parameter.getReferencedAssets().forEach(parameterValueBuilder::addReferencedAsset); + parameterValues.add(parameterValueBuilder.build()); + } + + return parameterValues; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/FlowControllerProcessGroupFacadeFactory.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/FlowControllerProcessGroupFacadeFactory.java new file mode 100644 index 000000000000..53bbe05b9ff2 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/FlowControllerProcessGroupFacadeFactory.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.controller.flow; + +import org.apache.nifi.components.connector.ProcessGroupFacadeFactory; +import org.apache.nifi.components.connector.components.ProcessGroupFacade; +import org.apache.nifi.components.connector.facades.standalone.ComponentContextProvider; +import org.apache.nifi.components.connector.facades.standalone.StandaloneProcessGroupFacade; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.ProcessScheduler; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.parameter.ParameterContext; +import org.apache.nifi.registry.flow.mapping.ComponentIdLookup; +import org.apache.nifi.registry.flow.mapping.FlowMappingOptions; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup; + +public class FlowControllerProcessGroupFacadeFactory implements ProcessGroupFacadeFactory { + private final FlowController flowController; + + public FlowControllerProcessGroupFacadeFactory(final FlowController flowController) { + this.flowController = flowController; + } + + @Override + public ProcessGroupFacade create(final ProcessGroup processGroup, final ComponentLog connectorLogger) { + final FlowMappingOptions flowMappingOptions = new FlowMappingOptions.Builder() + .mapSensitiveConfiguration(true) + .mapPropertyDescriptors(true) + .stateLookup(VersionedComponentStateLookup.IDENTITY_LOOKUP) + .sensitiveValueEncryptor(value -> value) + .componentIdLookup(ComponentIdLookup.VERSIONED_OR_GENERATE) + .mapInstanceIdentifiers(true) + .mapControllerServiceReferencesToVersionedId(true) + .mapFlowRegistryClientId(true) + .mapAssetReferences(true) + .build(); + + final VersionedComponentFlowMapper flowMapper = new VersionedComponentFlowMapper(flowController.getExtensionManager(), flowMappingOptions); + + final VersionedProcessGroup versionedManagedGroup = flowMapper.mapProcessGroup(processGroup, flowController.getControllerServiceProvider(), + flowController.getFlowManager(), true); + final ComponentContextProvider componentContextProvider = new StandardComponentContextProvider(flowController); + final ParameterContext parameterContext = processGroup.getParameterContext(); + final ProcessScheduler processScheduler = flowController.getProcessScheduler(); + final ExtensionManager extensionManager = flowController.getExtensionManager(); + + return new StandaloneProcessGroupFacade(processGroup, versionedManagedGroup, + processScheduler, parameterContext, flowController.getControllerServiceProvider(), + componentContextProvider, connectorLogger, extensionManager, flowController.getAssetManager()); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardComponentContextProvider.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardComponentContextProvider.java new file mode 100644 index 000000000000..89e1c215a7a0 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardComponentContextProvider.java @@ -0,0 +1,105 @@ +/* + * 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.flow; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.connector.facades.standalone.ComponentContextProvider; +import org.apache.nifi.components.state.StateManager; +import org.apache.nifi.controller.ComponentNode; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.lifecycle.TaskTermination; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.StandardConfigurationContext; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.parameter.ParameterLookup; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.StandardProcessContext; + +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; + +public class StandardComponentContextProvider implements ComponentContextProvider { + private final FlowController flowController; + + public StandardComponentContextProvider(final FlowController flowController) { + this.flowController = flowController; + } + + @Override + public ProcessContext createProcessContext(final ProcessorNode processorNode, final ParameterLookup parameterLookup) { + return createProcessContext(processorNode, Map.of(), parameterLookup); + } + + @Override + public ProcessContext createProcessContext(final ProcessorNode processorNode, final Map propertiesOverride, final ParameterLookup parameterLookup) { + final StateManager stateManager = flowController.getStateManagerProvider().getStateManager(processorNode.getIdentifier()); + final TaskTermination taskTermination = () -> false; + final Map serviceReferencedProperties = resolveServiceReferences(processorNode, processorNode.getProcessGroup(), propertiesOverride); + return new StandardProcessContext(processorNode, serviceReferencedProperties, null, parameterLookup, + flowController.getControllerServiceProvider(), stateManager, taskTermination, flowController); + } + + @Override + public ValidationContext createValidationContext(final ProcessorNode processorNode, final Map properties, final ParameterLookup parameterLookup) { + final Map serviceReferencedProperties = resolveServiceReferences(processorNode, processorNode.getProcessGroup(), properties); + return processorNode.createValidationContext(serviceReferencedProperties, processorNode.getAnnotationData(), parameterLookup, true); + } + + private Map resolveServiceReferences(final ComponentNode processorNode, final ProcessGroup processGroup, final Map propertiesOverride) { + final Map versionedToInstanceIds = processGroup.findAllControllerServices().stream() + .filter(cs -> cs.getVersionedComponentId().isPresent()) + .collect(Collectors.toMap(cs -> cs.getVersionedComponentId().get(), ComponentNode::getIdentifier)); + + final Map resolved = new HashMap<>(propertiesOverride); + for (final PropertyDescriptor descriptor : processorNode.getPropertyDescriptors()) { + if (descriptor.getControllerServiceDefinition() == null) { + continue; + } + + final String overriddenValue = propertiesOverride.get(descriptor.getName()); + final String currentValue = overriddenValue == null ? processorNode.getRawPropertyValue(descriptor) : overriddenValue; + final String instanceId = versionedToInstanceIds.get(currentValue); + if (instanceId != null) { + resolved.put(descriptor.getName(), instanceId); + } + } + + return resolved; + } + + @Override + public ConfigurationContext createConfigurationContext(final ControllerServiceNode serviceNode, final ParameterLookup parameterLookup) { + return createConfigurationContext(serviceNode, Map.of(), parameterLookup); + } + + @Override + public ConfigurationContext createConfigurationContext(final ControllerServiceNode serviceNode, final Map propertiesOverride, final ParameterLookup parameterLookup) { + final Map resolvedProperties = resolveServiceReferences(serviceNode, serviceNode.getProcessGroup(), propertiesOverride); + return new StandardConfigurationContext(serviceNode, resolvedProperties, null, parameterLookup, flowController.getControllerServiceProvider(), null); + } + + @Override + public ValidationContext createValidationContext(final ControllerServiceNode serviceNode, final Map properties, final ParameterLookup parameterLookup) { + final Map resolvedProperties = resolveServiceReferences(serviceNode, serviceNode.getProcessGroup(), properties); + return serviceNode.createValidationContext(resolvedProperties, serviceNode.getAnnotationData(), parameterLookup, true); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardFlowManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardFlowManager.java index e6ba2c0d4d26..144f3cde8c56 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardFlowManager.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardFlowManager.java @@ -27,6 +27,15 @@ import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.connector.ComponentBundleLookup; +import org.apache.nifi.components.connector.Connector; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorRepository; +import org.apache.nifi.components.connector.ConnectorStateTransition; +import org.apache.nifi.components.connector.FlowContextFactory; +import org.apache.nifi.components.connector.ProcessGroupFactory; +import org.apache.nifi.components.connector.StandardComponentBundleLookup; +import org.apache.nifi.components.connector.StandardConnectorConfigurationContext; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; @@ -56,12 +65,15 @@ import org.apache.nifi.controller.service.StandardConfigurationContext; import org.apache.nifi.deprecation.log.DeprecationLogger; import org.apache.nifi.deprecation.log.DeprecationLoggerFactory; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedParameterContext; import org.apache.nifi.flowanalysis.FlowAnalysisRule; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.groups.RemoteProcessGroup; import org.apache.nifi.groups.StandardProcessGroup; import org.apache.nifi.groups.StatelessGroupNodeFactory; +import org.apache.nifi.logging.ConnectorLogObserver; import org.apache.nifi.logging.ControllerServiceLogObserver; import org.apache.nifi.logging.FlowAnalysisRuleLogObserver; import org.apache.nifi.logging.FlowRegistryClientLogObserver; @@ -73,10 +85,18 @@ import org.apache.nifi.logging.ReportingTaskLogObserver; import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.NarCloseable; +import org.apache.nifi.parameter.ParameterContext; import org.apache.nifi.parameter.ParameterContextManager; import org.apache.nifi.parameter.ParameterProvider; +import org.apache.nifi.parameter.ParameterReferenceManager; +import org.apache.nifi.parameter.StandardParameterReferenceManager; import org.apache.nifi.processor.Processor; import org.apache.nifi.registry.flow.FlowRegistryClientNode; +import org.apache.nifi.registry.flow.mapping.ComponentIdLookup; +import org.apache.nifi.registry.flow.mapping.FlowMappingOptions; +import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedProcessGroup; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup; import org.apache.nifi.remote.PublicPort; import org.apache.nifi.remote.StandardPublicPort; import org.apache.nifi.remote.StandardRemoteProcessGroup; @@ -92,6 +112,7 @@ import java.lang.reflect.InvocationTargetException; import java.net.URL; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -100,6 +121,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; @@ -279,13 +301,14 @@ public Port createLocalOutputPort(String id, String name) { } @Override - public ProcessGroup createProcessGroup(final String id) { + public ProcessGroup createProcessGroup(final String id, final String connectorId) { final StatelessGroupNodeFactory statelessGroupNodeFactory = new StandardStatelessGroupNodeFactory(flowController, sslContext, flowController.createKerberosConfig(nifiProperties)); final ProcessGroup group = new StandardProcessGroup(requireNonNull(id), flowController.getControllerServiceProvider(), processScheduler, flowController.getEncryptor(), flowController.getExtensionManager(), flowController.getStateManagerProvider(), this, flowController.getReloadComponent(), flowController, nifiProperties, statelessGroupNodeFactory, - flowController.getAssetManager()); + flowController.getAssetManager(), connectorId); + onProcessGroupAdded(group); return group; @@ -718,6 +741,126 @@ public void removeRootControllerService(final ControllerServiceNode service) { logger.info("{} removed from Flow Controller", service); } + @Override + public ConnectorNode createConnector(final String type, final String id, final BundleCoordinate coordinate, final boolean firstTimeAdded, final boolean registerLogObserver) { + + requireNonNull(type, "Connector Type"); + requireNonNull(id, "Connector ID"); + requireNonNull(coordinate, "Bundle Coordinate"); + + final ExtensionManager extensionManager = flowController.getExtensionManager(); + + final String managedGroupId = UUID.nameUUIDFromBytes((id + "-root").getBytes(StandardCharsets.UTF_8)).toString(); + final ProcessGroup managedRootGroup = createProcessGroup(managedGroupId, id); + + final String paramContextId = UUID.nameUUIDFromBytes((id + "-parameter-context").getBytes(StandardCharsets.UTF_8)).toString(); + final String paramContextName = "Connector " + id + " Parameter Context"; + final String parameterContextDescription = "Implicit Parameter Context for Connector " + id; + final ParameterReferenceManager referenceManager = new StandardParameterReferenceManager(() -> managedRootGroup); + final ParameterContext managedParameterContext = createParameterContext(paramContextId, paramContextName, + parameterContextDescription, Collections.emptyMap(), Collections.emptyList(), null, referenceManager, false); + managedRootGroup.setParameterContext(managedParameterContext); + + final ConnectorRepository connectorRepository = flowController.getConnectorRepository(); + final ConnectorStateTransition stateTransition = connectorRepository.createStateTransition(type, id); + final StandardConnectorConfigurationContext activeConfigurationContext = new StandardConnectorConfigurationContext( + flowController.getConnectorAssetManager(), flowController.getConnectorRepository().getSecretsManager()); + + final ProcessGroupFactory processGroupFactory = groupId -> createProcessGroup(groupId, id); + final FlowContextFactory flowContextFactory = new FlowControllerFlowContextFactory(flowController, managedRootGroup, activeConfigurationContext, processGroupFactory); + final ComponentBundleLookup componentBundleLookup = new StandardComponentBundleLookup(extensionManager); + + final ConnectorNode connectorNode = new ExtensionBuilder() + .identifier(id) + .type(type) + .bundleCoordinate(coordinate) + .extensionManager(extensionManager) + .managedProcessGroup(managedRootGroup) + .activeConfigurationContext(activeConfigurationContext) + .flowContextFactory(flowContextFactory) + .flowController(flowController) + .connectorStateTransition(stateTransition) + .connectorInitializationContextBuilder(flowController.getConnectorRepository().createInitializationContextBuilder()) + .connectorValidationTrigger(flowController.getConnectorValidationTrigger()) + .componentBundleLookup(componentBundleLookup) + .buildConnector(firstTimeAdded); + + // Establish the Connector as the parent authorizable of the managed root group + managedRootGroup.setExplicitParentAuthorizable(connectorNode); + + // Set up logging for the connector + final LogRepository logRepository = LogRepositoryFactory.getRepository(id); + logRepository.setLogger(connectorNode.getComponentLog()); + if (registerLogObserver) { + logRepository.addObserver(LogLevel.WARN, new ConnectorLogObserver(bulletinRepository, connectorNode)); + } + + if (firstTimeAdded) { + final Connector connector = connectorNode.getConnector(); + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, connector.getClass(), connectorNode.getIdentifier())) { + ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, connector); + } catch (final Exception e) { + if (registerLogObserver) { + logRepository.removeAllObservers(); + } + throw new ComponentLifeCycleException("Failed to invoke @OnAdded methods of " + connectorNode.getConnector(), e); + } + + flowController.getConnectorRepository().addConnector(connectorNode); + } else { + flowController.getConnectorRepository().restoreConnector(connectorNode); + } + + return connectorNode; + } + + private void copyGroupContents(final ProcessGroup sourceGroup, final ProcessGroup destinationGroup, final String componentIdSeed) { + final FlowMappingOptions flowMappingOptions = new FlowMappingOptions.Builder() + .mapSensitiveConfiguration(true) + .mapPropertyDescriptors(true) + .stateLookup(VersionedComponentStateLookup.ENABLED_OR_DISABLED) + .sensitiveValueEncryptor(value -> value) + .componentIdLookup(ComponentIdLookup.VERSIONED_OR_GENERATE) + .mapInstanceIdentifiers(true) + .mapControllerServiceReferencesToVersionedId(true) + .mapFlowRegistryClientId(true) + .mapAssetReferences(true) + .build(); + + final VersionedComponentFlowMapper flowMapper = new VersionedComponentFlowMapper(flowController.getExtensionManager(), flowMappingOptions); + final Map parameterContexts = flowMapper.mapParameterContexts(sourceGroup, true, Map.of()); + final InstantiatedVersionedProcessGroup versionedGroup = flowMapper.mapProcessGroup(sourceGroup, flowController.getControllerServiceProvider(), this, true); + final VersionedExternalFlow versionedExternalFlow = new VersionedExternalFlow(); + versionedExternalFlow.setFlowContents(versionedGroup); + versionedExternalFlow.setExternalControllerServices(Map.of()); + versionedExternalFlow.setParameterProviders(Map.of()); + versionedExternalFlow.setParameterContexts(parameterContexts); + + destinationGroup.updateFlow(versionedExternalFlow, componentIdSeed, false, true, true); + } + + private void gatherParameterContexts(final ProcessGroup sourceGroup, final Map parameterContexts) { + final ParameterContext parameterContext = sourceGroup.getParameterContext(); + if (parameterContext != null && !parameterContexts.containsKey(parameterContext.getIdentifier())) { + parameterContexts.put(parameterContext.getIdentifier(), parameterContext); + } + + for (final ProcessGroup childGroup : sourceGroup.getProcessGroups()) { + gatherParameterContexts(childGroup, parameterContexts); + } + } + + + @Override + public List getAllConnectors() { + return flowController.getConnectorRepository().getConnectors(); + } + + @Override + public ConnectorNode getConnector(final String id) { + return flowController.getConnectorRepository().getConnector(id); + } + @Override public ControllerServiceNode createControllerService(final String type, final String id, final BundleCoordinate bundleCoordinate, final Set additionalUrls, final boolean firstTimeAdded, final boolean registerLogObserver, final String classloaderIsolationKey) { diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardStatelessGroupNodeFactory.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardStatelessGroupNodeFactory.java index c5e56cb7bf41..eef4bb0da959 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardStatelessGroupNodeFactory.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flow/StandardStatelessGroupNodeFactory.java @@ -61,7 +61,7 @@ import org.apache.nifi.registry.flow.mapping.ComponentIdLookup; import org.apache.nifi.registry.flow.mapping.FlowMappingOptions; import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedProcessGroup; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup; import org.apache.nifi.reporting.BulletinRepository; import org.apache.nifi.stateless.engine.ProcessContextFactory; @@ -186,12 +186,22 @@ public ProcessGroup createStatelessProcessGroup(final ProcessGroup group, final } private VersionedExternalFlow createVersionedExternalFlow(final ProcessGroup group, final FlowMappingOptions flowMappingOptions) { - final NiFiRegistryFlowMapper flowMapper = new NiFiRegistryFlowMapper(flowController.getExtensionManager(), flowMappingOptions); + final VersionedComponentFlowMapper flowMapper = new VersionedComponentFlowMapper(flowController.getExtensionManager(), flowMappingOptions); final InstantiatedVersionedProcessGroup versionedGroup = flowMapper.mapNonVersionedProcessGroup(group, flowController.getControllerServiceProvider()); - final Map parameterContexts = flowMapper.mapParameterContexts(group, true, new HashMap<>()); final Map externalControllerServiceReferences = Optional.ofNullable(versionedGroup.getExternalControllerServiceReferences()).orElse(Collections.emptyMap()); + // If the Process Group is within a Connector, then we do not want to map Parameter Contexts + // because we do not use the standard management for Parameter Contexts and instead use only an implicit + // Parameter Context for the entire Connector. + final boolean inConnector = group.getConnectorIdentifier().isPresent(); + final Map parameterContexts; + if (inConnector) { + parameterContexts = Collections.emptyMap(); + } else { + parameterContexts = flowMapper.mapParameterContexts(group, true, new HashMap<>()); + } + final VersionedExternalFlow versionedExternalFlow = new VersionedExternalFlow(); versionedExternalFlow.setFlowContents(versionedGroup); versionedExternalFlow.setExternalControllerServices(externalControllerServiceReferences); @@ -299,6 +309,13 @@ public Future> fetch(final Set bundleCoordinates, child.synchronizeFlow(versionedExternalFlow, synchronizationOptions, flowMappingOptions); child.setParent(group); + // If this Process Group is within a Connector, explicitly set the Parameter Context + // on the child Process Group to be the same as the parent group because we want all + // groups within a Connector to share the same Parameter Context. + if (group.getConnectorIdentifier().isPresent()) { + child.setParameterContext(group.getParameterContext()); + } + return child; } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flowanalysis/StandardFlowAnalysisContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flowanalysis/StandardFlowAnalysisContext.java index 70775f285d6e..d5efbab84c5e 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flowanalysis/StandardFlowAnalysisContext.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flowanalysis/StandardFlowAnalysisContext.java @@ -24,7 +24,7 @@ import org.apache.nifi.flow.VersionedControllerService; import org.apache.nifi.flowanalysis.FlowAnalysisContext; import org.apache.nifi.nar.ExtensionManager; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import java.util.Collections; import java.util.HashMap; @@ -43,7 +43,7 @@ public VersionedControllerServiceLookup getVersionedControllerServiceLookup() { ControllerServiceProvider controllerServiceProvider = flowController.getControllerServiceProvider(); ExtensionManager extensionManager = flowController.getExtensionManager(); - NiFiRegistryFlowMapper mapper = FlowAnalysisUtil.createMapper(extensionManager); + VersionedComponentFlowMapper mapper = FlowAnalysisUtil.createMapper(extensionManager); ControllerServiceNode controllerServiceNode = controllerServiceProvider.getControllerServiceNode(id); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flowanalysis/StandardFlowAnalysisRuleNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flowanalysis/StandardFlowAnalysisRuleNode.java index 2404f48cc5c4..9dda4fa789f2 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flowanalysis/StandardFlowAnalysisRuleNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/flowanalysis/StandardFlowAnalysisRuleNode.java @@ -22,6 +22,7 @@ import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.authorization.resource.ResourceFactory; import org.apache.nifi.authorization.resource.ResourceType; +import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.validation.ValidationTrigger; import org.apache.nifi.controller.FlowAnalysisRuleNode; @@ -97,7 +98,7 @@ public FlowAnalysisRuleContext getFlowAnalysisRuleContext() { } @Override - protected List validateConfig() { + protected List validateConfig(final ValidationContext validationContext) { return Collections.emptyList(); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/AbstractFlowFileQueue.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/AbstractFlowFileQueue.java index 6ae3f36bfc4e..dfcfeecfeb04 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/AbstractFlowFileQueue.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/AbstractFlowFileQueue.java @@ -387,12 +387,16 @@ public long getPenaltyExpirationMillis() { } protected QueueSize drop(final List flowFiles, final String requestor) throws IOException { + return dropWithDetails(flowFiles, "FlowFile Queue emptied by " + requestor); + } + + protected QueueSize dropWithDetails(final List flowFiles, final String details) throws IOException { // Create a Provenance Event and a FlowFile Repository record for each FlowFile final List provenanceEvents = new ArrayList<>(flowFiles.size()); final List flowFileRepoRecords = new ArrayList<>(flowFiles.size()); long dropContentSize = 0L; for (final FlowFileRecord flowFile : flowFiles) { - provenanceEvents.add(createDropProvenanceEvent(flowFile, requestor)); + provenanceEvents.add(createDropProvenanceEvent(flowFile, details)); flowFileRepoRecords.add(createDeleteRepositoryRecord(flowFile)); dropContentSize += flowFile.getSize(); } @@ -402,7 +406,31 @@ protected QueueSize drop(final List flowFiles, final String requ return new QueueSize(flowFiles.size(), dropContentSize); } - private ProvenanceEventRecord createDropProvenanceEvent(final FlowFileRecord flowFile, final String requestor) { + protected FlowFileRepository getFlowFileRepository() { + return flowFileRepository; + } + + protected ProvenanceEventRepository getProvenanceRepository() { + return provRepository; + } + + protected List createDeleteRepositoryRecords(final List flowFiles) { + final List records = new ArrayList<>(flowFiles.size()); + for (final FlowFileRecord flowFile : flowFiles) { + records.add(createDeleteRepositoryRecord(flowFile)); + } + return records; + } + + protected List createDropProvenanceEvents(final List flowFiles, final String details) { + final List events = new ArrayList<>(flowFiles.size()); + for (final FlowFileRecord flowFile : flowFiles) { + events.add(createDropProvenanceEvent(flowFile, details)); + } + return events; + } + + private ProvenanceEventRecord createDropProvenanceEvent(final FlowFileRecord flowFile, final String details) { final ProvenanceEventBuilder builder = provRepository.eventBuilder(); builder.fromFlowFile(flowFile); builder.setEventType(ProvenanceEventType.DROP); @@ -410,7 +438,7 @@ private ProvenanceEventRecord createDropProvenanceEvent(final FlowFileRecord flo builder.setComponentId(getIdentifier()); builder.setComponentType("Connection"); builder.setAttributes(flowFile.getAttributes(), Collections.emptyMap()); - builder.setDetails("FlowFile Queue emptied by " + requestor); + builder.setDetails(details); builder.setSourceQueueIdentifier(getIdentifier()); final ContentClaim contentClaim = flowFile.getContentClaim(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/SelectiveDropResult.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/SelectiveDropResult.java new file mode 100644 index 000000000000..3d2e7bf2778b --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/SelectiveDropResult.java @@ -0,0 +1,75 @@ +/* + * 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.queue; + +import org.apache.nifi.controller.repository.FlowFileRecord; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Represents the result of a selective drop operation on a SwappablePriorityQueue. + * Contains the dropped FlowFiles and information about swap file updates. + */ +public class SelectiveDropResult { + + private final List droppedFlowFiles; + private final Map swapLocationUpdates; + + /** + * Creates a new SelectiveDropResult. + * + * @param droppedFlowFiles the FlowFiles that were dropped + * @param swapLocationUpdates a map from old swap location to new swap location. + * If the new location is null, the swap file was deleted entirely. + */ + public SelectiveDropResult(final List droppedFlowFiles, final Map swapLocationUpdates) { + this.droppedFlowFiles = droppedFlowFiles; + this.swapLocationUpdates = swapLocationUpdates; + } + + /** + * @return the FlowFiles that were dropped from the queue + */ + public List getDroppedFlowFiles() { + return Collections.unmodifiableList(droppedFlowFiles); + } + + /** + * @return a map from old swap location to new swap location. If the new location is null, + * all FlowFiles in that swap file matched the predicate and the file should be deleted. + */ + public Map getSwapLocationUpdates() { + return Collections.unmodifiableMap(swapLocationUpdates); + } + + /** + * @return the total number of FlowFiles that were dropped + */ + public int getDroppedCount() { + return droppedFlowFiles.size(); + } + + /** + * @return the total size in bytes of all dropped FlowFiles + */ + public long getDroppedBytes() { + return droppedFlowFiles.stream().mapToLong(FlowFileRecord::getSize).sum(); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardFlowFileQueue.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardFlowFileQueue.java index 8e50ab7b7575..8a2a9895d4b6 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardFlowFileQueue.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardFlowFileQueue.java @@ -16,25 +16,34 @@ */ package org.apache.nifi.controller.queue; +import org.apache.nifi.components.connector.DropFlowFileSummary; import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.repository.FlowFileRecord; import org.apache.nifi.controller.repository.FlowFileRepository; import org.apache.nifi.controller.repository.FlowFileSwapManager; +import org.apache.nifi.controller.repository.RepositoryRecord; +import org.apache.nifi.controller.repository.RepositoryRecordType; +import org.apache.nifi.controller.repository.StandardRepositoryRecord; import org.apache.nifi.controller.repository.SwapSummary; import org.apache.nifi.controller.status.FlowFileAvailability; import org.apache.nifi.events.EventReporter; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.processor.FlowFileFilter; +import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventRepository; import org.apache.nifi.util.concurrency.TimedLock; import java.io.IOException; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Predicate; /** * A FlowFileQueue is used to queue FlowFile objects that are awaiting further @@ -203,6 +212,54 @@ protected void dropFlowFiles(final DropFlowFileRequest dropRequest, final String queue.dropFlowFiles(dropRequest, requestor); } + @Override + public DropFlowFileSummary dropFlowFiles(final Predicate predicate) throws IOException { + lock(); + try { + // Perform the selective drop on the queue, which returns the dropped FlowFiles and swap location updates + final SelectiveDropResult dropResult = queue.dropFlowFiles(predicate); + + if (dropResult.getDroppedCount() == 0) { + return new DropFlowFileSummary(0, 0L); + } + + // Create repository records for the dropped FlowFiles + final List droppedFlowFiles = dropResult.getDroppedFlowFiles(); + final List repositoryRecords = new ArrayList<>(createDeleteRepositoryRecords(droppedFlowFiles)); + + // Create repository records for swap file changes so the FlowFile Repository can track valid swap locations + for (final Map.Entry entry : dropResult.getSwapLocationUpdates().entrySet()) { + final String oldSwapLocation = entry.getKey(); + final String newSwapLocation = entry.getValue(); + + final StandardRepositoryRecord swapRecord = new StandardRepositoryRecord(this); + if (newSwapLocation == null) { + swapRecord.setSwapLocation(oldSwapLocation, RepositoryRecordType.SWAP_FILE_DELETED); + } else { + swapRecord.setSwapFileRenamed(oldSwapLocation, newSwapLocation); + } + repositoryRecords.add(swapRecord); + } + + // Update the FlowFile Repository + getFlowFileRepository().updateRepository(repositoryRecords); + + // Create and register provenance events + final List provenanceEvents = createDropProvenanceEvents(droppedFlowFiles, "Selective drop by predicate"); + getProvenanceRepository().registerEvents(provenanceEvents); + + // Delete old swap files that were replaced + for (final Map.Entry entry : dropResult.getSwapLocationUpdates().entrySet()) { + final String oldSwapLocation = entry.getKey(); + swapManager.deleteSwapFile(oldSwapLocation); + } + + return new DropFlowFileSummary(dropResult.getDroppedCount(), dropResult.getDroppedBytes()); + } finally { + unlock(); + } + } + /** * Lock the queue so that other threads are unable to interact with the queue */ diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardLocalQueuePartitionDiagnostics.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardLocalQueuePartitionDiagnostics.java index ff31e77e89be..b376f7ab72b4 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardLocalQueuePartitionDiagnostics.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/StandardLocalQueuePartitionDiagnostics.java @@ -21,11 +21,16 @@ public class StandardLocalQueuePartitionDiagnostics implements LocalQueuePartiti private final FlowFileQueueSize queueSize; private final boolean anyPenalized; private final boolean allPenalized; + private final QueueSize penalizedQueueSize; + private final QueueSize totalSwapFileQueueSize; - public StandardLocalQueuePartitionDiagnostics(final FlowFileQueueSize queueSize, final boolean anyPenalized, final boolean allPenalized) { + public StandardLocalQueuePartitionDiagnostics(final FlowFileQueueSize queueSize, final boolean anyPenalized, final boolean allPenalized, + final QueueSize penalizedQueueSize, final QueueSize totalSwapFileQueueSize) { this.queueSize = queueSize; this.anyPenalized = anyPenalized; this.allPenalized = allPenalized; + this.penalizedQueueSize = penalizedQueueSize; + this.totalSwapFileQueueSize = totalSwapFileQueueSize; } @Override @@ -57,4 +62,14 @@ public boolean isAnyActiveFlowFilePenalized() { public boolean isAllActiveFlowFilesPenalized() { return allPenalized; } + + @Override + public QueueSize getPenalizedQueueSize() { + return penalizedQueueSize; + } + + @Override + public QueueSize getTotalSwapFileQueueSize() { + return totalSwapFileQueueSize; + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/SwappablePriorityQueue.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/SwappablePriorityQueue.java index 694679770048..827a2a29fef1 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/SwappablePriorityQueue.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/SwappablePriorityQueue.java @@ -43,6 +43,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -52,13 +53,15 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Predicate; public class SwappablePriorityQueue { private static final Logger logger = LoggerFactory.getLogger(SwappablePriorityQueue.class); - private static final int SWAP_RECORD_POLL_SIZE = 10_000; + private static final int MAX_SWAP_RECORD_POLL_SIZE = 10_000; private static final int MAX_EXPIRED_RECORDS_PER_ITERATION = 10_000; private final int swapThreshold; + private final int swapRecordPollSize; private final FlowFileSwapManager swapManager; private final EventReporter eventReporter; private final FlowFileQueue flowFileQueue; @@ -74,7 +77,7 @@ public class SwappablePriorityQueue { // We keep an "active queue" and a "swap queue" that both are able to hold records in heap. When // FlowFiles are added to this FlowFileQueue, we first check if we are in "swap mode" and if so // we add to the 'swap queue' instead of the 'active queue'. The code would be much simpler if we - // eliminated the 'swap queue' and instead just used the active queue and swapped out the 10,000 + // eliminated the 'swap queue' and instead just used the active queue and swapped out the // lowest priority FlowFiles from that. However, doing that would cause problems with the ordering // of FlowFiles. If we swap out some FlowFiles, and then allow a new FlowFile to be written to the // active queue, then we would end up processing the newer FlowFile before the swapped FlowFile. By @@ -95,6 +98,7 @@ public SwappablePriorityQueue(final FlowFileSwapManager swapManager, final int s final DropFlowFileAction dropAction, final String swapPartitionName) { this.swapManager = swapManager; this.swapThreshold = swapThreshold; + this.swapRecordPollSize = Math.min(MAX_SWAP_RECORD_POLL_SIZE, swapThreshold); this.activeQueue = new PriorityQueue<>(20, new QueuePrioritizer(Collections.emptyList())); this.swapQueue = new ArrayList<>(); @@ -132,10 +136,34 @@ public void setPriorities(final List newPriorities) { public LocalQueuePartitionDiagnostics getQueueDiagnostics() { readLock.lock(); try { - final boolean anyPenalized = !activeQueue.isEmpty() && activeQueue.peek().isPenalized(); - final boolean allPenalized = anyPenalized && activeQueue.stream().anyMatch(FlowFileRecord::isPenalized); + int penalizedCount = 0; + long penalizedBytes = 0; + for (final FlowFileRecord flowFile : activeQueue) { + if (flowFile.isPenalized()) { + penalizedCount++; + penalizedBytes += flowFile.getSize(); + } + } + + final boolean anyPenalized = penalizedCount > 0; + final boolean allPenalized = anyPenalized && penalizedCount == activeQueue.size(); + final QueueSize penalizedQueueSize = new QueueSize(penalizedCount, penalizedBytes); + + int totalSwapFlowFiles = 0; + long totalSwapBytes = 0; + for (final String swapLocation : swapLocations) { + try { + final SwapSummary summary = swapManager.getSwapSummary(swapLocation); + final QueueSize queueSize = summary.getQueueSize(); + totalSwapFlowFiles += queueSize.getObjectCount(); + totalSwapBytes += queueSize.getByteCount(); + } catch (final IOException e) { + logger.warn("Unable to read swap file summary for {}: {}", swapLocation, e.getMessage()); + } + } + final QueueSize totalSwapFileQueueSize = new QueueSize(totalSwapFlowFiles, totalSwapBytes); - return new StandardLocalQueuePartitionDiagnostics(getFlowFileQueueSize(), anyPenalized, allPenalized); + return new StandardLocalQueuePartitionDiagnostics(getFlowFileQueueSize(), anyPenalized, allPenalized, penalizedQueueSize, totalSwapFileQueueSize); } finally { readLock.unlock("getQueueDiagnostics"); } @@ -158,16 +186,16 @@ public boolean isUnacknowledgedFlowFile() { * This method MUST be called with the write lock held */ private void writeSwapFilesIfNecessary() { - if (swapQueue.size() < SWAP_RECORD_POLL_SIZE) { + if (swapQueue.size() < swapRecordPollSize) { return; } migrateSwapToActive(); - if (swapQueue.size() < SWAP_RECORD_POLL_SIZE) { + if (swapQueue.size() < swapRecordPollSize) { return; } - final int numSwapFiles = swapQueue.size() / SWAP_RECORD_POLL_SIZE; + final int numSwapFiles = swapQueue.size() / swapRecordPollSize; int originalSwapQueueCount = swapQueue.size(); long originalSwapQueueBytes = 0L; @@ -190,9 +218,9 @@ private void writeSwapFilesIfNecessary() { long totalSwapQueueDatesThisIteration = 0L; long minQueueDateThisIteration = Long.MAX_VALUE; - // Create a new swap file for the next SWAP_RECORD_POLL_SIZE records - final List toSwap = new ArrayList<>(SWAP_RECORD_POLL_SIZE); - for (int j = 0; j < SWAP_RECORD_POLL_SIZE; j++) { + // Create a new swap file for the next swapRecordPollSize records + final List toSwap = new ArrayList<>(swapRecordPollSize); + for (int j = 0; j < swapRecordPollSize; j++) { final FlowFileRecord flowFile = tempQueue.poll(); toSwap.add(flowFile); bytesSwappedThisIteration += flowFile.getSize(); @@ -894,6 +922,210 @@ public void dropFlowFiles(final DropFlowFileRequest dropRequest, final String re } } + /** + * Performs a selective drop of FlowFiles that match the given predicate. This method: + * 1. Filters the active queue and swap queue, collecting matched FlowFiles + * 2. For each swap file, peeks to deserialize FlowFiles, filters them, and writes remaining + * FlowFiles to a new swap file (if any remain) + * 3. Returns a SelectiveDropResult containing all dropped FlowFiles and swap location updates + * + * Note: This method does NOT update the FlowFile Repository or Provenance Repository. + * The caller is responsible for updating the repositories after this method returns. + * + * @param predicate the predicate to determine which FlowFiles to drop + * @return a SelectiveDropResult containing dropped FlowFiles and swap location updates + * @throws IOException if an error occurs while reading or writing swap files + */ + public SelectiveDropResult dropFlowFiles(final Predicate predicate) throws IOException { + final List droppedFlowFiles = new ArrayList<>(); + final Map swapLocationUpdates = new LinkedHashMap<>(); + + writeLock.lock(); + try { + // Process swap files FIRST - this is done first because it can throw IOException. + // By processing swap files first, we avoid modifying in-memory state if swap file operations fail. + // We use a two-phase approach: first create all new swap files, then if all succeed, update state. + + // Phase 1: Process all swap files and create new ones as needed + // Track what we need to do for each swap file + final List swapFileResults = new ArrayList<>(); + final List newlyCreatedSwapFiles = new ArrayList<>(); + + try { + for (final String swapLocation : swapLocations) { + final SwapFileDropResult result = processSwapFileForDrop(swapLocation, predicate); + if (result != null) { + swapFileResults.add(result); + if (result.newSwapLocation() != null) { + newlyCreatedSwapFiles.add(result.newSwapLocation()); + } + } + } + } catch (final IOException ioe) { + // Failed to process swap files - delete all newly created swap files and throw exception + logger.error("Failed to process swap files during selective drop; rolling back all changes", ioe); + for (final String newSwapFile : newlyCreatedSwapFiles) { + try { + swapManager.deleteSwapFile(newSwapFile); + logger.debug("Deleted newly created swap file {} during rollback", newSwapFile); + } catch (final Exception deleteException) { + logger.warn("Failed to delete newly created swap file {} during rollback", newSwapFile, deleteException); + } + } + throw ioe; + } + + // Phase 2: All swap file operations succeeded - now update state + for (final SwapFileDropResult result : swapFileResults) { + final String oldSwapLocation = result.oldSwapLocation(); + final String newSwapLocation = result.newSwapLocation(); + + swapLocationUpdates.put(oldSwapLocation, newSwapLocation); + swapLocations.remove(oldSwapLocation); + + if (newSwapLocation != null) { + // Some FlowFiles remain in new swap file + swapLocations.add(newSwapLocation); + incrementSwapQueueSize(-result.droppedFlowFiles().size(), -result.droppedBytes(), 0); + + // Update metrics for the new swap location + minQueueDateInSwapLocation.put(newSwapLocation, result.remainingMinQueueDate()); + totalQueueDateInSwapLocation.put(newSwapLocation, result.remainingTotalQueueDate()); + + logger.debug("Selective drop removed {} FlowFiles from swap file {}, wrote remaining to {}", + result.droppedFlowFiles().size(), oldSwapLocation, newSwapLocation); + } else { + // All FlowFiles were dropped - swap file count decreases + incrementSwapQueueSize(-result.droppedFlowFiles().size(), -result.droppedBytes(), -1); + logger.debug("Selective drop removed all {} FlowFiles from swap file {}", result.droppedFlowFiles().size(), oldSwapLocation); + } + + // Remove metrics for the old swap location + minQueueDateInSwapLocation.remove(oldSwapLocation); + totalQueueDateInSwapLocation.remove(oldSwapLocation); + + droppedFlowFiles.addAll(result.droppedFlowFiles()); + } + + // Filter the active queue + final Queue newActiveQueue = new PriorityQueue<>(Math.max(20, activeQueue.size()), new QueuePrioritizer(getPriorities())); + int droppedFromActiveCount = 0; + long droppedFromActiveBytes = 0L; + + for (final FlowFileRecord flowFile : activeQueue) { + if (predicate.test(flowFile)) { + droppedFlowFiles.add(flowFile); + droppedFromActiveCount++; + droppedFromActiveBytes += flowFile.getSize(); + } else { + newActiveQueue.add(flowFile); + } + } + activeQueue = newActiveQueue; + if (droppedFromActiveCount > 0) { + incrementActiveQueueSize(-droppedFromActiveCount, -droppedFromActiveBytes); + logger.debug("Selective drop removed {} FlowFiles ({} bytes) from active queue", droppedFromActiveCount, droppedFromActiveBytes); + } + + // Filter the swap queue + final List newSwapQueue = new ArrayList<>(); + int droppedFromSwapQueueCount = 0; + long droppedFromSwapQueueBytes = 0L; + + for (final FlowFileRecord flowFile : swapQueue) { + if (predicate.test(flowFile)) { + droppedFlowFiles.add(flowFile); + droppedFromSwapQueueCount++; + droppedFromSwapQueueBytes += flowFile.getSize(); + } else { + newSwapQueue.add(flowFile); + } + } + swapQueue = newSwapQueue; + if (droppedFromSwapQueueCount > 0) { + incrementSwapQueueSize(-droppedFromSwapQueueCount, -droppedFromSwapQueueBytes, 0); + logger.debug("Selective drop removed {} FlowFiles ({} bytes) from swap queue", droppedFromSwapQueueCount, droppedFromSwapQueueBytes); + } + + // Update swap mode if we've drained all queues + if (swapQueue.isEmpty() && swapLocations.isEmpty()) { + swapMode = false; + } + + updateTopPenaltyExpiration(); + + logger.info("Selective drop completed for queue {}: dropped {} FlowFiles ({} bytes)", + getQueueIdentifier(), droppedFlowFiles.size(), droppedFlowFiles.stream().mapToLong(FlowFileRecord::getSize).sum()); + + return new SelectiveDropResult(droppedFlowFiles, swapLocationUpdates); + } finally { + writeLock.unlock("Selective Drop FlowFiles"); + } + } + + /** + * Processes a single swap file for selective drop, filtering FlowFiles based on the predicate. + * If any FlowFiles match the predicate, a new swap file is created with the remaining FlowFiles + * (or the swap file is marked for deletion if all FlowFiles match). + * + * @param swapLocation the location of the swap file to process + * @param predicate the predicate to determine which FlowFiles to drop + * @return a SwapFileDropResult if any FlowFiles were dropped, or null if no changes are needed + * @throws IOException if an error occurs reading or writing swap files + */ + private SwapFileDropResult processSwapFileForDrop(final String swapLocation, final Predicate predicate) throws IOException { + final SwapContents swapContents; + try { + swapContents = swapManager.peek(swapLocation, flowFileQueue); + } catch (final IncompleteSwapFileException isfe) { + logger.warn("Failed to read swap file {} due to incomplete file; some FlowFiles may not be filtered", swapLocation); + if (eventReporter != null) { + eventReporter.reportEvent(Severity.WARNING, "Selective Drop", "Failed to read swap file " + swapLocation + + " because the file was incomplete. Some FlowFiles may not be filtered."); + } + return null; + } + + final List swappedFlowFiles = swapContents.getFlowFiles(); + final List remainingFlowFiles = new ArrayList<>(); + final List droppedFromThisSwapFile = new ArrayList<>(); + long droppedFromSwapFileBytes = 0L; + long remainingMinQueueDate = Long.MAX_VALUE; + long remainingTotalQueueDate = 0L; + + for (final FlowFileRecord flowFile : swappedFlowFiles) { + if (predicate.test(flowFile)) { + droppedFromThisSwapFile.add(flowFile); + droppedFromSwapFileBytes += flowFile.getSize(); + } else { + remainingFlowFiles.add(flowFile); + remainingMinQueueDate = Math.min(remainingMinQueueDate, flowFile.getLastQueueDate()); + remainingTotalQueueDate += flowFile.getLastQueueDate(); + } + } + + if (droppedFromThisSwapFile.isEmpty()) { + return null; + } + + if (remainingFlowFiles.isEmpty()) { + // All FlowFiles matched - swap file will be deleted + return new SwapFileDropResult(swapLocation, null, droppedFromThisSwapFile, droppedFromSwapFileBytes, 0L, 0L); + } + + // Some FlowFiles remain - write to new swap file + final String newSwapLocation = swapManager.swapOut(remainingFlowFiles, flowFileQueue, swapPartitionName); + return new SwapFileDropResult(swapLocation, newSwapLocation, droppedFromThisSwapFile, + droppedFromSwapFileBytes, remainingMinQueueDate, remainingTotalQueueDate); + } + + /** + * Holds the result of processing a single swap file during selective drop. + */ + private record SwapFileDropResult(String oldSwapLocation, String newSwapLocation, List droppedFlowFiles, + long droppedBytes, long remainingMinQueueDate, long remainingTotalQueueDate) { + } + public SwapSummary recoverSwappedFlowFiles() { int swapFlowFileCount = 0; long swapByteCount = 0L; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java index 51b68806e377..39c1c0c9b656 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/SocketLoadBalancedFlowFileQueue.java @@ -22,6 +22,7 @@ import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.components.connector.DropFlowFileSummary; import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.queue.AbstractFlowFileQueue; import org.apache.nifi.controller.queue.DropFlowFileRequest; @@ -35,6 +36,7 @@ import org.apache.nifi.controller.queue.QueueDiagnostics; import org.apache.nifi.controller.queue.QueueSize; import org.apache.nifi.controller.queue.RemoteQueuePartitionDiagnostics; +import org.apache.nifi.controller.queue.SelectiveDropResult; import org.apache.nifi.controller.queue.StandardQueueDiagnostics; import org.apache.nifi.controller.queue.SwappablePriorityQueue; import org.apache.nifi.controller.queue.clustered.client.async.AsyncLoadBalanceClientRegistry; @@ -55,6 +57,7 @@ import org.apache.nifi.controller.repository.FlowFileRepository; import org.apache.nifi.controller.repository.FlowFileSwapManager; import org.apache.nifi.controller.repository.RepositoryRecord; +import org.apache.nifi.controller.repository.RepositoryRecordType; import org.apache.nifi.controller.repository.StandardRepositoryRecord; import org.apache.nifi.controller.repository.SwapSummary; import org.apache.nifi.controller.repository.claim.ContentClaim; @@ -62,6 +65,7 @@ import org.apache.nifi.controller.status.FlowFileAvailability; import org.apache.nifi.controller.swap.StandardSwapSummary; import org.apache.nifi.events.EventReporter; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.processor.FlowFileFilter; import org.apache.nifi.provenance.ProvenanceEventBuilder; @@ -91,6 +95,7 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; +import java.util.function.Predicate; import java.util.stream.Collectors; public class SocketLoadBalancedFlowFileQueue extends AbstractFlowFileQueue implements LoadBalancedFlowFileQueue { @@ -1098,6 +1103,67 @@ protected void dropFlowFiles(final DropFlowFileRequest dropRequest, final String } } + @Override + public DropFlowFileSummary dropFlowFiles(final Predicate predicate) throws IOException { + lock(); + try { + final List allDroppedFlowFiles = new ArrayList<>(); + final Map allSwapLocationUpdates = new HashMap<>(); + + for (final QueuePartition partition : queuePartitions) { + final SelectiveDropResult partitionResult = partition.dropFlowFiles(predicate); + allDroppedFlowFiles.addAll(partitionResult.getDroppedFlowFiles()); + allSwapLocationUpdates.putAll(partitionResult.getSwapLocationUpdates()); + adjustSize(-partitionResult.getDroppedCount(), -partitionResult.getDroppedBytes()); + } + + // Also drop from the rebalancing partition + final SelectiveDropResult rebalanceResult = rebalancingPartition.dropFlowFiles(predicate); + allDroppedFlowFiles.addAll(rebalanceResult.getDroppedFlowFiles()); + allSwapLocationUpdates.putAll(rebalanceResult.getSwapLocationUpdates()); + adjustSize(-rebalanceResult.getDroppedCount(), -rebalanceResult.getDroppedBytes()); + + if (allDroppedFlowFiles.isEmpty()) { + return new DropFlowFileSummary(0, 0L); + } + + // Create repository records for the dropped FlowFiles + final List repositoryRecords = new ArrayList<>(createDeleteRepositoryRecords(allDroppedFlowFiles)); + + // Create repository records for swap file changes so the FlowFile Repository can track valid swap locations + for (final Map.Entry entry : allSwapLocationUpdates.entrySet()) { + final String oldSwapLocation = entry.getKey(); + final String newSwapLocation = entry.getValue(); + + final StandardRepositoryRecord swapRecord = new StandardRepositoryRecord(this); + if (newSwapLocation == null) { + swapRecord.setSwapLocation(oldSwapLocation, RepositoryRecordType.SWAP_FILE_DELETED); + } else { + swapRecord.setSwapFileRenamed(oldSwapLocation, newSwapLocation); + } + repositoryRecords.add(swapRecord); + } + + // Update the FlowFile Repository + flowFileRepo.updateRepository(repositoryRecords); + + // Create and register provenance events + final List provenanceEvents = createDropProvenanceEvents(allDroppedFlowFiles, "Selective drop by predicate"); + provRepo.registerEvents(provenanceEvents); + + // Delete old swap files that were replaced + for (final String oldSwapLocation : allSwapLocationUpdates.keySet()) { + swapManager.deleteSwapFile(oldSwapLocation); + } + + final int totalDroppedCount = allDroppedFlowFiles.size(); + final long totalDroppedBytes = allDroppedFlowFiles.stream().mapToLong(FlowFileRecord::getSize).sum(); + return new DropFlowFileSummary(totalDroppedCount, totalDroppedBytes); + } finally { + unlock(); + } + } + @Override public void lock() { partitionReadLock.lock(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/QueuePartition.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/QueuePartition.java index 60259d111b4b..2bf59de2808f 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/QueuePartition.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/QueuePartition.java @@ -21,13 +21,17 @@ import org.apache.nifi.controller.queue.DropFlowFileRequest; import org.apache.nifi.controller.queue.FlowFileQueueContents; import org.apache.nifi.controller.queue.QueueSize; +import org.apache.nifi.controller.queue.SelectiveDropResult; import org.apache.nifi.controller.repository.FlowFileRecord; import org.apache.nifi.controller.repository.SwapSummary; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.FlowFilePrioritizer; +import java.io.IOException; import java.util.Collection; import java.util.List; import java.util.Optional; +import java.util.function.Predicate; /** * Represents a portion of a FlowFile Queue such that a FlowFile Queue can be broken into @@ -69,6 +73,15 @@ public interface QueuePartition { */ void dropFlowFiles(DropFlowFileRequest dropRequest, String requestor); + /** + * Drops all FlowFiles in this partition that match the given predicate. + * + * @param predicate the predicate used to determine which FlowFiles should be dropped + * @return a SelectiveDropResult containing the dropped FlowFiles and swap location updates + * @throws IOException if an error occurs while reading or writing swap files + */ + SelectiveDropResult dropFlowFiles(Predicate predicate) throws IOException; + /** * Updates the prioritizers to use when queueing data * @param newPriorities the new priorities diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/RemoteQueuePartition.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/RemoteQueuePartition.java index b73f2c279654..277d50b22e08 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/RemoteQueuePartition.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/RemoteQueuePartition.java @@ -24,6 +24,7 @@ import org.apache.nifi.controller.queue.PollStrategy; import org.apache.nifi.controller.queue.QueueSize; import org.apache.nifi.controller.queue.RemoteQueuePartitionDiagnostics; +import org.apache.nifi.controller.queue.SelectiveDropResult; import org.apache.nifi.controller.queue.StandardRemoteQueuePartitionDiagnostics; import org.apache.nifi.controller.queue.SwappablePriorityQueue; import org.apache.nifi.controller.queue.clustered.TransferFailureDestination; @@ -39,6 +40,7 @@ import org.apache.nifi.controller.repository.SwapSummary; import org.apache.nifi.controller.repository.claim.ContentClaim; import org.apache.nifi.controller.repository.claim.ResourceClaim; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.provenance.ProvenanceEventBuilder; @@ -49,6 +51,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -58,6 +61,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.function.BooleanSupplier; +import java.util.function.Predicate; import java.util.stream.Collectors; /** @@ -135,6 +139,11 @@ public void dropFlowFiles(final DropFlowFileRequest dropRequest, final String re priorityQueue.dropFlowFiles(dropRequest, requestor); } + @Override + public SelectiveDropResult dropFlowFiles(final Predicate predicate) throws IOException { + return priorityQueue.dropFlowFiles(predicate); + } + @Override public SwapSummary recoverSwappedFlowFiles() { return priorityQueue.recoverSwappedFlowFiles(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/StandardRebalancingPartition.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/StandardRebalancingPartition.java index 439cd8374a05..9fa3dced3eb5 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/StandardRebalancingPartition.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/StandardRebalancingPartition.java @@ -25,20 +25,24 @@ import org.apache.nifi.controller.queue.LoadBalancedFlowFileQueue; import org.apache.nifi.controller.queue.PollStrategy; import org.apache.nifi.controller.queue.QueueSize; +import org.apache.nifi.controller.queue.SelectiveDropResult; import org.apache.nifi.controller.repository.FlowFileRecord; import org.apache.nifi.controller.repository.FlowFileSwapManager; import org.apache.nifi.controller.repository.SwapSummary; import org.apache.nifi.events.EventReporter; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Optional; import java.util.Set; +import java.util.function.Predicate; public class StandardRebalancingPartition implements RebalancingPartition { private static final Logger logger = LoggerFactory.getLogger(StandardRebalancingPartition.class); @@ -106,6 +110,11 @@ public void dropFlowFiles(DropFlowFileRequest dropRequest, String requestor) { queue.dropFlowFiles(dropRequest, requestor); } + @Override + public SelectiveDropResult dropFlowFiles(final Predicate predicate) throws IOException { + return queue.dropFlowFiles(predicate); + } + @Override public void setPriorities(final List newPriorities) { queue.setPriorities(newPriorities); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/SwappablePriorityQueueLocalPartition.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/SwappablePriorityQueueLocalPartition.java index 4674cce74e07..5d84760b4a51 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/SwappablePriorityQueueLocalPartition.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/clustered/partition/SwappablePriorityQueueLocalPartition.java @@ -25,12 +25,14 @@ import org.apache.nifi.controller.queue.LocalQueuePartitionDiagnostics; import org.apache.nifi.controller.queue.PollStrategy; import org.apache.nifi.controller.queue.QueueSize; +import org.apache.nifi.controller.queue.SelectiveDropResult; import org.apache.nifi.controller.queue.SwappablePriorityQueue; import org.apache.nifi.controller.repository.FlowFileRecord; import org.apache.nifi.controller.repository.FlowFileSwapManager; import org.apache.nifi.controller.repository.SwapSummary; import org.apache.nifi.controller.status.FlowFileAvailability; import org.apache.nifi.events.EventReporter; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.processor.FlowFileFilter; @@ -40,6 +42,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.function.Predicate; /** * A Local Queue Partition that whose implementation is based on the use of a {@link SwappablePriorityQueue}. @@ -157,6 +160,11 @@ public void dropFlowFiles(final DropFlowFileRequest dropRequest, final String re priorityQueue.dropFlowFiles(dropRequest, requestor); } + @Override + public SelectiveDropResult dropFlowFiles(final Predicate predicate) throws IOException { + return priorityQueue.dropFlowFiles(predicate); + } + @Override public SwapSummary recoverSwappedFlowFiles() { return priorityQueue.recoverSwappedFlowFiles(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java index dfaf6347309f..055a1d246eb0 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java @@ -22,6 +22,7 @@ import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.authorization.resource.ResourceFactory; import org.apache.nifi.authorization.resource.ResourceType; +import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.validation.ValidationTrigger; import org.apache.nifi.controller.FlowController; @@ -91,7 +92,7 @@ public ReportingContext getReportingContext() { } @Override - protected List validateConfig() { + protected List validateConfig(final ValidationContext validationContext) { return Collections.emptyList(); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/LiveSerializedRepositoryRecord.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/LiveSerializedRepositoryRecord.java index 01df125ac2bc..f5176e9ad846 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/LiveSerializedRepositoryRecord.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/LiveSerializedRepositoryRecord.java @@ -54,6 +54,11 @@ public String getSwapLocation() { return record.getSwapLocation(); } + @Override + public String getOriginalSwapLocation() { + return record.getOriginalSwapLocation(); + } + @Override public FlowFileRecord getFlowFileRecord() { return record.getCurrent(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java index a04b7527917d..aa880e8ba37a 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java @@ -418,7 +418,9 @@ public boolean isValidSwapLocationSuffix(final String swapLocationSuffix) { private void updateRepository(final Collection records, final boolean sync) throws IOException { for (final RepositoryRecord record : records) { if (record.getType() != RepositoryRecordType.DELETE && record.getType() != RepositoryRecordType.CONTENTMISSING - && record.getType() != RepositoryRecordType.CLEANUP_TRANSIENT_CLAIMS && record.getDestination() == null) { + && record.getType() != RepositoryRecordType.CLEANUP_TRANSIENT_CLAIMS + && record.getType() != RepositoryRecordType.SWAP_FILE_DELETED && record.getType() != RepositoryRecordType.SWAP_FILE_RENAMED + && record.getDestination() == null) { throw new IllegalArgumentException("Record " + record + " has no destination and Type is " + record.getType()); } } @@ -481,6 +483,17 @@ protected void updateContentClaims(Collection repositoryRecord final String swapLocation = record.getSwapLocation(); swapLocationsRemoved.add(swapLocation); swapLocationsAdded.remove(swapLocation); + } else if (record.getType() == RepositoryRecordType.SWAP_FILE_DELETED) { + final String swapLocation = record.getSwapLocation(); + swapLocationsRemoved.add(swapLocation); + swapLocationsAdded.remove(swapLocation); + } else if (record.getType() == RepositoryRecordType.SWAP_FILE_RENAMED) { + final String originalSwapLocation = record.getOriginalSwapLocation(); + final String newSwapLocation = record.getSwapLocation(); + swapLocationsRemoved.add(originalSwapLocation); + swapLocationsAdded.remove(originalSwapLocation); + swapLocationsAdded.add(newSwapLocation); + swapLocationsRemoved.remove(newSwapLocation); } } @@ -635,7 +648,7 @@ public void swapFlowFilesIn(final String swapLocation, final List recoveredSwapLocations = wal.getRecoveredSwapLocations(); synchronized (this.swapLocationSuffixes) { recoveredSwapLocations.forEach(loc -> this.swapLocationSuffixes.add(normalizeSwapLocation(loc))); + + for (final SerializedRepositoryRecord record : recordList) { + final RepositoryRecordType recordType = record.getType(); + if (recordType == RepositoryRecordType.SWAP_FILE_DELETED) { + this.swapLocationSuffixes.remove(normalizeSwapLocation(record.getSwapLocation())); + } else if (recordType == RepositoryRecordType.SWAP_FILE_RENAMED) { + this.swapLocationSuffixes.remove(normalizeSwapLocation(record.getOriginalSwapLocation())); + this.swapLocationSuffixes.add(normalizeSwapLocation(record.getSwapLocation())); + } + } + logger.debug("Recovered {} Swap Files: {}", swapLocationSuffixes.size(), swapLocationSuffixes); } @@ -727,6 +751,11 @@ public long loadFlowFiles(final QueueProvider queueProvider) throws IOException int numFlowFilesMissingQueue = 0; long maxId = 0; for (final SerializedRepositoryRecord record : recordList) { + final RepositoryRecordType recordType = record.getType(); + if (recordType == RepositoryRecordType.SWAP_FILE_DELETED || recordType == RepositoryRecordType.SWAP_FILE_RENAMED) { + continue; + } + final long recordId = serdeFactory.getRecordIdentifier(record); if (recordId > maxId) { maxId = recordId; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java index be1982054f4d..6c5645b28aa4 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java @@ -22,6 +22,7 @@ import org.apache.nifi.annotation.lifecycle.OnUnscheduled; import org.apache.nifi.annotation.notification.PrimaryNodeState; import org.apache.nifi.authorization.resource.ComponentAuthorizable; +import org.apache.nifi.components.connector.ConnectorNode; import org.apache.nifi.components.state.StateManager; import org.apache.nifi.components.state.StateManagerProvider; import org.apache.nifi.components.validation.ValidationStatus; @@ -31,8 +32,10 @@ import org.apache.nifi.controller.AbstractPort; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.NodeTypeProvider; import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ReloadComponent; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.ScheduledState; import org.apache.nifi.controller.SchedulingAgentCallback; @@ -48,6 +51,7 @@ import org.apache.nifi.lifecycle.ProcessorStopLifecycleMethods; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.logging.StandardLoggingContext; +import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.NarCloseable; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.Processor; @@ -88,13 +92,16 @@ public final class StandardProcessScheduler implements ProcessScheduler { private static final Logger LOG = LoggerFactory.getLogger(StandardProcessScheduler.class); - private final FlowController flowController; private final long administrativeYieldMillis; private final String administrativeYieldDuration; private final StateManagerProvider stateManagerProvider; private final long processorStartTimeoutMillis; private final LifecycleStateManager lifecycleStateManager; private final AtomicLong frameworkTaskThreadIndex = new AtomicLong(1L); + private final ExtensionManager extensionManager; + private final NodeTypeProvider nodeTypeProvider; + private final Supplier controllerServiceProviderFactory; + private final ReloadComponent reloadComponent; private final ConcurrentMap strategyAgentMap = new ConcurrentHashMap<>(); @@ -104,12 +111,24 @@ public final class StandardProcessScheduler implements ProcessScheduler { private final ScheduledExecutorService componentMonitoringThreadPool = new FlowEngine(2, "Monitor Processor Lifecycle", true); public StandardProcessScheduler(final FlowEngine componentLifecycleThreadPool, final FlowController flowController, + final StateManagerProvider stateManagerProvider, final NiFiProperties nifiProperties, final LifecycleStateManager lifecycleStateManager) { + + this(componentLifecycleThreadPool, flowController.getExtensionManager(), flowController, flowController::getControllerServiceProvider, + flowController.getReloadComponent(), stateManagerProvider, nifiProperties, lifecycleStateManager); + } + + public StandardProcessScheduler(final FlowEngine componentLifecycleThreadPool, final ExtensionManager extensionManager, final NodeTypeProvider nodeTypeProvider, + final Supplier controllerServiceProviderFactory, final ReloadComponent reloadComponent, final StateManagerProvider stateManagerProvider, final NiFiProperties nifiProperties, final LifecycleStateManager lifecycleStateManager) { + this.componentLifeCycleThreadPool = componentLifecycleThreadPool; - this.flowController = flowController; this.stateManagerProvider = stateManagerProvider; this.lifecycleStateManager = lifecycleStateManager; + this.extensionManager = extensionManager; + this.nodeTypeProvider = nodeTypeProvider; + this.controllerServiceProviderFactory = controllerServiceProviderFactory; + this.reloadComponent = reloadComponent; administrativeYieldDuration = nifiProperties.getAdministrativeYieldDuration(); administrativeYieldMillis = FormatUtils.getTimeDuration(administrativeYieldDuration, TimeUnit.MILLISECONDS); @@ -119,7 +138,7 @@ public StandardProcessScheduler(final FlowEngine componentLifecycleThreadPool, f } public ControllerServiceProvider getControllerServiceProvider() { - return flowController.getControllerServiceProvider(); + return controllerServiceProviderFactory.get(); } private StateManager getStateManager(final String componentId) { @@ -239,7 +258,7 @@ public void shutdown() { @Override public void shutdownReportingTask(final ReportingTaskNode reportingTask) { final ConfigurationContext configContext = reportingTask.getConfigurationContext(); - try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), reportingTask.getReportingTask().getClass(), reportingTask.getIdentifier())) { + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, reportingTask.getReportingTask().getClass(), reportingTask.getIdentifier())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, reportingTask.getReportingTask(), configContext); } } @@ -247,7 +266,7 @@ public void shutdownReportingTask(final ReportingTaskNode reportingTask) { @Override public void shutdownControllerService(final ControllerServiceNode serviceNode, final ControllerServiceProvider controllerServiceProvider) { final Class serviceImplClass = serviceNode.getControllerServiceImplementation().getClass(); - try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), serviceImplClass, serviceNode.getIdentifier())) { + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, serviceImplClass, serviceNode.getIdentifier())) { final ConfigurationContext configContext = new StandardConfigurationContext(serviceNode, controllerServiceProvider, null); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, serviceNode.getControllerServiceImplementation(), configContext); } @@ -295,7 +314,7 @@ public void run() { return; } - try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), reportingTask.getClass(), reportingTask.getIdentifier())) { + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, reportingTask.getClass(), reportingTask.getIdentifier())) { ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, reportingTask, taskNode.getConfigurationContext()); } @@ -310,7 +329,7 @@ public void run() { + "ReportingTask and will attempt to schedule it again after {}", reportingTask, e.toString(), administrativeYieldDuration, e); - try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), reportingTask.getClass(), reportingTask.getIdentifier())) { + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, reportingTask.getClass(), reportingTask.getIdentifier())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnUnscheduled.class, reportingTask, taskNode.getConfigurationContext()); ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, reportingTask, taskNode.getConfigurationContext()); } @@ -349,7 +368,7 @@ public Future unschedule(final ReportingTaskNode taskNode) { synchronized (lifecycleState) { lifecycleState.setScheduled(false); - try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), reportingTask.getClass(), reportingTask.getIdentifier())) { + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, reportingTask.getClass(), reportingTask.getIdentifier())) { ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, reportingTask, configurationContext); } catch (final Exception e) { final Throwable cause = e instanceof InvocationTargetException ? e.getCause() : e; @@ -390,7 +409,7 @@ public synchronized CompletableFuture startProcessor(final ProcessorNode p final LifecycleState lifecycleState = getLifecycleState(requireNonNull(procNode), true, false); final Supplier processContextFactory = () -> new StandardProcessContext(procNode, getControllerServiceProvider(), - getStateManager(procNode), lifecycleState::isTerminated, flowController); + getStateManager(procNode), lifecycleState::isTerminated, nodeTypeProvider); final boolean scheduleActions = procNode.getProcessGroup().resolveExecutionEngine() != ExecutionEngine.STATELESS; @@ -421,12 +440,21 @@ public void onTaskComplete() { }; LOG.info("Starting {}", procNode); + + procNode.reloadAdditionalResourcesIfNecessary(); + procNode.start(componentMonitoringThreadPool, administrativeYieldMillis, processorStartTimeoutMillis, processContextFactory, callback, failIfStopping, scheduleActions); return future; } @Override public synchronized CompletableFuture startStatelessGroup(final StatelessGroupNode groupNode) { + final ScheduledState initialState = groupNode.getDesiredState(); + if (initialState != ScheduledState.RUNNING) { + LOG.warn("{} scheduled to run but its desired state is {}, not RUNNING; will not start the group", groupNode, initialState); + return CompletableFuture.failedFuture(new IllegalStateException("Attempted to start %s but its desired state is %s, not RUNNING".formatted(groupNode, initialState))); + } + final LifecycleState lifecycleState = getLifecycleState(requireNonNull(groupNode), true, true); lifecycleState.setScheduled(true); @@ -495,7 +523,7 @@ public void trigger() { } } - LOG.info("{} is no scheduled to run. Disabling {} Controller Services", groupNode.getProcessGroup(), allServices.size()); + LOG.info("{} is no longer scheduled to run. Disabling {} Controller Services", groupNode.getProcessGroup(), allServices.size()); // Cancel all service start futures, interrupting them if they are waiting for (final Future serviceStartFuture : serviceStartFutures) { @@ -523,7 +551,7 @@ public Future runProcessorOnce(ProcessorNode procNode, final Callable processContextFactory = () -> new StandardProcessContext(procNode, getControllerServiceProvider(), - getStateManager(procNode), lifecycleState::isTerminated, flowController); + getStateManager(procNode), lifecycleState::isTerminated, nodeTypeProvider); final CompletableFuture future = new CompletableFuture<>(); final SchedulingAgentCallback callback = new SchedulingAgentCallback() { @@ -564,7 +592,7 @@ public synchronized CompletableFuture stopProcessor(final ProcessorNode pr final LifecycleState lifecycleState = getLifecycleState(procNode, false, false); final StandardProcessContext processContext = new StandardProcessContext(procNode, getControllerServiceProvider(), - getStateManager(procNode), lifecycleState::isTerminated, flowController); + getStateManager(procNode), lifecycleState::isTerminated, nodeTypeProvider); LOG.info("Stopping {}", procNode); return procNode.stop(this, this.componentLifeCycleThreadPool, processContext, getSchedulingAgent(procNode), lifecycleState, lifecycleMethods); @@ -597,7 +625,7 @@ public synchronized void terminateProcessor(final ProcessorNode procNode) { try { final Set additionalUrls = procNode.getAdditionalClasspathResources(procNode.getPropertyDescriptors()); - flowController.getReloadComponent().reload(procNode, procNode.getCanonicalClassName(), procNode.getBundleCoordinate(), additionalUrls); + reloadComponent.reload(procNode, procNode.getCanonicalClassName(), procNode.getBundleCoordinate(), additionalUrls); } 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); @@ -741,7 +769,7 @@ private synchronized void stopConnectable(final Connectable connectable) { if (!state.isScheduled() && state.getActiveThreadCount() == 0 && state.mustCallOnStoppedMethods()) { final StateManager stateManager = (connectable instanceof ProcessorNode) ? getStateManager((ProcessorNode) connectable) : getStateManager(connectable.getIdentifier()); final ConnectableProcessContext processContext = new ConnectableProcessContext(connectable, stateManager); - try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), connectable.getClass(), connectable.getIdentifier())) { + try (final NarCloseable ignored = NarCloseable.withComponentNarLoader(extensionManager, connectable.getClass(), connectable.getIdentifier())) { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, connectable, processContext); } } @@ -852,7 +880,18 @@ public CompletableFuture enableControllerService(final ControllerServiceNo return enableControllerService(service, true); } + @Override + public CompletableFuture enableControllerService(final ControllerServiceNode service, final ConfigurationContext configurationContext) { + return enableControllerService(service, true, configurationContext); + } + private CompletableFuture enableControllerService(final ControllerServiceNode service, final boolean completeFutureExceptionally) { + return enableControllerService(service, completeFutureExceptionally, null); + } + + private CompletableFuture enableControllerService(final ControllerServiceNode service, final boolean completeFutureExceptionally, + final ConfigurationContext configurationContext) { + if (service.isActive()) { LOG.debug("{} is already active, so not enabling it again", service); return CompletableFuture.completedFuture(null); @@ -860,15 +899,18 @@ private CompletableFuture enableControllerService(final ControllerServiceN LOG.info("Enabling {}", service); + service.reloadAdditionalResourcesIfNecessary(); + final List> futures = new ArrayList<>(); final List dependentServices = service.getRequiredControllerServices(); for (final ControllerServiceNode dependentService : dependentServices) { // Enable Controller Service but if it fails, do not complete the future Exceptionally. This allows us to wait up until the // timeout for the service to enable, even if it needs to retry in order to do so. - futures.add(enableControllerService(dependentService, completeFutureExceptionally)); + // Note: dependent services always use their own configuration, not the provided configurationContext + futures.add(enableControllerService(dependentService, completeFutureExceptionally, null)); } - futures.add(service.enable(this.componentLifeCycleThreadPool, this.administrativeYieldMillis, completeFutureExceptionally)); + futures.add(service.enable(this.componentLifeCycleThreadPool, this.administrativeYieldMillis, completeFutureExceptionally, configurationContext)); return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])); } @@ -890,6 +932,23 @@ public CompletableFuture disableControllerService(final ControllerServiceN } } + @Override + public CompletableFuture startConnector(final ConnectorNode connectorNode) { + // TODO: Implement + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture stopConnector(final ConnectorNode connectorNode) { + // TODO: Implement + throw new UnsupportedOperationException(); + } + + @Override + public void onConnectorRemoved(final ConnectorNode connectorNode) { + lifecycleStateManager.removeLifecycleState(connectorNode.getIdentifier()); + } + private CompletableFuture disableControllerServiceWithStandaloneThreadPool(final ControllerServiceNode service) { final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); try { diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedDataflowMapper.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedDataflowMapper.java index d55e8d6c620d..550cfbe25f80 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedDataflowMapper.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedDataflowMapper.java @@ -17,6 +17,7 @@ package org.apache.nifi.controller.serialization; +import org.apache.nifi.components.connector.ConnectorNode; import org.apache.nifi.connectable.Port; import org.apache.nifi.controller.FlowAnalysisRuleNode; import org.apache.nifi.controller.FlowController; @@ -27,6 +28,7 @@ import org.apache.nifi.controller.flow.VersionedFlowEncodingVersion; import org.apache.nifi.controller.service.ControllerServiceNode; import org.apache.nifi.flow.ScheduledState; +import org.apache.nifi.flow.VersionedConnector; import org.apache.nifi.flow.VersionedControllerService; import org.apache.nifi.flow.VersionedFlowAnalysisRule; import org.apache.nifi.flow.VersionedFlowRegistryClient; @@ -40,8 +42,8 @@ import org.apache.nifi.registry.flow.FlowRegistryClientNode; import org.apache.nifi.registry.flow.mapping.ComponentIdLookup; import org.apache.nifi.registry.flow.mapping.FlowMappingOptions; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; import org.apache.nifi.registry.flow.mapping.SensitiveValueEncryptor; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup; import java.util.ArrayList; @@ -52,7 +54,7 @@ public class VersionedDataflowMapper { private static final VersionedFlowEncodingVersion ENCODING_VERSION = new VersionedFlowEncodingVersion(2, 0); private final FlowController flowController; - private final NiFiRegistryFlowMapper flowMapper; + private final VersionedComponentFlowMapper flowMapper; private final ScheduledStateLookup stateLookup; public VersionedDataflowMapper(final FlowController flowController, final ExtensionManager extensionManager, final SensitiveValueEncryptor encryptor, final ScheduledStateLookup stateLookup) { @@ -73,7 +75,7 @@ public VersionedDataflowMapper(final FlowController flowController, final Extens .mapAssetReferences(true) .build(); - flowMapper = new NiFiRegistryFlowMapper(extensionManager, mappingOptions); + flowMapper = new VersionedComponentFlowMapper(extensionManager, mappingOptions); } public VersionedDataflow createMapping() { @@ -86,11 +88,23 @@ public VersionedDataflow createMapping() { dataflow.setReportingTasks(mapReportingTasks()); dataflow.setFlowAnalysisRules(mapFlowAnalysisRules()); dataflow.setParameterProviders(mapParameterProviders()); + dataflow.setConnectors(mapConnectors()); dataflow.setRootGroup(mapRootGroup()); return dataflow; } + private List mapConnectors() { + final List connectors = new ArrayList<>(); + + for (final ConnectorNode connectorNode : flowController.getConnectorRepository().getConnectors()) { + final VersionedConnector versionedConnector = flowMapper.mapConnector(connectorNode); + connectors.add(versionedConnector); + } + + return connectors; + } + private List mapControllerServices() { final List controllerServices = new ArrayList<>(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizer.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizer.java index e0a7ce607fd4..13eb54aec787 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizer.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizer.java @@ -26,6 +26,15 @@ import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.cluster.protocol.DataFlow; import org.apache.nifi.cluster.protocol.StandardDataFlow; +import org.apache.nifi.components.connector.AssetReference; +import org.apache.nifi.components.connector.ConnectorConfiguration; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorRepository; +import org.apache.nifi.components.connector.ConnectorValueReference; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.NamedStepConfiguration; +import org.apache.nifi.components.connector.SecretReference; +import org.apache.nifi.components.connector.StringLiteralValue; import org.apache.nifi.components.validation.ValidationStatus; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Position; @@ -55,6 +64,9 @@ import org.apache.nifi.flow.ScheduledState; import org.apache.nifi.flow.VersionedAsset; import org.apache.nifi.flow.VersionedComponent; +import org.apache.nifi.flow.VersionedConfigurationStep; +import org.apache.nifi.flow.VersionedConnector; +import org.apache.nifi.flow.VersionedConnectorValueReference; import org.apache.nifi.flow.VersionedControllerService; import org.apache.nifi.flow.VersionedExternalFlow; import org.apache.nifi.flow.VersionedFlowAnalysisRule; @@ -412,9 +424,15 @@ private void synchronizeFlow(final FlowController controller, final DataFlow exi versionedExternalFlow.setParameterContexts(versionedParameterContextMap); versionedExternalFlow.setFlowContents(versionedFlow.getRootGroup()); + // Inherit Parameter Providers and Connectors first. Because Connectors are a bit different, in that updates could result in Exceptions being thrown, + // due to the fact that they manipulate the flow, and changes can be aborted, we handle them first. This way, if there's any Exception, + // we can fail before updating parts of the flow that are not managed by Connectors. + // Because Connectors may depend on Parameter Providers, we need to ensure that we inherit Parameter Providers first. + inheritParameterProviders(controller, versionedFlow, affectedComponentSet); + inheritConnectors(controller, versionedFlow); + // Inherit controller-level components. inheritControllerServices(controller, versionedFlow, affectedComponentSet); - inheritParameterProviders(controller, versionedFlow, affectedComponentSet); inheritParameterContexts(controller, versionedFlow); inheritReportingTasks(controller, versionedFlow, affectedComponentSet); inheritFlowAnalysisRules(controller, versionedFlow, affectedComponentSet); @@ -1023,6 +1041,156 @@ private Set getAssetIds(final Parameter parameter) { .collect(Collectors.toSet()); } + private void inheritConnectors(final FlowController flowController, final VersionedDataflow dataflow) { + // TODO: We need to delete any Connectors that are no longer part of the flow. + // This means we need to drain the Connector first, then stop it, then delete it. If unable to drain, we must fail... + // perhaps we need a DRAINING state? Or do we just delete it and drop the data? + final ConnectorRepository connectorRepository = flowController.getConnectorRepository(); + + final Set proposedConnectorIds = new HashSet<>(); + if (dataflow.getConnectors() != null) { + for (final VersionedConnector versionedConnector : dataflow.getConnectors()) { + proposedConnectorIds.add(versionedConnector.getInstanceIdentifier()); + + final ConnectorNode existingConnector = connectorRepository.getConnector(versionedConnector.getInstanceIdentifier()); + if (existingConnector == null) { + logger.info("Connector {} of type {} with name {} is not in the current flow. Will add Connector.", + versionedConnector.getInstanceIdentifier(), versionedConnector.getType(), versionedConnector.getName()); + + addConnector(versionedConnector, connectorRepository, flowController.getFlowManager()); + } else if (isConnectorConfigurationUpdated(existingConnector, versionedConnector)) { + logger.info("{} configuration has changed, updating configuration", existingConnector); + updateConnector(versionedConnector, connectorRepository); + } else { + logger.debug("{} configuration is up to date, no update necessary", existingConnector); + } + } + } + + for (final ConnectorNode existingConnector : connectorRepository.getConnectors()) { + if (!proposedConnectorIds.contains(existingConnector.getIdentifier())) { + logger.info("Connector {} is no longer part of the proposed flow. Will remove Connector.", existingConnector); + connectorRepository.stopConnector(existingConnector); + connectorRepository.removeConnector(existingConnector.getIdentifier()); + } + } + } + + private boolean isConnectorConfigurationUpdated(final ConnectorNode existingConnector, final VersionedConnector versionedConnector) { + final ConnectorConfiguration activeConfig = existingConnector.getActiveFlowContext().getConfigurationContext().toConnectorConfiguration(); + final List versionedActiveConfig = versionedConnector.getActiveFlowConfiguration(); + final boolean activeContextChanged = isConnectorConfigurationUpdated(activeConfig, versionedActiveConfig); + + final ConnectorConfiguration workingConfig = existingConnector.getWorkingFlowContext().getConfigurationContext().toConnectorConfiguration(); + final List versionedWorkingConfig = versionedConnector.getWorkingFlowConfiguration(); + final boolean workingContextChanged = isConnectorConfigurationUpdated(workingConfig, versionedWorkingConfig); + + return activeContextChanged || workingContextChanged; + } + + private boolean isConnectorConfigurationUpdated(final ConnectorConfiguration existingConfiguration, final List versionedConfigurationSteps) { + if (versionedConfigurationSteps == null || versionedConfigurationSteps.isEmpty()) { + return existingConfiguration != null && !existingConfiguration.getNamedStepConfigurations().isEmpty(); + } + + final Set existingStepConfigurations = existingConfiguration.getNamedStepConfigurations(); + if (existingStepConfigurations.size() != versionedConfigurationSteps.size()) { + return true; + } + + final Map existingStepsByName = existingStepConfigurations.stream() + .collect(Collectors.toMap(NamedStepConfiguration::stepName, Function.identity())); + + for (final VersionedConfigurationStep versionedStep : versionedConfigurationSteps) { + final NamedStepConfiguration existingStep = existingStepsByName.get(versionedStep.getName()); + if (existingStep == null) { + return true; + } + + if (isConfigurationStepUpdated(existingStep, versionedStep)) { + return true; + } + } + + return false; + } + + private boolean isConfigurationStepUpdated(final NamedStepConfiguration existingStep, final VersionedConfigurationStep versionedStep) { + final Map existingProperties = existingStep.configuration().getPropertyValues(); + + final Map versionedProperties = versionedStep.getProperties(); + if (versionedProperties == null || versionedProperties.isEmpty()) { + return existingProperties != null && !existingProperties.isEmpty(); + } + + if (existingProperties == null || existingProperties.size() != versionedProperties.size()) { + return true; + } + + for (final Map.Entry versionedEntry : versionedProperties.entrySet()) { + final String propertyName = versionedEntry.getKey(); + final VersionedConnectorValueReference versionedRef = versionedEntry.getValue(); + final ConnectorValueReference existingRef = existingProperties.get(propertyName); + + if (!equals(versionedRef, existingRef)) { + return true; + } + } + + return false; + } + + private boolean equals(final VersionedConnectorValueReference versionedReference, final ConnectorValueReference existingReference) { + if (versionedReference == null && existingReference == null) { + return true; + } + if (versionedReference == null || existingReference == null) { + return false; + } + + final String versionedValueType = versionedReference.getValueType(); + final String existingValueType = existingReference.getValueType().name(); + if (!Objects.equals(versionedValueType, existingValueType)) { + return false; + } + + return switch (existingReference) { + case StringLiteralValue stringLiteral -> Objects.equals(stringLiteral.getValue(), versionedReference.getValue()); + case AssetReference assetRef -> Objects.equals(assetRef.getAssetIdentifiers(), versionedReference.getAssetIds()); + case SecretReference secretRef -> Objects.equals(secretRef.getProviderId(), versionedReference.getProviderId()) + && Objects.equals(secretRef.getSecretName(), versionedReference.getSecretName()); + }; + } + + private void addConnector(final VersionedConnector versionedConnector, final ConnectorRepository connectorRepository, final FlowManager flowManager) { + final BundleCoordinate coordinate = createBundleCoordinate(extensionManager, versionedConnector.getBundle(), versionedConnector.getType()); + final ConnectorNode connectorNode = flowManager.createConnector(versionedConnector.getType(), versionedConnector.getInstanceIdentifier(), coordinate, false, true); + connectorRepository.restoreConnector(connectorNode); + updateConnector(versionedConnector, connectorRepository); + } + + + private void updateConnector(final VersionedConnector versionedConnector, final ConnectorRepository connectorRepository) { + final ConnectorNode connectorNode = connectorRepository.getConnector(versionedConnector.getInstanceIdentifier()); + + connectorRepository.updateConnector(connectorNode, versionedConnector.getName()); + + try { + final List activeFlowConfig = versionedConnector.getActiveFlowConfiguration(); + final List workingFlowConfig = versionedConnector.getWorkingFlowConfiguration(); + connectorRepository.inheritConfiguration(connectorNode, activeFlowConfig, workingFlowConfig, versionedConnector.getBundle()); + + final ScheduledState desiredState = versionedConnector.getScheduledState(); + if (desiredState == ScheduledState.RUNNING) { + connectorRepository.startConnector(connectorNode); + } else if (desiredState == ScheduledState.ENABLED) { + connectorRepository.stopConnector(connectorNode); + } + } catch (final FlowUpdateException e) { + logger.error("{} failed to inherit configuration", connectorNode, e); + } + } + private void inheritControllerServices(final FlowController controller, final VersionedDataflow dataflow, final AffectedComponentSet affectedComponentSet) { final FlowManager flowManager = controller.getFlowManager(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedReportingTaskSnapshotMapper.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedReportingTaskSnapshotMapper.java index fb00f48da651..dbe4e7a12f62 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedReportingTaskSnapshotMapper.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedReportingTaskSnapshotMapper.java @@ -23,7 +23,7 @@ import org.apache.nifi.flow.VersionedReportingTask; import org.apache.nifi.flow.VersionedReportingTaskSnapshot; import org.apache.nifi.nar.ExtensionManager; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import java.util.ArrayList; import java.util.Collections; @@ -32,11 +32,11 @@ public class VersionedReportingTaskSnapshotMapper { - private final NiFiRegistryFlowMapper flowMapper; + private final VersionedComponentFlowMapper flowMapper; private final ControllerServiceProvider controllerServiceProvider; public VersionedReportingTaskSnapshotMapper(final ExtensionManager extensionManager, final ControllerServiceProvider controllerServiceProvider) { - this.flowMapper = new NiFiRegistryFlowMapper(extensionManager); + this.flowMapper = new VersionedComponentFlowMapper(extensionManager); this.controllerServiceProvider = controllerServiceProvider; } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/StatelessFlowTask.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/StatelessFlowTask.java index b1276eb52b4b..b34a8cd1e84a 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/StatelessFlowTask.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/StatelessFlowTask.java @@ -66,6 +66,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.function.Predicate; @@ -227,6 +228,8 @@ public synchronized void trigger() { fail(successfulInvocations, statelessProvRepo, e); } + updateFlowFileActivity(statelessProvRepo); + logger.debug("Acknowledging FlowFiles from {} invocations", allInvocations.size()); for (final Invocation invocation : allInvocations) { for (final PolledFlowFile polledFlowFile : invocation.getPolledFlowFiles()) { @@ -618,6 +621,72 @@ void updateEventRepository(final List invocations) { } } + + /** + * Updates the Stateless Group Node's FlowFileActivity. The latest activity time is obtained from the stateless flow's root group, + * which is updated by processors within the flow when their sessions commit. Transfer counts are computed from provenance events + * generated during the stateless flow execution, mirroring the behavior of StandardProcessSession.updateTransferCounts. + * + * @param statelessProvRepo the provenance event repository used during stateless flow execution + */ + void updateFlowFileActivity(final ProvenanceEventRepository statelessProvRepo) { + final OptionalLong latestActivityTime = flow.getLatestActivityTime(); + if (latestActivityTime.isPresent()) { + statelessGroupNode.getFlowFileActivity().updateLatestActivityTime(); + } + + updateTransferCounts(statelessProvRepo); + } + + private void updateTransferCounts(final ProvenanceEventRepository statelessProvRepo) { + int receivedCount = 0; + long receivedBytes = 0L; + int sentCount = 0; + long sentBytes = 0L; + + long firstProvEventId = 0; + while (true) { + try { + final List events = statelessProvRepo.getEvents(firstProvEventId, 1000); + if (events.isEmpty()) { + break; + } + + for (final ProvenanceEventRecord event : events) { + final ProvenanceEventType eventType = event.getEventType(); + switch (eventType) { + case RECEIVE: + case CREATE: + receivedCount++; + receivedBytes += event.getFileSize(); + break; + case FETCH: + receivedBytes += event.getFileSize(); + break; + case SEND: + sentCount++; + sentBytes += event.getFileSize(); + break; + default: + break; + } + } + + if (events.size() < 1000) { + break; + } + firstProvEventId += 1000; + } catch (final IOException e) { + logger.warn("Failed to obtain Provenance Events for FlowFile Activity tracking", e); + break; + } + } + + if (receivedCount > 0 || receivedBytes > 0L || sentCount > 0 || sentBytes > 0L) { + statelessGroupNode.getFlowFileActivity().updateTransferCounts(receivedCount, receivedBytes, sentCount, sentBytes); + } + } + private void expireRecords(final FlowFileQueue sourceQueue, final Set expiredRecords) throws IOException { if (expiredRecords.isEmpty()) { return; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/bootstrap/BootstrapDiagnosticsFactory.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/bootstrap/BootstrapDiagnosticsFactory.java index 994aafddcbef..9d4049e5b9b9 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/bootstrap/BootstrapDiagnosticsFactory.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/bootstrap/BootstrapDiagnosticsFactory.java @@ -25,6 +25,7 @@ import org.apache.nifi.diagnostics.ThreadDumpTask; import org.apache.nifi.diagnostics.bootstrap.tasks.ClusterDiagnosticTask; import org.apache.nifi.diagnostics.bootstrap.tasks.ComponentCountTask; +import org.apache.nifi.diagnostics.bootstrap.tasks.ConnectionDiagnosticTask; import org.apache.nifi.diagnostics.bootstrap.tasks.ContentRepositoryScanTask; import org.apache.nifi.diagnostics.bootstrap.tasks.DataValveDiagnosticsTask; import org.apache.nifi.diagnostics.bootstrap.tasks.DiagnosticAnalysisTask; @@ -82,6 +83,7 @@ public List getDiagnosticTasks() { tasks.add(new ClusterDiagnosticTask(flowController)); tasks.add(new GarbageCollectionDiagnosticTask(flowController)); tasks.add(new MemoryPoolPeakUsageTask()); + tasks.add(new ConnectionDiagnosticTask(flowController)); tasks.add(new RepositoryDiagnosticTask(flowController)); tasks.add(new ComponentCountTask(flowController)); tasks.add(new NiFiPropertiesDiagnosticTask(nifiProperties)); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/bootstrap/tasks/ConnectionDiagnosticTask.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/bootstrap/tasks/ConnectionDiagnosticTask.java new file mode 100644 index 000000000000..a7294307ac4b --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/diagnostics/bootstrap/tasks/ConnectionDiagnosticTask.java @@ -0,0 +1,265 @@ +/* + * 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.diagnostics.bootstrap.tasks; + +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorRepository; +import org.apache.nifi.connectable.Connection; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.queue.FlowFileQueue; +import org.apache.nifi.controller.queue.LoadBalanceCompression; +import org.apache.nifi.controller.queue.LoadBalanceStrategy; +import org.apache.nifi.controller.queue.LocalQueuePartitionDiagnostics; +import org.apache.nifi.controller.queue.QueueDiagnostics; +import org.apache.nifi.controller.queue.QueueSize; +import org.apache.nifi.controller.queue.RemoteQueuePartitionDiagnostics; +import org.apache.nifi.diagnostics.DiagnosticTask; +import org.apache.nifi.diagnostics.DiagnosticsDumpElement; +import org.apache.nifi.diagnostics.StandardDiagnosticsDumpElement; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.util.FormatUtils; + +import java.text.NumberFormat; +import java.util.ArrayList; +import java.util.List; + +public class ConnectionDiagnosticTask implements DiagnosticTask { + private static final String TABLE_FORMAT = "| %-40s | %-25s | %-25s | %20s | %20s | %20s | %-40s | %15s | %20s | %-19s | %-45s |"; + private static final String TABLE_HEADER = String.format(TABLE_FORMAT, + "Connection ID", "Source", "Destination", "Queued", "Active", "Swap Queue", "Swap Files", "Unacknowledged", "Penalized", "FlowFile Expiration", "Load Balancing"); + private static final String TABLE_SEPARATOR = "+" + "-".repeat(42) + "+" + "-".repeat(27) + "+" + "-".repeat(27) + "+" + + "-".repeat(22) + "+" + "-".repeat(22) + "+" + "-".repeat(22) + "+" + "-".repeat(42) + "+" + "-".repeat(17) + "+" + "-".repeat(22) + "+" + "-".repeat(21) + "+" + "-".repeat(47) + "+"; + + private final FlowController flowController; + + public ConnectionDiagnosticTask(final FlowController flowController) { + this.flowController = flowController; + } + + @Override + public DiagnosticsDumpElement captureDump(final boolean verbose) { + final List details = new ArrayList<>(); + + final ProcessGroup rootGroup = flowController.getFlowManager().getRootGroup(); + details.add("=== Connections in Primary Flow ==="); + captureConnectionDiagnostics(rootGroup, details); + + details.add(""); + final ConnectorRepository connectorRepository = flowController.getConnectorRepository(); + final List connectors = connectorRepository != null ? connectorRepository.getConnectors() : List.of(); + if (connectors.isEmpty()) { + details.add("This instance has no Connectors."); + } else { + details.add("=== Connections in Connectors ==="); + for (final ConnectorNode connector : connectors) { + details.add(""); + details.add("Connector: " + connector.getName() + " (ID: " + connector.getIdentifier() + ", State: " + connector.getCurrentState() + ")"); + final ProcessGroup managedGroup = connector.getActiveFlowContext().getManagedProcessGroup(); + captureConnectionDiagnostics(managedGroup, details); + } + } + + return new StandardDiagnosticsDumpElement("Connection Diagnostics", details); + } + + private void captureConnectionDiagnostics(final ProcessGroup group, final List details) { + final List connections = group.findAllConnections(); + + if (connections.isEmpty()) { + details.add("No connections"); + return; + } + + details.add(TABLE_SEPARATOR); + details.add(TABLE_HEADER); + details.add(TABLE_SEPARATOR); + + for (final Connection connection : connections) { + captureConnectionRow(connection, details); + capturePartitionRows(connection, details); + } + + details.add(TABLE_SEPARATOR); + details.add("Total: " + connections.size() + " connections"); + } + + private void captureConnectionRow(final Connection connection, final List details) { + final FlowFileQueue queue = connection.getFlowFileQueue(); + final String connectionId = connection.getIdentifier(); + final String sourceName = truncate(connection.getSource().getName(), 25); + final String destName = truncate(connection.getDestination().getName(), 25); + + final QueueSize totalSize = queue.size(); + final String queued = formatQueueSize(totalSize); + final String loadBalancing = formatLoadBalancing(queue); + + final String flowFileExpiration = queue.getFlowFileExpiration(); + final String expirationDisplay = "0 sec".equals(flowFileExpiration) ? "No Expiration" : flowFileExpiration; + + // Gather totals from all partitions + String active = "-"; + String swapQueue = "-"; + String swapFiles = "-"; + String unack = "-"; + String penalized = "0"; + + final QueueDiagnostics diagnostics = queue.getQueueDiagnostics(); + if (diagnostics != null) { + int totalActive = 0; + long totalActiveBytes = 0; + int totalSwapQueue = 0; + long totalSwapQueueBytes = 0; + int totalSwapFileCount = 0; + int totalSwapFlowFiles = 0; + long totalSwapBytes = 0; + int totalUnack = 0; + long totalUnackBytes = 0; + int totalPenalizedCount = 0; + long totalPenalizedBytes = 0; + + final LocalQueuePartitionDiagnostics localDiagnostics = diagnostics.getLocalQueuePartitionDiagnostics(); + if (localDiagnostics != null) { + totalActive += localDiagnostics.getActiveQueueSize().getObjectCount(); + totalActiveBytes += localDiagnostics.getActiveQueueSize().getByteCount(); + totalSwapQueue += localDiagnostics.getSwapQueueSize().getObjectCount(); + totalSwapQueueBytes += localDiagnostics.getSwapQueueSize().getByteCount(); + totalSwapFileCount += localDiagnostics.getSwapFileCount(); + totalSwapFlowFiles += localDiagnostics.getTotalSwapFileQueueSize().getObjectCount(); + totalSwapBytes += localDiagnostics.getTotalSwapFileQueueSize().getByteCount(); + totalUnack += localDiagnostics.getUnacknowledgedQueueSize().getObjectCount(); + totalUnackBytes += localDiagnostics.getUnacknowledgedQueueSize().getByteCount(); + totalPenalizedCount += localDiagnostics.getPenalizedQueueSize().getObjectCount(); + totalPenalizedBytes += localDiagnostics.getPenalizedQueueSize().getByteCount(); + } + + final List remotePartitions = diagnostics.getRemoteQueuePartitionDiagnostics(); + if (remotePartitions != null) { + for (final RemoteQueuePartitionDiagnostics remoteDiagnostics : remotePartitions) { + totalActive += remoteDiagnostics.getActiveQueueSize().getObjectCount(); + totalActiveBytes += remoteDiagnostics.getActiveQueueSize().getByteCount(); + totalSwapQueue += remoteDiagnostics.getSwapQueueSize().getObjectCount(); + totalSwapQueueBytes += remoteDiagnostics.getSwapQueueSize().getByteCount(); + totalSwapFileCount += remoteDiagnostics.getSwapFileCount(); + totalUnack += remoteDiagnostics.getUnacknowledgedQueueSize().getObjectCount(); + totalUnackBytes += remoteDiagnostics.getUnacknowledgedQueueSize().getByteCount(); + } + } + + active = formatQueueSize(new QueueSize(totalActive, totalActiveBytes)); + swapQueue = formatQueueSize(new QueueSize(totalSwapQueue, totalSwapQueueBytes)); + swapFiles = formatSwapFiles(totalSwapFileCount, totalSwapFlowFiles, totalSwapBytes); + unack = formatQueueSize(new QueueSize(totalUnack, totalUnackBytes)); + penalized = formatQueueSize(new QueueSize(totalPenalizedCount, totalPenalizedBytes)); + } + + details.add(String.format(TABLE_FORMAT, connectionId, sourceName, destName, queued, active, swapQueue, swapFiles, unack, penalized, expirationDisplay, loadBalancing)); + } + + private void capturePartitionRows(final Connection connection, final List details) { + final FlowFileQueue queue = connection.getFlowFileQueue(); + + // Only show partition rows if load balancing is enabled + if (queue.getLoadBalanceStrategy() == LoadBalanceStrategy.DO_NOT_LOAD_BALANCE) { + return; + } + + final QueueDiagnostics diagnostics = queue.getQueueDiagnostics(); + if (diagnostics == null) { + return; + } + + final LocalQueuePartitionDiagnostics localDiagnostics = diagnostics.getLocalQueuePartitionDiagnostics(); + if (localDiagnostics != null) { + final String active = formatQueueSize(localDiagnostics.getActiveQueueSize()); + final String swapQueue = formatQueueSize(localDiagnostics.getSwapQueueSize()); + final int swapFileCount = localDiagnostics.getSwapFileCount(); + final QueueSize swapFileQueueSize = localDiagnostics.getTotalSwapFileQueueSize(); + final String swapFiles = formatSwapFiles(swapFileCount, swapFileQueueSize.getObjectCount(), swapFileQueueSize.getByteCount()); + final QueueSize unacknowledgedSize = localDiagnostics.getUnacknowledgedQueueSize(); + final String unack = unacknowledgedSize.getObjectCount() > 0 ? formatQueueSize(unacknowledgedSize) : "-"; + final String penalized = formatQueueSize(localDiagnostics.getPenalizedQueueSize()); + + details.add(String.format(TABLE_FORMAT, " - Local Partition", "", "", "", active, swapQueue, swapFiles, unack, penalized, "", "")); + } + + final List remotePartitions = diagnostics.getRemoteQueuePartitionDiagnostics(); + if (remotePartitions != null && !remotePartitions.isEmpty()) { + for (final RemoteQueuePartitionDiagnostics remoteDiagnostics : remotePartitions) { + final String nodeId = remoteDiagnostics.getNodeIdentifier(); + final String active = formatQueueSize(remoteDiagnostics.getActiveQueueSize()); + final String swapQueue = formatQueueSize(remoteDiagnostics.getSwapQueueSize()); + final String swapFiles = String.valueOf(remoteDiagnostics.getSwapFileCount()); + final QueueSize unacknowledgedSize = remoteDiagnostics.getUnacknowledgedQueueSize(); + final String unack = unacknowledgedSize.getObjectCount() > 0 ? formatQueueSize(unacknowledgedSize) : "-"; + + details.add(String.format(TABLE_FORMAT, " - Remote Partition - " + nodeId, "", "", "", active, swapQueue, swapFiles, unack, "-", "", "")); + } + } + } + + private String formatLoadBalancing(final FlowFileQueue queue) { + final LoadBalanceStrategy strategy = queue.getLoadBalanceStrategy(); + if (strategy == LoadBalanceStrategy.DO_NOT_LOAD_BALANCE) { + return "None"; + } + + final StringBuilder builder = new StringBuilder(); + builder.append(strategy.name()); + + if (strategy == LoadBalanceStrategy.PARTITION_BY_ATTRIBUTE) { + final String partitioningAttribute = queue.getPartitioningAttribute(); + if (partitioningAttribute != null && !partitioningAttribute.isEmpty()) { + builder.append(" (").append(partitioningAttribute).append(")"); + } + } + + final LoadBalanceCompression compression = queue.getLoadBalanceCompression(); + if (compression != LoadBalanceCompression.DO_NOT_COMPRESS) { + builder.append(", ").append(compression.name()); + } + + return builder.toString(); + } + + private String formatQueueSize(final QueueSize queueSize) { + if (queueSize.getObjectCount() == 0) { + return "0"; + } + final String formattedCount = NumberFormat.getIntegerInstance().format(queueSize.getObjectCount()); + return formattedCount + " (" + FormatUtils.formatDataSize(queueSize.getByteCount()) + ")"; + } + + private String formatSwapFiles(final int fileCount, final int flowFileCount, final long totalBytes) { + if (fileCount == 0) { + return "0"; + } + final String files = fileCount == 1 ? "1 File" : NumberFormat.getIntegerInstance().format(fileCount) + " Files"; + final String flowFiles = NumberFormat.getIntegerInstance().format(flowFileCount) + " FlowFiles"; + final String size = FormatUtils.formatDataSize(totalBytes); + return files + " - " + flowFiles + " (" + size + ")"; + } + + private String truncate(final String value, final int maxLength) { + if (value == null) { + return ""; + } + if (value.length() <= maxLength) { + return value; + } + return value.substring(0, maxLength - 3) + "..."; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/flowanalysis/StandardFlowAnalyzer.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/flowanalysis/StandardFlowAnalyzer.java index 63ae900d5117..ce0beac1ad98 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/flowanalysis/StandardFlowAnalyzer.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/flowanalysis/StandardFlowAnalyzer.java @@ -30,7 +30,7 @@ import org.apache.nifi.flow.VersionedProcessGroup; import org.apache.nifi.flow.VersionedProcessor; import org.apache.nifi.nar.ExtensionManager; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import org.apache.nifi.validation.RuleViolation; import org.apache.nifi.validation.RuleViolationsManager; import org.slf4j.Logger; @@ -89,7 +89,7 @@ public void setFlowAnalysisRequired(boolean flowAnalysisRequired) { public void analyzeProcessor(ProcessorNode processorNode) { logger.debug("Running analysis on {}", processorNode); - final NiFiRegistryFlowMapper mapper = createMapper(); + final VersionedComponentFlowMapper mapper = createMapper(); VersionedProcessor versionedProcessor = mapper.mapProcessor( processorNode, @@ -105,7 +105,7 @@ public void analyzeProcessor(ProcessorNode processorNode) { public void analyzeControllerService(ControllerServiceNode controllerServiceNode) { logger.debug("Running analysis on {}", controllerServiceNode); - final NiFiRegistryFlowMapper mapper = createMapper(); + final VersionedComponentFlowMapper mapper = createMapper(); VersionedControllerService versionedControllerService = mapper.mapControllerService( controllerServiceNode, @@ -266,8 +266,8 @@ private String getDisplayName(VersionedComponent component) { return displayName; } - private NiFiRegistryFlowMapper createMapper() { - NiFiRegistryFlowMapper mapper = FlowAnalysisUtil.createMapper(extensionManager); + private VersionedComponentFlowMapper createMapper() { + VersionedComponentFlowMapper mapper = FlowAnalysisUtil.createMapper(extensionManager); return mapper; } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/framework/configuration/FlowControllerConfiguration.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/framework/configuration/FlowControllerConfiguration.java index 88d760b34160..63aff7636a6e 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/framework/configuration/FlowControllerConfiguration.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/framework/configuration/FlowControllerConfiguration.java @@ -25,11 +25,13 @@ import org.apache.nifi.asset.AssetSynchronizer; import org.apache.nifi.asset.StandardAssetComponentManager; import org.apache.nifi.asset.StandardAssetSynchronizer; +import org.apache.nifi.asset.StandardConnectorAssetSynchronizer; import org.apache.nifi.authorization.Authorizer; import org.apache.nifi.cluster.coordination.ClusterCoordinator; import org.apache.nifi.cluster.coordination.heartbeat.HeartbeatMonitor; import org.apache.nifi.cluster.protocol.NodeProtocolSender; import org.apache.nifi.cluster.protocol.impl.NodeProtocolSenderListener; +import org.apache.nifi.components.connector.ConnectorRequestReplicator; import org.apache.nifi.components.state.StateManagerProvider; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.StandardFlowService; @@ -95,34 +97,21 @@ public class FlowControllerConfiguration { private static final String COMPONENT_METRIC_REPORTER_IMPLEMENTATION = "nifi.component.metric.reporter.implementation"; private NiFiProperties properties; - private ExtensionDiscoveringManager extensionManager; - private AuditService auditService; - private Authorizer authorizer; - private RevisionManager revisionManager; - private LeaderElectionManager leaderElectionManager; - private SSLContext sslContext; - private X509KeyManager keyManager; - private X509TrustManager trustManager; - private StateManagerProvider stateManagerProvider; - private BulletinRepository bulletinRepository; - private NodeProtocolSender nodeProtocolSender; - private NodeProtocolSenderListener nodeProtocolSenderListener; - private HeartbeatMonitor heartbeatMonitor; - private ClusterCoordinator clusterCoordinator; + private ConnectorRequestReplicator connectorRequestReplicator; @Autowired public void setProperties(final NiFiProperties properties) { @@ -164,6 +153,11 @@ public void setBulletinRepository(final BulletinRepository bulletinRepository) { this.bulletinRepository = bulletinRepository; } + @Autowired(required = false) + public void setConnectorRequestReplicator(final ConnectorRequestReplicator connectorRequestReplicator) { + this.connectorRequestReplicator = connectorRequestReplicator; + } + @Autowired(required = false) public void setSslContext(final SSLContext sslContext) { this.sslContext = sslContext; @@ -200,6 +194,7 @@ public void setClusterCoordinator(final ClusterCoordinator clusterCoordinator) { this.clusterCoordinator = clusterCoordinator; } + /** * Flow Controller implementation depends on cluster configuration * @@ -223,7 +218,8 @@ public FlowController flowController() throws Exception { extensionManager, statusHistoryRepository(), ruleViolationsManager(), - stateManagerProvider + stateManagerProvider, + connectorRequestReplicator ); } else { flowController = FlowController.createClusteredInstance( @@ -243,7 +239,8 @@ public FlowController flowController() throws Exception { revisionManager, statusHistoryRepository(), ruleViolationsManager(), - stateManagerProvider + stateManagerProvider, + connectorRequestReplicator ); } @@ -266,7 +263,8 @@ public FlowService flowService(@Autowired final NarManager narManager) throws Ex properties, revisionManager, narManager, - assetSynchronizer(), + parameterContextAssetSynchronizer(), + connectorAssetSynchronizer(), authorizer ); } else { @@ -277,7 +275,8 @@ public FlowService flowService(@Autowired final NarManager narManager) throws Ex clusterCoordinator, revisionManager, narManager, - assetSynchronizer(), + parameterContextAssetSynchronizer(), + connectorAssetSynchronizer(), authorizer ); } @@ -466,7 +465,7 @@ public AssetManager assetManager() throws Exception { } /** - * Asset Synchronizer depends on ClusterCoordinator, WebClientService, and NiFiProperties + * Parameter Conext Asset Synchronizer depends on ClusterCoordinator, WebClientService, and NiFiProperties * * @return Asset Synchronizer */ @@ -475,6 +474,21 @@ public AssetSynchronizer assetSynchronizer() throws Exception { return new StandardAssetSynchronizer(flowController(), clusterCoordinator, webClientService(), properties, affectedComponentManager()); } + @Bean + public AssetSynchronizer parameterContextAssetSynchronizer() throws Exception { + return new StandardAssetSynchronizer(flowController(), clusterCoordinator, webClientService(), properties, affectedComponentManager()); + } + + /** + * Connector Asset Synchronizer depends on ClusterCoordinator, WebClientService, and NiFiProperties + * + * @return Connector Asset Synchronizer + */ + @Bean + public AssetSynchronizer connectorAssetSynchronizer() throws Exception { + return new StandardConnectorAssetSynchronizer(flowController(), clusterCoordinator, webClientService(), properties); + } + /** * Affected Component Manager depends on FlowController * diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardStatelessGroupNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardStatelessGroupNode.java index a6c076190d34..ac727bdccdb3 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardStatelessGroupNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardStatelessGroupNode.java @@ -23,8 +23,10 @@ import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.state.StateManagerProvider; import org.apache.nifi.components.state.StatelessStateManagerProvider; +import org.apache.nifi.connectable.ConnectableFlowFileActivity; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; +import org.apache.nifi.connectable.FlowFileActivity; import org.apache.nifi.connectable.Port; import org.apache.nifi.connectable.Position; import org.apache.nifi.controller.BackoffMechanism; @@ -120,6 +122,8 @@ public class StandardStatelessGroupNode implements StatelessGroupNode { private volatile long schedulingNanos = 1L; private ComponentLog logger; + private final ConnectableFlowFileActivity flowFileActivity = new ConnectableFlowFileActivity(); + private StandardStatelessGroupNode(final Builder builder) { this.processGroup = builder.rootGroup; this.controllerServiceProvider = builder.controllerServiceProvider; @@ -151,6 +155,7 @@ public void start(final ScheduledExecutorService executor, final SchedulingAgent return; } + flowFileActivity.reset(); currentState = ScheduledState.STARTING; logger.info("Starting {}", this); } finally { @@ -1036,4 +1041,9 @@ public StatelessGroupNode build() { } } + + @Override + public FlowFileActivity getFlowFileActivity() { + return flowFileActivity; + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/manifest/StandardRuntimeManifestService.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/manifest/StandardRuntimeManifestService.java index 1a7f3ef214a1..320f59af99b5 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/manifest/StandardRuntimeManifestService.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/manifest/StandardRuntimeManifestService.java @@ -401,6 +401,51 @@ private Map discoverAdditionalDetails(final BundleDetails bundleDe return additionalDetailsMap; } + @Override + public Map discoverStepDocumentation(final String group, final String artifact, final String version, final String connectorType) { + final BundleCoordinate bundleCoordinate = new BundleCoordinate(group, artifact, version); + final Bundle bundle = extensionManager.getBundle(bundleCoordinate); + + if (bundle == null) { + throw new ResourceNotFoundException("Unable to find bundle [" + bundleCoordinate + "]"); + } + + return discoverStepDocumentation(bundle.getBundleDetails(), connectorType); + } + + private Map discoverStepDocumentation(final BundleDetails bundleDetails, final String connectorType) { + final Map stepDocsMap = new LinkedHashMap<>(); + + final File stepDocsDir = new File(bundleDetails.getWorkingDirectory(), "META-INF/docs/steps/" + connectorType); + if (!stepDocsDir.exists()) { + LOGGER.debug("No steps directory found for [{}] under [{}]", connectorType, bundleDetails.getWorkingDirectory().getAbsolutePath()); + return stepDocsMap; + } + + final File[] stepDocFiles = stepDocsDir.listFiles(); + if (stepDocFiles == null) { + return stepDocsMap; + } + + for (final File stepDocFile : stepDocFiles) { + if (!stepDocFile.isFile() || !stepDocFile.getName().endsWith(".md")) { + LOGGER.debug("Skipping [{}], not a markdown file...", stepDocFile.getAbsolutePath()); + continue; + } + + final String fileName = stepDocFile.getName().substring(0, stepDocFile.getName().length() - 3); + final String stepName = fileNameToStepName(fileName); + stepDocsMap.put(stepName, stepDocFile); + LOGGER.debug("Discovered step documentation for step [{}] at [{}]", stepName, stepDocFile.getAbsolutePath()); + } + + return stepDocsMap; + } + + private String fileNameToStepName(final String fileName) { + return fileName.replace("_", " "); + } + private Map loadAdditionalDetails(final BundleDetails bundleDetails) { final Map additionalDetailsMap = discoverAdditionalDetails(bundleDetails); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/nar/NarInstallTask.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/nar/NarInstallTask.java index 8c2673353d54..ffdb2fa43565 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/nar/NarInstallTask.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/nar/NarInstallTask.java @@ -20,6 +20,7 @@ import org.apache.nifi.bundle.Bundle; import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.bundle.BundleDetails; +import org.apache.nifi.components.connector.Connector; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.service.ControllerServiceProvider; import org.apache.nifi.flowanalysis.FlowAnalysisRule; @@ -54,7 +55,8 @@ public class NarInstallTask implements Runnable { ReportingTask.class, FlowRegistryClient.class, FlowAnalysisRule.class, - ParameterProvider.class + ParameterProvider.class, + Connector.class ); private final NarNode narNode; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.components.connector.ConnectorRepository b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.components.connector.ConnectorRepository new file mode 100644 index 000000000000..8ce9c5d3ce38 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/META-INF/services/org.apache.nifi.components.connector.ConnectorRepository @@ -0,0 +1,16 @@ +# 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. + +org.apache.nifi.components.connector.StandardConnectorRepository diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/BlockingConnector.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/BlockingConnector.java new file mode 100644 index 000000000000..75c35154d693 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/BlockingConnector.java @@ -0,0 +1,132 @@ +/* + * 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.components.connector; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.flow.VersionedExternalFlow; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; + +public class BlockingConnector implements Connector { + private final CountDownLatch startLatch; + private final CountDownLatch stopLatch; + private final CountDownLatch finishUpdateLatch; + + public BlockingConnector(final CountDownLatch startLatch, final CountDownLatch stopLatch, final CountDownLatch finishUpdateLatch) { + this.startLatch = startLatch; + this.stopLatch = stopLatch; + this.finishUpdateLatch = finishUpdateLatch; + } + + + @Override + public void initialize(final ConnectorInitializationContext connectorInitializationContext) { + } + + @Override + public VersionedExternalFlow getInitialFlow() { + return null; + } + + @Override + public void start(final FlowContext activeContext) throws FlowUpdateException { + try { + startLatch.await(); + } catch (final InterruptedException e) { + throw new FlowUpdateException(e); + } + } + + @Override + public void stop(final FlowContext activeContext) throws FlowUpdateException { + try { + stopLatch.await(); + } catch (final InterruptedException e) { + throw new FlowUpdateException(e); + } + } + + @Override + public List validate(final FlowContext flowContext, final ConnectorValidationContext connectorValidationContext) { + return List.of(); + } + + @Override + public List validateConfigurationStep(final ConfigurationStep configurationStep, final ConnectorConfigurationContext connectorConfigurationContext, + final ConnectorValidationContext connectorValidationContext) { + return List.of(); + } + + @Override + public List getConfigurationSteps() { + return List.of(); + } + + @Override + public void onConfigurationStepConfigured(final String stepName, final FlowContext flowContext) { + } + + @Override + public void prepareForUpdate(final FlowContext workingContext, final FlowContext activeContext) { + } + + @Override + public void abortUpdate(final FlowContext workingContext, final Throwable throwable) { + + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + try { + finishUpdateLatch.await(); + } catch (final InterruptedException e) { + throw new FlowUpdateException(e); + } + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext flowContext) { + return List.of(); + } + + @Override + public List verify(final FlowContext flowContext) { + return List.of(); + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName, final FlowContext workingContext, final String filter) { + return List.of(); + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName, final FlowContext workingContext) { + return List.of(); + } + + @Override + public CompletableFuture drainFlowFiles(final FlowContext flowContext) { + return CompletableFuture.completedFuture(null); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/DynamicAllowableValuesConnector.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/DynamicAllowableValuesConnector.java new file mode 100644 index 000000000000..404a78b9727f --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/DynamicAllowableValuesConnector.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.components.connector; + +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.Validator; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.components.ProcessorFacade; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; + +import java.io.File; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class DynamicAllowableValuesConnector extends AbstractConnector { + static final ConnectorPropertyDescriptor FILE_PATH = new ConnectorPropertyDescriptor.Builder() + .name("File Path") + .description("The path to the file") + .addValidator(new SimpleFileExistsValidator()) + .required(true) + .build(); + + static final ConnectorPropertyGroup FILE_PROPERTY_GROUP = new ConnectorPropertyGroup.Builder() + .name("") + .addProperty(FILE_PATH) + .build(); + + static final ConfigurationStep FILE_STEP = new ConfigurationStep.Builder() + .name("File") + .propertyGroups(List.of(FILE_PROPERTY_GROUP)) + .build(); + + static final ConnectorPropertyDescriptor FIRST_PRIMARY_COLOR = new ConnectorPropertyDescriptor.Builder() + .name("First Primary Color") + .description("The first primary color") + .allowableValuesFetchable(true) + .required(true) + .build(); + + static final ConnectorPropertyGroup PRIMARY_COLORS_PROPERTY_GROUP = new ConnectorPropertyGroup.Builder() + .name("Primary Colors") + .addProperty(FIRST_PRIMARY_COLOR) + .build(); + + static final ConfigurationStep COLORS_STEP = new ConfigurationStep.Builder() + .name("Colors") + .propertyGroups(List.of(PRIMARY_COLORS_PROPERTY_GROUP)) + .dependsOn(FILE_STEP, FILE_PATH) + .build(); + + private static final List CONFIGURATION_STEPS = List.of(FILE_STEP, COLORS_STEP); + + + @Override + public VersionedExternalFlow getInitialFlow() { + return null; + } + + @Override + public List getConfigurationSteps() { + return CONFIGURATION_STEPS; + } + + @Override + protected void onStepConfigured(final String stepName, final FlowContext flowContext) { + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + final VersionedExternalFlow externalFlow = VersionedFlowUtils.loadFlowFromResource("flows/choose-color.json"); + final VersionedProcessGroup rootGroup = externalFlow.getFlowContents(); + final VersionedProcessor processor = rootGroup.getProcessors().iterator().next(); + processor.setProperties(Map.of("File", workingContext.getConfigurationContext().getProperty(FILE_STEP, FILE_PATH).getValue())); + + getInitializationContext().updateFlow(activeContext, externalFlow); + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext flowContext) { + return List.of(); + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName, final FlowContext flowContext) { + if ("Colors".equals(stepName) && "First Primary Color".equals(propertyName)) { + final Set processorFacades = flowContext.getRootGroup().getProcessors(); + if (!processorFacades.isEmpty()) { + final ProcessorFacade processorFacade = processorFacades.iterator().next(); + + try { + @SuppressWarnings("unchecked") + final List fileValues = (List) processorFacade.invokeConnectorMethod("getFileValues", Map.of()); + + return fileValues.stream() + .map(AllowableValue::new) + .map(DescribedValue.class::cast) + .toList(); + } catch (final InvocationFailedException e) { + throw new RuntimeException("Failed to fetch allowable values from connector.", e); + } + } + } + + return super.fetchAllowableValues(stepName, propertyName, flowContext); + } + + + public static class SimpleFileExistsValidator implements Validator { + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext validationContext) { + final File file = new File(input); + if (file.exists()) { + return new ValidationResult.Builder().subject(subject).input(input).valid(true).build(); + } + + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("File does not exist.").build(); + } + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/DynamicFlowConnector.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/DynamicFlowConnector.java new file mode 100644 index 000000000000..bf8148214393 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/DynamicFlowConnector.java @@ -0,0 +1,336 @@ +/* + * 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.components.connector; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.Validator; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.processors.CreateDummyFlowFile; +import org.apache.nifi.components.connector.processors.DuplicateFlowFile; +import org.apache.nifi.components.connector.processors.LogFlowFileContents; +import org.apache.nifi.components.connector.processors.OverwriteFlowFile; +import org.apache.nifi.components.connector.services.impl.StandardCounterService; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.ConnectableComponent; +import org.apache.nifi.flow.Position; +import org.apache.nifi.flow.VersionedConnection; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedPort; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; +import org.apache.nifi.processor.util.StandardValidators; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class DynamicFlowConnector extends AbstractConnector { + private volatile boolean initialized = false; + + static final ConnectorPropertyDescriptor SOURCE_TEXT = new ConnectorPropertyDescriptor.Builder() + .name("Source Text") + .type(PropertyType.STRING) + .addValidator(Validator.VALID) + .required(true) + .defaultValue("Hello World") + .build(); + + static final ConnectorPropertyDescriptor COUNT_FLOWFILES = new ConnectorPropertyDescriptor.Builder() + .name("Count FlowFiles") + .description("If true, the Counter Service will be used to count the number of FlowFiles created") + .type(PropertyType.BOOLEAN) + .allowableValues("true", "false") + .required(true) + .defaultValue("false") + .build(); + + static final ConnectorPropertyDescriptor NUM_COPIES = new ConnectorPropertyDescriptor.Builder() + .name("Number of Copies") + .type(PropertyType.INTEGER) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .required(true) + .defaultValue("1") + .build(); + + static final ConnectorPropertyDescriptor LOG_FLOWFILE_CONTENTS = new ConnectorPropertyDescriptor.Builder() + .name("Log FlowFile Contents") + .type(PropertyType.BOOLEAN) + .allowableValues("true", "false") + .required(true) + .defaultValue("false") + .build(); + + private static final ConnectorPropertyGroup SOURCE_GROUP = new ConnectorPropertyGroup.Builder() + .name("Source Settings") + .description("Settings for the source of FlowFiles") + .addProperty(SOURCE_TEXT) + .addProperty(COUNT_FLOWFILES) + .build(); + + private static final ConfigurationStep SOURCE_STEP = new ConfigurationStep.Builder() + .name("Source") + .propertyGroups(List.of(SOURCE_GROUP)) + .build(); + + private static final ConnectorPropertyGroup DUPLICATION_GROUP = new ConnectorPropertyGroup.Builder() + .addProperty(NUM_COPIES) + .build(); + + private static final ConfigurationStep DUPLICATION_STEP = new ConfigurationStep.Builder() + .name("Duplication") + .propertyGroups(List.of(DUPLICATION_GROUP)) + .build(); + + private static final ConnectorPropertyGroup DESTINATION_GROUP = new ConnectorPropertyGroup.Builder() + .addProperty(LOG_FLOWFILE_CONTENTS) + .build(); + + private static final ConfigurationStep DESTINATION_STEP = new ConfigurationStep.Builder() + .name("Destination") + .propertyGroups(List.of(DESTINATION_GROUP)) + .build(); + + private static final List configurationSteps = List.of( + SOURCE_STEP, + DUPLICATION_STEP, + DESTINATION_STEP + ); + + @Override + public List getConfigurationSteps() { + return configurationSteps; + } + + @Override + protected void init() { + initialized = true; + } + + @Override + public VersionedExternalFlow getInitialFlow() { + return VersionedFlowUtils.loadFlowFromResource("flows/generate-duplicate-log-flow.json"); + } + + public boolean isInitialized() { + return initialized; + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + // Apply the updates to the active flow that were made to the working flow + final VersionedExternalFlow versionedFlow = getFlow(workingContext); + getInitializationContext().updateFlow(activeContext, versionedFlow); + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext flowContext) { + return List.of(); + } + + @Override + public void onStepConfigured(final String stepName, final FlowContext workingContext) throws FlowUpdateException { + // Now that configuration is available, update the flow based on configured properties + final VersionedExternalFlow versionedFlow = getFlow(workingContext); + getInitializationContext().updateFlow(workingContext, versionedFlow); + } + + private VersionedExternalFlow getFlow(final FlowContext flowContext) { + final VersionedExternalFlow externalFlow = VersionedFlowUtils.loadFlowFromResource("flows/generate-duplicate-log-flow.json"); + final VersionedProcessGroup versionedProcessGroup = externalFlow.getFlowContents(); + + // Update the flow based on configured properties + updateSourceStep(flowContext, versionedProcessGroup); + updateDuplicationStep(flowContext, versionedProcessGroup); + updateDestinationStep(flowContext, versionedProcessGroup); + + return externalFlow; + } + + private void updateSourceStep(final FlowContext flowContext, final VersionedProcessGroup rootGroup) { + final ConnectorConfigurationContext configContext = flowContext.getConfigurationContext(); + final String sourceText = configContext.getProperty(SOURCE_STEP, SOURCE_TEXT).getValue(); + + final VersionedProcessor sourceTextProcessor = VersionedFlowUtils.findProcessor(rootGroup, + p -> p.getType().equals(OverwriteFlowFile.class.getName())).orElseThrow(); + sourceTextProcessor.getProperties().put(OverwriteFlowFile.CONTENT.getName(), sourceText); + + final boolean count = configContext.getProperty(SOURCE_STEP, COUNT_FLOWFILES).asBoolean(); + if (count) { + final Bundle systemBundle = new Bundle(); + systemBundle.setArtifact("system"); + systemBundle.setGroup("default"); + systemBundle.setVersion("unversioned"); + + final VersionedControllerService controllerService = VersionedFlowUtils.addControllerService(rootGroup, StandardCounterService.class.getName(), systemBundle, "Count"); + rootGroup.getControllerServices().add(controllerService); + + final VersionedProcessor generateProcessor = VersionedFlowUtils.findProcessor(rootGroup, + p -> p.getType().equals(CreateDummyFlowFile.class.getName())).orElseThrow(); + generateProcessor.getProperties().put("Counter Service", controllerService.getIdentifier()); + } + } + + private void updateDuplicationStep(final FlowContext flowContext, final VersionedProcessGroup rootGroup) { + final ConnectorConfigurationContext configContext = flowContext.getConfigurationContext(); + + final int numCopies = configContext.getProperty(DUPLICATION_STEP, NUM_COPIES).asInteger(); + final VersionedProcessor duplicateProcessor = VersionedFlowUtils.findProcessor(rootGroup, + p -> p.getType().equals(DuplicateFlowFile.class.getName())).orElseThrow(); + duplicateProcessor.getProperties().put(DuplicateFlowFile.NUM_DUPLICATES.getName(), String.valueOf(numCopies)); + + // Need to determine how many Connections exist going out of the DuplicateFlowFile processor + // and then add/remove connections as necessary to match the number of copies. + final VersionedProcessGroup duplicatesGroup = VersionedFlowUtils.findGroupForProcessor(rootGroup, duplicateProcessor); + if (duplicatesGroup == null) { + return; + } + + final List outboundConnections = VersionedFlowUtils.findOutboundConnections(rootGroup, duplicateProcessor); + final int currentConnections = outboundConnections.size(); + + if (numCopies > currentConnections) { + // Add new connections for the additional copies + addConnectionsForDuplicates(duplicatesGroup, duplicateProcessor, outboundConnections, numCopies); + } else if (numCopies < currentConnections) { + // Remove excess connections + removeExcessConnections(duplicatesGroup, outboundConnections, numCopies); + } + } + + private void updateDestinationStep(final FlowContext flowContext, final VersionedProcessGroup rootGroup) { + final ConnectorConfigurationContext configContext = flowContext.getConfigurationContext(); + + final boolean logContents = configContext.getProperty(DESTINATION_STEP, LOG_FLOWFILE_CONTENTS).asBoolean(); + if (!logContents) { + return; + } + + final VersionedProcessGroup destinationGroup = rootGroup.getProcessGroups().stream() + .filter(group -> group.getName().equals("Destination")) + .findFirst() + .orElseThrow(); + + // Add a LogFlowFileContents processor to the Destination group. Move the TerminateFlowFile processor down about + // 250 pixels and update connections so that it's Port -> LogFlowFileContents -> TerminateFlowFile + + // Find the existing TerminateFlowFile processor + final VersionedProcessor terminateProcessor = VersionedFlowUtils.findProcessor(destinationGroup, + p -> p.getType().equals("org.apache.nifi.components.connector.processors.TerminateFlowFile")).orElseThrow(); + + // Get the first (and only) input port + final VersionedPort inputPort = destinationGroup.getInputPorts().iterator().next(); + + // Create LogFlowFileContents processor + final VersionedProcessor logProcessor = createLogFlowFileContentsProcessor(destinationGroup, terminateProcessor); + + // Move TerminateFlowFile processor down by 250 pixels + final Position terminatePosition = terminateProcessor.getPosition(); + terminateProcessor.setPosition(new Position(terminatePosition.getX(), terminatePosition.getY() + 250)); + + // Update connections: Port -> LogFlowFileContents -> TerminateFlowFile + updateDestinationConnections(destinationGroup, inputPort, logProcessor, terminateProcessor); + } + + private void addConnectionsForDuplicates(final VersionedProcessGroup duplicatesGroup, final VersionedProcessor duplicateProcessor, + final List existingConnections, final int targetNumCopies) { + + if (existingConnections.isEmpty()) { + return; + } + + // Use the first existing connection as a template for creating new connections + final VersionedConnection templateConnection = existingConnections.getFirst(); + final ConnectableComponent sourceComponent = VersionedFlowUtils.createConnectableComponent(duplicateProcessor); + final ConnectableComponent destinationComponent = templateConnection.getDestination(); + + final int currentConnections = existingConnections.size(); + for (int i = currentConnections + 1; i <= targetNumCopies; i++) { + final Set relationships = Set.of(String.valueOf(i)); + VersionedFlowUtils.addConnection(duplicatesGroup, sourceComponent, destinationComponent, relationships); + } + } + + private void removeExcessConnections(final VersionedProcessGroup duplicatesGroup, final List outboundConnections, final int targetNumCopies) { + final Set connectionsToRemove = new HashSet<>(); + + // Sort connections by relationship name (which should be numeric) and remove the highest numbered ones + final List sortedConnections = new ArrayList<>(outboundConnections); + sortedConnections.sort((c1, c2) -> { + final String rel1 = c1.getSelectedRelationships().iterator().next(); + final String rel2 = c2.getSelectedRelationships().iterator().next(); + try { + final int num1 = Integer.parseInt(rel1); + final int num2 = Integer.parseInt(rel2); + return Integer.compare(num2, num1); // Sort descending to remove highest numbers first + } catch (final NumberFormatException e) { + return rel2.compareTo(rel1); + } + }); + + final int connectionsToRemoveCount = outboundConnections.size() - targetNumCopies; + for (int i = 0; i < connectionsToRemoveCount && i < sortedConnections.size(); i++) { + connectionsToRemove.add(sortedConnections.get(i)); + } + + // Remove the connections from the process group + final Set groupConnections = duplicatesGroup.getConnections(); + if (groupConnections != null) { + groupConnections.removeAll(connectionsToRemove); + } + } + + private void updateDestinationConnections(final VersionedProcessGroup destinationGroup, final VersionedPort inputPort, + final VersionedProcessor logProcessor, final VersionedProcessor terminateProcessor) { + // Find and remove the existing connection from input port to terminate processor + final Set connections = destinationGroup.getConnections(); + if (connections != null) { + connections.stream() + .filter(conn -> conn.getSource().getId().equals(inputPort.getIdentifier())) + .filter(conn -> conn.getDestination().getId().equals(terminateProcessor.getIdentifier())) + .findFirst() + .ifPresent(connections::remove); + } + + // Create connection from input port to LogFlowFileContents processor + final ConnectableComponent inputPortComponent = VersionedFlowUtils.createConnectableComponent(inputPort); + final ConnectableComponent logProcessorComponent = VersionedFlowUtils.createConnectableComponent(logProcessor); + VersionedFlowUtils.addConnection(destinationGroup, inputPortComponent, logProcessorComponent, Set.of("")); + + // Create connection from LogFlowFileContents processor to TerminateFlowFile processor + final ConnectableComponent terminateProcessorComponent = VersionedFlowUtils.createConnectableComponent(terminateProcessor); + VersionedFlowUtils.addConnection(destinationGroup, logProcessorComponent, terminateProcessorComponent, Set.of("success")); + } + + private VersionedProcessor createLogFlowFileContentsProcessor(final VersionedProcessGroup destinationGroup, final VersionedProcessor terminateProcessor) { + // Position new processor where terminate processor is; terminate processor will be moved down later + final Position terminatePosition = terminateProcessor.getPosition(); + final Position logProcessorPosition = new Position(terminatePosition.getX(), terminatePosition.getY()); + + return VersionedFlowUtils.addProcessor( + destinationGroup, + LogFlowFileContents.class.getName(), + terminateProcessor.getBundle(), + "Log FlowFile Contents", + logProcessorPosition + ); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/MissingBundleConnector.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/MissingBundleConnector.java new file mode 100644 index 000000000000..c21b55dfbbaf --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/MissingBundleConnector.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.components.connector; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.Position; +import org.apache.nifi.flow.ScheduledState; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +/** + * A test connector that returns an initial flow containing a processor with a bundle that does not exist. + * This is used to test the behavior when a connector's initial flow references unavailable components. + */ +public class MissingBundleConnector extends AbstractConnector { + + private static final List CONFIGURATION_STEPS = List.of(); + + @Override + public List getConfigurationSteps() { + return CONFIGURATION_STEPS; + } + + @Override + protected void init() { + } + + @Override + public VersionedExternalFlow getInitialFlow() { + final VersionedProcessGroup rootGroup = new VersionedProcessGroup(); + rootGroup.setIdentifier(UUID.randomUUID().toString()); + rootGroup.setInstanceIdentifier(UUID.randomUUID().toString()); + rootGroup.setName("Missing Bundle Connector Flow"); + rootGroup.setPosition(new Position(0.0, 0.0)); + rootGroup.setProcessGroups(new HashSet<>()); + rootGroup.setConnections(new HashSet<>()); + rootGroup.setInputPorts(new HashSet<>()); + rootGroup.setOutputPorts(new HashSet<>()); + rootGroup.setControllerServices(new HashSet<>()); + rootGroup.setFunnels(new HashSet<>()); + rootGroup.setLabels(new HashSet<>()); + + final VersionedProcessor missingProcessor = new VersionedProcessor(); + missingProcessor.setIdentifier(UUID.randomUUID().toString()); + missingProcessor.setInstanceIdentifier(UUID.randomUUID().toString()); + missingProcessor.setName("Missing Processor"); + missingProcessor.setType("com.example.nonexistent.MissingProcessor"); + missingProcessor.setPosition(new Position(100.0, 100.0)); + missingProcessor.setScheduledState(ScheduledState.ENABLED); + missingProcessor.setSchedulingPeriod("0 sec"); + missingProcessor.setSchedulingStrategy("TIMER_DRIVEN"); + missingProcessor.setExecutionNode("ALL"); + missingProcessor.setPenaltyDuration("30 sec"); + missingProcessor.setYieldDuration("1 sec"); + missingProcessor.setBulletinLevel("WARN"); + missingProcessor.setRunDurationMillis(0L); + missingProcessor.setConcurrentlySchedulableTaskCount(1); + missingProcessor.setAutoTerminatedRelationships(new HashSet<>()); + missingProcessor.setProperties(Map.of()); + missingProcessor.setPropertyDescriptors(Map.of()); + missingProcessor.setGroupIdentifier(rootGroup.getIdentifier()); + + final Bundle missingBundle = new Bundle(); + missingBundle.setGroup("com.example.nonexistent"); + missingBundle.setArtifact("missing-nar"); + missingBundle.setVersion("1.0.0"); + missingProcessor.setBundle(missingBundle); + + rootGroup.setProcessors(new HashSet<>(List.of(missingProcessor))); + + final VersionedExternalFlow externalFlow = new VersionedExternalFlow(); + externalFlow.setFlowContents(rootGroup); + return externalFlow; + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext flowContext) { + return List.of(); + } + + @Override + public void onStepConfigured(final String stepName, final FlowContext workingContext) throws FlowUpdateException { + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/OnPropertyModifiedConnector.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/OnPropertyModifiedConnector.java new file mode 100644 index 000000000000..bf5f242a939e --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/OnPropertyModifiedConnector.java @@ -0,0 +1,88 @@ +/* + * 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.components.connector; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.processor.util.StandardValidators; + +import java.util.List; +import java.util.Map; + +/** + * A test connector that creates a flow with an OnPropertyModifiedTracker processor. + * The processor's Configured Number property is bound to a parameter, allowing this connector + * to test that onPropertyModified is called when a Connector's applyUpdate changes a parameter value. + */ +public class OnPropertyModifiedConnector extends AbstractConnector { + + private static final String PARAMETER_NAME = "CONFIGURED_NUMBER"; + + static final ConnectorPropertyDescriptor NUMBER_VALUE = new ConnectorPropertyDescriptor.Builder() + .name("Number Value") + .description("The number value to set for the Configured Number parameter") + .type(PropertyType.STRING) + .addValidator(StandardValidators.INTEGER_VALIDATOR) + .required(true) + .build(); + + private static final ConnectorPropertyGroup PROPERTY_GROUP = new ConnectorPropertyGroup.Builder() + .name("Configuration") + .description("Configuration properties for the OnPropertyModified test") + .addProperty(NUMBER_VALUE) + .build(); + + private static final ConfigurationStep CONFIG_STEP = new ConfigurationStep.Builder() + .name("Configuration") + .description("Configure the number value for testing onPropertyModified") + .propertyGroups(List.of(PROPERTY_GROUP)) + .build(); + + @Override + public VersionedExternalFlow getInitialFlow() { + return VersionedFlowUtils.loadFlowFromResource("flows/on-property-modified-tracker.json"); + } + + @Override + public List getConfigurationSteps() { + return List.of(CONFIG_STEP); + } + + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) throws FlowUpdateException { + final VersionedExternalFlow versionedExternalFlow = getInitialFlow(); + final String number = workingContext.getConfigurationContext().getProperty(CONFIG_STEP, NUMBER_VALUE).getValue(); + VersionedFlowUtils.setParameterValue(versionedExternalFlow, PARAMETER_NAME, number); + getInitializationContext().updateFlow(workingContext, versionedExternalFlow); + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + final VersionedExternalFlow versionedExternalFlow = getInitialFlow(); + final String number = workingContext.getConfigurationContext().getProperty(CONFIG_STEP, NUMBER_VALUE).getValue(); + VersionedFlowUtils.setParameterValue(versionedExternalFlow, PARAMETER_NAME, number); + getInitializationContext().updateFlow(activeContext, versionedExternalFlow); + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext workingContext) { + return List.of(); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/ParameterConnector.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/ParameterConnector.java new file mode 100644 index 000000000000..bbc071aca0eb --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/ParameterConnector.java @@ -0,0 +1,131 @@ +/* + * 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.components.connector; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.Validator; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.components.ParameterValue; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.processor.util.StandardValidators; + +import java.util.List; +import java.util.Map; + +public class ParameterConnector extends AbstractConnector { + private volatile boolean initialized = false; + + static final ConnectorPropertyDescriptor TEXT_PROPERTY = new ConnectorPropertyDescriptor.Builder() + .name("Text") + .description("The text to write to FlowFiles") + .type(PropertyType.STRING) + .addValidator(Validator.VALID) + .required(true) + .defaultValue("Hello World") + .build(); + + static final ConnectorPropertyDescriptor SLEEP_DURATION = new ConnectorPropertyDescriptor.Builder() + .name("Sleep Duration") + .description("The duration to sleep when the Sleep Processor is stopped") + .type(PropertyType.STRING) + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .required(true) + .defaultValue("1 sec") + .build(); + + private static final ConnectorPropertyGroup TEXT_GROUP = new ConnectorPropertyGroup.Builder() + .name("Text Settings") + .description("Settings for the text to write to FlowFiles") + .addProperty(TEXT_PROPERTY) + .addProperty(SLEEP_DURATION) + .build(); + + private static final ConfigurationStep TEXT_STEP = new ConfigurationStep.Builder() + .name("Text Configuration") + .description("Configure the text to be written to FlowFiles") + .propertyGroups(List.of(TEXT_GROUP)) + .build(); + + @Override + protected void init() { + initialized = true; + } + + public boolean isInitialized() { + return initialized; + } + + @Override + public VersionedExternalFlow getInitialFlow() { + return VersionedFlowUtils.loadFlowFromResource("flows/generate-and-log-with-parameter.json"); + } + + @Override + public List getConfigurationSteps() { + return List.of(TEXT_STEP); + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) { + try { + updateTextParameter(workingContext, activeContext); + } catch (final FlowUpdateException e) { + getLogger().error("Failed to update parameters", e); + throw new RuntimeException("Failed to update parameters", e); + } + } + + @Override + public void onStepConfigured(final String stepName, final FlowContext workingContext) { + } + + @Override + public void prepareForUpdate(final FlowContext workingContext, final FlowContext activeContext) { + } + + @Override + public void abortUpdate(final FlowContext workingContext, final Throwable throwable) { + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext workingContext) { + return List.of(); + } + + private void updateTextParameter(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + final ConnectorConfigurationContext configContext = workingContext.getConfigurationContext(); + final String textValue = configContext.getProperty(TEXT_STEP, TEXT_PROPERTY).getValue(); + + // Update the "Text" parameter with the configured property value + final ParameterValue textParameter = new ParameterValue.Builder() + .name("Text") + .value(textValue) + .sensitive(false) + .build(); + + final ParameterValue sleepDurationParameter = new ParameterValue.Builder() + .name("Sleep Duration") + .value(configContext.getProperty(TEXT_STEP, SLEEP_DURATION).getValue()) + .sensitive(false) + .build(); + + final List parameterValues = List.of(textParameter, sleepDurationParameter); + activeContext.getParameterContext().updateParameters(parameterValues); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/SleepingConnector.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/SleepingConnector.java new file mode 100644 index 000000000000..6818b5b059cc --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/SleepingConnector.java @@ -0,0 +1,126 @@ +/* + * 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.components.connector; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.flow.VersionedExternalFlow; + +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +public class SleepingConnector implements Connector { + private final Duration sleepDuration; + + public SleepingConnector(final Duration sleepDuration) { + this.sleepDuration = sleepDuration; + } + + @Override + public void initialize(final ConnectorInitializationContext connectorInitializationContext) { + } + + @Override + public VersionedExternalFlow getInitialFlow() { + return null; + } + + @Override + public void start(final FlowContext activeContext) throws FlowUpdateException { + try { + Thread.sleep(sleepDuration); + } catch (final InterruptedException e) { + throw new FlowUpdateException(e); + } + } + + @Override + public void stop(final FlowContext activeContext) throws FlowUpdateException { + try { + Thread.sleep(sleepDuration); + } catch (final InterruptedException e) { + throw new FlowUpdateException(e); + } + } + + @Override + public List validate(final FlowContext flowContext, final ConnectorValidationContext connectorValidationContext) { + return List.of(); + } + + @Override + public List validateConfigurationStep(final ConfigurationStep configurationStep, final ConnectorConfigurationContext connectorConfigurationContext, + final ConnectorValidationContext connectorValidationContext) { + return List.of(); + } + + @Override + public List getConfigurationSteps() { + return List.of(); + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + try { + Thread.sleep(sleepDuration); + } catch (final InterruptedException e) { + throw new FlowUpdateException(e); + } + } + + @Override + public void onConfigurationStepConfigured(final String stepName, final FlowContext workingContext) { + } + + @Override + public void prepareForUpdate(final FlowContext workingContext, final FlowContext activeContext) { + } + + @Override + public void abortUpdate(final FlowContext workingContext, final Throwable throwable) { + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext activeContext) { + return List.of(); + } + + @Override + public List verify(final FlowContext flowContext) { + return List.of(); + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName, final FlowContext workingContext, final String filter) { + return List.of(); + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName, final FlowContext workingContext) { + return List.of(); + } + + @Override + public CompletableFuture drainFlowFiles(final FlowContext flowContext) { + return CompletableFuture.completedFuture(null); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/StandardConnectorNodeIT.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/StandardConnectorNodeIT.java new file mode 100644 index 000000000000..6b321eca8e86 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/StandardConnectorNodeIT.java @@ -0,0 +1,675 @@ +/* + * 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.components.connector; + +import org.apache.nifi.bundle.Bundle; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.connector.processors.CreateDummyFlowFile; +import org.apache.nifi.components.connector.processors.DuplicateFlowFile; +import org.apache.nifi.components.connector.processors.LogFlowFileContents; +import org.apache.nifi.components.connector.processors.OnPropertyModifiedTracker; +import org.apache.nifi.components.connector.processors.OverwriteFlowFile; +import org.apache.nifi.components.connector.processors.TerminateFlowFile; +import org.apache.nifi.components.connector.secrets.ParameterProviderSecretsManager; +import org.apache.nifi.components.connector.secrets.SecretsManager; +import org.apache.nifi.components.connector.services.CounterService; +import org.apache.nifi.components.state.StateManagerProvider; +import org.apache.nifi.components.validation.StandardVerifiableComponentFactory; +import org.apache.nifi.components.validation.ValidationState; +import org.apache.nifi.components.validation.ValidationStatus; +import org.apache.nifi.components.validation.ValidationTrigger; +import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.connectable.Connection; +import org.apache.nifi.connectable.StandardConnection; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.GarbageCollectionLog; +import org.apache.nifi.controller.MockStateManagerProvider; +import org.apache.nifi.controller.NodeTypeProvider; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.ReloadComponent; +import org.apache.nifi.controller.flow.StandardFlowManager; +import org.apache.nifi.controller.flowanalysis.FlowAnalyzer; +import org.apache.nifi.controller.queue.DropFlowFileRequest; +import org.apache.nifi.controller.queue.DropFlowFileState; +import org.apache.nifi.controller.queue.FlowFileQueue; +import org.apache.nifi.controller.queue.FlowFileQueueFactory; +import org.apache.nifi.controller.queue.LoadBalanceCompression; +import org.apache.nifi.controller.queue.LoadBalanceStrategy; +import org.apache.nifi.controller.queue.QueueSize; +import org.apache.nifi.controller.repository.ContentRepository; +import org.apache.nifi.controller.repository.FlowFileEventRepository; +import org.apache.nifi.controller.repository.FlowFileRecord; +import org.apache.nifi.controller.repository.FlowFileRepository; +import org.apache.nifi.controller.scheduling.LifecycleStateManager; +import org.apache.nifi.controller.scheduling.RepositoryContextFactory; +import org.apache.nifi.controller.scheduling.SchedulingAgent; +import org.apache.nifi.controller.scheduling.StandardLifecycleStateManager; +import org.apache.nifi.controller.scheduling.StandardProcessScheduler; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.engine.FlowEngine; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.mock.MockNodeTypeProvider; +import org.apache.nifi.nar.ExtensionDiscoveringManager; +import org.apache.nifi.nar.StandardExtensionDiscoveringManager; +import org.apache.nifi.nar.SystemBundle; +import org.apache.nifi.parameter.Parameter; +import org.apache.nifi.parameter.ParameterContext; +import org.apache.nifi.parameter.ParameterContextManager; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.provenance.ProvenanceRepository; +import org.apache.nifi.python.PythonBridge; +import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.scheduling.SchedulingStrategy; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.validation.RuleViolationsManager; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Predicate; + +import static java.util.Objects.requireNonNull; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyCollection; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.nullable; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class StandardConnectorNodeIT { + + private StandardProcessScheduler processScheduler; + private StandardFlowManager flowManager; + private FlowEngine componentLifecycleThreadPool; + private ConnectorRepository connectorRepository; + + @BeforeEach + public void setup() { + final ControllerServiceProvider controllerServiceProvider = mock(ControllerServiceProvider.class); + when(controllerServiceProvider.disableControllerServicesAsync(anyCollection())).thenReturn(CompletableFuture.completedFuture(null)); + connectorRepository = new StandardConnectorRepository(); + + final SecretsManager secretsManager = new ParameterProviderSecretsManager(); + final ConnectorRepositoryInitializationContext repoInitContext = mock(ConnectorRepositoryInitializationContext.class); + when(repoInitContext.getSecretsManager()).thenReturn(secretsManager); + connectorRepository.initialize(repoInitContext); + + final ExtensionDiscoveringManager extensionManager = new StandardExtensionDiscoveringManager(); + final BulletinRepository bulletinRepository = mock(BulletinRepository.class); + final StateManagerProvider stateManagerProvider = new MockStateManagerProvider(); + final LifecycleStateManager lifecycleStateManager = new StandardLifecycleStateManager(); + final ReloadComponent reloadComponent = mock(ReloadComponent.class); + + final NiFiProperties nifiProperties = NiFiProperties.createBasicNiFiProperties("src/test/resources/conf/nifi.properties"); + + final FlowController flowController = mock(FlowController.class); + when(flowController.isInitialized()).thenReturn(true); + when(flowController.getExtensionManager()).thenReturn(extensionManager); + when(flowController.getStateManagerProvider()).thenReturn(stateManagerProvider); + when(flowController.getReloadComponent()).thenReturn(reloadComponent); + when(flowController.getVerifiableComponentFactory()).thenReturn(new StandardVerifiableComponentFactory(flowController, nifiProperties)); + + final RepositoryContextFactory repoContextFactory = mock(RepositoryContextFactory.class); + final FlowFileRepository flowFileRepo = mock(FlowFileRepository.class); + final ProvenanceRepository provRepo = mock(ProvenanceRepository.class); + final ContentRepository contentRepo = mock(ContentRepository.class); + when(repoContextFactory.getFlowFileRepository()).thenReturn(flowFileRepo); + when(repoContextFactory.getProvenanceRepository()).thenReturn(provRepo); + when(repoContextFactory.getContentRepository()).thenReturn(contentRepo); + + when(flowController.getRepositoryContextFactory()).thenReturn(repoContextFactory); + when(flowController.getGarbageCollectionLog()).thenReturn(mock(GarbageCollectionLog.class)); + when(flowController.getControllerServiceProvider()).thenReturn(controllerServiceProvider); + when(flowController.getProvenanceRepository()).thenReturn(provRepo); + when(flowController.getBulletinRepository()).thenReturn(bulletinRepository); + when(flowController.getLifecycleStateManager()).thenReturn(lifecycleStateManager); + when(flowController.getFlowFileEventRepository()).thenReturn(mock(FlowFileEventRepository.class)); + when(flowController.getConnectorRepository()).thenReturn(connectorRepository); + when(flowController.getValidationTrigger()).thenReturn(mock(ValidationTrigger.class)); + when(flowController.getConnectorValidationTrigger()).thenReturn(mock(ConnectorValidationTrigger.class)); + + doAnswer(invocation -> { + return createConnection(invocation.getArgument(0), invocation.getArgument(1), invocation.getArgument(2), invocation.getArgument(3), invocation.getArgument(4)); + }).when(flowController).createConnection(anyString(), nullable(String.class), any(Connectable.class), any(Connectable.class), anyCollection()); + + final FlowFileEventRepository flowFileEventRepository = mock(FlowFileEventRepository.class); + final ParameterContextManager parameterContextManager = mock(ParameterContextManager.class); + + final NodeTypeProvider nodeTypeProvider = new MockNodeTypeProvider(); + componentLifecycleThreadPool = new FlowEngine(4, "Component Lifecycle Thread Pool", true); + processScheduler = new StandardProcessScheduler(componentLifecycleThreadPool, extensionManager, nodeTypeProvider, () -> controllerServiceProvider, + reloadComponent, stateManagerProvider, nifiProperties, lifecycleStateManager); + when(flowController.getProcessScheduler()).thenReturn(processScheduler); + processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, mock(SchedulingAgent.class)); + processScheduler.setSchedulingAgent(SchedulingStrategy.CRON_DRIVEN, mock(SchedulingAgent.class)); + + final Bundle systemBundle = SystemBundle.create(nifiProperties); + extensionManager.discoverExtensions(systemBundle, Set.of()); + + flowManager = new StandardFlowManager(nifiProperties, null, flowController, flowFileEventRepository, parameterContextManager); + flowManager.initialize(controllerServiceProvider, mock(PythonBridge.class), mock(FlowAnalyzer.class), mock(RuleViolationsManager.class)); + final ProcessGroup rootGroup = flowManager.createProcessGroup("root"); + rootGroup.setName("Root"); + flowManager.setRootGroup(rootGroup); + + when(flowController.getFlowManager()).thenReturn(flowManager); + } + + @AfterEach + public void tearDown() { + if (componentLifecycleThreadPool != null) { + componentLifecycleThreadPool.shutdown(); + } + } + + @Test + public void testConnectorDynamicallyCreatingFlow() throws FlowUpdateException { + final ConnectorNode connectorNode = initializeDynamicFlowConnector(); + final ProcessGroup rootGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + + // Configure the connector to log FlowFile contents, make 5 copies, and set text to "Second Iteration." + final ConnectorConfiguration configuration = createConnectorConfiguration("Second Iteration.", 5, true, false); + configure(connectorNode, configuration); + + final List processorsAfterUpdate = rootGroup.findAllProcessors(); + assertEquals(5, processorsAfterUpdate.size()); + + final List processorTypesAfterUpdate = processorsAfterUpdate.stream().map(ProcessorNode::getCanonicalClassName).toList(); + assertTrue(processorTypesAfterUpdate.contains(CreateDummyFlowFile.class.getName())); + assertTrue(processorTypesAfterUpdate.contains(OverwriteFlowFile.class.getName())); + assertTrue(processorTypesAfterUpdate.contains(TerminateFlowFile.class.getName())); + assertTrue(processorTypesAfterUpdate.contains(DuplicateFlowFile.class.getName())); + assertTrue(processorTypesAfterUpdate.contains(LogFlowFileContents.class.getName())); + + // Verify the OverwriteFlowFile processor has the correct content configured + final ProcessorNode overwriteProcessor = processorsAfterUpdate.stream() + .filter(p -> p.getCanonicalClassName().equals(OverwriteFlowFile.class.getName())) + .findFirst() + .orElseThrow(); + assertEquals("Second Iteration.", overwriteProcessor.getEffectivePropertyValue(OverwriteFlowFile.CONTENT)); + + // Verify the DuplicateFlowFile processor is configured for 5 copies + final ProcessorNode duplicateProcessor = processorsAfterUpdate.stream() + .filter(p -> p.getCanonicalClassName().equals(DuplicateFlowFile.class.getName())) + .findFirst() + .orElseThrow(); + assertEquals("5", duplicateProcessor.getEffectivePropertyValue(DuplicateFlowFile.NUM_DUPLICATES)); + + // Verify that all of the Connections were created + assertEquals(5, duplicateProcessor.getConnections().size()); + final String outputPortId = duplicateProcessor.getProcessGroup().getOutputPorts().iterator().next().getIdentifier(); + for (final Connection connection : duplicateProcessor.getConnections()) { + assertEquals(duplicateProcessor.getIdentifier(), connection.getSource().getIdentifier()); + assertEquals(outputPortId, connection.getDestination().getIdentifier()); + } + } + + private void configure(final ConnectorNode connectorNode, final ConnectorConfiguration configuration) throws FlowUpdateException { + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + for (final NamedStepConfiguration stepConfig : configuration.getNamedStepConfigurations()) { + connectorNode.setConfiguration(stepConfig.stepName(), stepConfig.configuration()); + } + connectorNode.applyUpdate(); + } + + private ConnectorNode initializeDynamicFlowConnector() { + final ConnectorNode connectorNode = flowManager.createConnector(DynamicFlowConnector.class.getName(), "dynamic-flow-connector", SystemBundle.SYSTEM_BUNDLE_COORDINATE, true, true); + assertNotNull(connectorNode); + + final Connector connector = connectorNode.getConnector(); + assertNotNull(connector); + assertInstanceOf(DynamicFlowConnector.class, connector); + + final DynamicFlowConnector flowConnector = (DynamicFlowConnector) connector; + assertTrue(flowConnector.isInitialized()); + + assertEquals(List.of(connectorNode), connectorRepository.getConnectors()); + + final ProcessGroup rootGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + assertEquals(3, rootGroup.getProcessGroups().size()); + + final List initialProcessors = rootGroup.findAllProcessors(); + assertEquals(4, initialProcessors.size()); + final List initialProcessorTypes = initialProcessors.stream().map(ProcessorNode::getComponentType).toList(); + assertTrue(initialProcessorTypes.contains(CreateDummyFlowFile.class.getSimpleName())); + assertTrue(initialProcessorTypes.contains(OverwriteFlowFile.class.getSimpleName())); + assertTrue(initialProcessorTypes.contains(TerminateFlowFile.class.getSimpleName())); + assertTrue(initialProcessorTypes.contains(DuplicateFlowFile.class.getSimpleName())); + + return connectorNode; + } + + private ConnectorNode initializeParameterConnector() { + final ConnectorNode connectorNode = flowManager.createConnector(ParameterConnector.class.getName(), "parameter-connector", SystemBundle.SYSTEM_BUNDLE_COORDINATE, true, true); + assertNotNull(connectorNode); + assertEquals(List.of(connectorNode), connectorRepository.getConnectors()); + + final ProcessGroup rootGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + assertEquals(3, rootGroup.getProcessors().size()); + + return connectorNode; + } + + // Test a scenario where the connector is initialized and the Connector then changes the value of a parameter. + // This should result in any Processor that references that parameter being stopped and restarted. It should not result + // in non-referencing Processors being restarted. + @Test + public void testParameterUpdateRestartsReferencingProcessors() throws FlowUpdateException { + final ConnectorNode connectorNode = initializeParameterConnector(); + final Connector connector = connectorNode.getConnector(); + assertNotNull(connector); + assertInstanceOf(ParameterConnector.class, connector); + + final ParameterConnector flowConnector = (ParameterConnector) connector; + assertTrue(flowConnector.isInitialized()); + + final ProcessGroup rootGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + final ParameterContext parameterContext = rootGroup.getParameterContext(); + assertNotNull(parameterContext); + assertEquals(2, parameterContext.getParameters().size()); + final Optional optionalParameter = parameterContext.getParameter("Text"); + assertTrue(optionalParameter.isPresent()); + + final Parameter textParameter = optionalParameter.get(); + assertEquals("Hello", textParameter.getValue()); + + // Set the value of the 'Text' property to Hi. This should result in the parameter context being updated. + final StepConfiguration sourceStepConfig = new StepConfiguration(Map.of("Text", new StringLiteralValue("Hi."))); + final NamedStepConfiguration sourceConfigurationStepConfiguration = new NamedStepConfiguration("Text Configuration", sourceStepConfig); + final ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(Set.of(sourceConfigurationStepConfiguration)); + configure(connectorNode, connectorConfiguration); + + assertEquals("Hi.", parameterContext.getParameter("Text").orElseThrow().getValue()); + } + + @Test + @Timeout(10) + public void testOnPropertyModifiedCalledOnApplyUpdate() throws FlowUpdateException { + final ConnectorNode connectorNode = flowManager.createConnector(OnPropertyModifiedConnector.class.getName(), + "on-property-modified-connector", SystemBundle.SYSTEM_BUNDLE_COORDINATE, true, true); + assertNotNull(connectorNode); + + final StepConfiguration initialConfig = new StepConfiguration(Map.of("Number Value", new StringLiteralValue("0"))); + final NamedStepConfiguration initialStepConfig = new NamedStepConfiguration("Configuration", initialConfig); + configure(connectorNode, new ConnectorConfiguration(Set.of(initialStepConfig))); + + final ProcessGroup activeGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + final ProcessorNode processorNode = activeGroup.getProcessors().iterator().next(); + final OnPropertyModifiedTracker tracker = (OnPropertyModifiedTracker) processorNode.getProcessor(); + + assertEquals(0, tracker.getPropertyChangeCount()); + + final StepConfiguration updatedConfig = new StepConfiguration(Map.of("Number Value", new StringLiteralValue("1"))); + final NamedStepConfiguration updatedStepConfig = new NamedStepConfiguration("Configuration", updatedConfig); + + connectorNode.setConfiguration("Configuration", updatedStepConfig.configuration()); + + final ProcessGroup workingGroup = connectorNode.getWorkingFlowContext().getManagedProcessGroup(); + final ProcessorNode workingProcessorNode = workingGroup.getProcessors().iterator().next(); + final OnPropertyModifiedTracker workingTracker = (OnPropertyModifiedTracker) workingProcessorNode.getProcessor(); + + assertEquals(1, workingTracker.getPropertyChangeCount()); + assertEquals("0", workingTracker.getPropertyChanges().getFirst().oldValue()); + assertEquals("1", workingTracker.getPropertyChanges().getFirst().newValue()); + + workingTracker.clearPropertyChanges(); + + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + connectorNode.applyUpdate(); + + assertEquals(1, tracker.getPropertyChangeCount()); + assertEquals("0", tracker.getPropertyChanges().getFirst().oldValue()); + assertEquals("1", tracker.getPropertyChanges().getFirst().newValue()); + + // Ensure that no parameter contexts are registered + final Set registeredContexts = flowManager.getParameterContextManager().getParameterContexts(); + assertEquals(Set.of(), registeredContexts); + } + + @Test + public void testControllerServices() throws FlowUpdateException { + final ConnectorNode connectorNode = initializeDynamicFlowConnector(); + + final ConnectorConfiguration configuration = createConnectorConfiguration("Second Iteration", 5, true, true); + configure(connectorNode, configuration); + + final ProcessGroup managedGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + final Set serviceNodes = managedGroup.getControllerServices(true); + assertNotNull(serviceNodes); + assertEquals(1, serviceNodes.size()); + assertInstanceOf(CounterService.class, serviceNodes.iterator().next().getControllerServiceImplementation()); + } + + @Test + public void testUpdateProcessorPropertyDataQueued() throws FlowUpdateException { + final ConnectorNode connectorNode = initializeDynamicFlowConnector(); + final ProcessGroup rootGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + final Connection connection = queueDataBySource(rootGroup, "Create FlowFile"); + + // Update Connector Property to ensure that the change is allowed with data queued + final ConnectorConfiguration configuration = createConnectorConfiguration("Second Iteration", 5, true, false); + configure(connectorNode, configuration); + + assertEquals(1, connection.getFlowFileQueue().size().getObjectCount()); + } + + @Test + public void testRemoveConnectionDataQueued() throws FlowUpdateException { + final ConnectorNode connectorNode = initializeDynamicFlowConnector(); + final ProcessGroup rootGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + + // Create a configuration that will result in the LogFlowFileContents processor being removed. + // Because the component is being removed, it should drain the queues before doing so. + final ConnectorConfiguration addLogConfiguration = createConnectorConfiguration("Second Iteration", 5, true, false); + configure(connectorNode, addLogConfiguration); + + // Queue data between LogFlowFileContents and TerminateFlowFile + final Connection connection = queueDataByDestination(rootGroup, "Terminate FlowFile"); + + // Create a configuration that will result in the LogFlowFileContents processor being removed. + // Because the component is being removed and there's data queued in its incoming connection, it should fail. + final ConnectorConfiguration removeLogConfiguration = createConnectorConfiguration("Second Iteration", 5, false, false); + + final Throwable cause = assertThrows(FlowUpdateException.class, () -> configure(connectorNode, removeLogConfiguration)); + connectorNode.abortUpdate(cause); + + rootGroup.findAllConnections().contains(connection); + assertFalse(connection.getFlowFileQueue().isEmpty()); + } + + + @Test + public void testDynamicProperties() throws IOException, FlowUpdateException { + final ConnectorNode connectorNode = flowManager.createConnector(DynamicAllowableValuesConnector.class.getName(), "dynamic-allowable-values-connector", + SystemBundle.SYSTEM_BUNDLE_COORDINATE, true, true); + assertNotNull(connectorNode); + + assertEquals(List.of("File", "Colors"), getConfigurationStepNames(connectorNode)); + + final Path tempFile = Files.createTempFile("StandardConnectorNodeIT", ".txt"); + Files.writeString(tempFile, String.join("\n", "red", "blue", "yellow")); + + final ConnectorConfiguration configuration = createFileConfiguration(tempFile.toFile().getAbsolutePath()); + configure(connectorNode, configuration); + + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + assertEquals(List.of("File", "Colors"), getConfigurationStepNames(connectorNode)); + + final List allowableValues = connectorNode.fetchAllowableValues("Colors", "First Primary Color").stream() + .map(DescribedValue::getValue) + .toList(); + assertEquals(List.of("red", "blue", "yellow"), allowableValues); + } + + @Test + public void testSimpleValidation() throws FlowUpdateException { + final ConnectorNode connectorNode = flowManager.createConnector(DynamicAllowableValuesConnector.class.getName(), "dynamic-allowable-values-connector", + SystemBundle.SYSTEM_BUNDLE_COORDINATE, true, true); + assertNotNull(connectorNode); + + assertEquals(List.of("File", "Colors"), getConfigurationStepNames(connectorNode)); + + final ConnectorConfiguration configuration = createFileConfiguration("/non/existent/file"); + configure(connectorNode, configuration); + + final ValidationState validationState = connectorNode.performValidation(); + assertNotNull(validationState); + assertEquals(ValidationStatus.INVALID, validationState.getStatus()); + assertEquals(2, validationState.getValidationErrors().size()); + + final boolean hasFileError = validationState.getValidationErrors().stream() + .anyMatch(result -> result.getInput() != null && result.getInput().contains("/non/existent/file")); + assertTrue(hasFileError); + + final boolean hasColorError = validationState.getValidationErrors().stream() + .anyMatch(result -> result.getSubject() != null && result.getSubject().contains("First Primary Color")); + assertTrue(hasColorError); + + final File colorsFile = new File("src/test/resources/colors.txt"); + final ConnectorConfiguration validConfig = createFileAndColorsConfiguration(colorsFile.getAbsolutePath(), "red"); + configure(connectorNode, validConfig); + + final ValidationState updatedValidationState = connectorNode.performValidation(); + assertEquals(ValidationStatus.VALID, updatedValidationState.getStatus(), + "Expected valid state but invalid due to " + updatedValidationState.getValidationErrors()); + assertEquals(List.of(), updatedValidationState.getValidationErrors()); + } + + @Test + public void testValidationWithParameterContext() throws FlowUpdateException { + final ConnectorNode connectorNode = initializeParameterConnector(); + + final ValidationState initialValidationState = connectorNode.performValidation(); + assertNotNull(initialValidationState); + assertEquals(ValidationStatus.VALID, initialValidationState.getStatus()); + assertEquals(List.of(), initialValidationState.getValidationErrors()); + + final StepConfiguration sourceStepConfig = new StepConfiguration(Map.of("Sleep Duration", new StringLiteralValue("Hi."))); + final NamedStepConfiguration sourceConfigurationStepConfiguration = new NamedStepConfiguration("Text Configuration", sourceStepConfig); + final ConnectorConfiguration connectorConfiguration = new ConnectorConfiguration(Set.of(sourceConfigurationStepConfiguration)); + configure(connectorNode, connectorConfiguration); + + final ValidationState validationState = connectorNode.performValidation(); + assertNotNull(validationState); + assertEquals(ValidationStatus.INVALID, validationState.getStatus()); + assertEquals(1, validationState.getValidationErrors().size()); + } + + @Test + public void testPurgeFlowFilesEmptiesQueues() throws ExecutionException, InterruptedException, TimeoutException { + final ConnectorNode connectorNode = initializeDynamicFlowConnector(); + final ProcessGroup rootGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + + final Connection connection = queueDataBySource(rootGroup, "Create FlowFile"); + assertEquals(1, connection.getFlowFileQueue().size().getObjectCount()); + + final Future purgeFuture = connectorNode.purgeFlowFiles("test-user"); + purgeFuture.get(10, TimeUnit.SECONDS); + + assertTrue(connection.getFlowFileQueue().isEmpty()); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + } + + @Test + public void testPurgeFlowFilesMultipleQueues() throws ExecutionException, InterruptedException, TimeoutException { + final ConnectorNode connectorNode = initializeDynamicFlowConnector(); + final ProcessGroup rootGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + + final Connection connection1 = queueDataBySource(rootGroup, "Create FlowFile"); + final Connection connection2 = queueDataByDestination(rootGroup, "Terminate FlowFile"); + assertEquals(1, connection1.getFlowFileQueue().size().getObjectCount()); + assertEquals(1, connection2.getFlowFileQueue().size().getObjectCount()); + + final Future purgeFuture = connectorNode.purgeFlowFiles("test-user"); + purgeFuture.get(10, TimeUnit.SECONDS); + + assertTrue(connection1.getFlowFileQueue().isEmpty()); + assertTrue(connection2.getFlowFileQueue().isEmpty()); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + } + + @Test + public void testPurgeFlowFilesRequiresStoppedState() { + final ConnectorNode connectorNode = initializeDynamicFlowConnector(); + final ProcessGroup rootGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + queueDataBySource(rootGroup, "Create FlowFile"); + + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + + connectorNode.start(componentLifecycleThreadPool); + assertEquals(ConnectorState.RUNNING, connectorNode.getDesiredState()); + + final IllegalStateException exception = assertThrows(IllegalStateException.class, () -> connectorNode.purgeFlowFiles("test-user")); + assertTrue(exception.getMessage().contains("must be STOPPED")); + + connectorNode.stop(componentLifecycleThreadPool); + } + + @Test + public void testInitialFlowWithMissingBundleResultsInInvalidConnector() { + final ConnectorNode connectorNode = flowManager.createConnector(MissingBundleConnector.class.getName(), "missing-bundle-connector", SystemBundle.SYSTEM_BUNDLE_COORDINATE, true, true); + assertNotNull(connectorNode); + + assertFalse(connectorNode.isExtensionMissing()); + + final ValidationState validationState = connectorNode.performValidation(); + assertNotNull(validationState); + assertEquals(ValidationStatus.INVALID, validationState.getStatus()); + assertEquals(1, validationState.getValidationErrors().size()); + + final String explanation = validationState.getValidationErrors().iterator().next().getExplanation(); + assertTrue(explanation.contains("com.example.nonexistent:missing-nar:1.0.0"), "Expected explanation to mention the missing bundle coordinates but was: " + explanation); + assertTrue(explanation.contains("com.example.nonexistent.MissingProcessor"), "Expected explanation to mention the missing processor type but was: " + explanation); + } + + private List getConfigurationStepNames(final ConnectorNode connectorNode) { + return connectorNode.getConfigurationSteps().stream() + .map(ConfigurationStep::getName) + .toList(); + } + + private Connection queueDataBySource(final ProcessGroup group, final String sourceComponentName) { + return queueData(group, conn -> conn.getSource().getName().equals(sourceComponentName)); + } + + private Connection queueDataByDestination(final ProcessGroup group, final String destinationComponentName) { + return queueData(group, conn -> conn.getDestination().getName().equals(destinationComponentName)); + } + + private Connection queueData(final ProcessGroup group, final Predicate connectionTest) { + final Connection connection = group.findAllConnections().stream() + .filter(connectionTest) + .findFirst() + .orElseThrow(); + + final FlowFileRecord flowFile = mock(FlowFileRecord.class); + connection.getFlowFileQueue().put(flowFile); + return connection; + } + + private Connection createConnection(final String id, final String name, final Connectable source, final Connectable destination, final Collection relationshipNames) { + final List relationships = relationshipNames.stream() + .map(relName -> new Relationship.Builder().name(relName).build()) + .toList(); + + final FlowFileQueue flowFileQueue = mock(FlowFileQueue.class); + final List flowFileList = new ArrayList<>(); + + // Update mock to add FlowFiles to the queue + doAnswer(invocation -> { + flowFileList.add(invocation.getArgument(0)); + return null; + }).when(flowFileQueue).put(any(FlowFileRecord.class)); + + // Update mock to return queue size and isEmpty status + when(flowFileQueue.size()).thenAnswer(invocation -> new QueueSize(flowFileList.size(), flowFileList.size())); + when(flowFileQueue.isEmpty()).thenAnswer(invocation -> flowFileList.isEmpty()); + when(flowFileQueue.getLoadBalanceStrategy()).thenReturn(LoadBalanceStrategy.DO_NOT_LOAD_BALANCE); + when(flowFileQueue.getLoadBalanceCompression()).thenReturn(LoadBalanceCompression.DO_NOT_COMPRESS); + + // Mock dropFlowFiles to clear the list and return a completed status + when(flowFileQueue.dropFlowFiles(anyString(), anyString())).thenAnswer(invocation -> { + final String requestId = invocation.getArgument(0); + final int originalCount = flowFileList.size(); + flowFileList.clear(); + + final DropFlowFileRequest dropRequest = new DropFlowFileRequest(requestId); + dropRequest.setOriginalSize(new QueueSize(originalCount, originalCount)); + dropRequest.setCurrentSize(new QueueSize(0, 0)); + dropRequest.setDroppedSize(new QueueSize(originalCount, originalCount)); + dropRequest.setState(DropFlowFileState.COMPLETE); + return dropRequest; + }); + + final FlowFileQueueFactory flowFileQueueFactory = (loadBalanceStrategy, partitioningAttribute, processGroup) -> flowFileQueue; + + final Connection connection = new StandardConnection.Builder(processScheduler) + .id(id) + .name(name) + .processGroup(destination.getProcessGroup()) + .relationships(relationships) + .source(requireNonNull(source)) + .destination(destination) + .flowFileQueueFactory(flowFileQueueFactory) + .build(); + + return connection; + } + + private ConnectorConfiguration createConnectorConfiguration(final String sourceText, final int numberOfCopies, final boolean logContents, final boolean countFlowFiles) { + // Source configuration step + final StepConfiguration sourceStepConfig = new StepConfiguration(Map.of( + "Source Text", new StringLiteralValue(sourceText), + "Count FlowFiles", new StringLiteralValue(Boolean.toString(countFlowFiles)))); + final NamedStepConfiguration sourceConfigurationStepConfiguration = new NamedStepConfiguration("Source", sourceStepConfig); + + // Duplication configuration step + final StepConfiguration duplicationStepConfig = new StepConfiguration(Map.of("Number of Copies", new StringLiteralValue(Integer.toString(numberOfCopies)))); + final NamedStepConfiguration duplicationConfigurationStepConfiguration = new NamedStepConfiguration("Duplication", duplicationStepConfig); + + // Destination configuration step + final StepConfiguration destinationStepConfig = new StepConfiguration(Map.of("Log FlowFile Contents", new StringLiteralValue(Boolean.toString(logContents)))); + final NamedStepConfiguration destinationConfigurationStepConfiguration = new NamedStepConfiguration("Destination", destinationStepConfig); + + return new ConnectorConfiguration(Set.of(sourceConfigurationStepConfiguration, duplicationConfigurationStepConfiguration, destinationConfigurationStepConfiguration)); + } + + private ConnectorConfiguration createFileConfiguration(final String filename) { + final StepConfiguration fileStepConfig = new StepConfiguration(Map.of("File Path", new StringLiteralValue(filename))); + final NamedStepConfiguration fileConfigurationStepConfiguration = new NamedStepConfiguration("File", fileStepConfig); + + return new ConnectorConfiguration(Set.of(fileConfigurationStepConfiguration)); + } + + private ConnectorConfiguration createFileAndColorsConfiguration(final String filename, final String color) { + final StepConfiguration fileStepConfig = new StepConfiguration(Map.of("File Path", new StringLiteralValue(filename))); + final NamedStepConfiguration fileConfigurationStepConfiguration = new NamedStepConfiguration("File", fileStepConfig); + + final StepConfiguration colorsStepConfig = new StepConfiguration(Map.of("First Primary Color", new StringLiteralValue(color))); + final NamedStepConfiguration colorsConfigurationStepConfiguration = new NamedStepConfiguration("Colors", colorsStepConfig); + + return new ConnectorConfiguration(Set.of(fileConfigurationStepConfiguration, colorsConfigurationStepConfiguration)); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardComponentBundleLookup.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardComponentBundleLookup.java new file mode 100644 index 000000000000..c0916d433593 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardComponentBundleLookup.java @@ -0,0 +1,246 @@ +/* + * 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.components.connector; + +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.bundle.BundleDetails; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.nar.ExtensionManager; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.util.List; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestStandardComponentBundleLookup { + private static final String COMPONENT_TYPE = "org.apache.nifi.processors.TestProcessor"; + + private ExtensionManager extensionManager; + private StandardComponentBundleLookup lookup; + + @BeforeEach + void setup() { + extensionManager = mock(ExtensionManager.class); + lookup = new StandardComponentBundleLookup(extensionManager); + } + + @Nested + class GetLatestBundle { + @Test + void testMinorVersion() { + setupBundles( + createBundle("group", "artifact", "1.0.0"), + createBundle("group", "artifact", "1.2.0") + ); + + final Optional latestBundle = lookup.getLatestBundle(COMPONENT_TYPE); + assertTrue(latestBundle.isPresent()); + assertEquals("1.2.0", latestBundle.get().getVersion()); + } + + @Test + void testMajorVersion() { + setupBundles( + createBundle("group", "artifact", "1.0.0"), + createBundle("group", "artifact", "2.0.0") + ); + + final Optional latestBundle = lookup.getLatestBundle(COMPONENT_TYPE); + assertTrue(latestBundle.isPresent()); + assertEquals("2.0.0", latestBundle.get().getVersion()); + } + + @Test + void testMoreDigitsLater() { + setupBundles( + createBundle("group", "artifact", "1.0"), + createBundle("group", "artifact", "1.0.1") + ); + + final Optional latestBundle = lookup.getLatestBundle(COMPONENT_TYPE); + assertTrue(latestBundle.isPresent()); + assertEquals("1.0.1", latestBundle.get().getVersion()); + } + + @Test + void testMoreDigitsFirst() { + setupBundles( + createBundle("group", "artifact", "1.0.1"), + createBundle("group", "artifact", "1.0") + ); + + final Optional latestBundle = lookup.getLatestBundle(COMPONENT_TYPE); + assertTrue(latestBundle.isPresent()); + assertEquals("1.0.1", latestBundle.get().getVersion()); + } + + @Test + void testWithSnapshotAndSameVersion() { + setupBundles( + createBundle("group", "artifact", "1.0.0-SNAPSHOT"), + createBundle("group", "artifact", "1.0.0") + ); + + final Optional latestBundle = lookup.getLatestBundle(COMPONENT_TYPE); + assertTrue(latestBundle.isPresent()); + assertEquals("1.0.0", latestBundle.get().getVersion()); + } + + @Test + void testWithSnapshotAndDifferentVersion() { + setupBundles( + createBundle("group", "artifact", "1.0.1-SNAPSHOT"), + createBundle("group", "artifact", "1.0.0") + ); + + final Optional latestBundle = lookup.getLatestBundle(COMPONENT_TYPE); + assertTrue(latestBundle.isPresent()); + assertEquals("1.0.1-SNAPSHOT", latestBundle.get().getVersion()); + } + + @Test + void testEmptyList() { + setupBundles(); + assertTrue(lookup.getLatestBundle(COMPONENT_TYPE).isEmpty()); + } + + @Test + void testNonNumericVersionPart() { + setupBundles( + createBundle("group", "artifact", "4.0.0"), + createBundle("group", "artifact", "4.0.next") + ); + + final Optional latestBundle = lookup.getLatestBundle(COMPONENT_TYPE); + assertTrue(latestBundle.isPresent()); + assertEquals("4.0.0", latestBundle.get().getVersion()); + } + + @Test + void testFullyNonNumericVersionVsNumeric() { + setupBundles( + createBundle("group", "artifact", "1.0.0"), + createBundle("group", "artifact", "undefined") + ); + + final Optional latestBundle = lookup.getLatestBundle(COMPONENT_TYPE); + assertTrue(latestBundle.isPresent()); + assertEquals("1.0.0", latestBundle.get().getVersion()); + } + + @Test + void testTwoFullyNonNumericVersions() { + setupBundles( + createBundle("group", "artifact", "undefined"), + createBundle("group", "artifact", "unknown") + ); + + final Optional latestBundle = lookup.getLatestBundle(COMPONENT_TYPE); + assertTrue(latestBundle.isPresent()); + assertEquals("unknown", latestBundle.get().getVersion()); + } + + @Test + void testQualifierOrdering() { + setupBundles( + createBundle("group", "artifact", "2.0.0-SNAPSHOT"), + createBundle("group", "artifact", "2.0.0-M1"), + createBundle("group", "artifact", "2.0.0-M4"), + createBundle("group", "artifact", "2.0.0-RC1"), + createBundle("group", "artifact", "2.0.0-RC2"), + createBundle("group", "artifact", "2.0.0") + ); + + final Optional latestBundle = lookup.getLatestBundle(COMPONENT_TYPE); + assertTrue(latestBundle.isPresent()); + assertEquals("2.0.0", latestBundle.get().getVersion()); + } + + @Test + void testQualifierOrderingWithoutRelease() { + setupBundles( + createBundle("group", "artifact", "2.0.0-SNAPSHOT"), + createBundle("group", "artifact", "2.0.0-M1"), + createBundle("group", "artifact", "2.0.0-M4"), + createBundle("group", "artifact", "2.0.0-RC1"), + createBundle("group", "artifact", "2.0.0-RC2") + ); + + final Optional latestBundle = lookup.getLatestBundle(COMPONENT_TYPE); + assertTrue(latestBundle.isPresent()); + assertEquals("2.0.0-RC2", latestBundle.get().getVersion()); + } + + @Test + void testMilestoneVersionsOrdering() { + setupBundles( + createBundle("group", "artifact", "2.0.0-M1"), + createBundle("group", "artifact", "2.0.0-M4"), + createBundle("group", "artifact", "2.0.0-SNAPSHOT") + ); + + final Optional latestBundle = lookup.getLatestBundle(COMPONENT_TYPE); + assertTrue(latestBundle.isPresent()); + assertEquals("2.0.0-M4", latestBundle.get().getVersion()); + } + + @Test + void testCalendarDateFormat() { + setupBundles( + createBundle("group", "artifact", "2025.12.31"), + createBundle("group", "artifact", "2026.01.01") + ); + + final Optional latestBundle = lookup.getLatestBundle(COMPONENT_TYPE); + assertTrue(latestBundle.isPresent()); + assertEquals("2026.01.01", latestBundle.get().getVersion()); + } + + @Test + void testCalendarDateFormatWithBuildNumber() { + setupBundles( + createBundle("group", "artifact", "2025.12.31.999"), + createBundle("group", "artifact", "2026.01.01.451") + ); + + final Optional latestBundle = lookup.getLatestBundle(COMPONENT_TYPE); + assertTrue(latestBundle.isPresent()); + assertEquals("2026.01.01.451", latestBundle.get().getVersion()); + } + + private void setupBundles(final org.apache.nifi.bundle.Bundle... bundles) { + when(extensionManager.getBundles(COMPONENT_TYPE)).thenReturn(List.of(bundles)); + } + } + + private org.apache.nifi.bundle.Bundle createBundle(final String group, final String artifact, final String version) { + final BundleCoordinate coordinate = new BundleCoordinate(group, artifact, version); + final BundleDetails details = new BundleDetails.Builder() + .coordinate(coordinate) + .workingDir(new File(".")) + .build(); + return new org.apache.nifi.bundle.Bundle(details, this.getClass().getClassLoader()); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorConfigurationContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorConfigurationContext.java new file mode 100644 index 000000000000..6a73137ac762 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorConfigurationContext.java @@ -0,0 +1,256 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.components.connector; + +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.connector.secrets.SecretProvider; +import org.apache.nifi.components.connector.secrets.SecretsManager; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestStandardConnectorConfigurationContext { + private StandardConnectorConfigurationContext context; + + @BeforeEach + public void setUp() { + final AssetManager assetManager = mock(AssetManager.class); + final SecretsManager secretsManager = mock(SecretsManager.class); + context = new StandardConnectorConfigurationContext(assetManager, secretsManager); + } + + private StepConfiguration toStepConfiguration(final Map stringProperties) { + final Map valueReferences = new HashMap<>(); + for (final Map.Entry entry : stringProperties.entrySet()) { + final String value = entry.getValue(); + valueReferences.put(entry.getKey(), value == null ? null : new StringLiteralValue(value)); + } + return new StepConfiguration(valueReferences); + } + + @Test + public void testSetPropertiesWithNoExistingConfigurations() { + final Map properties = new HashMap<>(); + properties.put("key1", "value1"); + properties.put("key2", "value2"); + + context.setProperties("step1", toStepConfiguration(properties)); + + assertEquals("value1", context.getProperty("step1", "key1").getValue()); + assertEquals("value2", context.getProperty("step1", "key2").getValue()); + } + + @Test + public void testSetPropertiesAddsNewProperties() { + final Map initialProperties = new HashMap<>(); + initialProperties.put("key1", "value1"); + context.setProperties("step1", toStepConfiguration(initialProperties)); + + final Map newProperties = new HashMap<>(); + newProperties.put("key3", "value3"); + context.setProperties("step1", toStepConfiguration(newProperties)); + + assertEquals("value1", context.getProperty("step1", "key1").getValue()); + assertEquals("value3", context.getProperty("step1", "key3").getValue()); + } + + @Test + public void testSetPropertiesMergesExistingPropertiesWithNewValues() { + final Map initialProperties = new HashMap<>(); + initialProperties.put("key1", "value1"); + initialProperties.put("key2", "value2"); + context.setProperties("step1", toStepConfiguration(initialProperties)); + + final Map updatedProperties = new HashMap<>(); + updatedProperties.put("key2", "updatedValue2"); + updatedProperties.put("key3", "value3"); + context.setProperties("step1", toStepConfiguration(updatedProperties)); + + assertEquals("value1", context.getProperty("step1", "key1").getValue()); + assertEquals("updatedValue2", context.getProperty("step1", "key2").getValue()); + assertEquals("value3", context.getProperty("step1", "key3").getValue()); + } + + @Test + public void testSetPropertiesRemovesKeyWhenValueIsNull() { + final Map initialProperties = new HashMap<>(); + initialProperties.put("key1", "value1"); + initialProperties.put("key2", "value2"); + initialProperties.put("key3", "value3"); + context.setProperties("step1", toStepConfiguration(initialProperties)); + + assertEquals("value2", context.getProperty("step1", "key2").getValue()); + + final Map updatedProperties = new HashMap<>(); + updatedProperties.put("key2", null); + context.setProperties("step1", toStepConfiguration(updatedProperties)); + + assertEquals("value1", context.getProperty("step1", "key1").getValue()); + assertNull(context.getProperty("step1", "key2").getValue()); + assertEquals("value3", context.getProperty("step1", "key3").getValue()); + } + + @Test + public void testSetPropertiesLeavesUnprovidedKeysAsIs() { + final Map initialProperties = new HashMap<>(); + initialProperties.put("key1", "value1"); + initialProperties.put("key2", "value2"); + initialProperties.put("key3", "value3"); + context.setProperties("step1", toStepConfiguration(initialProperties)); + + final Map updatedProperties = new HashMap<>(); + updatedProperties.put("key2", "updatedValue2"); + context.setProperties("step1", toStepConfiguration(updatedProperties)); + + assertEquals("value1", context.getProperty("step1", "key1").getValue()); + assertEquals("updatedValue2", context.getProperty("step1", "key2").getValue()); + assertEquals("value3", context.getProperty("step1", "key3").getValue()); + } + + @Test + public void testSetPropertiesForDifferentSteps() { + final Map step1Properties = new HashMap<>(); + step1Properties.put("key1", "value1"); + context.setProperties("step1", toStepConfiguration(step1Properties)); + + final Map step2Properties = new HashMap<>(); + step2Properties.put("key2", "value2"); + context.setProperties("step2", toStepConfiguration(step2Properties)); + + assertEquals("value1", context.getProperty("step1", "key1").getValue()); + assertNull(context.getProperty("step1", "key2").getValue()); + assertEquals("value2", context.getProperty("step2", "key2").getValue()); + assertNull(context.getProperty("step2", "key1").getValue()); + } + + @Test + public void testGetPropertyReturnsEmptyForNonExistentStep() { + final ConnectorPropertyValue propertyValue = context.getProperty("nonExistentStep", "someProperty"); + assertNull(propertyValue.getValue()); + } + + @Test + public void testGetPropertyReturnsEmptyForNonExistentProperty() { + final Map properties = new HashMap<>(); + properties.put("key1", "value1"); + context.setProperties("step1", toStepConfiguration(properties)); + + final ConnectorPropertyValue propertyValue = context.getProperty("step1", "nonExistentProperty"); + assertNull(propertyValue.getValue()); + } + + @Test + public void testComplexMergingScenario() { + final Map initialProps = new HashMap<>(); + initialProps.put("a", "1"); + initialProps.put("b", "2"); + initialProps.put("c", "3"); + initialProps.put("d", "4"); + initialProps.put("e", "5"); + initialProps.put("f", "6"); + context.setProperties("step1", toStepConfiguration(initialProps)); + + final Map updateProps = new HashMap<>(); + updateProps.put("b", null); + updateProps.put("c", "30"); + updateProps.put("g", "7"); + updateProps.put("h", "8"); + context.setProperties("step1", toStepConfiguration(updateProps)); + + assertEquals("1", context.getProperty("step1", "a").getValue()); + assertNull(context.getProperty("step1", "b").getValue()); + assertEquals("30", context.getProperty("step1", "c").getValue()); + assertEquals("4", context.getProperty("step1", "d").getValue()); + assertEquals("5", context.getProperty("step1", "e").getValue()); + assertEquals("6", context.getProperty("step1", "f").getValue()); + assertEquals("7", context.getProperty("step1", "g").getValue()); + assertEquals("8", context.getProperty("step1", "h").getValue()); + } + + @Test + public void testCloneHasCorrectResolvedValues() { + final Map step1Properties = new HashMap<>(); + step1Properties.put("key1", "value1"); + step1Properties.put("key2", "value2"); + step1Properties.put("key3", "value3"); + context.setProperties("step1", toStepConfiguration(step1Properties)); + + final Map step2Properties = new HashMap<>(); + step2Properties.put("keyA", "valueA"); + step2Properties.put("keyB", "valueB"); + context.setProperties("step2", toStepConfiguration(step2Properties)); + + final MutableConnectorConfigurationContext clonedContext = context.clone(); + + assertEquals("value1", clonedContext.getProperty("step1", "key1").getValue()); + assertEquals("value2", clonedContext.getProperty("step1", "key2").getValue()); + assertEquals("value3", clonedContext.getProperty("step1", "key3").getValue()); + assertEquals("valueA", clonedContext.getProperty("step2", "keyA").getValue()); + assertEquals("valueB", clonedContext.getProperty("step2", "keyB").getValue()); + assertNull(clonedContext.getProperty("step1", "nonExistent").getValue()); + assertNull(clonedContext.getProperty("nonExistentStep", "key1").getValue()); + } + + @Test + public void testResolvePropertyValuesResolvesSecretsThatWereInitiallyUnresolvable() { + final String providerId = "provider-1"; + final String providerName = "TestProvider"; + final String secretName = "mySecret"; + final String fullyQualifiedName = "TestProvider.mySecret"; + final String secretValue = "super-secret-value"; + + final SecretProvider secretProvider = mock(SecretProvider.class); + when(secretProvider.getProviderId()).thenReturn(providerId); + when(secretProvider.getProviderName()).thenReturn(providerName); + + final Secret secret = mock(Secret.class); + when(secret.getValue()).thenReturn(secretValue); + when(secretProvider.getSecrets(List.of(fullyQualifiedName))).thenReturn(List.of(secret)); + + final SecretsManager secretsManager = mock(SecretsManager.class); + when(secretsManager.getSecretProviders()).thenReturn(Collections.emptySet()); + + final AssetManager assetManager = mock(AssetManager.class); + final StandardConnectorConfigurationContext testContext = new StandardConnectorConfigurationContext(assetManager, secretsManager); + + final SecretReference secretRef = new SecretReference(providerId, providerName, secretName, fullyQualifiedName); + final Map properties = new HashMap<>(); + properties.put("plainProp", new StringLiteralValue("plainValue")); + properties.put("secretProp", secretRef); + testContext.setProperties("authStep", new StepConfiguration(properties)); + + assertEquals("plainValue", testContext.getProperty("authStep", "plainProp").getValue()); + assertNull(testContext.getProperty("authStep", "secretProp").getValue()); + + when(secretsManager.getSecretProviders()).thenReturn(Set.of(secretProvider)); + testContext.resolvePropertyValues(); + + assertEquals("plainValue", testContext.getProperty("authStep", "plainProp").getValue()); + assertEquals(secretValue, testContext.getProperty("authStep", "secretProp").getValue()); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorInitializationContext.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorInitializationContext.java new file mode 100644 index 000000000000..565ddb05f9c7 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorInitializationContext.java @@ -0,0 +1,387 @@ +/* + * 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.components.connector; + +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.components.connector.components.ParameterValue; +import org.apache.nifi.components.connector.secrets.SecretsManager; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.ComponentType; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedParameter; +import org.apache.nifi.flow.VersionedParameterContext; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; +import org.apache.nifi.logging.ComponentLog; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestStandardConnectorInitializationContext { + + + @Test + public void testCreateParameterValuesSingleContext() { + final VersionedParameterContext context = new VersionedParameterContext(); + context.setName("ctx"); + context.setInheritedParameterContexts(List.of()); + + final Set parameters = new HashSet<>(); + parameters.add(createVersionedParameter("p1", "v1", false)); + parameters.add(createVersionedParameter("p2", "secret", true)); + context.setParameters(parameters); + + final List parameterValues = createParameterValues(List.of(context)); + assertEquals(2, parameterValues.size()); + + final Map byName = indexByName(parameterValues); + assertEquals("v1", byName.get("p1").getValue()); + assertFalse(byName.get("p1").isSensitive()); + assertEquals("secret", byName.get("p2").getValue()); + assertTrue(byName.get("p2").isSensitive()); + } + + @Test + public void testCreateParameterValuesInheritedPrecedenceOrder() { + // Inherited contexts listed highest precedence first; reverse traversal ensures last applied wins + final VersionedParameterContext high = new VersionedParameterContext(); + high.setName("high"); + high.setInheritedParameterContexts(List.of()); + high.setParameters(Set.of(createVersionedParameter("p", "H", false))); + + final VersionedParameterContext low = new VersionedParameterContext(); + low.setName("low"); + low.setInheritedParameterContexts(List.of()); + low.setParameters(Set.of(createVersionedParameter("p", "L", false))); + + final VersionedParameterContext child = new VersionedParameterContext(); + child.setName("child"); + child.setInheritedParameterContexts(List.of("high", "low")); + child.setParameters(Set.of()); + + final Collection contexts = List.of(child, high, low); + final List parameterValues = createParameterValues(contexts); + final Map byName = indexByName(parameterValues); + assertEquals(1, byName.size()); + assertEquals("H", byName.get("p").getValue()); + } + + @Test + public void testCreateParameterValuesLocalOverridesInherited() { + final VersionedParameterContext base = new VersionedParameterContext(); + base.setName("base"); + base.setInheritedParameterContexts(List.of()); + base.setParameters(Set.of( + createVersionedParameter("x", "1", false), + createVersionedParameter("y", "Y", false) + )); + + final VersionedParameterContext mid = new VersionedParameterContext(); + mid.setName("mid"); + mid.setInheritedParameterContexts(List.of("base")); + mid.setParameters(Set.of( + createVersionedParameter("x", "2", false) + )); + + final VersionedParameterContext top = new VersionedParameterContext(); + top.setName("top"); + top.setInheritedParameterContexts(List.of("mid")); + top.setParameters(Set.of( + createVersionedParameter("x", "3", false), + createVersionedParameter("z", "Z", true) + )); + + final List contexts = new ArrayList<>(); + contexts.add(top); + contexts.add(mid); + contexts.add(base); + + final List parameterValues = createParameterValues(contexts); + final Map byName = indexByName(parameterValues); + assertEquals(3, byName.size()); + assertEquals("3", byName.get("x").getValue()); + assertEquals("Y", byName.get("y").getValue()); + assertEquals("Z", byName.get("z").getValue()); + assertTrue(byName.get("z").isSensitive()); + } + + private VersionedParameter createVersionedParameter(final String name, final String value, final boolean sensitive) { + final VersionedParameter parameter = new VersionedParameter(); + parameter.setName(name); + parameter.setValue(value); + parameter.setSensitive(sensitive); + return parameter; + } + + private Map indexByName(final List values) { + final Map byName = new HashMap<>(); + for (final ParameterValue value : values) { + byName.put(value.getName(), value); + } + return byName; + } + + private List createParameterValues(final Collection contexts) { + final ConnectorParameterLookup parameterLookup = new ConnectorParameterLookup(contexts, null); + return parameterLookup.getParameterValues(); + } + + @Test + public void testResolveBundlesRequireExactBundle() { + final Bundle unavailableBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "0.0.0-NONEXISTENT"); + final Bundle availableBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "2.0.0"); + + final ComponentBundleLookup bundleLookup = mock(ComponentBundleLookup.class); + when(bundleLookup.getAvailableBundles("org.apache.nifi.processors.standard.GenerateFlowFile")).thenReturn(List.of(availableBundle)); + + final VersionedProcessGroup group = createProcessGroupWithProcessor("GenerateFlowFile", + "org.apache.nifi.processors.standard.GenerateFlowFile", unavailableBundle); + + final VersionedExternalFlow externalFlow = new VersionedExternalFlow(); + externalFlow.setFlowContents(group); + externalFlow.setParameterContexts(Map.of()); + + final StandardConnectorInitializationContext context = createContext(bundleLookup); + context.resolveBundles(group, BundleCompatibility.REQUIRE_EXACT_BUNDLE); + + final VersionedProcessor processor = group.getProcessors().iterator().next(); + assertEquals("0.0.0-NONEXISTENT", processor.getBundle().getVersion(), "REQUIRE_EXACT_BUNDLE should not change the bundle"); + } + + @Test + public void testResolveBundlesResolveBundleWithSingleAvailable() { + final Bundle unavailableBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "0.0.0-NONEXISTENT"); + final Bundle availableBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "2.0.0"); + + final ComponentBundleLookup bundleLookup = mock(ComponentBundleLookup.class); + when(bundleLookup.getAvailableBundles("org.apache.nifi.processors.standard.GenerateFlowFile")).thenReturn(List.of(availableBundle)); + + final VersionedProcessGroup group = createProcessGroupWithProcessor("GenerateFlowFile", "org.apache.nifi.processors.standard.GenerateFlowFile", unavailableBundle); + + final StandardConnectorInitializationContext context = createContext(bundleLookup); + context.resolveBundles(group, BundleCompatibility.RESOLVE_BUNDLE); + + final VersionedProcessor processor = group.getProcessors().iterator().next(); + assertEquals("2.0.0", processor.getBundle().getVersion(), "RESOLVE_BUNDLE should resolve to the single available bundle"); + } + + @Test + public void testResolveBundlesResolveBundleWithMultipleAvailable() { + final Bundle unavailableBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "0.0.0-NONEXISTENT"); + final Bundle availableBundle1 = createBundle("org.apache.nifi", "nifi-standard-nar", "1.0.0"); + final Bundle availableBundle2 = createBundle("org.apache.nifi", "nifi-standard-nar", "2.0.0"); + + final ComponentBundleLookup bundleLookup = mock(ComponentBundleLookup.class); + when(bundleLookup.getAvailableBundles("org.apache.nifi.processors.standard.GenerateFlowFile")).thenReturn(List.of(availableBundle1, availableBundle2)); + + final VersionedProcessGroup group = createProcessGroupWithProcessor("GenerateFlowFile", "org.apache.nifi.processors.standard.GenerateFlowFile", unavailableBundle); + + final StandardConnectorInitializationContext context = createContext(bundleLookup); + context.resolveBundles(group, BundleCompatibility.RESOLVE_BUNDLE); + + final VersionedProcessor processor = group.getProcessors().iterator().next(); + assertEquals("0.0.0-NONEXISTENT", processor.getBundle().getVersion(), "RESOLVE_BUNDLE should not change bundle when multiple are available"); + } + + @Test + public void testResolveBundlesResolveBundleWithNoneAvailable() { + final Bundle unavailableBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "0.0.0-NONEXISTENT"); + + final ComponentBundleLookup bundleLookup = mock(ComponentBundleLookup.class); + when(bundleLookup.getAvailableBundles("org.apache.nifi.processors.standard.GenerateFlowFile")).thenReturn(List.of()); + + final VersionedProcessGroup group = createProcessGroupWithProcessor("GenerateFlowFile", "org.apache.nifi.processors.standard.GenerateFlowFile", unavailableBundle); + + final StandardConnectorInitializationContext context = createContext(bundleLookup); + context.resolveBundles(group, BundleCompatibility.RESOLVE_BUNDLE); + + final VersionedProcessor processor = group.getProcessors().iterator().next(); + assertEquals("0.0.0-NONEXISTENT", processor.getBundle().getVersion(), "RESOLVE_BUNDLE should not change bundle when none are available"); + } + + @Test + public void testResolveBundlesResolveNewestBundle() { + final Bundle unavailableBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "0.0.0-NONEXISTENT"); + final Bundle newestBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "2.0.0"); + + final ComponentBundleLookup bundleLookup = mock(ComponentBundleLookup.class); + when(bundleLookup.getAvailableBundles("org.apache.nifi.processors.standard.GenerateFlowFile")).thenReturn(List.of(createBundle("org.apache.nifi", "nifi-standard-nar", "1.0.0"), newestBundle)); + when(bundleLookup.getLatestBundle("org.apache.nifi.processors.standard.GenerateFlowFile")).thenReturn(Optional.of(newestBundle)); + + final VersionedProcessGroup group = createProcessGroupWithProcessor("GenerateFlowFile", "org.apache.nifi.processors.standard.GenerateFlowFile", unavailableBundle); + + final StandardConnectorInitializationContext context = createContext(bundleLookup); + context.resolveBundles(group, BundleCompatibility.RESOLVE_NEWEST_BUNDLE); + + final VersionedProcessor processor = group.getProcessors().iterator().next(); + assertEquals("2.0.0", processor.getBundle().getVersion(), "RESOLVE_NEWEST_BUNDLE should resolve to the newest available bundle"); + } + + @Test + public void testResolveBundlesResolveNewestBundleWithNoneAvailable() { + final Bundle unavailableBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "0.0.0-NONEXISTENT"); + + final ComponentBundleLookup bundleLookup = mock(ComponentBundleLookup.class); + when(bundleLookup.getAvailableBundles("org.apache.nifi.processors.standard.GenerateFlowFile")) + .thenReturn(List.of()); + + final VersionedProcessGroup group = createProcessGroupWithProcessor("GenerateFlowFile", + "org.apache.nifi.processors.standard.GenerateFlowFile", unavailableBundle); + + final StandardConnectorInitializationContext context = createContext(bundleLookup); + context.resolveBundles(group, BundleCompatibility.RESOLVE_NEWEST_BUNDLE); + + final VersionedProcessor processor = group.getProcessors().iterator().next(); + assertEquals("0.0.0-NONEXISTENT", processor.getBundle().getVersion(), "RESOLVE_NEWEST_BUNDLE should not change bundle when none are available"); + } + + @Test + public void testResolveBundlesWithControllerService() { + final Bundle unavailableBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "0.0.0-NONEXISTENT"); + final Bundle availableBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "2.0.0"); + + final ComponentBundleLookup bundleLookup = mock(ComponentBundleLookup.class); + when(bundleLookup.getAvailableBundles("org.apache.nifi.ssl.StandardSSLContextService")) + .thenReturn(List.of(availableBundle)); + + final VersionedProcessGroup group = createProcessGroupWithControllerService("SSLContextService", + "org.apache.nifi.ssl.StandardSSLContextService", unavailableBundle); + + final StandardConnectorInitializationContext context = createContext(bundleLookup); + context.resolveBundles(group, BundleCompatibility.RESOLVE_BUNDLE); + + final VersionedControllerService service = group.getControllerServices().iterator().next(); + assertEquals("2.0.0", service.getBundle().getVersion(), "RESOLVE_BUNDLE should resolve controller service bundle"); + } + + @Test + public void testResolveBundlesWithNestedProcessGroups() { + final Bundle unavailableBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "0.0.0-NONEXISTENT"); + final Bundle availableBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "2.0.0"); + + final ComponentBundleLookup bundleLookup = mock(ComponentBundleLookup.class); + when(bundleLookup.getAvailableBundles("org.apache.nifi.processors.standard.GenerateFlowFile")) + .thenReturn(List.of(availableBundle)); + + final VersionedProcessGroup childGroup = createProcessGroupWithProcessor("GenerateFlowFile", + "org.apache.nifi.processors.standard.GenerateFlowFile", unavailableBundle); + childGroup.setIdentifier("child-group"); + + final VersionedProcessGroup rootGroup = new VersionedProcessGroup(); + rootGroup.setIdentifier("root-group"); + rootGroup.setName("Root"); + rootGroup.setProcessors(new HashSet<>()); + rootGroup.setControllerServices(new HashSet<>()); + rootGroup.setProcessGroups(Set.of(childGroup)); + + final StandardConnectorInitializationContext context = createContext(bundleLookup); + context.resolveBundles(rootGroup, BundleCompatibility.RESOLVE_BUNDLE); + + final VersionedProcessor processor = childGroup.getProcessors().iterator().next(); + assertEquals("2.0.0", processor.getBundle().getVersion(), "RESOLVE_BUNDLE should resolve bundles in nested process groups"); + } + + @Test + public void testResolveBundlesDoesNotChangeAvailableBundle() { + final Bundle specifiedBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "1.0.0"); + final Bundle newerBundle = createBundle("org.apache.nifi", "nifi-standard-nar", "2.0.0"); + + final ComponentBundleLookup bundleLookup = mock(ComponentBundleLookup.class); + when(bundleLookup.getAvailableBundles("org.apache.nifi.processors.standard.GenerateFlowFile")) + .thenReturn(List.of(specifiedBundle, newerBundle)); + + final VersionedProcessGroup group = createProcessGroupWithProcessor("GenerateFlowFile", + "org.apache.nifi.processors.standard.GenerateFlowFile", specifiedBundle); + + final StandardConnectorInitializationContext context = createContext(bundleLookup); + context.resolveBundles(group, BundleCompatibility.RESOLVE_NEWEST_BUNDLE); + + final VersionedProcessor processor = group.getProcessors().iterator().next(); + assertEquals("1.0.0", processor.getBundle().getVersion(), "Should not change bundle when it is already available"); + } + + private Bundle createBundle(final String group, final String artifact, final String version) { + final Bundle bundle = new Bundle(); + bundle.setGroup(group); + bundle.setArtifact(artifact); + bundle.setVersion(version); + return bundle; + } + + private VersionedProcessGroup createProcessGroupWithProcessor(final String name, final String type, final Bundle bundle) { + final VersionedProcessor processor = new VersionedProcessor(); + processor.setIdentifier("processor-1"); + processor.setName(name); + processor.setType(type); + processor.setBundle(bundle); + processor.setComponentType(ComponentType.PROCESSOR); + processor.setProperties(new HashMap<>()); + processor.setPropertyDescriptors(new HashMap<>()); + + final VersionedProcessGroup group = new VersionedProcessGroup(); + group.setIdentifier("test-group"); + group.setName("Test Group"); + group.setProcessors(new HashSet<>(Set.of(processor))); + group.setControllerServices(new HashSet<>()); + group.setProcessGroups(new HashSet<>()); + return group; + } + + private VersionedProcessGroup createProcessGroupWithControllerService(final String name, final String type, final Bundle bundle) { + final VersionedControllerService service = new VersionedControllerService(); + service.setIdentifier("service-1"); + service.setName(name); + service.setType(type); + service.setBundle(bundle); + service.setComponentType(ComponentType.CONTROLLER_SERVICE); + service.setProperties(new HashMap<>()); + service.setPropertyDescriptors(new HashMap<>()); + + final VersionedProcessGroup group = new VersionedProcessGroup(); + group.setIdentifier("test-group"); + group.setName("Test Group"); + group.setProcessors(new HashSet<>()); + group.setControllerServices(new HashSet<>(Set.of(service))); + group.setProcessGroups(new HashSet<>()); + return group; + } + + private StandardConnectorInitializationContext createContext(final ComponentBundleLookup bundleLookup) { + return new StandardConnectorInitializationContext.Builder() + .identifier("test-connector") + .name("Test Connector") + .componentLog(mock(ComponentLog.class)) + .secretsManager(mock(SecretsManager.class)) + .assetManager(mock(AssetManager.class)) + .componentBundleLookup(bundleLookup) + .build(); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorNode.java new file mode 100644 index 000000000000..d306ac45aa83 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorNode.java @@ -0,0 +1,848 @@ +/* + * 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.components.connector; + +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.components.FlowContextType; +import org.apache.nifi.components.connector.secrets.SecretsManager; +import org.apache.nifi.controller.flow.FlowManager; +import org.apache.nifi.controller.queue.QueueSize; +import org.apache.nifi.engine.FlowEngine; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.util.MockComponentLog; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.time.Duration; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.anySet; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestStandardConnectorNode { + + private FlowEngine scheduler; + + @Mock + private ExtensionManager extensionManager; + @Mock + private ProcessGroup managedProcessGroup; + @Mock + private AssetManager assetManager; + @Mock + private SecretsManager secretsManager; + + private FlowContextFactory flowContextFactory; + + @BeforeEach + public void setUp() { + MockitoAnnotations.openMocks(this); + scheduler = new FlowEngine(1, "flow-engine"); + + when(managedProcessGroup.purge()).thenReturn(CompletableFuture.completedFuture(null)); + when(managedProcessGroup.getQueueSize()).thenReturn(new QueueSize(0, 0L)); + + flowContextFactory = new FlowContextFactory() { + @Override + public FrameworkFlowContext createActiveFlowContext(final String connectorId, final ComponentLog connectorLogger, final Bundle bundle) { + final MutableConnectorConfigurationContext activeConfigurationContext = new StandardConnectorConfigurationContext(assetManager, secretsManager); + final ProcessGroupFacadeFactory processGroupFacadeFactory = mock(ProcessGroupFacadeFactory.class); + final ParameterContextFacadeFactory parameterContextFacadeFactory = mock(ParameterContextFacadeFactory.class); + return new StandardFlowContext(managedProcessGroup, activeConfigurationContext, processGroupFacadeFactory, + parameterContextFacadeFactory, connectorLogger, FlowContextType.ACTIVE, bundle); + } + + @Override + public FrameworkFlowContext createWorkingFlowContext(final String connectorId, final ComponentLog connectorLogger, + final MutableConnectorConfigurationContext currentConfiguration, final Bundle bundle) { + + final ProcessGroupFacadeFactory processGroupFacadeFactory = mock(ProcessGroupFacadeFactory.class); + final ParameterContextFacadeFactory parameterContextFacadeFactory = mock(ParameterContextFacadeFactory.class); + + return new StandardFlowContext(managedProcessGroup, currentConfiguration, processGroupFacadeFactory, + parameterContextFacadeFactory, connectorLogger, FlowContextType.WORKING, bundle); + } + }; + } + + @AfterEach + public void teardown() { + if (scheduler != null) { + scheduler.close(); + } + } + + @Test + public void testStartFromStoppedState() throws Exception { + final StandardConnectorNode connectorNode = createConnectorNode(); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + + final Future startFuture = connectorNode.start(scheduler); + + startFuture.get(5, TimeUnit.SECONDS); + + assertEquals(ConnectorState.RUNNING, connectorNode.getCurrentState()); + assertEquals(ConnectorState.RUNNING, connectorNode.getDesiredState()); + assertTrue(startFuture.isDone()); + assertFalse(startFuture.isCancelled()); + } + + @Test + public void testStopFromRunningState() throws Exception { + final StandardConnectorNode connectorNode = createConnectorNode(); + final Future startFuture = connectorNode.start(scheduler); + startFuture.get(5, TimeUnit.SECONDS); + assertEquals(ConnectorState.RUNNING, connectorNode.getCurrentState()); + + final Future stopFuture = connectorNode.stop(scheduler); + stopFuture.get(5, TimeUnit.SECONDS); + + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + assertEquals(ConnectorState.STOPPED, connectorNode.getDesiredState()); + assertTrue(stopFuture.isDone()); + assertFalse(stopFuture.isCancelled()); + } + + @Test + public void testStartFutureCompletedOnlyWhenRunning() throws Exception { + final StandardConnectorNode connectorNode = createConnectorNode(); + final Future startFuture = connectorNode.start(scheduler); + + startFuture.get(5, TimeUnit.SECONDS); + assertEquals(ConnectorState.RUNNING, connectorNode.getCurrentState()); + assertTrue(startFuture.isDone()); + } + + @Test + public void testStopFutureCompletedOnlyWhenStopped() throws Exception { + final StandardConnectorNode connectorNode = createConnectorNode(); + connectorNode.start(scheduler).get(5, TimeUnit.SECONDS); + assertEquals(ConnectorState.RUNNING, connectorNode.getCurrentState()); + + final Future stopFuture = connectorNode.stop(scheduler); + stopFuture.get(5, TimeUnit.SECONDS); + + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + assertTrue(stopFuture.isDone()); + } + + @Test + @Timeout(value = 5, unit = TimeUnit.SECONDS) + public void testMultipleStartCallsReturnCompletedFutures() throws Exception { + final CountDownLatch startLatch = new CountDownLatch(1); + final BlockingConnector blockingConnector = new BlockingConnector(startLatch, new CountDownLatch(0), new CountDownLatch(0)); + final StandardConnectorNode connectorNode = createConnectorNode(blockingConnector); + + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + + final Future startFuture1 = connectorNode.start(scheduler); + assertEquals(ConnectorState.STARTING, connectorNode.getCurrentState()); + assertEquals(ConnectorState.RUNNING, connectorNode.getDesiredState()); + + final Future startFuture2 = connectorNode.start(scheduler); + assertEquals(ConnectorState.STARTING, connectorNode.getCurrentState()); + assertEquals(ConnectorState.RUNNING, connectorNode.getDesiredState()); + + // Allow the connector to start + startLatch.countDown(); + startFuture1.get(5, TimeUnit.SECONDS); + startFuture2.get(5, TimeUnit.SECONDS); + + assertEquals(ConnectorState.RUNNING, connectorNode.getCurrentState()); + assertEquals(ConnectorState.RUNNING, connectorNode.getDesiredState()); + + assertTrue(startFuture1.isDone()); + assertTrue(startFuture2.isDone()); + } + + @Test + public void testVerifyCanDeleteWhenStopped() throws FlowUpdateException { + final StandardConnectorNode connectorNode = createConnectorNode(); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + connectorNode.verifyCanDelete(); + } + + @Test + public void testCannotDeleteWhenRunning() throws Exception { + final StandardConnectorNode connectorNode = createConnectorNode(); + connectorNode.start(scheduler).get(5, TimeUnit.SECONDS); + assertEquals(ConnectorState.RUNNING, connectorNode.getCurrentState()); + + assertThrows(IllegalStateException.class, connectorNode::verifyCanDelete); + } + + @Test + public void testVerifyCanStartWhenStopped() throws FlowUpdateException { + final StandardConnectorNode connectorNode = createConnectorNode(); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + connectorNode.verifyCanStart(); + } + + @Test + public void testStartAlreadyRunningReturnsImmediately() throws Exception { + final StandardConnectorNode connectorNode = createConnectorNode(); + connectorNode.start(scheduler).get(5, TimeUnit.SECONDS); + assertEquals(ConnectorState.RUNNING, connectorNode.getCurrentState()); + + final Future startFuture = connectorNode.start(scheduler); + assertTrue(startFuture.isDone()); + + assertEquals(ConnectorState.RUNNING, connectorNode.getCurrentState()); + } + + @Test + public void testStopAlreadyStoppedReturnsImmediately() throws FlowUpdateException { + final StandardConnectorNode connectorNode = createConnectorNode(); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + + final Future stopFuture = connectorNode.stop(scheduler); + assertTrue(stopFuture.isDone()); + + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + } + + @Test + public void testStartWhileStoppingQueuesStartFuture() throws Exception { + final CountDownLatch stopLatch = new CountDownLatch(1); + final BlockingConnector blockingConnector = new BlockingConnector(new CountDownLatch(0), stopLatch, new CountDownLatch(0)); + final StandardConnectorNode connectorNode = createConnectorNode(blockingConnector); + + connectorNode.start(scheduler).get(5, TimeUnit.SECONDS); + assertEquals(ConnectorState.RUNNING, connectorNode.getCurrentState()); + assertEquals(ConnectorState.RUNNING, connectorNode.getDesiredState()); + + final Future stopFuture = connectorNode.stop(scheduler); + assertEquals(ConnectorState.STOPPING, connectorNode.getCurrentState()); + assertEquals(ConnectorState.STOPPED, connectorNode.getDesiredState()); + + final Future startFuture = connectorNode.start(scheduler); + assertEquals(ConnectorState.STOPPING, connectorNode.getCurrentState()); + assertEquals(ConnectorState.RUNNING, connectorNode.getDesiredState()); + + stopLatch.countDown(); + + stopFuture.get(5, TimeUnit.SECONDS); + startFuture.get(5, TimeUnit.SECONDS); + + assertEquals(ConnectorState.RUNNING, connectorNode.getCurrentState()); + assertTrue(stopFuture.isDone()); + assertTrue(startFuture.isDone()); + } + + @Test + public void testCannotDeleteWhenStarting() throws Exception { + // Use a slow-starting connector to test deletion during STARTING state + final CountDownLatch startLatch = new CountDownLatch(1); + final BlockingConnector blockingConnector = new BlockingConnector(startLatch, new CountDownLatch(0), new CountDownLatch(0)); + final StandardConnectorNode slowNode = createConnectorNode(blockingConnector); + + // Start the connector - this will take time + final Future startFuture = slowNode.start(scheduler); + + // While starting, verify we cannot delete + assertEquals(ConnectorState.STARTING, slowNode.getCurrentState()); + + assertThrows(IllegalStateException.class, slowNode::verifyCanDelete); + + // Wait for start to complete + startLatch.countDown(); + startFuture.get(5, TimeUnit.SECONDS); + assertEquals(ConnectorState.RUNNING, slowNode.getCurrentState()); + } + + @Test + public void testSetConfigurationWhenStopped() throws FlowUpdateException { + final StandardConnectorNode connectorNode = createConnectorNode(); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + assertEquals(ConnectorState.STOPPED, connectorNode.getDesiredState()); + + final ConnectorConfiguration newConfiguration = createTestConfiguration(); + + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + connectorNode.setConfiguration("testGroup", createStepConfiguration()); + connectorNode.applyUpdate(); + + assertEquals(newConfiguration, connectorNode.getActiveFlowContext().getConfigurationContext().toConnectorConfiguration()); + } + + @Test + public void testSetConfigurationWithPropertyChanges() throws FlowUpdateException, ExecutionException, InterruptedException, TimeoutException { + final StandardConnectorNode connectorNode = createConnectorNode(); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + connectorNode.setConfiguration("step1", createStepConfiguration(Map.of("prop1", "value1"))); + connectorNode.applyUpdate(); + + final ConnectorConfiguration newConfiguration = createTestConfiguration("step1", "prop1", "value2"); + + connectorNode.stop(scheduler).get(5, TimeUnit.SECONDS); + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + connectorNode.setConfiguration("step1", createStepConfiguration(Map.of("prop1", "value2"))); + connectorNode.applyUpdate(); + assertEquals(newConfiguration, connectorNode.getActiveFlowContext().getConfigurationContext().toConnectorConfiguration()); + } + + @Test + public void testSetConfigurationWithNewConfigurationStep() throws FlowUpdateException, ExecutionException, InterruptedException, TimeoutException { + final StandardConnectorNode connectorNode = createConnectorNode(); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + connectorNode.setConfiguration("configurationStep1", createStepConfiguration(Map.of("prop1", "value1"))); + connectorNode.applyUpdate(); + + final ConnectorConfiguration newConfiguration = createTestConfigurationWithMultipleGroups(); + + // Wait for Connector to fully stop + connectorNode.stop(scheduler).get(5, TimeUnit.SECONDS); + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + connectorNode.setConfiguration("configurationStep1", createStepConfiguration(Map.of("prop1", "value1"))); + connectorNode.setConfiguration("configurationStep2", createStepConfiguration(Map.of("prop2", "value2"))); + connectorNode.applyUpdate(); + + assertEquals(newConfiguration, connectorNode.getActiveFlowContext().getConfigurationContext().toConnectorConfiguration()); + } + + @Test + public void testSetConfigurationWithRemovedConfigurationStep() throws FlowUpdateException, ExecutionException, InterruptedException, TimeoutException { + final StandardConnectorNode connectorNode = createConnectorNode(); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + connectorNode.setConfiguration("configurationStep1", createStepConfiguration(Map.of("prop1", "value1"))); + connectorNode.setConfiguration("configurationStep2", createStepConfiguration(Map.of("prop2", "value2"))); + connectorNode.applyUpdate(); + + connectorNode.stop(scheduler).get(5, TimeUnit.SECONDS); + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + connectorNode.setConfiguration("configurationStep1", createStepConfiguration(Map.of("prop1", "value1"))); + connectorNode.applyUpdate(); + + final Set expectedSteps = Set.of( + new NamedStepConfiguration("configurationStep1", new StepConfiguration(Map.of("prop1", new StringLiteralValue("value1")))), + new NamedStepConfiguration("configurationStep2", new StepConfiguration(Map.of("prop2", new StringLiteralValue("value2")))) + ); + final ConnectorConfiguration expectedConfiguration = new ConnectorConfiguration(expectedSteps); + assertEquals(expectedConfiguration, connectorNode.getActiveFlowContext().getConfigurationContext().toConnectorConfiguration()); + } + + @Test + public void testSetConfigurationCallsOnConfigured() throws FlowUpdateException { + final TrackingConnector trackingConnector = new TrackingConnector(); + final StandardConnectorNode connectorNode = createConnectorNode(trackingConnector); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + connectorNode.setConfiguration("testGroup", createStepConfiguration()); + connectorNode.applyUpdate(); + } + + @Test + public void testSetConfigurationCallsOnPropertyGroupConfiguredForChangedConfigurationSteps() throws FlowUpdateException, ExecutionException, InterruptedException, TimeoutException { + final TrackingConnector trackingConnector = new TrackingConnector(); + final StandardConnectorNode connectorNode = createConnectorNode(trackingConnector); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + connectorNode.setConfiguration("configurationStep1", createStepConfiguration(Map.of("prop1", "value1"))); + connectorNode.applyUpdate(); + trackingConnector.reset(); + + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + connectorNode.setConfiguration("configurationStep1", createStepConfiguration(Map.of("prop1", "value2"))); + connectorNode.applyUpdate(); + + assertTrue(trackingConnector.wasOnPropertyGroupConfiguredCalled("configurationStep1")); + } + + @Test + public void testVerifyConfigurationStepSurfacesInvalidValidationResults() throws FlowUpdateException { + final ValidationFailingConnector validationFailingConnector = new ValidationFailingConnector(); + final StandardConnectorNode connectorNode = createConnectorNode(validationFailingConnector); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + + connectorNode.transitionStateForUpdating(); + connectorNode.prepareForUpdate(); + final List results = connectorNode.verifyConfigurationStep("testStep", new StepConfiguration(Map.of("testProperty", new StringLiteralValue("invalidValue")))); + + assertFalse(results.isEmpty()); + final List failedResults = results.stream() + .filter(result -> result.getOutcome() == ConfigVerificationResult.Outcome.FAILED) + .toList(); + assertFalse(failedResults.isEmpty()); + + final ConfigVerificationResult failedResult = failedResults.getFirst(); + assertEquals("Property Validation - Test Property", failedResult.getVerificationStepName()); + assertEquals("Test Property", failedResult.getSubject()); + assertEquals("The property value is invalid", failedResult.getExplanation()); + } + + @Test + @Timeout(value = 5, unit = TimeUnit.SECONDS) + public void testDrainFlowFilesTransitionsStateToDraining() throws FlowUpdateException { + final CompletableFuture drainCompletionFuture = new CompletableFuture<>(); + final DrainBlockingConnector drainBlockingConnector = new DrainBlockingConnector(drainCompletionFuture); + final StandardConnectorNode connectorNode = createConnectorNode(drainBlockingConnector); + + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + + final Future drainFuture = connectorNode.drainFlowFiles(); + + assertEquals(ConnectorState.DRAINING, connectorNode.getCurrentState()); + assertFalse(drainFuture.isDone()); + + drainCompletionFuture.complete(null); + + try { + drainFuture.get(2, TimeUnit.SECONDS); + } catch (final Exception e) { + throw new RuntimeException("Drain future failed to complete", e); + } + + assertTrue(drainFuture.isDone()); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + } + + @Test + @Timeout(value = 5, unit = TimeUnit.SECONDS) + public void testDrainFlowFilesFutureDoesNotCompleteUntilDrainFinishes() throws FlowUpdateException, InterruptedException { + final CompletableFuture drainCompletionFuture = new CompletableFuture<>(); + final DrainBlockingConnector drainBlockingConnector = new DrainBlockingConnector(drainCompletionFuture); + final StandardConnectorNode connectorNode = createConnectorNode(drainBlockingConnector); + + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + + final Future drainFuture = connectorNode.drainFlowFiles(); + assertEquals(ConnectorState.DRAINING, connectorNode.getCurrentState()); + + Thread.sleep(200); + assertFalse(drainFuture.isDone()); + assertEquals(ConnectorState.DRAINING, connectorNode.getCurrentState()); + + drainCompletionFuture.complete(null); + + try { + drainFuture.get(2, TimeUnit.SECONDS); + } catch (final Exception e) { + throw new RuntimeException("Drain future failed to complete", e); + } + + assertTrue(drainFuture.isDone()); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + } + + @Test + @Timeout(value = 5, unit = TimeUnit.SECONDS) + public void testDrainFlowFilesStateTransitionsBackToStoppedOnCompletion() throws FlowUpdateException { + final CompletableFuture drainCompletionFuture = new CompletableFuture<>(); + final DrainBlockingConnector drainBlockingConnector = new DrainBlockingConnector(drainCompletionFuture); + final StandardConnectorNode connectorNode = createConnectorNode(drainBlockingConnector); + + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + assertEquals(ConnectorState.STOPPED, connectorNode.getDesiredState()); + + final Future drainFuture = connectorNode.drainFlowFiles(); + assertEquals(ConnectorState.DRAINING, connectorNode.getCurrentState()); + + drainCompletionFuture.complete(null); + + try { + drainFuture.get(2, TimeUnit.SECONDS); + } catch (final Exception e) { + throw new RuntimeException("Drain future failed to complete", e); + } + + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + assertEquals(ConnectorState.STOPPED, connectorNode.getDesiredState()); + } + + @Test + @Timeout(value = 10, unit = TimeUnit.SECONDS) + public void testCancelDrainFlowFilesInterruptsConnector() throws Exception { + final BusyLoopDrainConnector busyLoopConnector = new BusyLoopDrainConnector(); + final StandardConnectorNode connectorNode = createConnectorNode(busyLoopConnector); + + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + + connectorNode.drainFlowFiles(); + + assertTrue(busyLoopConnector.awaitDrainStarted(2, TimeUnit.SECONDS)); + assertEquals(ConnectorState.DRAINING, connectorNode.getCurrentState()); + + Thread.sleep(1000); + + assertEquals(ConnectorState.DRAINING, connectorNode.getCurrentState()); + assertFalse(busyLoopConnector.wasInterrupted()); + assertFalse(busyLoopConnector.wasStopCalled()); + + connectorNode.cancelDrainFlowFiles(); + + assertTrue(busyLoopConnector.awaitDrainCompleted(2, TimeUnit.SECONDS)); + assertTrue(busyLoopConnector.wasInterrupted()); + assertTrue(busyLoopConnector.wasStopCalled()); + assertEquals(ConnectorState.STOPPED, connectorNode.getCurrentState()); + } + + private StandardConnectorNode createConnectorNode() throws FlowUpdateException { + final SleepingConnector sleepingConnector = new SleepingConnector(Duration.ofMillis(1)); + return createConnectorNode(sleepingConnector); + } + + private StandardConnectorNode createConnectorNode(final Connector connector) throws FlowUpdateException { + final ConnectorStateTransition stateTransition = new StandardConnectorStateTransition("TestConnectorNode"); + final ConnectorValidationTrigger validationTrigger = new SynchronousConnectorValidationTrigger(); + final StandardConnectorNode node = new StandardConnectorNode( + "test-connector-id", + mock(FlowManager.class), + extensionManager, + null, + createConnectorDetails(connector), + "TestConnector", + connector.getClass().getCanonicalName(), + new StandardConnectorConfigurationContext(assetManager, secretsManager), + stateTransition, + flowContextFactory, + validationTrigger, + false); + + // mock secrets manager + final SecretsManager secretsManager = mock(SecretsManager.class); + when(secretsManager.getAllSecrets()).thenReturn(List.of()); + when(secretsManager.getSecrets(anySet())).thenReturn(Collections.emptyMap()); + + final FrameworkConnectorInitializationContext initializationContext = mock(FrameworkConnectorInitializationContext.class); + when(initializationContext.getSecretsManager()).thenReturn(secretsManager); + + node.initializeConnector(initializationContext); + node.loadInitialFlow(); + return node; + } + + private static class SynchronousConnectorValidationTrigger implements ConnectorValidationTrigger { + @Override + public void triggerAsync(final ConnectorNode connector) { + trigger(connector); + } + + @Override + public void trigger(final ConnectorNode connector) { + connector.performValidation(); + } + } + + private ConnectorDetails createConnectorDetails(final Connector connector) { + final ComponentLog componentLog = new MockComponentLog("TestConnector", connector); + final BundleCoordinate bundleCoordinate = new BundleCoordinate("org.apache.nifi", "test-standard-connector-node", "1.0.0"); + return new ConnectorDetails(connector, bundleCoordinate, componentLog); + } + + private StepConfiguration createStepConfiguration() { + return createStepConfiguration(Map.of("testProperty", "testValue")); + } + + private StepConfiguration createStepConfiguration(final Map properties) { + final Map valueReferences = new java.util.HashMap<>(); + for (final Map.Entry entry : properties.entrySet()) { + valueReferences.put(entry.getKey(), new StringLiteralValue(entry.getValue())); + } + return new StepConfiguration(valueReferences); + } + + private ConnectorConfiguration createTestConfiguration() { + return createTestConfiguration("testGroup", "testProperty", "testValue"); + } + + private ConnectorConfiguration createTestConfiguration(final String configurationStepName, final String propertyName, final String propertyValue) { + final StepConfiguration stepConfig = new StepConfiguration(Map.of(propertyName, new StringLiteralValue(propertyValue))); + final NamedStepConfiguration configurationStepConfiguration = new NamedStepConfiguration(configurationStepName, stepConfig); + return new ConnectorConfiguration(Set.of(configurationStepConfiguration)); + } + + private ConnectorConfiguration createTestConfigurationWithMultipleGroups() { + final StepConfiguration firstStepConfig = new StepConfiguration(Map.of("prop1", new StringLiteralValue("value1"))); + final NamedStepConfiguration firstConfigurationStepConfiguration = new NamedStepConfiguration("configurationStep1", firstStepConfig); + + final StepConfiguration secondStepConfig = new StepConfiguration(Map.of("prop2", new StringLiteralValue("value2"))); + final NamedStepConfiguration secondConfigurationStepConfiguration = new NamedStepConfiguration("configurationStep2", secondStepConfig); + + return new ConnectorConfiguration(Set.of(firstConfigurationStepConfiguration, secondConfigurationStepConfiguration)); + } + + /** + * Test connector that tracks method calls for verification + */ + private static class TrackingConnector extends AbstractConnector { + private final Set onConfigurationStepConfiguredCalls = new HashSet<>(); + + @Override + public VersionedExternalFlow getInitialFlow() { + return null; + } + + @Override + public void prepareForUpdate(final FlowContext workingContext, final FlowContext activeContext) { + } + + @Override + public List getConfigurationSteps() { + return List.of(); + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) { + } + + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) { + onConfigurationStepConfiguredCalls.add(stepName); + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext flowContext) { + return List.of(); + } + + public boolean wasOnPropertyGroupConfiguredCalled(final String stepName) { + return onConfigurationStepConfiguredCalls.contains(stepName); + } + + public void reset() { + onConfigurationStepConfiguredCalls.clear(); + } + } + + /** + * Test connector that returns invalid validation results from validateConfigurationStep + */ + private static class ValidationFailingConnector extends AbstractConnector { + @Override + public VersionedExternalFlow getInitialFlow() { + return null; + } + + @Override + public void prepareForUpdate(final FlowContext workingContext, final FlowContext activeContext) { + } + + @Override + public List validateConfigurationStep(final ConfigurationStep configurationStep, final ConnectorConfigurationContext connectorConfigurationContext, + final ConnectorValidationContext connectorValidationContext) { + + final ValidationResult invalidResult = new ValidationResult.Builder() + .subject("Test Property") + .valid(false) + .explanation("The property value is invalid") + .build(); + return List.of(invalidResult); + } + + @Override + public List getConfigurationSteps() { + final ConfigurationStep testStep = new ConfigurationStep.Builder() + .name("testStep") + .build(); + return List.of(testStep); + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) { + } + + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) { + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext flowContext) { + return List.of(); + } + } + + /** + * Test connector that allows control over when drainFlowFiles completes via a CompletableFuture + */ + private static class DrainBlockingConnector extends AbstractConnector { + private final CompletableFuture drainCompletionFuture; + + public DrainBlockingConnector(final CompletableFuture drainCompletionFuture) { + this.drainCompletionFuture = drainCompletionFuture; + } + + @Override + public VersionedExternalFlow getInitialFlow() { + return null; + } + + @Override + public void prepareForUpdate(final FlowContext workingContext, final FlowContext activeContext) { + } + + @Override + public List getConfigurationSteps() { + return List.of(); + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) { + } + + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) { + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext flowContext) { + return List.of(); + } + + @Override + public CompletableFuture drainFlowFiles(final FlowContext flowContext) { + return drainCompletionFuture; + } + } + + /** + * Test connector that uses a busy loop for draining that never completes naturally, + * but can be interrupted via cancellation. + */ + private static class BusyLoopDrainConnector extends AbstractConnector { + private volatile boolean interrupted = false; + private volatile boolean stopCalled = false; + private final AtomicReference drainThreadRef = new AtomicReference<>(); + private final CountDownLatch drainStartedLatch = new CountDownLatch(1); + private final CountDownLatch drainCompletedLatch = new CountDownLatch(1); + + @Override + public VersionedExternalFlow getInitialFlow() { + return null; + } + + @Override + public void prepareForUpdate(final FlowContext workingContext, final FlowContext activeContext) { + } + + @Override + public List getConfigurationSteps() { + return List.of(); + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) { + } + + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) { + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext flowContext) { + return List.of(); + } + + @Override + public void stop(final FlowContext activeContext) { + stopCalled = true; + } + + @Override + public CompletableFuture drainFlowFiles(final FlowContext flowContext) { + final CompletableFuture future = new CompletableFuture<>(); + + final Thread drainThread = new Thread(() -> { + drainStartedLatch.countDown(); + try { + while (!Thread.currentThread().isInterrupted()) { + // Busy loop that never completes naturally + } + } finally { + interrupted = Thread.currentThread().isInterrupted(); + drainCompletedLatch.countDown(); + } + }); + + drainThreadRef.set(drainThread); + + future.whenComplete((result, throwable) -> { + final Thread thread = drainThreadRef.get(); + if (thread != null) { + thread.interrupt(); + } + }); + + drainThread.start(); + + return future; + } + + public boolean awaitDrainStarted(final long timeout, final TimeUnit unit) throws InterruptedException { + return drainStartedLatch.await(timeout, unit); + } + + public boolean awaitDrainCompleted(final long timeout, final TimeUnit unit) throws InterruptedException { + return drainCompletedLatch.await(timeout, unit); + } + + public boolean wasInterrupted() { + return interrupted; + } + + public boolean wasStopCalled() { + return stopCalled; + } + } + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorRepository.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorRepository.java new file mode 100644 index 000000000000..4089b274819e --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/TestStandardConnectorRepository.java @@ -0,0 +1,877 @@ +/* + * 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.components.connector; + +import org.apache.nifi.asset.Asset; +import org.apache.nifi.asset.AssetManager; +import org.apache.nifi.flow.VersionedConfigurationStep; +import org.apache.nifi.flow.VersionedConnectorValueReference; +import org.apache.nifi.nar.ExtensionManager; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.timeout; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.when; + +public class TestStandardConnectorRepository { + + @Test + public void testAddAndGetConnectors() { + final StandardConnectorRepository repository = new StandardConnectorRepository(); + + final ConnectorNode connector1 = mock(ConnectorNode.class); + final ConnectorNode connector2 = mock(ConnectorNode.class); + when(connector1.getIdentifier()).thenReturn("connector-1"); + when(connector2.getIdentifier()).thenReturn("connector-2"); + + repository.addConnector(connector1); + repository.addConnector(connector2); + + final List connectors = repository.getConnectors(); + assertEquals(2, connectors.size()); + assertTrue(connectors.contains(connector1)); + assertTrue(connectors.contains(connector2)); + } + + @Test + public void testRemoveConnector() { + final StandardConnectorRepository repository = new StandardConnectorRepository(); + + final ConnectorRepositoryInitializationContext initContext = mock(ConnectorRepositoryInitializationContext.class); + final ExtensionManager extensionManager = mock(ExtensionManager.class); + when(initContext.getExtensionManager()).thenReturn(extensionManager); + repository.initialize(initContext); + + final Connector mockConnector = mock(Connector.class); + final ConnectorNode connector = mock(ConnectorNode.class); + when(connector.getIdentifier()).thenReturn("connector-1"); + when(connector.getConnector()).thenReturn(mockConnector); + + repository.addConnector(connector); + repository.removeConnector("connector-1"); + + assertEquals(0, repository.getConnectors().size()); + assertNull(repository.getConnector("connector-1")); + } + + @Test + public void testRestoreConnector() { + final StandardConnectorRepository repository = new StandardConnectorRepository(); + + final ConnectorNode connector = mock(ConnectorNode.class); + when(connector.getIdentifier()).thenReturn("connector-1"); + + repository.restoreConnector(connector); + assertEquals(1, repository.getConnectors().size()); + assertEquals(connector, repository.getConnector("connector-1")); + } + + @Test + public void testGetConnectorsReturnsNewListInstances() { + final StandardConnectorRepository repository = new StandardConnectorRepository(); + + final ConnectorNode connector1 = mock(ConnectorNode.class); + final ConnectorNode connector2 = mock(ConnectorNode.class); + when(connector1.getIdentifier()).thenReturn("connector-1"); + when(connector2.getIdentifier()).thenReturn("connector-2"); + + repository.addConnector(connector1); + repository.addConnector(connector2); + + final List connectors1 = repository.getConnectors(); + final List connectors2 = repository.getConnectors(); + + assertEquals(2, connectors1.size()); + assertEquals(2, connectors2.size()); + assertEquals(connectors1, connectors2); + assertNotSame(connectors1, connectors2); + } + + @Test + public void testAddConnectorWithDuplicateIdReplaces() { + final StandardConnectorRepository repository = new StandardConnectorRepository(); + + final ConnectorNode connector1 = mock(ConnectorNode.class); + final ConnectorNode connector2 = mock(ConnectorNode.class); + when(connector1.getIdentifier()).thenReturn("same-id"); + when(connector2.getIdentifier()).thenReturn("same-id"); + + repository.addConnector(connector1); + repository.addConnector(connector2); + + final List connectors = repository.getConnectors(); + assertEquals(1, connectors.size()); + assertEquals(connector2, repository.getConnector("same-id")); + } + + @Test + public void testDiscardWorkingConfigurationPreservesWorkingAssets() { + final String connectorId = "test-connector"; + final String activeAssetId = "active-asset-id"; + final String workingAssetId = "working-asset-id"; + final String unreferencedAssetId = "unreferenced-asset-id"; + + final StandardConnectorRepository repository = new StandardConnectorRepository(); + + final AssetManager assetManager = mock(AssetManager.class); + final ConnectorRepositoryInitializationContext initContext = mock(ConnectorRepositoryInitializationContext.class); + when(initContext.getExtensionManager()).thenReturn(mock(ExtensionManager.class)); + when(initContext.getAssetManager()).thenReturn(assetManager); + repository.initialize(initContext); + + final Asset activeAsset = mock(Asset.class); + when(activeAsset.getIdentifier()).thenReturn(activeAssetId); + when(activeAsset.getName()).thenReturn("active-asset.jar"); + + final Asset workingAsset = mock(Asset.class); + when(workingAsset.getIdentifier()).thenReturn(workingAssetId); + when(workingAsset.getName()).thenReturn("working-asset.jar"); + + final Asset unreferencedAsset = mock(Asset.class); + when(unreferencedAsset.getIdentifier()).thenReturn(unreferencedAssetId); + when(unreferencedAsset.getName()).thenReturn("unreferenced-asset.jar"); + + final MutableConnectorConfigurationContext activeConfigContext = mock(MutableConnectorConfigurationContext.class); + final StepConfiguration activeStepConfig = new StepConfiguration( + Map.of("prop1", new AssetReference(Set.of(activeAssetId))) + ); + final ConnectorConfiguration activeConfig = new ConnectorConfiguration( + Set.of(new NamedStepConfiguration("step1", activeStepConfig)) + ); + when(activeConfigContext.toConnectorConfiguration()).thenReturn(activeConfig); + + final MutableConnectorConfigurationContext workingConfigContext = mock(MutableConnectorConfigurationContext.class); + final StepConfiguration workingStepConfig = new StepConfiguration( + Map.of("prop1", new AssetReference(Set.of(workingAssetId))) + ); + final ConnectorConfiguration workingConfig = new ConnectorConfiguration( + Set.of(new NamedStepConfiguration("step1", workingStepConfig)) + ); + when(workingConfigContext.toConnectorConfiguration()).thenReturn(workingConfig); + + final FrameworkFlowContext activeFlowContext = mock(FrameworkFlowContext.class); + when(activeFlowContext.getConfigurationContext()).thenReturn(activeConfigContext); + + final FrameworkFlowContext workingFlowContext = mock(FrameworkFlowContext.class); + when(workingFlowContext.getConfigurationContext()).thenReturn(workingConfigContext); + + final ConnectorNode connector = mock(ConnectorNode.class); + when(connector.getIdentifier()).thenReturn(connectorId); + when(connector.getActiveFlowContext()).thenReturn(activeFlowContext); + when(connector.getWorkingFlowContext()).thenReturn(workingFlowContext); + + when(assetManager.getAssets(connectorId)).thenReturn(List.of(activeAsset, workingAsset, unreferencedAsset)); + + repository.addConnector(connector); + repository.discardWorkingConfiguration(connector); + + verify(assetManager, never()).deleteAsset(activeAssetId); + verify(assetManager, never()).deleteAsset(workingAssetId); + verify(assetManager).deleteAsset(unreferencedAssetId); + } + + @Test + public void testGetConnectorWithProviderOverridesWorkingConfig() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + final MutableConnectorConfigurationContext workingConfigContext = mock(MutableConnectorConfigurationContext.class); + final ConnectorNode connector = createConnectorNodeWithWorkingConfig("connector-1", "Original Name", workingConfigContext); + repository.addConnector(connector); + + final ConnectorWorkingConfiguration externalConfig = new ConnectorWorkingConfiguration(); + externalConfig.setName("External Name"); + final VersionedConfigurationStep externalStep = createVersionedStep("step1", Map.of("prop1", createStringLiteralRef("external-value"))); + externalConfig.setWorkingFlowConfiguration(List.of(externalStep)); + when(provider.load("connector-1")).thenReturn(Optional.of(externalConfig)); + + final ConnectorNode result = repository.getConnector("connector-1"); + + assertNotNull(result); + verify(connector).setName("External Name"); + verify(workingConfigContext).replaceProperties(eq("step1"), any(StepConfiguration.class)); + } + + @Test + public void testGetConnectorWithProviderReturnsEmpty() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + final ConnectorNode connector = createSimpleConnectorNode("connector-1", "Original Name"); + repository.addConnector(connector); + + when(provider.load("connector-1")).thenReturn(Optional.empty()); + + final ConnectorNode result = repository.getConnector("connector-1"); + + assertNotNull(result); + verify(connector, never()).setName(anyString()); + } + + @Test + public void testGetConnectorWithProviderThrowsException() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + final ConnectorNode connector = createSimpleConnectorNode("connector-1", "Original Name"); + repository.addConnector(connector); + + when(provider.load("connector-1")).thenThrow(new ConnectorConfigurationProviderException("Provider failure")); + + assertThrows(ConnectorConfigurationProviderException.class, () -> repository.getConnector("connector-1")); + verify(connector, never()).setName(anyString()); + } + + @Test + public void testGetConnectorWithNullProvider() { + final StandardConnectorRepository repository = createRepositoryWithProvider(null); + + final ConnectorNode connector = createSimpleConnectorNode("connector-1", "Original Name"); + repository.addConnector(connector); + + final ConnectorNode result = repository.getConnector("connector-1"); + + assertNotNull(result); + verify(connector, never()).setName(anyString()); + } + + @Test + public void testGetConnectorsWithProviderOverrides() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + final MutableConnectorConfigurationContext workingConfig1 = mock(MutableConnectorConfigurationContext.class); + final MutableConnectorConfigurationContext workingConfig2 = mock(MutableConnectorConfigurationContext.class); + final ConnectorNode connector1 = createConnectorNodeWithWorkingConfig("connector-1", "Name 1", workingConfig1); + final ConnectorNode connector2 = createConnectorNodeWithWorkingConfig("connector-2", "Name 2", workingConfig2); + repository.addConnector(connector1); + repository.addConnector(connector2); + + final ConnectorWorkingConfiguration externalConfig1 = new ConnectorWorkingConfiguration(); + externalConfig1.setName("External Name 1"); + externalConfig1.setWorkingFlowConfiguration(List.of()); + when(provider.load("connector-1")).thenReturn(Optional.of(externalConfig1)); + when(provider.load("connector-2")).thenReturn(Optional.empty()); + + final List results = repository.getConnectors(); + + assertEquals(2, results.size()); + verify(connector1).setName("External Name 1"); + verify(connector2, never()).setName(anyString()); + } + + @Test + public void testConfigureConnectorSavesToProviderBeforeModifyingNode() throws FlowUpdateException { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + final ConnectorNode connector = createConnectorNodeWithEmptyWorkingConfig("connector-1", "Test Connector"); + repository.addConnector(connector); + + when(provider.load("connector-1")).thenReturn(Optional.empty()); + + final StepConfiguration incomingConfig = new StepConfiguration(Map.of("prop1", new StringLiteralValue("new-value"))); + repository.configureConnector(connector, "step1", incomingConfig); + + final ArgumentCaptor configCaptor = ArgumentCaptor.forClass(ConnectorWorkingConfiguration.class); + verify(provider).save(eq("connector-1"), configCaptor.capture()); + + final ConnectorWorkingConfiguration savedConfig = configCaptor.getValue(); + assertNotNull(savedConfig); + assertEquals("Test Connector", savedConfig.getName()); + + final List savedSteps = savedConfig.getWorkingFlowConfiguration(); + assertNotNull(savedSteps); + final VersionedConfigurationStep savedStep = savedSteps.stream() + .filter(s -> "step1".equals(s.getName())).findFirst().orElse(null); + assertNotNull(savedStep); + assertEquals("STRING_LITERAL", savedStep.getProperties().get("prop1").getValueType()); + assertEquals("new-value", savedStep.getProperties().get("prop1").getValue()); + + verify(connector).setConfiguration("step1", incomingConfig); + } + + @Test + public void testConfigureConnectorProviderSaveFailsDoesNotModifyNode() throws FlowUpdateException { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + final ConnectorNode connector = createConnectorNodeWithEmptyWorkingConfig("connector-1", "Test Connector"); + repository.addConnector(connector); + + when(provider.load("connector-1")).thenReturn(Optional.empty()); + doThrow(new RuntimeException("Save failed")).when(provider).save(anyString(), any(ConnectorWorkingConfiguration.class)); + + final StepConfiguration incomingConfig = new StepConfiguration(Map.of("prop1", new StringLiteralValue("new-value"))); + + assertThrows(RuntimeException.class, () -> repository.configureConnector(connector, "step1", incomingConfig)); + + verify(connector, never()).setConfiguration(anyString(), any(StepConfiguration.class)); + } + + @Test + public void testConfigureConnectorMergesPartialStepConfig() throws FlowUpdateException { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + final ConnectorNode connector = createConnectorNodeWithEmptyWorkingConfig("connector-1", "Test Connector"); + repository.addConnector(connector); + + final VersionedConfigurationStep existingStep = createVersionedStep("step1", + Map.of("propA", createStringLiteralRef("old-A"), "propB", createStringLiteralRef("old-B"))); + final ConnectorWorkingConfiguration existingConfig = new ConnectorWorkingConfiguration(); + existingConfig.setName("Test Connector"); + existingConfig.setWorkingFlowConfiguration(new ArrayList<>(List.of(existingStep))); + when(provider.load("connector-1")).thenReturn(Optional.of(existingConfig)); + + final Map incomingProps = new HashMap<>(); + incomingProps.put("propA", new StringLiteralValue("new-A")); + incomingProps.put("propC", new StringLiteralValue("new-C")); + final StepConfiguration incomingConfig = new StepConfiguration(incomingProps); + + repository.configureConnector(connector, "step1", incomingConfig); + + final ArgumentCaptor configCaptor = ArgumentCaptor.forClass(ConnectorWorkingConfiguration.class); + verify(provider).save(eq("connector-1"), configCaptor.capture()); + + final ConnectorWorkingConfiguration savedConfig = configCaptor.getValue(); + final VersionedConfigurationStep savedStep = savedConfig.getWorkingFlowConfiguration().stream() + .filter(s -> "step1".equals(s.getName())).findFirst().orElse(null); + assertNotNull(savedStep); + + final Map savedProps = savedStep.getProperties(); + assertEquals("new-A", savedProps.get("propA").getValue()); + assertEquals("old-B", savedProps.get("propB").getValue()); + assertEquals("new-C", savedProps.get("propC").getValue()); + + verify(connector).setConfiguration("step1", incomingConfig); + } + + @Test + public void testDiscardWorkingConfigurationCallsProviderDiscard() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + final ConnectorNode connector = createConnectorNodeWithEmptyWorkingConfig("connector-1", "Test Connector"); + repository.addConnector(connector); + + repository.discardWorkingConfiguration(connector); + + verify(provider).discard("connector-1"); + verify(provider, never()).save(anyString(), any(ConnectorWorkingConfiguration.class)); + verify(connector).discardWorkingConfiguration(); + } + + @Test + public void testDiscardWorkingConfigurationProviderThrowsException() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + final ConnectorNode connector = createConnectorNodeWithEmptyWorkingConfig("connector-1", "Test Connector"); + repository.addConnector(connector); + + doThrow(new ConnectorConfigurationProviderException("Discard failed")).when(provider).discard("connector-1"); + + assertThrows(ConnectorConfigurationProviderException.class, () -> repository.discardWorkingConfiguration(connector)); + } + + @Test + public void testRemoveConnectorCallsProviderDelete() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + final Connector mockConnector = mock(Connector.class); + final ConnectorNode connector = mock(ConnectorNode.class); + when(connector.getIdentifier()).thenReturn("connector-1"); + when(connector.getConnector()).thenReturn(mockConnector); + repository.addConnector(connector); + + repository.removeConnector("connector-1"); + + verify(provider).delete("connector-1"); + } + + @Test + public void testRemoveConnectorProviderThrowsException() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + final Connector mockConnector = mock(Connector.class); + final ConnectorNode connector = mock(ConnectorNode.class); + when(connector.getIdentifier()).thenReturn("connector-1"); + when(connector.getConnector()).thenReturn(mockConnector); + repository.addConnector(connector); + + doThrow(new ConnectorConfigurationProviderException("Delete failed")).when(provider).delete("connector-1"); + + assertThrows(ConnectorConfigurationProviderException.class, () -> repository.removeConnector("connector-1")); + } + + @Test + public void testUpdateConnectorSavesNameToProvider() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + final ConnectorNode connector = createConnectorNodeWithEmptyWorkingConfig("connector-1", "Old Name"); + repository.addConnector(connector); + + repository.updateConnector(connector, "New Name"); + + final ArgumentCaptor configCaptor = ArgumentCaptor.forClass(ConnectorWorkingConfiguration.class); + verify(provider).save(eq("connector-1"), configCaptor.capture()); + assertEquals("New Name", configCaptor.getValue().getName()); + + verify(connector).setName("New Name"); + } + + @Test + public void testInheritConfigurationDoesNotCallProvider() throws FlowUpdateException { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + when(provider.load("connector-1")).thenReturn(Optional.empty()); + + final ConnectorNode connector = mock(ConnectorNode.class); + when(connector.getIdentifier()).thenReturn("connector-1"); + repository.addConnector(connector); + + // Reset interactions so we can verify that inheritConfiguration itself does not call the provider + reset(provider); + + repository.inheritConfiguration(connector, List.of(), List.of(), null); + + verifyNoInteractions(provider); + } + + @Test + public void testVerifyCreateWithExistingConnectorThrows() { + final StandardConnectorRepository repository = createRepositoryWithProvider(null); + + final ConnectorNode connector = createSimpleConnectorNode("connector-1", "Test Connector"); + repository.addConnector(connector); + + assertThrows(IllegalStateException.class, () -> repository.verifyCreate("connector-1")); + } + + @Test + public void testVerifyCreateWithNewConnectorSucceeds() { + final StandardConnectorRepository repository = createRepositoryWithProvider(null); + + repository.verifyCreate("connector-1"); + } + + @Test + public void testVerifyCreateDelegatesToProvider() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + repository.verifyCreate("connector-1"); + + verify(provider).verifyCreate("connector-1"); + } + + @Test + public void testVerifyCreateProviderRejectsThrows() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + doThrow(new ConnectorConfigurationProviderException("Create not supported")).when(provider).verifyCreate("connector-1"); + + assertThrows(ConnectorConfigurationProviderException.class, () -> repository.verifyCreate("connector-1")); + } + + @Test + public void testVerifyCreateExistingConnectorDoesNotCallProvider() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + when(provider.load("connector-1")).thenReturn(Optional.empty()); + + final ConnectorNode connector = createSimpleConnectorNode("connector-1", "Test Connector"); + repository.addConnector(connector); + + assertThrows(IllegalStateException.class, () -> repository.verifyCreate("connector-1")); + verify(provider, never()).verifyCreate(anyString()); + } + + // --- Asset Lifecycle Tests --- + + @Test + public void testStoreAssetDelegatesToProviderAndReturnsStoredAsset() throws IOException { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final AssetManager assetManager = mock(AssetManager.class); + final StandardConnectorRepository repository = createRepositoryWithProviderAndAssetManager(provider, assetManager); + + final Asset storedAsset = mock(Asset.class); + when(storedAsset.getIdentifier()).thenReturn("nifi-uuid-1"); + // After the provider stores the asset locally, the framework retrieves it via getAsset() + when(assetManager.getAsset("nifi-uuid-1")).thenReturn(Optional.of(storedAsset)); + + final InputStream content = new ByteArrayInputStream("test-content".getBytes()); + final Asset result = repository.storeAsset("connector-1", "nifi-uuid-1", "driver.jar", content); + + assertNotNull(result); + assertEquals("nifi-uuid-1", result.getIdentifier()); + // Provider handles both local storage and external upload -- no direct assetManager.saveAsset() call + verify(provider).storeAsset(eq("connector-1"), eq("nifi-uuid-1"), eq("driver.jar"), any(InputStream.class)); + verify(assetManager, never()).saveAsset(any(), any(), any(), any()); + } + + @Test + public void testStoreAssetWrapsProviderFailureAsIOException() throws IOException { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final AssetManager assetManager = mock(AssetManager.class); + final StandardConnectorRepository repository = createRepositoryWithProviderAndAssetManager(provider, assetManager); + + doThrow(new RuntimeException("Provider upload failed")) + .when(provider).storeAsset(eq("connector-1"), eq("nifi-uuid-1"), eq("driver.jar"), any(InputStream.class)); + + final InputStream content = new ByteArrayInputStream("test-content".getBytes()); + assertThrows(IOException.class, () -> repository.storeAsset("connector-1", "nifi-uuid-1", "driver.jar", content)); + + // Rollback of local storage is the provider's responsibility; the framework does not call deleteAsset + verify(assetManager, never()).deleteAsset(anyString()); + } + + @Test + public void testStoreAssetWithoutProviderDelegatesToAssetManagerOnly() throws IOException { + final AssetManager assetManager = mock(AssetManager.class); + final StandardConnectorRepository repository = createRepositoryWithProviderAndAssetManager(null, assetManager); + + final Asset localAsset = mock(Asset.class); + when(localAsset.getIdentifier()).thenReturn("nifi-uuid-1"); + when(assetManager.saveAsset(eq("connector-1"), eq("nifi-uuid-1"), eq("driver.jar"), any(InputStream.class))) + .thenReturn(localAsset); + + final Asset result = repository.storeAsset("connector-1", "nifi-uuid-1", "driver.jar", + new ByteArrayInputStream("content".getBytes())); + assertNotNull(result); + verify(assetManager).saveAsset(eq("connector-1"), eq("nifi-uuid-1"), eq("driver.jar"), any(InputStream.class)); + } + + @Test + public void testGetAssetDelegatesToAssetManager() { + final AssetManager assetManager = mock(AssetManager.class); + final StandardConnectorRepository repository = createRepositoryWithProviderAndAssetManager(null, assetManager); + + final Asset mockAsset = mock(Asset.class); + when(mockAsset.getIdentifier()).thenReturn("asset-1"); + when(assetManager.getAsset("asset-1")).thenReturn(Optional.of(mockAsset)); + + final Optional result = repository.getAsset("asset-1"); + assertTrue(result.isPresent()); + assertEquals("asset-1", result.get().getIdentifier()); + } + + @Test + public void testGetAssetsDelegatesToAssetManager() { + final AssetManager assetManager = mock(AssetManager.class); + final StandardConnectorRepository repository = createRepositoryWithProviderAndAssetManager(null, assetManager); + + final Asset asset1 = mock(Asset.class); + final Asset asset2 = mock(Asset.class); + when(assetManager.getAssets("connector-1")).thenReturn(List.of(asset1, asset2)); + + final List result = repository.getAssets("connector-1"); + assertEquals(2, result.size()); + } + + @Test + public void testDeleteAssetsDelegatesToProviderByNifiUuid() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final AssetManager assetManager = mock(AssetManager.class); + final StandardConnectorRepository repository = createRepositoryWithProviderAndAssetManager(provider, assetManager); + + final Asset asset = mock(Asset.class); + when(asset.getIdentifier()).thenReturn("nifi-uuid-1"); + when(assetManager.getAssets("connector-1")).thenReturn(List.of(asset)); + + repository.deleteAssets("connector-1"); + + // Provider receives NiFi UUID; provider handles external cleanup and local deletion + verify(provider).deleteAsset("connector-1", "nifi-uuid-1"); + verify(assetManager, never()).deleteAsset(anyString()); + } + + @Test + public void testDeleteAssetsWithoutProviderDelegatesToAssetManager() { + final AssetManager assetManager = mock(AssetManager.class); + final StandardConnectorRepository repository = createRepositoryWithProviderAndAssetManager(null, assetManager); + + final Asset asset = mock(Asset.class); + when(asset.getIdentifier()).thenReturn("nifi-uuid-1"); + when(assetManager.getAssets("connector-1")).thenReturn(List.of(asset)); + + repository.deleteAssets("connector-1"); + + verify(assetManager).deleteAsset("nifi-uuid-1"); + } + + @Test + public void testSyncFromProviderAppliesNifiUuidsDirectly() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + final MutableConnectorConfigurationContext workingConfigContext = mock(MutableConnectorConfigurationContext.class); + final ConnectorNode connector = createConnectorNodeWithWorkingConfig("connector-1", "Test Connector", workingConfigContext); + repository.addConnector(connector); + + // Provider returns config with NiFi UUIDs (no translation needed in framework) + final VersionedConnectorValueReference assetRef = new VersionedConnectorValueReference(); + assetRef.setValueType("ASSET_REFERENCE"); + assetRef.setAssetIds(Set.of("nifi-uuid-from-provider")); + final VersionedConfigurationStep step = createVersionedStep("step1", Map.of("driver", assetRef)); + + final ConnectorWorkingConfiguration config = new ConnectorWorkingConfiguration(); + config.setName("Test Connector"); + config.setWorkingFlowConfiguration(List.of(step)); + when(provider.load("connector-1")).thenReturn(Optional.of(config)); + + repository.getConnector("connector-1"); + + // Working config is updated with NiFi UUIDs as-is -- no translation in the repository + verify(workingConfigContext).replaceProperties(eq("step1"), any(StepConfiguration.class)); + } + + @Test + public void testSyncAssetsFromProviderCallsSyncAssetsThenReloads() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final StandardConnectorRepository repository = createRepositoryWithProvider(provider); + + final MutableConnectorConfigurationContext workingConfigContext = mock(MutableConnectorConfigurationContext.class); + final ConnectorNode connector = createConnectorNodeWithWorkingConfig("connector-1", "Test Connector", workingConfigContext); + repository.addConnector(connector); + + final ConnectorWorkingConfiguration config = new ConnectorWorkingConfiguration(); + config.setName("Test Connector"); + config.setWorkingFlowConfiguration(List.of()); + when(provider.load("connector-1")).thenReturn(Optional.of(config)); + + repository.syncAssetsFromProvider(connector); + + // Step 1: provider.syncAssets() called + verify(provider).syncAssets("connector-1"); + // Step 2: provider.load() called to reload updated config (may also have been called during addConnector) + verify(provider, org.mockito.Mockito.atLeastOnce()).load("connector-1"); + } + + @Test + public void testSyncAssetsFromProviderNoOpWithoutProvider() { + final AssetManager assetManager = mock(AssetManager.class); + final StandardConnectorRepository repository = createRepositoryWithProviderAndAssetManager(null, assetManager); + + final ConnectorNode connector = createSimpleConnectorNode("connector-1", "Test"); + repository.addConnector(connector); + + repository.syncAssetsFromProvider(connector); + + verifyNoInteractions(assetManager); + } + + @Test + public void testCleanUpAssetsCallsProviderDeleteForUnreferencedAssets() { + final ConnectorConfigurationProvider provider = mock(ConnectorConfigurationProvider.class); + final AssetManager assetManager = mock(AssetManager.class); + final StandardConnectorRepository repository = createRepositoryWithProviderAndAssetManager(provider, assetManager); + + final String connectorId = "connector-1"; + final String referencedAssetId = "referenced-uuid"; + final String unreferencedAssetId = "unreferenced-uuid"; + + final Asset referencedAsset = mock(Asset.class); + when(referencedAsset.getIdentifier()).thenReturn(referencedAssetId); + when(referencedAsset.getName()).thenReturn("referenced.jar"); + + final Asset unreferencedAsset = mock(Asset.class); + when(unreferencedAsset.getIdentifier()).thenReturn(unreferencedAssetId); + when(unreferencedAsset.getName()).thenReturn("unreferenced.jar"); + + final MutableConnectorConfigurationContext activeConfigContext = mock(MutableConnectorConfigurationContext.class); + final ConnectorConfiguration activeConfig = new ConnectorConfiguration(Set.of( + new NamedStepConfiguration("step1", new StepConfiguration( + Map.of("prop", new AssetReference(Set.of(referencedAssetId))))) + )); + when(activeConfigContext.toConnectorConfiguration()).thenReturn(activeConfig); + final FrameworkFlowContext activeFlowContext = mock(FrameworkFlowContext.class); + when(activeFlowContext.getConfigurationContext()).thenReturn(activeConfigContext); + + final MutableConnectorConfigurationContext workingConfigContext = mock(MutableConnectorConfigurationContext.class); + final ConnectorConfiguration workingConfig = new ConnectorConfiguration(Set.of()); + when(workingConfigContext.toConnectorConfiguration()).thenReturn(workingConfig); + final FrameworkFlowContext workingFlowContext = mock(FrameworkFlowContext.class); + when(workingFlowContext.getConfigurationContext()).thenReturn(workingConfigContext); + + final ConnectorNode connector = mock(ConnectorNode.class); + when(connector.getIdentifier()).thenReturn(connectorId); + when(connector.getActiveFlowContext()).thenReturn(activeFlowContext); + when(connector.getWorkingFlowContext()).thenReturn(workingFlowContext); + + when(assetManager.getAssets(connectorId)).thenReturn(List.of(referencedAsset, unreferencedAsset)); + when(provider.load(connectorId)).thenReturn(Optional.empty()); + repository.addConnector(connector); + + repository.discardWorkingConfiguration(connector); + + // Provider.deleteAsset called with NiFi UUID for unreferenced asset + verify(provider).deleteAsset(connectorId, unreferencedAssetId); + // Referenced asset is not deleted + verify(provider, never()).deleteAsset(connectorId, referencedAssetId); + // AssetManager.deleteAsset NOT called directly since provider handles local deletion + verify(assetManager, never()).deleteAsset(anyString()); + } + + @Test + public void testInheritConfigurationFailureCallsAbortUpdateAndMarkInvalid() throws FlowUpdateException { + final StandardConnectorRepository repository = createRepositoryWithProvider(null); + + final ConnectorNode connector = mock(ConnectorNode.class); + when(connector.getIdentifier()).thenReturn("connector-1"); + doThrow(new FlowUpdateException("Simulated failure")) + .when(connector).inheritConfiguration(any(), any(), any()); + + repository.addConnector(connector); + + assertThrows(FlowUpdateException.class, () -> repository.inheritConfiguration(connector, List.of(), List.of(), null)); + + verify(connector).abortUpdate(any(FlowUpdateException.class)); + verify(connector).markInvalid(eq("Flow Update Failure"), eq("The flow could not be updated: Simulated failure")); + } + + @Test + public void testApplyUpdateFailureCallsAbortUpdateButNotMarkInvalid() throws FlowUpdateException { + final AssetManager assetManager = mock(AssetManager.class); + when(assetManager.getAssets("connector-1")).thenReturn(List.of()); + final StandardConnectorRepository repository = createRepositoryWithProviderAndAssetManager(null, assetManager); + + final ConnectorNode connector = mock(ConnectorNode.class); + when(connector.getIdentifier()).thenReturn("connector-1"); + when(connector.getDesiredState()).thenReturn(ConnectorState.STOPPED); + doThrow(new FlowUpdateException("Simulated failure")) + .when(connector).prepareForUpdate(); + + repository.addConnector(connector); + + repository.applyUpdate(connector, mock(ConnectorUpdateContext.class)); + + verify(connector, timeout(5000)).abortUpdate(any(FlowUpdateException.class)); + verify(connector, never()).markInvalid(anyString(), anyString()); + } + + // --- Helper Methods --- + + private StandardConnectorRepository createRepositoryWithProviderAndAssetManager( + final ConnectorConfigurationProvider provider, final AssetManager assetManager) { + final StandardConnectorRepository repository = new StandardConnectorRepository(); + final ConnectorRepositoryInitializationContext initContext = mock(ConnectorRepositoryInitializationContext.class); + when(initContext.getExtensionManager()).thenReturn(mock(ExtensionManager.class)); + when(initContext.getAssetManager()).thenReturn(assetManager); + when(initContext.getConnectorConfigurationProvider()).thenReturn(provider); + repository.initialize(initContext); + return repository; + } + + private StandardConnectorRepository createRepositoryWithProvider(final ConnectorConfigurationProvider provider) { + final StandardConnectorRepository repository = new StandardConnectorRepository(); + final ConnectorRepositoryInitializationContext initContext = mock(ConnectorRepositoryInitializationContext.class); + when(initContext.getExtensionManager()).thenReturn(mock(ExtensionManager.class)); + when(initContext.getAssetManager()).thenReturn(mock(AssetManager.class)); + when(initContext.getConnectorConfigurationProvider()).thenReturn(provider); + repository.initialize(initContext); + return repository; + } + + private ConnectorNode createSimpleConnectorNode(final String id, final String name) { + final ConnectorNode connector = mock(ConnectorNode.class); + when(connector.getIdentifier()).thenReturn(id); + when(connector.getName()).thenReturn(name); + return connector; + } + + private ConnectorNode createConnectorNodeWithWorkingConfig(final String id, final String name, final MutableConnectorConfigurationContext workingConfigContext) { + final ConnectorNode connector = mock(ConnectorNode.class); + when(connector.getIdentifier()).thenReturn(id); + when(connector.getName()).thenReturn(name); + + final FrameworkFlowContext workingFlowContext = mock(FrameworkFlowContext.class); + when(workingFlowContext.getConfigurationContext()).thenReturn(workingConfigContext); + when(connector.getWorkingFlowContext()).thenReturn(workingFlowContext); + + return connector; + } + + private ConnectorNode createConnectorNodeWithEmptyWorkingConfig(final String id, final String name) { + final ConnectorNode connector = mock(ConnectorNode.class); + when(connector.getIdentifier()).thenReturn(id); + when(connector.getName()).thenReturn(name); + + final MutableConnectorConfigurationContext workingConfigContext = mock(MutableConnectorConfigurationContext.class); + final ConnectorConfiguration emptyConfig = new ConnectorConfiguration(Set.of()); + when(workingConfigContext.toConnectorConfiguration()).thenReturn(emptyConfig); + + final FrameworkFlowContext workingFlowContext = mock(FrameworkFlowContext.class); + when(workingFlowContext.getConfigurationContext()).thenReturn(workingConfigContext); + when(connector.getWorkingFlowContext()).thenReturn(workingFlowContext); + + final FrameworkFlowContext activeFlowContext = mock(FrameworkFlowContext.class); + final MutableConnectorConfigurationContext activeConfigContext = mock(MutableConnectorConfigurationContext.class); + when(activeConfigContext.toConnectorConfiguration()).thenReturn(emptyConfig); + when(activeFlowContext.getConfigurationContext()).thenReturn(activeConfigContext); + when(connector.getActiveFlowContext()).thenReturn(activeFlowContext); + + return connector; + } + + private VersionedConfigurationStep createVersionedStep(final String name, final Map properties) { + final VersionedConfigurationStep step = new VersionedConfigurationStep(); + step.setName(name); + step.setProperties(new HashMap<>(properties)); + return step; + } + + private VersionedConnectorValueReference createStringLiteralRef(final String value) { + final VersionedConnectorValueReference ref = new VersionedConnectorValueReference(); + ref.setValueType("STRING_LITERAL"); + ref.setValue(value); + return ref; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/facades/standalone/TestStandaloneProcessGroupLifecycle.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/facades/standalone/TestStandaloneProcessGroupLifecycle.java new file mode 100644 index 000000000000..6aa4670aa5d5 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/facades/standalone/TestStandaloneProcessGroupLifecycle.java @@ -0,0 +1,292 @@ +/* + * 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.components.connector.facades.standalone; + +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.connector.components.StatelessGroupLifecycle; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.parameter.ParameterLookup; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class TestStandaloneProcessGroupLifecycle { + + private static final String SERVICE_ID = "service-1"; + private static final String SERVICE_ID_2 = "service-2"; + + private static final AllowableValue VALUE_A = new AllowableValue("A", "Value A"); + private static final AllowableValue VALUE_B = new AllowableValue("B", "Value B"); + + private static final PropertyDescriptor MODE_PROPERTY = new PropertyDescriptor.Builder() + .name("mode") + .displayName("Mode") + .allowableValues(VALUE_A, VALUE_B) + .defaultValue(VALUE_A.getValue()) + .required(true) + .build(); + + private static final PropertyDescriptor SERVICE_PROPERTY_NO_DEPENDENCY = new PropertyDescriptor.Builder() + .name("service-no-dep") + .displayName("Service (No Dependency)") + .identifiesControllerService(ControllerService.class) + .build(); + + private static final PropertyDescriptor SERVICE_PROPERTY_WITH_DEPENDENCY = new PropertyDescriptor.Builder() + .name("service-with-dep") + .displayName("Service (With Dependency)") + .identifiesControllerService(ControllerService.class) + .dependsOn(MODE_PROPERTY, VALUE_B) + .build(); + + @Mock + private ProcessGroup processGroup; + + @Mock + private ControllerServiceProvider controllerServiceProvider; + + @Mock + private StatelessGroupLifecycle statelessGroupLifecycle; + + private StandaloneProcessGroupLifecycle lifecycle; + + @BeforeEach + void setUp() { + lifecycle = new StandaloneProcessGroupLifecycle(processGroup, controllerServiceProvider, statelessGroupLifecycle, id -> null); + lenient().when(processGroup.getProcessGroups()).thenReturn(Collections.emptySet()); + } + + @Test + void testFindReferencedServicesIncludesServiceWithNoDependency() { + final ControllerServiceNode serviceNode = mock(ControllerServiceNode.class); + when(serviceNode.getRequiredControllerServices()).thenReturn(Collections.emptyList()); + + final ProcessorNode processor = createProcessorNode( + List.of(SERVICE_PROPERTY_NO_DEPENDENCY), + Map.of(SERVICE_PROPERTY_NO_DEPENDENCY, SERVICE_ID) + ); + + when(processGroup.getProcessors()).thenReturn(List.of(processor)); + when(controllerServiceProvider.getControllerServiceNode(SERVICE_ID)).thenReturn(serviceNode); + + final Set result = lifecycle.findReferencedServices(false); + + assertTrue(result.contains(serviceNode), "Service with no property dependency should be included"); + assertEquals(1, result.size()); + } + + @Test + void testFindReferencedServicesExcludesServiceWhenDependencyNotSatisfied() { + final ProcessorNode processor = createProcessorNode( + List.of(MODE_PROPERTY, SERVICE_PROPERTY_WITH_DEPENDENCY), + Map.of(MODE_PROPERTY, VALUE_A.getValue(), SERVICE_PROPERTY_WITH_DEPENDENCY, SERVICE_ID) + ); + + when(processGroup.getProcessors()).thenReturn(List.of(processor)); + + final Set result = lifecycle.findReferencedServices(false); + + assertTrue(result.isEmpty(), "Service should not be included when property dependency is not satisfied"); + } + + @Test + void testFindReferencedServicesIncludesServiceWhenDependencySatisfied() { + final ControllerServiceNode serviceNode = mock(ControllerServiceNode.class); + when(serviceNode.getRequiredControllerServices()).thenReturn(Collections.emptyList()); + + final ProcessorNode processor = createProcessorNode( + List.of(MODE_PROPERTY, SERVICE_PROPERTY_WITH_DEPENDENCY), + Map.of(MODE_PROPERTY, VALUE_B.getValue(), SERVICE_PROPERTY_WITH_DEPENDENCY, SERVICE_ID) + ); + + when(processGroup.getProcessors()).thenReturn(List.of(processor)); + when(controllerServiceProvider.getControllerServiceNode(SERVICE_ID)).thenReturn(serviceNode); + + final Set result = lifecycle.findReferencedServices(false); + + assertTrue(result.contains(serviceNode), "Service should be included when property dependency is satisfied"); + assertEquals(1, result.size()); + } + + @Test + void testFindReferencedServicesUsesDefaultValueForDependencyCheck() { + // A property that depends on MODE_PROPERTY having VALUE_A (which is the default) + final PropertyDescriptor servicePropertyDependsOnDefault = new PropertyDescriptor.Builder() + .name("service-dep-default") + .displayName("Service (Depends on Default)") + .identifiesControllerService(ControllerService.class) + .dependsOn(MODE_PROPERTY, VALUE_A) + .build(); + + final ControllerServiceNode serviceNode = mock(ControllerServiceNode.class); + when(serviceNode.getRequiredControllerServices()).thenReturn(Collections.emptyList()); + + // Mode property not explicitly set; the default value "A" should satisfy the dependency + final ProcessorNode processor = createProcessorNode( + List.of(MODE_PROPERTY, servicePropertyDependsOnDefault), + Map.of(servicePropertyDependsOnDefault, SERVICE_ID) + ); + + when(processGroup.getProcessors()).thenReturn(List.of(processor)); + when(controllerServiceProvider.getControllerServiceNode(SERVICE_ID)).thenReturn(serviceNode); + + final Set result = lifecycle.findReferencedServices(false); + + assertTrue(result.contains(serviceNode), "Service should be included when dependency property uses default value that satisfies dependency"); + assertEquals(1, result.size()); + } + + @Test + void testFindReferencedServicesTransitiveServiceExcludedWhenDependencyNotSatisfied() { + // Service 1 is directly referenced by the processor (no dependency on service property) + final ControllerServiceNode serviceNode1 = mock(ControllerServiceNode.class); + // getRequiredControllerServices() already filters based on dependencies, so it returns empty + when(serviceNode1.getRequiredControllerServices()).thenReturn(Collections.emptyList()); + + final ProcessorNode processor = createProcessorNode( + List.of(SERVICE_PROPERTY_NO_DEPENDENCY), + Map.of(SERVICE_PROPERTY_NO_DEPENDENCY, SERVICE_ID) + ); + + when(processGroup.getProcessors()).thenReturn(List.of(processor)); + when(controllerServiceProvider.getControllerServiceNode(SERVICE_ID)).thenReturn(serviceNode1); + + final Set result = lifecycle.findReferencedServices(false); + + assertTrue(result.contains(serviceNode1), "Directly referenced service should be included"); + assertEquals(1, result.size(), "Only the directly referenced service should be in the result"); + } + + @Test + void testFindReferencedServicesTransitiveServiceIncludedWhenDependencySatisfied() { + final ControllerServiceNode serviceNode1 = mock(ControllerServiceNode.class); + final ControllerServiceNode serviceNode2 = mock(ControllerServiceNode.class); + // Service 1 requires Service 2 (dependency is satisfied) + when(serviceNode1.getRequiredControllerServices()).thenReturn(List.of(serviceNode2)); + when(serviceNode2.getRequiredControllerServices()).thenReturn(Collections.emptyList()); + + final ProcessorNode processor = createProcessorNode( + List.of(SERVICE_PROPERTY_NO_DEPENDENCY), + Map.of(SERVICE_PROPERTY_NO_DEPENDENCY, SERVICE_ID) + ); + + when(processGroup.getProcessors()).thenReturn(List.of(processor)); + when(controllerServiceProvider.getControllerServiceNode(SERVICE_ID)).thenReturn(serviceNode1); + + final Set result = lifecycle.findReferencedServices(false); + + assertTrue(result.contains(serviceNode1), "Directly referenced service should be included"); + assertTrue(result.contains(serviceNode2), "Transitively required service should be included"); + assertEquals(2, result.size()); + } + + @Test + void testFindReferencedServicesMultiplePropertiesMixedDependencies() { + final ControllerServiceNode serviceNode1 = mock(ControllerServiceNode.class); + when(serviceNode1.getRequiredControllerServices()).thenReturn(Collections.emptyList()); + final ControllerServiceNode serviceNode2 = mock(ControllerServiceNode.class); + + // Processor with two service properties: one with no dependency (service1) and one with unsatisfied dependency (service2) + // Mode is "A", which does NOT satisfy the dependency on VALUE_B for SERVICE_PROPERTY_WITH_DEPENDENCY + final ProcessorNode processor = createProcessorNode( + List.of(MODE_PROPERTY, SERVICE_PROPERTY_NO_DEPENDENCY, SERVICE_PROPERTY_WITH_DEPENDENCY), + Map.of(MODE_PROPERTY, VALUE_A.getValue(), SERVICE_PROPERTY_NO_DEPENDENCY, SERVICE_ID, SERVICE_PROPERTY_WITH_DEPENDENCY, SERVICE_ID_2) + ); + + when(processGroup.getProcessors()).thenReturn(List.of(processor)); + when(controllerServiceProvider.getControllerServiceNode(SERVICE_ID)).thenReturn(serviceNode1); + + final Set result = lifecycle.findReferencedServices(false); + + assertTrue(result.contains(serviceNode1), "Service from property with no dependency should be included"); + assertFalse(result.contains(serviceNode2), "Service from property with unsatisfied dependency should not be included"); + assertEquals(1, result.size()); + } + + /** + * Creates a mock ProcessorNode with the given property descriptors and effective property values. + * The mock is configured with a ValidationContext whose isDependencySatisfied calls the real default + * implementation, using property values derived from the provided effective values. + */ + private ProcessorNode createProcessorNode(final List descriptors, + final Map effectiveValues) { + final ProcessorNode processor = mock(ProcessorNode.class); + when(processor.getPropertyDescriptors()).thenReturn(descriptors); + + for (final PropertyDescriptor descriptor : descriptors) { + lenient().when(processor.getPropertyDescriptor(descriptor.getName())).thenReturn(descriptor); + lenient().when(processor.getEffectivePropertyValue(descriptor)).thenReturn(effectiveValues.get(descriptor)); + } + + // Build the effective property value map for creating the validation context + final Map effectivePropertyValues = new LinkedHashMap<>(); + for (final PropertyDescriptor descriptor : descriptors) { + final String value = effectiveValues.get(descriptor); + if (value != null) { + effectivePropertyValues.put(descriptor, value); + } + } + when(processor.getEffectivePropertyValues()).thenReturn(effectivePropertyValues); + + // Set up a ValidationContext that delegates isDependencySatisfied to the real default implementation + final ValidationContext validationContext = mock(ValidationContext.class); + when(validationContext.isDependencySatisfied(any(PropertyDescriptor.class), any(Function.class))).thenCallRealMethod(); + + // Mock getProperty() so the real isDependencySatisfied default method can resolve property values + for (final PropertyDescriptor descriptor : descriptors) { + final String value = effectiveValues.get(descriptor); + if (value != null) { + final PropertyValue propertyValue = mock(PropertyValue.class); + lenient().when(propertyValue.getValue()).thenReturn(value); + lenient().when(validationContext.getProperty(descriptor)).thenReturn(propertyValue); + } + } + + lenient().when(processor.getAnnotationData()).thenReturn(null); + lenient().when(processor.getParameterLookup()).thenReturn(ParameterLookup.EMPTY); + when(processor.createValidationContext(any(Map.class), any(), any(ParameterLookup.class), anyBoolean())).thenReturn(validationContext); + + return processor; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/CreateDummyFlowFile.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/CreateDummyFlowFile.java new file mode 100644 index 000000000000..98ead292e1b6 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/CreateDummyFlowFile.java @@ -0,0 +1,84 @@ +/* + * 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.components.connector.processors; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.connector.services.CounterService; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Set; + +@InputRequirement(Requirement.INPUT_FORBIDDEN) +public class CreateDummyFlowFile extends AbstractProcessor { + + static final PropertyDescriptor TEXT = new PropertyDescriptor.Builder() + .name("Text") + .description("The text to write to the FlowFile content") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + static final PropertyDescriptor COUNT_SERVICE = new PropertyDescriptor.Builder() + .name("Counter Service") + .description("The Counter Service to use for incrementing a counter each time a FlowFile is created") + .required(false) + .identifiesControllerService(CounterService.class) + .build(); + + private static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("All FlowFiles are routed to this relationship") + .build(); + + @Override + protected List getSupportedPropertyDescriptors() { + return List.of(TEXT); + } + + @Override + public Set getRelationships() { + return Set.of(REL_SUCCESS); + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + FlowFile flowFile = session.create(); + final String text = context.getProperty(TEXT).getValue(); + if (text != null) { + flowFile = session.write(flowFile, out -> out.write(text.getBytes(StandardCharsets.UTF_8))); + } + + final CounterService service = context.getProperty(COUNT_SERVICE).asControllerService(CounterService.class); + if (service != null) { + final long count = service.increment(); + flowFile = session.putAttribute(flowFile, "counter.value", String.valueOf(count)); + } + + session.transfer(flowFile, REL_SUCCESS); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/DuplicateFlowFile.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/DuplicateFlowFile.java new file mode 100644 index 000000000000..34c438ba61a3 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/DuplicateFlowFile.java @@ -0,0 +1,128 @@ +/* + * 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.components.connector.processors; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +@Tags({"test", "duplicate", "connector"}) +@InputRequirement(Requirement.INPUT_REQUIRED) +@CapabilityDescription("Duplicates incoming FlowFiles to numbered relationships based on the configured number of duplicates. " + + "Each duplicate is sent to a relationship numbered 1, 2, 3, etc., up to the configured number.") +public class DuplicateFlowFile extends AbstractProcessor { + + public static final PropertyDescriptor NUM_DUPLICATES = new PropertyDescriptor.Builder() + .name("Number of Duplicates") + .description("Specifies how many duplicates of each incoming FlowFile will be created") + .required(true) + .defaultValue("3") + .addValidator(StandardValidators.createLongValidator(1L, 10L, true)) + .build(); + + private static final List PROPERTY_DESCRIPTORS = List.of(NUM_DUPLICATES); + + private final AtomicReference> relationshipsRef = new AtomicReference<>(); + private final AtomicReference> sortedRelationshipsRef = new AtomicReference<>(); + + @Override + protected List getSupportedPropertyDescriptors() { + return PROPERTY_DESCRIPTORS; + } + + @Override + public Set getRelationships() { + final Set relationships = relationshipsRef.get(); + return relationships == null ? Set.of() : relationships; + } + + @Override + public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { + if (NUM_DUPLICATES.equals(descriptor)) { + updateRelationships(newValue); + } + } + + private void updateRelationships(final String numDuplicatesValue) { + if (numDuplicatesValue == null) { + relationshipsRef.set(Set.of()); + sortedRelationshipsRef.set(List.of()); + return; + } + + try { + final int numDuplicates = Integer.parseInt(numDuplicatesValue); + final Set relationships = new HashSet<>(); + final List sortedRelationships = new ArrayList<>(); + + for (int i = 1; i <= numDuplicates; i++) { + final Relationship relationship = new Relationship.Builder() + .name(String.valueOf(i)) + .description("Relationship for duplicate " + i) + .build(); + relationships.add(relationship); + sortedRelationships.add(relationship); + } + + relationshipsRef.set(relationships); + sortedRelationshipsRef.set(sortedRelationships); + } catch (final NumberFormatException e) { + relationshipsRef.set(Set.of()); + sortedRelationshipsRef.set(List.of()); + } + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; + } + + final List sortedRelationships = sortedRelationshipsRef.get(); + + if (sortedRelationships == null || sortedRelationships.isEmpty()) { + session.rollback(); + return; + } + + // Create duplicates and send to numbered relationships + for (final Relationship relationship : sortedRelationships) { + final FlowFile duplicate = session.clone(flowFile); + session.transfer(duplicate, relationship); + } + + // Remove the original FlowFile + session.remove(flowFile); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/ExposeFileValues.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/ExposeFileValues.java new file mode 100644 index 000000000000..154e3d2b5fe5 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/ExposeFileValues.java @@ -0,0 +1,72 @@ +/* + * 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.components.connector.processors; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.connector.components.ComponentState; +import org.apache.nifi.components.connector.components.ConnectorMethod; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.List; + +public class ExposeFileValues extends AbstractProcessor { + + static final PropertyDescriptor FILE = new PropertyDescriptor.Builder() + .name("File") + .description("The name of the file whose contents will be exposed via a ConnectorMethod.") + .required(true) + .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR) + .build(); + + private volatile String filename; + + @Override + protected List getSupportedPropertyDescriptors() { + return List.of(FILE); + } + + @Override + public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { + if (descriptor.equals(FILE)) { + this.filename = newValue; + } + } + + @ConnectorMethod( + name = "getFileValues", + description = "Reads all lines from the configured file and returns them as a List of String.", + allowedStates = ComponentState.STOPPED + ) + public List getFileValues() throws IOException { + final File file = new File(filename); + return Files.readAllLines(file.toPath(), StandardCharsets.UTF_8); + } + + @Override + public void onTrigger(final ProcessContext processContext, final ProcessSession processSession) throws ProcessException { + + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/LogFlowFileContents.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/LogFlowFileContents.java new file mode 100644 index 000000000000..795387601924 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/LogFlowFileContents.java @@ -0,0 +1,58 @@ +/* + * 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.components.connector.processors; + +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; + +import java.util.Set; + +public class LogFlowFileContents extends AbstractProcessor { + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("FlowFiles that have been successfully overwritten") + .build(); + + private static final Set RELATIONSHIPS = Set.of(REL_SUCCESS); + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; + } + + session.read(flowFile, in -> { + final byte[] contents = in.readAllBytes(); + final String contentStr = new String(contents); + getLogger().info("FlowFile Content:\n{}", contentStr); + }); + + session.transfer(flowFile, REL_SUCCESS); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/OnPropertyModifiedTracker.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/OnPropertyModifiedTracker.java new file mode 100644 index 000000000000..ad8de1eda663 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/OnPropertyModifiedTracker.java @@ -0,0 +1,116 @@ +/* + * 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.components.connector.processors; + +import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +/** + * A test processor that tracks calls to onPropertyModified when the Configured Number property changes. + * This is used to verify that onPropertyModified is called correctly when a Connector's applyUpdate + * changes a parameter value. + */ +public class OnPropertyModifiedTracker extends AbstractProcessor { + + static final PropertyDescriptor CONFIGURED_NUMBER = new PropertyDescriptor.Builder() + .name("Configured Number") + .displayName("Configured Number") + .description("A number property that is tracked for changes via onPropertyModified") + .required(false) + .addValidator(StandardValidators.INTEGER_VALIDATOR) + .build(); + + static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("All FlowFiles are routed here") + .build(); + + private volatile boolean configurationRestored = false; + private final List propertyChanges = Collections.synchronizedList(new ArrayList<>()); + + @Override + protected List getSupportedPropertyDescriptors() { + return List.of(CONFIGURED_NUMBER); + } + + @Override + public Set getRelationships() { + return Set.of(REL_SUCCESS); + } + + @OnConfigurationRestored + public void onConfigurationRestored() { + this.configurationRestored = true; + } + + @Override + public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { + if (!CONFIGURED_NUMBER.getName().equals(descriptor.getName())) { + return; + } + + if (!configurationRestored) { + return; + } + + if (oldValue == null) { + getLogger().info("Property [{}] initialized to [{}]", descriptor.getName(), newValue); + return; + } + + getLogger().info("Property [{}] changed from [{}] to [{}]", descriptor.getName(), oldValue, newValue); + propertyChanges.add(new PropertyChange(oldValue, newValue)); + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; + } + + session.transfer(flowFile, REL_SUCCESS); + } + + public List getPropertyChanges() { + return new ArrayList<>(propertyChanges); + } + + public int getPropertyChangeCount() { + return propertyChanges.size(); + } + + public void clearPropertyChanges() { + propertyChanges.clear(); + } + + public record PropertyChange(String oldValue, String newValue) { + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/OverwriteFlowFile.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/OverwriteFlowFile.java new file mode 100644 index 000000000000..83632081e46e --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/OverwriteFlowFile.java @@ -0,0 +1,92 @@ +/* + * 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.components.connector.processors; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.expression.ExpressionLanguageScope; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.io.OutputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Set; + +@Tags({"test", "overwrite", "content", "connector"}) +@InputRequirement(Requirement.INPUT_REQUIRED) +@CapabilityDescription("Overwrites the content of incoming FlowFiles with a configurable string value and transfers them to the success relationship.") +public class OverwriteFlowFile extends AbstractProcessor { + + public static final PropertyDescriptor CONTENT = new PropertyDescriptor.Builder() + .name("Content") + .description("The content to write to the FlowFile") + .required(true) + .defaultValue("Hello World") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) + .build(); + + private static final List PROPERTY_DESCRIPTORS = List.of(CONTENT); + + public static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("FlowFiles that have been successfully overwritten") + .build(); + + private static final Set RELATIONSHIPS = Set.of(REL_SUCCESS); + + @Override + protected List getSupportedPropertyDescriptors() { + return PROPERTY_DESCRIPTORS; + } + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; + } + + final String content = context.getProperty(CONTENT).evaluateAttributeExpressions(flowFile).getValue(); + + flowFile = session.write(flowFile, new OutputStreamCallback() { + @Override + public void process(final OutputStream out) throws IOException { + out.write(content.getBytes(StandardCharsets.UTF_8)); + } + }); + + session.transfer(flowFile, REL_SUCCESS); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/Sleep.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/Sleep.java new file mode 100644 index 000000000000..f936ece3f2f8 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/Sleep.java @@ -0,0 +1,151 @@ +/* + * 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.components.connector.processors; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.expression.ExpressionLanguageScope; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.util.StandardValidators; + +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +@Tags({"test", "sleep", "delay", "timing"}) +@CapabilityDescription("Test processor that introduces configurable delays during different lifecycle events. " + + "Can sleep during OnScheduled, OnStopped, and/or onTrigger methods.") +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) +@SideEffectFree +public class Sleep extends AbstractProcessor { + + static final PropertyDescriptor SLEEP_DURATION = new PropertyDescriptor.Builder() + .name("Sleep Duration") + .displayName("Sleep Duration") + .description("Length of time to sleep when enabled.") + .required(true) + .defaultValue("5 sec") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .expressionLanguageSupported(ExpressionLanguageScope.NONE) + .build(); + + static final PropertyDescriptor SLEEP_ON_SCHEDULED = new PropertyDescriptor.Builder() + .name("Sleep On Scheduled") + .displayName("Sleep On Scheduled") + .description("Sleep during OnScheduled when enabled.") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .build(); + + static final PropertyDescriptor SLEEP_ON_TRIGGER = new PropertyDescriptor.Builder() + .name("Sleep On Trigger") + .displayName("Sleep On Trigger") + .description("Sleep during onTrigger when enabled.") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .build(); + + static final PropertyDescriptor SLEEP_ON_STOPPED = new PropertyDescriptor.Builder() + .name("Sleep On Stopped") + .displayName("Sleep On Stopped") + .description("Sleep during OnStopped when enabled.") + .required(true) + .allowableValues("true", "false") + .defaultValue("false") + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .build(); + + static final Relationship REL_SUCCESS = new Relationship.Builder() + .name("success") + .description("All FlowFiles are routed to success after optional sleeping.") + .build(); + + private static final List DESCRIPTORS = List.of( + SLEEP_DURATION, + SLEEP_ON_SCHEDULED, + SLEEP_ON_TRIGGER, + SLEEP_ON_STOPPED + ); + + private static final Set RELATIONSHIPS = Set.of(REL_SUCCESS); + + @Override + public Set getRelationships() { + return RELATIONSHIPS; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return DESCRIPTORS; + } + + @OnScheduled + public void onScheduled(final ProcessContext context) { + if (context.getProperty(SLEEP_ON_SCHEDULED).asBoolean()) { + sleep(context); + } + } + + @OnStopped + public void onStopped(final ProcessContext context) { + if (context.getProperty(SLEEP_ON_STOPPED).asBoolean()) { + sleep(context); + } + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) { + if (context.getProperty(SLEEP_ON_TRIGGER).asBoolean()) { + sleep(context); + } + + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; + } + + session.transfer(flowFile, REL_SUCCESS); + } + + private void sleep(final ProcessContext context) { + final long durationMillis = context.getProperty(SLEEP_DURATION).asTimePeriod(TimeUnit.MILLISECONDS); + final ComponentLog logger = getLogger(); + try { + if (durationMillis > 0) { + Thread.sleep(durationMillis); + } + } catch (final InterruptedException interruptedException) { + Thread.currentThread().interrupt(); + logger.warn("Sleep processor interrupted while sleeping for {} ms", durationMillis); + } + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/TerminateFlowFile.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/TerminateFlowFile.java new file mode 100644 index 000000000000..b507b8071dc5 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/processors/TerminateFlowFile.java @@ -0,0 +1,37 @@ +/* + * 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.components.connector.processors; + +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; + +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) +public class TerminateFlowFile extends AbstractProcessor { + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if (flowFile != null) { + session.remove(flowFile); + } + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/services/CounterService.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/services/CounterService.java new file mode 100644 index 000000000000..fe9cebad30f0 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/services/CounterService.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.components.connector.services; + +import org.apache.nifi.controller.ControllerService; + +public interface CounterService extends ControllerService { + long increment(); + + long getCount(); +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/services/impl/StandardCounterService.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/services/impl/StandardCounterService.java new file mode 100644 index 000000000000..99515f450116 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/components/connector/services/impl/StandardCounterService.java @@ -0,0 +1,37 @@ +/* + * 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.components.connector.services.impl; + +import org.apache.nifi.components.connector.services.CounterService; +import org.apache.nifi.controller.AbstractControllerService; + +import java.util.concurrent.atomic.AtomicLong; + +public class StandardCounterService extends AbstractControllerService implements CounterService { + private final AtomicLong counter = new AtomicLong(); + + @Override + public long increment() { + return counter.incrementAndGet(); + } + + @Override + public long getCount() { + return counter.get(); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/MockStateManagerProvider.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/MockStateManagerProvider.java new file mode 100644 index 000000000000..e0766c2b3ee0 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/MockStateManagerProvider.java @@ -0,0 +1,72 @@ +/* + * 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.components.state.Scope; +import org.apache.nifi.components.state.StateManager; +import org.apache.nifi.components.state.StateManagerProvider; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.controller.state.StandardStateMap; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.Collections; +import java.util.Optional; + +import static org.mockito.ArgumentMatchers.any; + +public class MockStateManagerProvider implements StateManagerProvider { + @Override + public StateManager getStateManager(final String componentId) { + final StateManager stateManager = Mockito.mock(StateManager.class); + final StateMap emptyStateMap = new StandardStateMap(Collections.emptyMap(), Optional.empty()); + try { + Mockito.when(stateManager.getState(any(Scope.class))).thenReturn(emptyStateMap); + } catch (IOException e) { + throw new AssertionError(); + } + + return stateManager; + } + + @Override + public StateManager getStateManager(final String componentId, final boolean dropStateKeySupported) { + return getStateManager(componentId); + } + + @Override + public void shutdown() { + } + + @Override + public void enableClusterProvider() { + } + + @Override + public void disableClusterProvider() { + } + + @Override + public boolean isClusterProviderEnabled() { + return false; + } + + @Override + public void onComponentRemoved(final String componentId) { + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/MockSwapManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/MockSwapManager.java index 90a0dfbb4a45..3c390be62e43 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/MockSwapManager.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/MockSwapManager.java @@ -173,6 +173,11 @@ public void purge() { swappedOut.clear(); } + @Override + public void deleteSwapFile(final String swapLocation) throws IOException { + swappedOut.remove(swapLocation); + } + @Override public String getQueueIdentifier(final String swapLocation) { return null; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardFlowFileQueue.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardFlowFileQueue.java index 8a6469f3e353..e1aac575a0a9 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardFlowFileQueue.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardFlowFileQueue.java @@ -17,6 +17,7 @@ package org.apache.nifi.controller; +import org.apache.nifi.components.connector.DropFlowFileSummary; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connection; import org.apache.nifi.controller.queue.DropFlowFileState; @@ -27,6 +28,8 @@ import org.apache.nifi.controller.queue.StandardFlowFileQueue; import org.apache.nifi.controller.repository.FlowFileRecord; import org.apache.nifi.controller.repository.FlowFileRepository; +import org.apache.nifi.controller.repository.RepositoryRecord; +import org.apache.nifi.controller.repository.RepositoryRecordType; import org.apache.nifi.processor.FlowFileFilter; import org.apache.nifi.processor.FlowFileFilter.FlowFileFilterResult; import org.apache.nifi.provenance.ProvenanceEventRecord; @@ -58,19 +61,20 @@ public class TestStandardFlowFileQueue { private MockSwapManager swapManager = null; private StandardFlowFileQueue queue = null; - private Connection connection = null; private FlowFileRepository flowFileRepo = null; private ProvenanceEventRepository provRepo = null; private ProcessScheduler scheduler = null; - private List provRecords = new ArrayList<>(); + private final List provRecords = new ArrayList<>(); + private final List repoRecords = new ArrayList<>(); @BeforeEach @SuppressWarnings("unchecked") - public void setup() { + public void setup() throws Exception { provRecords.clear(); + repoRecords.clear(); - connection = Mockito.mock(Connection.class); + final Connection connection = Mockito.mock(Connection.class); Mockito.when(connection.getSource()).thenReturn(Mockito.mock(Connectable.class)); Mockito.when(connection.getDestination()).thenReturn(Mockito.mock(Connectable.class)); @@ -89,6 +93,12 @@ public void setup() { return null; }).when(provRepo).registerEvents(Mockito.any(Iterable.class)); + Mockito.doAnswer((Answer) invocation -> { + final Collection records = (Collection) invocation.getArguments()[0]; + repoRecords.addAll(records); + return null; + }).when(flowFileRepo).updateRepository(Mockito.any(Collection.class)); + queue = new StandardFlowFileQueue("id", flowFileRepo, provRepo, scheduler, swapManager, null, 10000, "0 sec", 0L, "0 B"); MockFlowFileRecord.resetIdGenerator(); } @@ -338,24 +348,25 @@ public void testSwapIn() { @Test public void testSwapInWhenThresholdIsLessThanSwapSize() { // create a queue where the swap threshold is less than 10k + // With threshold of 1000, swapRecordPollSize will also be 1000 (min of 10000 and threshold) queue = new StandardFlowFileQueue("id", flowFileRepo, provRepo, scheduler, swapManager, null, 1000, "0 sec", 0L, "0 B"); for (int i = 1; i <= 20000; i++) { queue.put(new MockFlowFileRecord()); } - assertEquals(1, swapManager.swappedOut.size()); + // With swapRecordPollSize = 1000, we expect 19 swap files (19,000 FlowFiles) + assertEquals(19, swapManager.swappedOut.size()); queue.put(new MockFlowFileRecord()); - assertEquals(1, swapManager.swappedOut.size()); + assertEquals(19, swapManager.swappedOut.size()); final Set exp = new HashSet<>(); // At this point there should be: - // 1k FlowFiles in the active queue - // 9,001 FlowFiles in the swap queue - // 10k FlowFiles swapped to disk - - for (int i = 0; i < 999; i++) { // + // 1k flow files in the active queue + // 1 flow file in the swap queue + // 19k flow files swapped to disk (19 swap files with 1k each) + for (int i = 0; i < 999; i++) { final FlowFileRecord flowFile = queue.poll(exp); assertNotNull(flowFile); assertEquals(1, queue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getUnacknowledgedQueueSize().getObjectCount()); @@ -373,13 +384,13 @@ public void testSwapInWhenThresholdIsLessThanSwapSize() { assertEquals(0, swapManager.swapInCalledCount); assertEquals(0, queue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getActiveQueueSize().getObjectCount()); - assertEquals(1, swapManager.swapOutCalledCount); + assertEquals(19, swapManager.swapOutCalledCount); - assertNotNull(queue.poll(exp)); // this should trigger a swap-in of 10,000 records, and then pull 1 off the top. + assertNotNull(queue.poll(exp)); // this should trigger a swap-in of 1,000 records, and then pull 1 off the top. assertEquals(1, swapManager.swapInCalledCount); - assertEquals(9999, queue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getActiveQueueSize().getObjectCount()); + assertEquals(999, queue.getQueueDiagnostics().getLocalQueuePartitionDiagnostics().getActiveQueueSize().getObjectCount()); - assertTrue(swapManager.swappedOut.isEmpty()); + assertEquals(18, swapManager.swappedOut.size()); queue.poll(exp); } @@ -609,4 +620,81 @@ public void testGetMinLastQueueDate() { assertEquals(500, now - queue.getMinLastQueueDate()); } + + @Test + public void testSelectiveDropCreatesDeleteRecords() throws Exception { + for (int i = 0; i < 10; i++) { + queue.put(new MockFlowFileRecord(i)); + } + + final DropFlowFileSummary summary = queue.dropFlowFiles(ff -> ff.getSize() < 5); + assertEquals(5, summary.getDroppedCount()); + + final long deleteRecordCount = repoRecords.stream().filter(r -> r.getType() == RepositoryRecordType.DELETE).count(); + assertEquals(5, deleteRecordCount); + + assertEquals(5, provRecords.size()); + for (final ProvenanceEventRecord event : provRecords) { + assertEquals(ProvenanceEventType.DROP, event.getEventType()); + } + + assertEquals(5, queue.size().getObjectCount()); + } + + @Test + public void testSelectiveDropWithSwappedFlowFilesCreatesSwapFileRenamedRecords() throws Exception { + for (int i = 0; i < 20000; i++) { + queue.put(new MockFlowFileRecord(i % 10)); + } + + assertEquals(1, swapManager.swappedOut.size()); + + repoRecords.clear(); + + final DropFlowFileSummary summary = queue.dropFlowFiles(ff -> ff.getSize() < 5); + assertEquals(10000, summary.getDroppedCount()); + + final long deleteRecordCount = repoRecords.stream().filter(r -> r.getType() == RepositoryRecordType.DELETE).count(); + assertEquals(10000, deleteRecordCount); + + final long swapFileRenamedCount = repoRecords.stream().filter(r -> r.getType() == RepositoryRecordType.SWAP_FILE_RENAMED).count(); + assertEquals(1, swapFileRenamedCount); + + final RepositoryRecord swapRenamedRecord = repoRecords.stream() + .filter(r -> r.getType() == RepositoryRecordType.SWAP_FILE_RENAMED) + .findFirst() + .orElseThrow(); + assertNotNull(swapRenamedRecord.getOriginalSwapLocation()); + assertNotNull(swapRenamedRecord.getSwapLocation()); + + assertEquals(10000, queue.size().getObjectCount()); + } + + @Test + public void testSelectiveDropWithAllSwappedFlowFilesCreatesSwapFileDeletedRecords() throws Exception { + for (int i = 0; i < 20000; i++) { + queue.put(new MockFlowFileRecord(1)); + } + + assertEquals(1, swapManager.swappedOut.size()); + + repoRecords.clear(); + + final DropFlowFileSummary summary = queue.dropFlowFiles(ff -> ff.getSize() == 1); + assertEquals(20000, summary.getDroppedCount()); + + final long deleteRecordCount = repoRecords.stream().filter(r -> r.getType() == RepositoryRecordType.DELETE).count(); + assertEquals(20000, deleteRecordCount); + + final long swapFileDeletedCount = repoRecords.stream().filter(r -> r.getType() == RepositoryRecordType.SWAP_FILE_DELETED).count(); + assertEquals(1, swapFileDeletedCount); + + final RepositoryRecord swapDeletedRecord = repoRecords.stream() + .filter(r -> r.getType() == RepositoryRecordType.SWAP_FILE_DELETED) + .findFirst() + .orElseThrow(); + assertNotNull(swapDeletedRecord.getSwapLocation()); + + assertEquals(0, queue.size().getObjectCount()); + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/flow/NopConnector.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/flow/NopConnector.java new file mode 100644 index 000000000000..c2dc1ab49a89 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/flow/NopConnector.java @@ -0,0 +1,167 @@ +/* + * 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.flow; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.Connector; +import org.apache.nifi.components.connector.ConnectorConfigurationContext; +import org.apache.nifi.components.connector.ConnectorInitializationContext; +import org.apache.nifi.components.connector.ConnectorValidationContext; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.flow.VersionedExternalFlow; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** + * A simple no-op Connector implementation for testing purposes. + * This connector does nothing but provides the basic interface implementation + * required for testing StandardFlowManager's connector creation functionality. + */ +public class NopConnector implements Connector { + + private ConnectorInitializationContext context; + private boolean initialized = false; + private boolean started = false; + private boolean configured = false; + + @Override + public void initialize(final ConnectorInitializationContext context) { + this.context = context; + this.initialized = true; + } + + @Override + public VersionedExternalFlow getInitialFlow() { + return null; + } + + @Override + public void start(final FlowContext activeContext) throws FlowUpdateException { + if (!initialized) { + throw new FlowUpdateException("Connector must be initialized before starting"); + } + started = true; + } + + @Override + public void stop(final FlowContext activeContext) throws FlowUpdateException { + started = false; + } + + @Override + public List validate(final FlowContext activeContext, final ConnectorValidationContext validationContext) { + if (!initialized) { + return List.of(new ValidationResult.Builder() + .subject("Initialization") + .valid(false) + .explanation("Connector has not been initialized") + .build()); + } + + return List.of(new ValidationResult.Builder() + .subject("Test Connector") + .valid(true) + .explanation("Test connector is valid") + .build()); + } + + @Override + public List validateConfigurationStep(final ConfigurationStep configurationStep, final ConnectorConfigurationContext connectorConfigurationContext, + final ConnectorValidationContext connectorValidationContext) { + return List.of(); + } + + @Override + public List getConfigurationSteps() { + return List.of(new ConfigurationStep.Builder() + .name("Test Group") + .description("A test configuration step") + .build()); + } + + @Override + public void onConfigurationStepConfigured(final String stepName, final FlowContext flowContext) { + } + + @Override + public void prepareForUpdate(final FlowContext workingContext, final FlowContext activeContext) { + + } + + @Override + public void abortUpdate(final FlowContext flowContext, final Throwable throwable) { + + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + if (!initialized) { + throw new FlowUpdateException("Connector must be initialized before configuration"); + } + configured = true; + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext flowContext) { + return List.of(); + } + + @Override + public List verify(final FlowContext flowContext) { + return List.of(); + } + + + // Getters for test assertions + public boolean isInitialized() { + return initialized; + } + + public boolean isStarted() { + return started; + } + + public boolean isConfigured() { + return configured; + } + + public ConnectorInitializationContext getContext() { + return context; + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName, final FlowContext workingContext, final String filter) { + return List.of(); + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName, final FlowContext workingContext) { + return List.of(); + } + + @Override + public CompletableFuture drainFlowFiles(final FlowContext flowContext) { + return CompletableFuture.completedFuture(null); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/flow/TestStandardFlowManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/flow/TestStandardFlowManager.java new file mode 100644 index 000000000000..473962890fdd --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/flow/TestStandardFlowManager.java @@ -0,0 +1,174 @@ +/* + * 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.flow; + +import org.apache.nifi.bundle.Bundle; +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.components.connector.Connector; +import org.apache.nifi.components.connector.ConnectorInitializationContext; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorRepository; +import org.apache.nifi.components.connector.ConnectorValidationTrigger; +import org.apache.nifi.components.connector.StandardConnectorInitializationContext; +import org.apache.nifi.components.connector.StandardConnectorStateTransition; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.GarbageCollectionLog; +import org.apache.nifi.controller.MockStateManagerProvider; +import org.apache.nifi.controller.ReloadComponent; +import org.apache.nifi.controller.repository.FlowFileEventRepository; +import org.apache.nifi.controller.repository.FlowFileRepository; +import org.apache.nifi.controller.scheduling.LifecycleStateManager; +import org.apache.nifi.controller.scheduling.RepositoryContextFactory; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.nar.StandardExtensionDiscoveringManager; +import org.apache.nifi.nar.SystemBundle; +import org.apache.nifi.parameter.ParameterContextManager; +import org.apache.nifi.provenance.ProvenanceRepository; +import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.util.NiFiProperties; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.util.Collections; +import javax.net.ssl.SSLContext; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TestStandardFlowManager { + + private StandardFlowManager flowManager; + + @Mock + private NiFiProperties nifiProperties; + + @Mock + private SSLContext sslContext; + + @Mock + private FlowController flowController; + + @Mock + private FlowFileEventRepository flowFileEventRepository; + + @Mock + private ParameterContextManager parameterContextManager; + + @Mock + private ExtensionManager extensionManager; + + @Mock + private BundleCoordinate bundleCoordinate; + + @Mock + private Bundle bundle; + + @BeforeEach + public void setUp() { + MockitoAnnotations.openMocks(this); + + when(flowController.isInitialized()).thenReturn(true); + when(flowController.getExtensionManager()).thenReturn(extensionManager); + when(extensionManager.getBundle(bundleCoordinate)).thenReturn(bundle); + when(bundle.getClassLoader()).thenReturn(NopConnector.class.getClassLoader()); + + flowManager = new StandardFlowManager(nifiProperties, sslContext, flowController, flowFileEventRepository, parameterContextManager); + when(flowController.getFlowManager()).thenReturn(flowManager); + } + + + @Test + public void testCreateConnectorParameterValidation() { + final String validConnectorType = NopConnector.class.getName(); + final String validConnectorId = "test-connector-123"; + + final NullPointerException typeException = assertThrows(NullPointerException.class, () -> flowManager.createConnector(null, validConnectorId, bundleCoordinate, true, true)); + assertEquals("Connector Type", typeException.getMessage()); + + final NullPointerException idException = assertThrows(NullPointerException.class, () -> flowManager.createConnector(validConnectorType, null, bundleCoordinate, true, true)); + assertEquals("Connector ID", idException.getMessage()); + + final NullPointerException bundleException = assertThrows(NullPointerException.class, () -> flowManager.createConnector(validConnectorType, validConnectorId, null, true, true)); + assertEquals("Bundle Coordinate", bundleException.getMessage()); + } + + @Test + public void testCreateConnectorInitializesConnector() { + // Prepare a real discovering manager to provide Connector type + final StandardExtensionDiscoveringManager discoveringManager = new StandardExtensionDiscoveringManager(); + + // Discover from system bundle to pick up test services on classpath + final NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi-with-remote.properties"); + final Bundle systemBundle = SystemBundle.create(properties); + discoveringManager.discoverExtensions(systemBundle, Collections.emptySet()); + + // mock FlowController methods necessary for connector creation + // Most of the methods are used in order to create the ProcessGroup that will be managed by the Connector + when(flowController.getExtensionManager()).thenReturn(discoveringManager); + when(flowController.getStateManagerProvider()).thenReturn(new MockStateManagerProvider()); + final RepositoryContextFactory repositoryContextFactory = mock(RepositoryContextFactory.class); + when(repositoryContextFactory.getFlowFileRepository()).thenReturn(mock(FlowFileRepository.class)); + when(flowController.getRepositoryContextFactory()).thenReturn(repositoryContextFactory); + when(flowController.getGarbageCollectionLog()).thenReturn(mock(GarbageCollectionLog.class)); + when(flowController.getControllerServiceProvider()).thenReturn(mock(ControllerServiceProvider.class)); + when(flowController.getProvenanceRepository()).thenReturn(mock(ProvenanceRepository.class)); + when(flowController.getBulletinRepository()).thenReturn(mock(BulletinRepository.class)); + when(flowController.getLifecycleStateManager()).thenReturn(mock(LifecycleStateManager.class)); + when(flowController.getFlowFileEventRepository()).thenReturn(mock(FlowFileEventRepository.class)); + when(flowController.getReloadComponent()).thenReturn(mock(ReloadComponent.class)); + + final ConnectorValidationTrigger validationTrigger = mock(ConnectorValidationTrigger.class); + when(flowController.getConnectorValidationTrigger()).thenReturn(validationTrigger); + + final ConnectorRepository connectorRepository = mock(ConnectorRepository.class); + when(connectorRepository.createInitializationContextBuilder()).thenAnswer( + invocation -> new StandardConnectorInitializationContext.Builder()); + when(flowController.getConnectorRepository()).thenReturn(connectorRepository); + when(connectorRepository.createStateTransition(anyString(), anyString())).thenReturn(new StandardConnectorStateTransition("test component")); + + // Create the connector + final String type = NopConnector.class.getName(); + final String id = "connector-init-test"; + final ConnectorNode connectorNode = flowManager.createConnector(type, id, systemBundle.getBundleDetails().getCoordinate(), true, false); + + // Verify initialized via context presence + final Connector connector = connectorNode.getConnector(); + assertInstanceOf(NopConnector.class, connector); + + final NopConnector nopConnector = (NopConnector) connector; + assertTrue(nopConnector.isInitialized()); + assertFalse(nopConnector.isStarted()); + assertFalse(nopConnector.isConfigured()); + + final ConnectorInitializationContext initializationContext = nopConnector.getContext(); + assertNotNull(initializationContext.getLogger()); + assertEquals(id, initializationContext.getIdentifier()); + assertEquals(NopConnector.class.getSimpleName(), initializationContext.getName()); + } + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/queue/clustered/TestSocketLoadBalancedFlowFileQueue.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/queue/clustered/TestSocketLoadBalancedFlowFileQueue.java index 3736856ef710..5d108bcf3dba 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/queue/clustered/TestSocketLoadBalancedFlowFileQueue.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/queue/clustered/TestSocketLoadBalancedFlowFileQueue.java @@ -21,6 +21,7 @@ import org.apache.nifi.cluster.coordination.ClusterTopologyEventListener; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.components.connector.DropFlowFileSummary; import org.apache.nifi.connectable.Connection; import org.apache.nifi.controller.MockFlowFileRecord; import org.apache.nifi.controller.MockSwapManager; @@ -33,11 +34,16 @@ import org.apache.nifi.controller.repository.ContentRepository; import org.apache.nifi.controller.repository.FlowFileRecord; import org.apache.nifi.controller.repository.FlowFileRepository; +import org.apache.nifi.controller.repository.RepositoryRecord; +import org.apache.nifi.controller.repository.RepositoryRecordType; import org.apache.nifi.controller.repository.SwapSummary; import org.apache.nifi.controller.status.FlowFileAvailability; import org.apache.nifi.events.EventReporter; import org.apache.nifi.flowfile.FlowFilePrioritizer; +import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventRepository; +import org.apache.nifi.provenance.ProvenanceEventType; +import org.apache.nifi.provenance.StandardProvenanceEventRecord; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; @@ -47,6 +53,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -80,9 +87,16 @@ public class TestSocketLoadBalancedFlowFileQueue { private List nodeIds; private int nodePort = 4096; + private List repoRecords = new ArrayList<>(); + private List provRecords = new ArrayList<>(); + @BeforeEach - public void setup() { + @SuppressWarnings("unchecked") + public void setup() throws Exception { MockFlowFileRecord.resetIdGenerator(); + repoRecords.clear(); + provRecords.clear(); + Connection connection = mock(Connection.class); when(connection.getIdentifier()).thenReturn("unit-test"); @@ -109,6 +123,21 @@ public void setup() { return null; }).when(clusterCoordinator).registerEventListener(Mockito.any(ClusterTopologyEventListener.class)); + when(provRepo.eventBuilder()).thenReturn(new StandardProvenanceEventRecord.Builder()); + doAnswer((Answer) invocation -> { + final Iterable iterable = (Iterable) invocation.getArguments()[0]; + for (final ProvenanceEventRecord record : iterable) { + provRecords.add(record); + } + return null; + }).when(provRepo).registerEvents(Mockito.any(Iterable.class)); + + doAnswer((Answer) invocation -> { + final Collection records = (Collection) invocation.getArguments()[0]; + repoRecords.addAll(records); + return null; + }).when(flowFileRepo).updateRepository(Mockito.any(Collection.class)); + final ProcessScheduler scheduler = mock(ProcessScheduler.class); final AsyncLoadBalanceClientRegistry registry = mock(AsyncLoadBalanceClientRegistry.class); @@ -649,4 +678,90 @@ public boolean isRebalanceOnFailure() { return false; } } + + @Test + public void testSelectiveDropCreatesDeleteRecordsAndProvenanceEvents() throws Exception { + for (int i = 0; i < 10; i++) { + queue.put(new MockFlowFileRecord(i)); + } + + final DropFlowFileSummary summary = queue.dropFlowFiles(ff -> ff.getSize() < 5); + assertEquals(5, summary.getDroppedCount()); + + final long deleteRecordCount = repoRecords.stream().filter(r -> r.getType() == RepositoryRecordType.DELETE).count(); + assertEquals(5, deleteRecordCount); + + assertEquals(5, provRecords.size()); + for (final ProvenanceEventRecord event : provRecords) { + assertEquals(ProvenanceEventType.DROP, event.getEventType()); + } + + assertEquals(5, queue.size().getObjectCount()); + } + + @Test + public void testSelectiveDropWithSwappedFlowFilesCreatesSwapFileRenamedRecords() throws Exception { + for (int i = 0; i < 20000; i++) { + queue.put(new MockFlowFileRecord(i % 10)); + } + + assertEquals(1, swapManager.swappedOut.size()); + + repoRecords.clear(); + provRecords.clear(); + + final DropFlowFileSummary summary = queue.dropFlowFiles(ff -> ff.getSize() < 5); + assertEquals(10000, summary.getDroppedCount()); + + final long deleteRecordCount = repoRecords.stream().filter(r -> r.getType() == RepositoryRecordType.DELETE).count(); + assertEquals(10000, deleteRecordCount); + + final long swapFileRenamedCount = repoRecords.stream().filter(r -> r.getType() == RepositoryRecordType.SWAP_FILE_RENAMED).count(); + assertEquals(1, swapFileRenamedCount); + + final RepositoryRecord swapRenamedRecord = repoRecords.stream() + .filter(r -> r.getType() == RepositoryRecordType.SWAP_FILE_RENAMED) + .findFirst() + .orElseThrow(); + assertNotNull(swapRenamedRecord.getOriginalSwapLocation()); + assertNotNull(swapRenamedRecord.getSwapLocation()); + + assertEquals(10000, provRecords.size()); + for (final ProvenanceEventRecord event : provRecords) { + assertEquals(ProvenanceEventType.DROP, event.getEventType()); + } + + assertEquals(10000, queue.size().getObjectCount()); + } + + @Test + public void testSelectiveDropWithAllSwappedFlowFilesCreatesSwapFileDeletedRecords() throws Exception { + for (int i = 0; i < 20000; i++) { + queue.put(new MockFlowFileRecord(1)); + } + + assertEquals(1, swapManager.swappedOut.size()); + + repoRecords.clear(); + provRecords.clear(); + + final DropFlowFileSummary summary = queue.dropFlowFiles(ff -> ff.getSize() == 1); + assertEquals(20000, summary.getDroppedCount()); + + final long deleteRecordCount = repoRecords.stream().filter(r -> r.getType() == RepositoryRecordType.DELETE).count(); + assertEquals(20000, deleteRecordCount); + + final long swapFileDeletedCount = repoRecords.stream().filter(r -> r.getType() == RepositoryRecordType.SWAP_FILE_DELETED).count(); + assertEquals(1, swapFileDeletedCount); + + final RepositoryRecord swapDeletedRecord = repoRecords.stream() + .filter(r -> r.getType() == RepositoryRecordType.SWAP_FILE_DELETED) + .findFirst() + .orElseThrow(); + assertNotNull(swapDeletedRecord.getSwapLocation()); + + assertEquals(20000, provRecords.size()); + + assertEquals(0, queue.size().getObjectCount()); + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/queue/clustered/TestSwappablePriorityQueue.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/queue/clustered/TestSwappablePriorityQueue.java index 382f2ae35668..05550aaa3b8c 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/queue/clustered/TestSwappablePriorityQueue.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/queue/clustered/TestSwappablePriorityQueue.java @@ -24,6 +24,7 @@ import org.apache.nifi.controller.queue.FlowFileQueue; import org.apache.nifi.controller.queue.PollStrategy; import org.apache.nifi.controller.queue.QueueSize; +import org.apache.nifi.controller.queue.SelectiveDropResult; import org.apache.nifi.controller.queue.SwappablePriorityQueue; import org.apache.nifi.controller.repository.FlowFileRecord; import org.apache.nifi.events.EventReporter; @@ -45,6 +46,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Predicate; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -547,24 +549,25 @@ public void testSwapIn() { @Test public void testSwapInWhenThresholdIsLessThanSwapSize() { // create a queue where the swap threshold is less than 10k + // With threshold of 1000, swapRecordPollSize will also be 1000 (min of 10000 and threshold) queue = new SwappablePriorityQueue(swapManager, 1000, eventReporter, flowFileQueue, dropAction, null); for (int i = 1; i <= 20000; i++) { queue.put(new MockFlowFileRecord()); } - assertEquals(1, swapManager.swappedOut.size()); + // With swapRecordPollSize = 1000, we expect 19 swap files (19,000 FlowFiles) + assertEquals(19, swapManager.swappedOut.size()); queue.put(new MockFlowFileRecord()); - assertEquals(1, swapManager.swappedOut.size()); + assertEquals(19, swapManager.swappedOut.size()); final Set exp = new HashSet<>(); // At this point there should be: - // 1k FlowFiles in the active queue - // 9,001 FlowFiles in the swap queue - // 10k FlowFiles swapped to disk - - for (int i = 0; i < 999; i++) { // + // 1k flow files in the active queue + // 1 flow file in the swap queue + // 19k flow files swapped to disk (19 swap files with 1k each) + for (int i = 0; i < 999; i++) { final FlowFileRecord flowFile = queue.poll(exp, 500000); assertNotNull(flowFile); assertEquals(1, queue.getQueueDiagnostics().getUnacknowledgedQueueSize().getObjectCount()); @@ -582,13 +585,13 @@ public void testSwapInWhenThresholdIsLessThanSwapSize() { assertEquals(0, swapManager.swapInCalledCount); assertEquals(0, queue.getQueueDiagnostics().getActiveQueueSize().getObjectCount()); - assertEquals(1, swapManager.swapOutCalledCount); + assertEquals(19, swapManager.swapOutCalledCount); - assertNotNull(queue.poll(exp, 500000)); // this should trigger a swap-in of 10,000 records, and then pull 1 off the top. + assertNotNull(queue.poll(exp, 500000)); // this should trigger a swap-in of 1,000 records, and then pull 1 off the top. assertEquals(1, swapManager.swapInCalledCount); - assertEquals(9999, queue.getQueueDiagnostics().getActiveQueueSize().getObjectCount()); + assertEquals(999, queue.getQueueDiagnostics().getActiveQueueSize().getObjectCount()); - assertTrue(swapManager.swappedOut.isEmpty()); + assertEquals(18, swapManager.swappedOut.size()); queue.poll(exp, 500000); } @@ -672,6 +675,101 @@ public void testDropSwappedFlowFiles() { assertEquals(2, swapManager.swapInCalledCount); } + @Test + @Timeout(120) + public void testSelectiveDropFromActiveQueue() throws IOException { + final Predicate evenSizePredicate = flowFile -> flowFile.getSize() % 2 == 0; + + for (int i = 0; i < 100; i++) { + queue.put(new MockFlowFileRecord(i)); + } + assertEquals(100, queue.size().getObjectCount()); + + final SelectiveDropResult result = queue.dropFlowFiles(evenSizePredicate); + + assertEquals(50, result.getDroppedCount()); + assertEquals(50, queue.size().getObjectCount()); + + final Set expired = new HashSet<>(); + for (int i = 0; i < 50; i++) { + final FlowFileRecord flowFile = queue.poll(expired, 0L); + assertNotNull(flowFile); + assertEquals(1, flowFile.getSize() % 2); + } + assertNull(queue.poll(expired, 0L)); + } + + @Test + @Timeout(120) + public void testSelectiveDropFromSwapQueue() throws IOException { + final Predicate evenSizePredicate = flowFile -> flowFile.getSize() % 2 == 0; + + for (int i = 0; i < 15000; i++) { + queue.put(new MockFlowFileRecord(i)); + } + assertEquals(15000, queue.size().getObjectCount()); + assertEquals(10000, queue.getQueueDiagnostics().getActiveQueueSize().getObjectCount()); + + final SelectiveDropResult result = queue.dropFlowFiles(evenSizePredicate); + + assertEquals(7500, result.getDroppedCount()); + assertEquals(7500, queue.size().getObjectCount()); + } + + @Test + @Timeout(120) + public void testSelectiveDropFromSwapFile() throws IOException { + final Predicate evenSizePredicate = flowFile -> flowFile.getSize() % 2 == 0; + + for (int i = 0; i < 30000; i++) { + queue.put(new MockFlowFileRecord(i)); + } + assertEquals(30000, queue.size().getObjectCount()); + assertEquals(2, swapManager.swappedOut.size()); + + final SelectiveDropResult result = queue.dropFlowFiles(evenSizePredicate); + + assertEquals(15000, result.getDroppedCount()); + assertEquals(15000, queue.size().getObjectCount()); + assertEquals(2, result.getSwapLocationUpdates().size()); + } + + @Test + @Timeout(120) + public void testSelectiveDropRemovesEntireSwapFileWhenAllMatch() throws IOException { + final Predicate allMatch = flowFile -> true; + + for (int i = 0; i < 30000; i++) { + queue.put(new MockFlowFileRecord(i)); + } + assertEquals(30000, queue.size().getObjectCount()); + assertEquals(2, swapManager.swappedOut.size()); + + final SelectiveDropResult result = queue.dropFlowFiles(allMatch); + + assertEquals(30000, result.getDroppedCount()); + assertEquals(0, queue.size().getObjectCount()); + + long nullNewLocations = result.getSwapLocationUpdates().values().stream().filter(v -> v == null).count(); + assertEquals(2, nullNewLocations); + } + + @Test + @Timeout(120) + public void testSelectiveDropWithNoMatches() throws IOException { + final Predicate noMatch = flowFile -> false; + + for (int i = 0; i < 100; i++) { + queue.put(new MockFlowFileRecord(i)); + } + assertEquals(100, queue.size().getObjectCount()); + + final SelectiveDropResult result = queue.dropFlowFiles(noMatch); + + assertEquals(0, result.getDroppedCount()); + assertEquals(100, queue.size().getObjectCount()); + } + @Test @Timeout(5) public void testGetActiveFlowFilesReturnsAllActiveFlowFiles() throws InterruptedException { @@ -746,6 +844,7 @@ public void testOOMEFollowedBySuccessfulSwapIn() { // To truly test this we need to get both the in-memory swap queue and swap "on disk" involved. public void testLastQueueDateMetrics() throws IOException { Set flowFileRecords = new HashSet<>(11001); + // With threshold of 1000, swapRecordPollSize will also be 1000 queue = new SwappablePriorityQueue(swapManager, 1000, eventReporter, flowFileQueue, dropAction, "testGetMinLastQueueDate"); long minQueueDate = Long.MAX_VALUE; long totalQueueDate = 0L; @@ -759,10 +858,11 @@ public void testLastQueueDateMetrics() throws IOException { } // Assert the queue has a max of active, in-memory swap, and on-disk swap + // With swapRecordPollSize = 1000, we have 10 swap files (10,000 FlowFiles) + 1 in swap queue + 1,000 in active assertEquals(1000, queue.getActiveFlowFiles().size()); assertEquals(10001, queue.getFlowFileQueueSize().getSwappedCount()); - assertEquals(1, queue.getFlowFileQueueSize().getSwapFileCount()); - assertEquals(10000, swapManager.getSwapSummary(swapManager.recoverSwapLocations(flowFileQueue, "testGetMinLastQueueDate").get(0)).getQueueSize().getObjectCount()); + assertEquals(10, queue.getFlowFileQueueSize().getSwapFileCount()); + assertEquals(1000, swapManager.getSwapSummary(swapManager.recoverSwapLocations(flowFileQueue, "testGetMinLastQueueDate").get(0)).getQueueSize().getObjectCount()); // Ensure that the min and totals are correct long now = System.currentTimeMillis(); @@ -773,11 +873,13 @@ public void testLastQueueDateMetrics() throws IOException { List polledRecords = queue.poll(1000, Collections.emptySet(), -1); polledRecords.addAll(queue.poll(2, Collections.emptySet(), -1)); - // Assert that the lone swap file was recovered into memory and that all numbers are as we still expect them to be. - assertEquals(9998, queue.getActiveFlowFiles().size()); - assertEquals(1, queue.getFlowFileQueueSize().getSwappedCount()); - assertEquals(0, queue.getFlowFileQueueSize().getSwapFileCount()); - assertTrue(swapManager.recoverSwapLocations(flowFileQueue, "testGetMinLastQueueDate").isEmpty()); + // Assert that one swap file was recovered into memory and that all numbers are as we still expect them to be. + // After polling 1,002: 1,000 active polled, then swap-in of 1,000 (from first swap file), then 2 more polled + // So: 998 active, 1 in swap queue, 9 swap files remaining + assertEquals(998, queue.getActiveFlowFiles().size()); + assertEquals(9001, queue.getFlowFileQueueSize().getSwappedCount()); + assertEquals(9, queue.getFlowFileQueueSize().getSwapFileCount()); + assertEquals(9, swapManager.recoverSwapLocations(flowFileQueue, "testGetMinLastQueueDate").size()); // Ensure that the min and total are still correct flowFileRecords.removeAll(polledRecords); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java index e11c20be6ac6..936979f79dfa 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java @@ -155,7 +155,7 @@ public void testV1SwapInCantHandleLongAttributeName() throws IOException { } attributes.put(stringBuilder.toString(), "testValue"); StandardRepositoryRecord record = createCreateFlowFileRecord(attributes); - record.setSwapLocation("fake"); + record.setSwapLocation("fake", SWAP_IN); assertEquals(SWAP_IN, record.getType()); schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(record), dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1); @@ -176,7 +176,7 @@ public void testV1SwapInCantHandleLongAttributeValue() throws IOException { } attributes.put("testName", stringBuilder.toString()); StandardRepositoryRecord record = createCreateFlowFileRecord(attributes); - record.setSwapLocation("fake"); + record.setSwapLocation("fake", SWAP_IN); assertEquals(SWAP_IN, record.getType()); schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(record), dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1); @@ -197,7 +197,7 @@ public void testV2SwapInCanHandleLongAttributeName() throws IOException { } attributes.put(stringBuilder.toString(), "testValue"); StandardRepositoryRecord record = createCreateFlowFileRecord(attributes); - record.setSwapLocation("fake"); + record.setSwapLocation("fake", SWAP_IN); assertEquals(SWAP_IN, record.getType()); schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(record), dataOutputStream); @@ -217,7 +217,7 @@ public void testV2SwapInCanHandleLongAttributeValue() throws IOException { } attributes.put("testName", stringBuilder.toString()); StandardRepositoryRecord record = createCreateFlowFileRecord(attributes); - record.setSwapLocation("fake"); + record.setSwapLocation("fake", SWAP_IN); assertEquals(SWAP_IN, record.getType()); schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(record), dataOutputStream); @@ -233,7 +233,7 @@ public void testRoundTripSwapInV1ToV2() throws IOException { Map attributes = new HashMap<>(); attributes.put("testName", "testValue"); StandardRepositoryRecord record = createCreateFlowFileRecord(attributes); - record.setSwapLocation("fake"); + record.setSwapLocation("fake", SWAP_IN); assertEquals(SWAP_IN, record.getType()); schemaRepositoryRecordSerde.serializeRecord(new LiveSerializedRepositoryRecord(record), dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/StandardProcessSessionIT.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/StandardProcessSessionIT.java index 8eb5444e47ea..7835da135301 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/StandardProcessSessionIT.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/StandardProcessSessionIT.java @@ -20,6 +20,7 @@ import org.apache.nifi.components.state.Scope; import org.apache.nifi.components.state.StateMap; import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.connectable.ConnectableFlowFileActivity; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; import org.apache.nifi.controller.BackoffMechanism; @@ -199,6 +200,7 @@ public void setup() throws IOException { when(connectable.getComponentType()).thenReturn("Unit Test Component"); when(connectable.getBackoffMechanism()).thenReturn(BackoffMechanism.PENALIZE_FLOWFILE); when(connectable.getMaxBackoffPeriod()).thenReturn("1 sec"); + when(connectable.getFlowFileActivity()).thenReturn(new ConnectableFlowFileActivity()); Mockito.doAnswer((Answer>) invocation -> { final Object[] arguments = invocation.getArguments(); @@ -213,7 +215,7 @@ public void setup() throws IOException { }).when(connectable).getConnections(Mockito.any(Relationship.class)); when(connectable.getConnections()).thenReturn(new HashSet<>(connList)); - + when(connectable.getFlowFileActivity()).thenReturn(new ConnectableFlowFileActivity()); contentRepo = new MockContentRepository(); contentRepo.initialize(new StandardContentRepositoryContext(new StandardResourceClaimManager(), EventReporter.NO_OP)); flowFileRepo = new MockFlowFileRepository(contentRepo); @@ -2958,6 +2960,7 @@ public Connectable createProcessorConnectable() { when(connectable.getIdentifier()).thenReturn("connectable-1"); when(connectable.getConnectableType()).thenReturn(ConnectableType.PROCESSOR); when(connectable.getComponentType()).thenReturn("Unit Test Component"); + when(connectable.getFlowFileActivity()).thenReturn(new ConnectableFlowFileActivity()); Mockito.doAnswer((Answer>) invocation -> { final Object[] arguments = invocation.getArguments(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java index 9d6118e8ebe6..dcbbd1e80c40 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java @@ -16,6 +16,7 @@ */ package org.apache.nifi.controller.repository; +import org.apache.nifi.components.connector.DropFlowFileSummary; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connection; import org.apache.nifi.controller.MockFlowFileRecord; @@ -39,6 +40,7 @@ import org.apache.nifi.controller.status.FlowFileAvailability; import org.apache.nifi.controller.swap.StandardSwapContents; import org.apache.nifi.controller.swap.StandardSwapSummary; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.processor.FlowFileFilter; import org.apache.nifi.util.MockFlowFile; @@ -69,6 +71,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.function.Predicate; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -287,6 +290,11 @@ public DropFlowFileStatus cancelDropFlowFileRequest(String requestIdentifier) { return null; } + @Override + public DropFlowFileSummary dropFlowFiles(Predicate predicate) { + return new DropFlowFileSummary(0, 0L); + } + @Override public ListFlowFileStatus listFlowFiles(String requestIdentifier, int maxResults) { return null; @@ -314,7 +322,7 @@ public void verifyCanList() throws IllegalStateException { @Override public QueueDiagnostics getQueueDiagnostics() { final FlowFileQueueSize size = new FlowFileQueueSize(size().getObjectCount(), size().getByteCount(), 0, 0, 0, 0, 0); - return new StandardQueueDiagnostics(new StandardLocalQueuePartitionDiagnostics(size, false, false), Collections.emptyList()); + return new StandardQueueDiagnostics(new StandardLocalQueuePartitionDiagnostics(size, false, false, new QueueSize(0, 0), new QueueSize(0, 0)), Collections.emptyList()); } @Override @@ -682,6 +690,126 @@ public void testRestartWithOneRecord() throws IOException { repo2.close(); } + @Test + public void testSelectiveDropWithSwappedFlowFilesAndRecovery() throws IOException { + final Path path = Paths.get("target/test-repo"); + if (Files.exists(path)) { + FileUtils.deleteFile(path.toFile(), true); + } + + final ResourceClaimManager claimManager = new StandardResourceClaimManager(); + final TestQueueProvider queueProvider = new TestQueueProvider(); + final Connection connection = Mockito.mock(Connection.class); + when(connection.getIdentifier()).thenReturn("selective-drop-test"); + when(connection.getDestination()).thenReturn(Mockito.mock(Connectable.class)); + + final MockFlowFileSwapManager swapMgr = new MockFlowFileSwapManager(); + final FlowFileQueue queue = new StandardFlowFileQueue("selective-drop-test", null, null, null, swapMgr, null, 10000, "0 sec", 0L, "0 B"); + when(connection.getFlowFileQueue()).thenReturn(queue); + queueProvider.addConnection(connection); + + final List allFlowFiles = new ArrayList<>(); + final String originalSwapLocation; + final String newSwapLocation; + + try (final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(niFiProperties)) { + repo.initialize(claimManager); + repo.loadFlowFiles(queueProvider); + + for (int i = 0; i < 20; i++) { + final FlowFileRecord flowFile = new StandardFlowFileRecord.Builder() + .id(i + 1) + .addAttribute("uuid", UUID.randomUUID().toString()) + .addAttribute("index", String.valueOf(i)) + .size(i) + .build(); + + final StandardRepositoryRecord record = new StandardRepositoryRecord(queue); + record.setWorking(flowFile, false); + record.setDestination(queue); + repo.updateRepository(Collections.singletonList(record)); + allFlowFiles.add(flowFile); + } + + final List flowFilesToSwap = new ArrayList<>(); + for (int i = 10; i < 20; i++) { + flowFilesToSwap.add(allFlowFiles.get(i)); + } + originalSwapLocation = swapMgr.swapOut(flowFilesToSwap, queue, null); + repo.swapFlowFilesOut(flowFilesToSwap, queue, originalSwapLocation); + + final List selectiveDropRecords = new ArrayList<>(); + + for (int i = 0; i < 5; i++) { + final FlowFileRecord ff = allFlowFiles.get(i); + final StandardRepositoryRecord dropRecord = new StandardRepositoryRecord(queue, ff); + dropRecord.markForDelete(); + selectiveDropRecords.add(dropRecord); + } + + for (int i = 10; i < 15; i++) { + final FlowFileRecord ff = allFlowFiles.get(i); + final StandardRepositoryRecord dropRecord = new StandardRepositoryRecord(queue, ff); + dropRecord.markForDelete(); + selectiveDropRecords.add(dropRecord); + } + + final List remainingSwapped = new ArrayList<>(); + for (int i = 15; i < 20; i++) { + remainingSwapped.add(allFlowFiles.get(i)); + } + newSwapLocation = swapMgr.swapOut(remainingSwapped, queue, null); + + for (final FlowFileRecord ff : remainingSwapped) { + final StandardRepositoryRecord swapRecord = new StandardRepositoryRecord(queue, ff, newSwapLocation); + swapRecord.setDestination(queue); + selectiveDropRecords.add(swapRecord); + } + + final StandardRepositoryRecord swapFileRenamedRecord = new StandardRepositoryRecord(queue); + swapFileRenamedRecord.setSwapFileRenamed(originalSwapLocation, newSwapLocation); + selectiveDropRecords.add(swapFileRenamedRecord); + + repo.updateRepository(selectiveDropRecords); + + swapMgr.deleteSwapFile(originalSwapLocation); + } + + final List recoveredFlowFiles = new ArrayList<>(); + final ResourceClaimManager recoveryClaimManager = new StandardResourceClaimManager(); + + final Connection recoveryConnection = Mockito.mock(Connection.class); + when(recoveryConnection.getIdentifier()).thenReturn("selective-drop-test"); + when(recoveryConnection.getDestination()).thenReturn(Mockito.mock(Connectable.class)); + + final FlowFileQueue recoveryQueue = Mockito.mock(FlowFileQueue.class); + when(recoveryQueue.getIdentifier()).thenReturn("selective-drop-test"); + doAnswer(invocation -> { + recoveredFlowFiles.add((FlowFileRecord) invocation.getArguments()[0]); + return null; + }).when(recoveryQueue).put(any(FlowFileRecord.class)); + + when(recoveryConnection.getFlowFileQueue()).thenReturn(recoveryQueue); + + final TestQueueProvider recoveryQueueProvider = new TestQueueProvider(); + recoveryQueueProvider.addConnection(recoveryConnection); + + try (final WriteAheadFlowFileRepository repo = new WriteAheadFlowFileRepository(niFiProperties)) { + repo.initialize(recoveryClaimManager); + repo.loadFlowFiles(recoveryQueueProvider); + + assertEquals(5, recoveredFlowFiles.size()); + + for (final FlowFileRecord ff : recoveredFlowFiles) { + final int index = Integer.parseInt(ff.getAttribute("index")); + assertTrue(index >= 5 && index < 10, "Expected active FlowFiles with index 5-9, but found index " + index); + } + + assertFalse(repo.isValidSwapLocationSuffix(originalSwapLocation), "Original swap location should not be valid after selective drop"); + assertTrue(repo.isValidSwapLocationSuffix(newSwapLocation), "New swap location should be valid after selective drop"); + } + } + private static class TestQueueProvider implements QueueProvider { private List connectionList = new ArrayList<>(); @@ -795,6 +923,13 @@ public void purge() { this.swappedRecords.clear(); } + @Override + public void deleteSwapFile(final String swapLocation) { + for (final Map> swapMap : swappedRecords.values()) { + swapMap.remove(swapLocation); + } + } + @Override public String getQueueIdentifier(final String swapLocation) { return null; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java index dd2c6ae70e9e..abadd3b0477b 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java @@ -48,6 +48,7 @@ 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.StandardConfigurationContext; import org.apache.nifi.controller.service.StandardControllerServiceProvider; import org.apache.nifi.controller.service.mock.MockProcessGroup; import org.apache.nifi.engine.FlowEngine; @@ -64,6 +65,7 @@ import org.apache.nifi.processor.StandardProcessorInitializationContext; import org.apache.nifi.processor.StandardValidationContextFactory; import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.reporting.AbstractReportingTask; import org.apache.nifi.reporting.InitializationException; import org.apache.nifi.reporting.ReportingContext; @@ -88,6 +90,7 @@ import java.util.Map; import java.util.Random; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; @@ -95,6 +98,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -491,6 +495,51 @@ public void validateNeverEnablingServiceCanStillBeDisabled() throws Exception { assertEquals(0, ts.disableInvocationCount()); } + @Test + @Timeout(10) + public void testEnableControllerServiceWithConfigurationContext() throws Exception { + final ControllerServiceNode serviceNode = flowManager.createControllerService(PropertyTrackingService.class.getName(), + "property-tracking-service", systemBundle.getBundleDetails().getCoordinate(), null, false, true, null); + + rootGroup.addControllerService(serviceNode); + serviceNode.setProperties(Map.of(PropertyTrackingService.TRACKING_PROPERTY.getName(), "original-value")); + serviceNode.performValidation(); + + final ConfigurationContext overrideContext = new StandardConfigurationContext( + serviceNode, Map.of(PropertyTrackingService.TRACKING_PROPERTY.getName(), "overridden-value"), null, + rootGroup.getParameterContext(), serviceProvider, null); + + final CompletableFuture future = scheduler.enableControllerService(serviceNode, overrideContext); + future.get(5, TimeUnit.SECONDS); + + final PropertyTrackingService service = (PropertyTrackingService) serviceNode.getControllerServiceImplementation(); + assertEquals(1, service.enableInvocationCount()); + assertEquals("overridden-value", service.getEnabledPropertyValue()); + assertEquals(ControllerServiceState.ENABLED, serviceNode.getState()); + } + + @Test + @Timeout(10) + public void testEnableControllerServiceWithConfigurationContextUsesOverriddenProperties() throws ExecutionException, InterruptedException, TimeoutException { + final ControllerServiceNode serviceNode = flowManager.createControllerService(PropertyTrackingService.class.getName(), + "property-tracking-service-2", systemBundle.getBundleDetails().getCoordinate(), null, false, true, null); + + rootGroup.addControllerService(serviceNode); + serviceNode.performValidation(); + + final ConfigurationContext validOverrideContext = new StandardConfigurationContext( + serviceNode, Map.of(PropertyTrackingService.TRACKING_PROPERTY.getName(), "override-value"), null, + rootGroup.getParameterContext(), serviceProvider, null); + + final CompletableFuture future = scheduler.enableControllerService(serviceNode, validOverrideContext); + future.get(5, TimeUnit.SECONDS); + + final PropertyTrackingService service = (PropertyTrackingService) serviceNode.getControllerServiceImplementation(); + assertEquals(1, service.enableInvocationCount()); + assertEquals("override-value", service.getEnabledPropertyValue()); + assertEquals(ControllerServiceState.ENABLED, serviceNode.getState()); + } + // Test that if processor throws Exception in @OnScheduled, it keeps getting scheduled @Test @Timeout(10) @@ -626,6 +675,38 @@ public int disableInvocationCount() { } } + public static class PropertyTrackingService extends AbstractControllerService { + public static final PropertyDescriptor TRACKING_PROPERTY = new PropertyDescriptor.Builder() + .name("Tracking Property") + .description("A property for tracking what value was used during enabling") + .required(false) + .defaultValue("default-value") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + private volatile String enabledPropertyValue; + private final AtomicInteger enableCounter = new AtomicInteger(); + + @Override + protected List getSupportedPropertyDescriptors() { + return List.of(TRACKING_PROPERTY); + } + + @OnEnabled + public void enable(final ConfigurationContext context) { + this.enabledPropertyValue = context.getProperty(TRACKING_PROPERTY).getValue(); + this.enableCounter.incrementAndGet(); + } + + public String getEnabledPropertyValue() { + return enabledPropertyValue; + } + + public int enableInvocationCount() { + return enableCounter.get(); + } + } + private StandardProcessScheduler createScheduler() { return new StandardProcessScheduler(new FlowEngine(1, "Unit Test", true), Mockito.mock(FlowController.class), stateMgrProvider, nifiProperties, new StandardLifecycleStateManager()); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizerTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizerTest.java index b01317c83ecc..f526e25cb444 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizerTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizerTest.java @@ -18,6 +18,7 @@ import org.apache.nifi.cluster.protocol.DataFlow; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorRepository; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.ReportingTaskNode; import org.apache.nifi.controller.SnippetManager; @@ -51,6 +52,7 @@ import java.io.File; import java.nio.charset.StandardCharsets; import java.nio.file.Path; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.UUID; @@ -278,5 +280,9 @@ private void setFlowController() { when(flowController.getEncryptor()).thenReturn(encryptor); when(flowController.createVersionedComponentStateLookup(any())).thenReturn(stateLookup); when(flowController.getControllerServiceProvider()).thenReturn(controllerServiceProvider); + + final ConnectorRepository connectorRepository = mock(ConnectorRepository.class); + when(connectorRepository.getConnectors()).thenReturn(Collections.emptyList()); + when(flowController.getConnectorRepository()).thenReturn(connectorRepository); } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java index 0e81a6e7bb98..217caa0a5416 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java @@ -20,10 +20,8 @@ import org.apache.nifi.bundle.Bundle; import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.components.state.Scope; import org.apache.nifi.components.state.StateManager; import org.apache.nifi.components.state.StateManagerProvider; -import org.apache.nifi.components.state.StateMap; import org.apache.nifi.components.validation.ValidationStatus; import org.apache.nifi.components.validation.ValidationTrigger; import org.apache.nifi.components.validation.VerifiableComponentFactory; @@ -31,6 +29,7 @@ import org.apache.nifi.controller.ExtensionBuilder; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.LoggableComponent; +import org.apache.nifi.controller.MockStateManagerProvider; import org.apache.nifi.controller.NodeTypeProvider; import org.apache.nifi.controller.ProcessScheduler; import org.apache.nifi.controller.ProcessorNode; @@ -45,7 +44,6 @@ import org.apache.nifi.controller.service.mock.ServiceA; import org.apache.nifi.controller.service.mock.ServiceB; import org.apache.nifi.controller.service.mock.ServiceC; -import org.apache.nifi.controller.state.StandardStateMap; import org.apache.nifi.engine.FlowEngine; import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.nar.ExtensionDiscoveringManager; @@ -64,14 +62,12 @@ import org.mockito.Mockito; import org.mockito.stubbing.Answer; -import java.io.IOException; import java.net.URL; import java.util.Arrays; import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -92,41 +88,7 @@ public class TestStandardControllerServiceProvider { - private static final StateManagerProvider stateManagerProvider = new StateManagerProvider() { - @Override - public StateManager getStateManager(final String componentId, final boolean dropStateKeySupported) { - final StateManager stateManager = Mockito.mock(StateManager.class); - final StateMap emptyStateMap = new StandardStateMap(Collections.emptyMap(), Optional.empty()); - try { - Mockito.when(stateManager.getState(any(Scope.class))).thenReturn(emptyStateMap); - } catch (IOException e) { - throw new AssertionError(); - } - - return stateManager; - } - - @Override - public void shutdown() { - } - - @Override - public void enableClusterProvider() { - } - - @Override - public void disableClusterProvider() { - } - - @Override - public void onComponentRemoved(final String componentId) { - } - - @Override - public boolean isClusterProviderEnabled() { - return false; - } - }; + private static final StateManagerProvider stateManagerProvider = new MockStateManagerProvider(); private static NiFiProperties niFiProperties; private static ExtensionDiscoveringManager extensionManager; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java index cfd0eeefe552..6beb29d409eb 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java @@ -21,10 +21,12 @@ import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connection; +import org.apache.nifi.connectable.FlowFileActivity; import org.apache.nifi.connectable.Funnel; import org.apache.nifi.connectable.Port; import org.apache.nifi.connectable.Position; import org.apache.nifi.connectable.Positionable; +import org.apache.nifi.connectable.ProcessGroupFlowFileActivity; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.Snippet; import org.apache.nifi.controller.flow.FlowManager; @@ -125,6 +127,11 @@ public void setName(final String name) { } + @Override + public Optional getConnectorIdentifier() { + return Optional.empty(); + } + @Override public void setPosition(final Position position) { @@ -713,7 +720,7 @@ public ComponentAdditions addVersionedComponents(VersionedComponentAdditions add } @Override - public void updateFlow(VersionedExternalFlow proposedFlow, String componentIdSeed, boolean verifyNotDirty, boolean updateSettings, boolean updateDescendantVerisonedFlows) { + public void updateFlow(VersionedExternalFlow proposedFlow, String componentIdSeed, boolean verifyNotDirty, boolean updateSettings, boolean updateDescendantVersionedFlows) { } @Override @@ -903,6 +910,15 @@ public String getStatelessFlowTimeout() { return null; } + @Override + public FlowFileActivity getFlowFileActivity() { + return new ProcessGroupFlowFileActivity(this); + } + + @Override + public void setExplicitParentAuthorizable(final Authorizable parent) { + } + @Override public void setLogFileSuffix(String logFileSuffix) { @@ -912,4 +928,12 @@ public void setLogFileSuffix(String logFileSuffix) { public void terminateProcessor(ProcessorNode processor) { } + @Override + public CompletableFuture purge() { + processorMap.clear(); + serviceMap.clear(); + inputPortMap.clear(); + outputPortMap.clear(); + return CompletableFuture.completedFuture(null); + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestStatelessFlowTask.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestStatelessFlowTask.java index d1c122f2e5b4..5b6335082a88 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestStatelessFlowTask.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/tasks/TestStatelessFlowTask.java @@ -17,8 +17,11 @@ package org.apache.nifi.controller.tasks; +import org.apache.nifi.connectable.ConnectableFlowFileActivity; import org.apache.nifi.connectable.Connection; import org.apache.nifi.connectable.ConnectionUtils.FlowFileCloneResult; +import org.apache.nifi.connectable.FlowFileActivity; +import org.apache.nifi.connectable.FlowFileTransferCounts; import org.apache.nifi.connectable.Port; import org.apache.nifi.controller.MockFlowFileRecord; import org.apache.nifi.controller.queue.FlowFileQueue; @@ -63,6 +66,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.OptionalLong; import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -90,6 +94,8 @@ public class TestStatelessFlowTask { private List statelessProvenanceEvents; private Map flowFileEventsByComponentId; private ProvenanceEventRepository statelessProvRepo; + private FlowFileActivity groupNodeFlowFileActivity; + private StatelessDataflow statelessFlow; @BeforeEach public void setup() throws IOException { @@ -123,10 +129,13 @@ public void setup() throws IOException { return statelessProvenanceEvents.subList((int) startEventId, (int) lastEvent); }).when(statelessProvRepo).getEvents(anyLong(), anyInt()); - final StatelessDataflow statelessFlow = mock(StatelessDataflow.class); + statelessFlow = mock(StatelessDataflow.class); + when(statelessFlow.getLatestActivityTime()).thenReturn(OptionalLong.empty()); final StatelessGroupNode statelessGroupNode = mock(StatelessGroupNode.class); when(statelessGroupNode.getProcessGroup()).thenReturn(rootGroup); + groupNodeFlowFileActivity = new ConnectableFlowFileActivity(); + when(statelessGroupNode.getFlowFileActivity()).thenReturn(groupNodeFlowFileActivity); final FlowFileRepository flowFileRepo = mock(FlowFileRepository.class); @@ -530,6 +539,135 @@ public void testUpdateEventRepositoryWithInputFlowFile() { assertEquals(10, inputPortEvent.getContentSizeOut()); } + + @Test + public void testUpdateFlowFileActivityWithNoEvents() { + task.updateFlowFileActivity(statelessProvRepo); + + final FlowFileTransferCounts counts = groupNodeFlowFileActivity.getTransferCounts(); + assertEquals(0L, counts.getReceivedCount()); + assertEquals(0L, counts.getReceivedBytes()); + assertEquals(0L, counts.getSentCount()); + assertEquals(0L, counts.getSentBytes()); + assertTrue(groupNodeFlowFileActivity.getLatestActivityTime().isEmpty()); + } + + @Test + public void testUpdateFlowFileActivityCountsReceiveAndSendEvents() { + statelessProvenanceEvents.add(createProvenanceEvent(ProvenanceEventType.RECEIVE, 100L)); + statelessProvenanceEvents.add(createProvenanceEvent(ProvenanceEventType.RECEIVE, 200L)); + statelessProvenanceEvents.add(createProvenanceEvent(ProvenanceEventType.SEND, 50L)); + + task.updateFlowFileActivity(statelessProvRepo); + + final FlowFileTransferCounts counts = groupNodeFlowFileActivity.getTransferCounts(); + assertEquals(2L, counts.getReceivedCount()); + assertEquals(300L, counts.getReceivedBytes()); + assertEquals(1L, counts.getSentCount()); + assertEquals(50L, counts.getSentBytes()); + } + + @Test + public void testUpdateFlowFileActivityCountsCreateEvents() { + statelessProvenanceEvents.add(createProvenanceEvent(ProvenanceEventType.CREATE, 75L)); + + task.updateFlowFileActivity(statelessProvRepo); + + final FlowFileTransferCounts counts = groupNodeFlowFileActivity.getTransferCounts(); + assertEquals(1L, counts.getReceivedCount()); + assertEquals(75L, counts.getReceivedBytes()); + assertEquals(0L, counts.getSentCount()); + assertEquals(0L, counts.getSentBytes()); + } + + @Test + public void testUpdateFlowFileActivityCountsFetchBytesButNotCount() { + statelessProvenanceEvents.add(createProvenanceEvent(ProvenanceEventType.FETCH, 500L)); + + task.updateFlowFileActivity(statelessProvRepo); + + final FlowFileTransferCounts counts = groupNodeFlowFileActivity.getTransferCounts(); + assertEquals(0L, counts.getReceivedCount()); + assertEquals(500L, counts.getReceivedBytes()); + assertEquals(0L, counts.getSentCount()); + assertEquals(0L, counts.getSentBytes()); + } + + @Test + public void testUpdateFlowFileActivityIgnoresOtherEventTypes() { + statelessProvenanceEvents.add(createProvenanceEvent(ProvenanceEventType.ATTRIBUTES_MODIFIED, 100L)); + statelessProvenanceEvents.add(createProvenanceEvent(ProvenanceEventType.CONTENT_MODIFIED, 200L)); + statelessProvenanceEvents.add(createProvenanceEvent(ProvenanceEventType.ROUTE, 300L)); + + task.updateFlowFileActivity(statelessProvRepo); + + final FlowFileTransferCounts counts = groupNodeFlowFileActivity.getTransferCounts(); + assertEquals(0L, counts.getReceivedCount()); + assertEquals(0L, counts.getReceivedBytes()); + assertEquals(0L, counts.getSentCount()); + assertEquals(0L, counts.getSentBytes()); + } + + @Test + public void testUpdateFlowFileActivitySetsActivityTimeFromFlow() { + when(statelessFlow.getLatestActivityTime()).thenReturn(OptionalLong.of(System.currentTimeMillis())); + + task.updateFlowFileActivity(statelessProvRepo); + + assertTrue(groupNodeFlowFileActivity.getLatestActivityTime().isPresent()); + } + + @Test + public void testUpdateFlowFileActivityNoActivityTimeWhenFlowReportsNone() { + when(statelessFlow.getLatestActivityTime()).thenReturn(OptionalLong.empty()); + statelessProvenanceEvents.add(createProvenanceEvent(ProvenanceEventType.RECEIVE, 100L)); + + task.updateFlowFileActivity(statelessProvRepo); + + assertTrue(groupNodeFlowFileActivity.getLatestActivityTime().isEmpty()); + } + + @Test + public void testUpdateFlowFileActivityAccumulatesAcrossMultipleCalls() { + statelessProvenanceEvents.add(createProvenanceEvent(ProvenanceEventType.RECEIVE, 100L)); + task.updateFlowFileActivity(statelessProvRepo); + + statelessProvenanceEvents.clear(); + statelessProvenanceEvents.add(createProvenanceEvent(ProvenanceEventType.SEND, 50L)); + task.updateFlowFileActivity(statelessProvRepo); + + final FlowFileTransferCounts counts = groupNodeFlowFileActivity.getTransferCounts(); + assertEquals(1L, counts.getReceivedCount()); + assertEquals(100L, counts.getReceivedBytes()); + assertEquals(1L, counts.getSentCount()); + assertEquals(50L, counts.getSentBytes()); + } + + private ProvenanceEventRecord createProvenanceEvent(final ProvenanceEventType eventType, final long fileSize) { + final StandardProvenanceEventRecord.Builder builder = new StandardProvenanceEventRecord.Builder() + .setEventType(eventType) + .setComponentId("component-1") + .setEventTime(System.currentTimeMillis()) + .setFlowFileUUID("uuid-1") + .setComponentType("Unit Test") + .setCurrentContentClaim(null, null, null, null, fileSize); + + switch (eventType) { + case RECEIVE: + case SEND: + case FETCH: + builder.setTransitUri("http://localhost/test"); + break; + case ROUTE: + builder.setRelationship("success"); + break; + default: + break; + } + + return builder.build(); + } + private FlowFileRecord createFlowFile() { final ResourceClaim resourceClaim = new StandardResourceClaim(resourceClaimManager, "container", "section", "1", false); final ContentClaim contentClaim = new StandardContentClaim(resourceClaim, 0L); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/manifest/StandardRuntimeManifestServiceTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/manifest/StandardRuntimeManifestServiceTest.java index fae3db4f85e7..fe9b3d93ef7c 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/manifest/StandardRuntimeManifestServiceTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/manifest/StandardRuntimeManifestServiceTest.java @@ -31,6 +31,7 @@ import org.apache.nifi.nar.PythonBundle; import org.apache.nifi.processor.Processor; import org.apache.nifi.python.PythonProcessorDetails; +import org.apache.nifi.web.ResourceNotFoundException; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -38,11 +39,14 @@ import java.util.Arrays; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import static java.util.Collections.emptySet; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -163,6 +167,50 @@ public void testGetPythonManifest() { assertEquals(0, controllerServiceDefinitions.size()); } + @Test + public void testDiscoverStepDocumentation() { + when(extensionManager.getBundle(testComponentsBundle.getBundleDetails().getCoordinate())).thenReturn(testComponentsBundle); + + final BundleCoordinate coordinate = testComponentsBundle.getBundleDetails().getCoordinate(); + final Map stepDocs = runtimeManifestService.discoverStepDocumentation( + coordinate.getGroup(), coordinate.getId(), coordinate.getVersion(), "org.example.TestConnector"); + + assertNotNull(stepDocs); + assertEquals(2, stepDocs.size()); + assertTrue(stepDocs.containsKey("Test Step")); + assertTrue(stepDocs.containsKey("Another Test Step")); + + final File testStepFile = stepDocs.get("Test Step"); + assertNotNull(testStepFile); + assertTrue(testStepFile.exists()); + assertEquals("Test_Step.md", testStepFile.getName()); + + final File anotherTestStepFile = stepDocs.get("Another Test Step"); + assertNotNull(anotherTestStepFile); + assertTrue(anotherTestStepFile.exists()); + assertEquals("Another_Test_Step.md", anotherTestStepFile.getName()); + } + + @Test + public void testDiscoverStepDocumentationWithNonExistentConnector() { + when(extensionManager.getBundle(testComponentsBundle.getBundleDetails().getCoordinate())).thenReturn(testComponentsBundle); + + final BundleCoordinate coordinate = testComponentsBundle.getBundleDetails().getCoordinate(); + final Map stepDocs = runtimeManifestService.discoverStepDocumentation( + coordinate.getGroup(), coordinate.getId(), coordinate.getVersion(), "org.example.NonExistentConnector"); + + assertNotNull(stepDocs); + assertTrue(stepDocs.isEmpty()); + } + + @Test + public void testDiscoverStepDocumentationWithNonExistentBundle() { + when(extensionManager.getBundle(new BundleCoordinate("org.example", "nonexistent", "1.0.0"))).thenReturn(null); + + assertThrows(ResourceNotFoundException.class, () -> + runtimeManifestService.discoverStepDocumentation("org.example", "nonexistent", "1.0.0", "org.example.TestConnector")); + } + /** * Override getFrameworkBundle to provide a mocked Bundle. */ diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapperTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/VersionedComponentFlowMapperTest.java similarity index 99% rename from nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapperTest.java rename to nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/VersionedComponentFlowMapperTest.java index 489db2e7c5e1..196fdc8cc646 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapperTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/VersionedComponentFlowMapperTest.java @@ -119,7 +119,7 @@ @ExtendWith(MockitoExtension.class) @MockitoSettings(strictness = Strictness.LENIENT) -public class NiFiRegistryFlowMapperTest { +public class VersionedComponentFlowMapperTest { private static final String PARAMETER_PROVIDER_ID = "id"; @Mock @@ -135,7 +135,7 @@ public class NiFiRegistryFlowMapperTest { @Mock private ParameterProvider parameterProvider; - private final NiFiRegistryFlowMapper flowMapper = new NiFiRegistryFlowMapper(extensionManager); + private final VersionedComponentFlowMapper flowMapper = new VersionedComponentFlowMapper(extensionManager); private int counter = 1; diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.components.connector.Connector b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.components.connector.Connector new file mode 100644 index 000000000000..93fb310b0aca --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.components.connector.Connector @@ -0,0 +1,18 @@ +# 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. + +org.apache.nifi.controller.flow.NopConnector +org.apache.nifi.components.connector.DynamicFlowConnector +org.apache.nifi.components.connector.DynamicAllowableValuesConnector diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.controller.ControllerService index 373a1f8e1d22..d9e0a9a8385c 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.controller.ControllerService +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.controller.ControllerService @@ -12,8 +12,11 @@ # 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. + org.apache.nifi.controller.service.util.TestControllerService org.apache.nifi.controller.service.mock.ServiceA org.apache.nifi.controller.service.mock.ServiceB org.apache.nifi.controller.service.mock.ServiceC -org.apache.nifi.controller.service.mock.ServiceD \ No newline at end of file +org.apache.nifi.controller.service.mock.ServiceD + +org.apache.nifi.components.connector.services.impl.StandardCounterService \ No newline at end of file diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor index 56740ff7c350..ff061c320029 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -12,6 +12,15 @@ # 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. + org.apache.nifi.test.processors.ModifiesClasspathProcessor org.apache.nifi.test.processors.ModifiesClasspathNoAnnotationProcessor -org.apache.nifi.controller.service.mock.DummyProcessor \ No newline at end of file +org.apache.nifi.controller.service.mock.DummyProcessor +org.apache.nifi.components.connector.processors.CreateDummyFlowFile +org.apache.nifi.components.connector.processors.DuplicateFlowFile +org.apache.nifi.components.connector.processors.OverwriteFlowFile +org.apache.nifi.components.connector.processors.TerminateFlowFile +org.apache.nifi.components.connector.processors.LogFlowFileContents +org.apache.nifi.components.connector.processors.ExposeFileValues +org.apache.nifi.components.connector.processors.Sleep +org.apache.nifi.components.connector.processors.OnPropertyModifiedTracker \ No newline at end of file diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/TestRuntimeManifest/nifi-test-components-nar/META-INF/docs/steps/org.example.TestConnector/Another_Test_Step.md b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/TestRuntimeManifest/nifi-test-components-nar/META-INF/docs/steps/org.example.TestConnector/Another_Test_Step.md new file mode 100644 index 000000000000..4673395a0c94 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/TestRuntimeManifest/nifi-test-components-nar/META-INF/docs/steps/org.example.TestConnector/Another_Test_Step.md @@ -0,0 +1,4 @@ +# Another Test Step + +Documentation for another step. + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/TestRuntimeManifest/nifi-test-components-nar/META-INF/docs/steps/org.example.TestConnector/Test_Step.md b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/TestRuntimeManifest/nifi-test-components-nar/META-INF/docs/steps/org.example.TestConnector/Test_Step.md new file mode 100644 index 000000000000..22a6c822b828 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/TestRuntimeManifest/nifi-test-components-nar/META-INF/docs/steps/org.example.TestConnector/Test_Step.md @@ -0,0 +1,8 @@ +# Test Step Documentation + +This is test documentation for the Test Step. + +## Configuration + +Configure the test step properties as needed. + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/colors.txt b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/colors.txt new file mode 100644 index 000000000000..1fedccaeb90b --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/colors.txt @@ -0,0 +1,3 @@ +red +green +blue \ No newline at end of file diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/choose-color.json b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/choose-color.json new file mode 100644 index 000000000000..cd29c728565c --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/choose-color.json @@ -0,0 +1,76 @@ +{ + "flowContents": { + "identifier": "9caa5337-2e56-3711-a344-b29bce96d3e2", + "instanceIdentifier": "77e61da1-0199-1000-ffff-ffff90f30b8d", + "name": "Choose Color", + "comments": "", + "position": { + "x": 61.0, + "y": -181.0 + }, + "processGroups": [], + "remoteProcessGroups": [], + "processors": [ + { + "identifier": "2abf1573-d8ee-38ad-a745-083fdb1a8903", + "instanceIdentifier": "09b902c7-c4b7-3e92-8246-ef0a0bb685e7", + "name": "Choose Color", + "comments": "", + "position": { + "x": -355.5, + "y": -399.0 + }, + "type": "org.apache.nifi.components.connector.processors.ExposeFileValues", + "bundle": { + "group": "default", + "artifact": "system", + "version": "unversioned" + }, + "properties": { + }, + "propertyDescriptors": { + }, + "style": {}, + "schedulingPeriod": "1 min", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 0, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "9caa5337-2e56-3711-a344-b29bce96d3e2" + } + ], + "inputPorts": [], + "outputPorts": [], + "connections": [ + ], + "labels": [], + "funnels": [], + "controllerServices": [], + "defaultFlowFileExpiration": "0 sec", + "defaultBackPressureObjectThreshold": 10000, + "defaultBackPressureDataSizeThreshold": "1 GB", + "scheduledState": "ENABLED", + "executionEngine": "INHERITED", + "maxConcurrentTasks": 1, + "statelessFlowTimeout": "1 min", + "flowFileConcurrency": "UNBOUNDED", + "flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE", + "componentType": "PROCESS_GROUP" + }, + "externalControllerServices": {}, + "parameterContexts": { + }, + "flowEncodingVersion": "1.0", + "parameterProviders": {}, + "latest": false +} \ No newline at end of file diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/generate-and-log-with-parameter.json b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/generate-and-log-with-parameter.json new file mode 100644 index 000000000000..5b46489242e5 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/generate-and-log-with-parameter.json @@ -0,0 +1,253 @@ +{ + "flowContents": { + "identifier": "9caa5337-2e56-3711-a344-b29bce96d3e2", + "instanceIdentifier": "77e61da1-0199-1000-ffff-ffff90f30b8d", + "name": "Generate and Log with Parameter", + "comments": "", + "position": { + "x": 61.0, + "y": -181.0 + }, + "processGroups": [], + "remoteProcessGroups": [], + "processors": [ + { + "identifier": "2abf1573-d8ee-38ad-a745-083fdb1a8903", + "instanceIdentifier": "09b902c7-c4b7-3e92-8246-ef0a0bb685e7", + "name": "CreateDummyFlowFile", + "comments": "", + "position": { + "x": -355.5, + "y": -399.0 + }, + "type": "org.apache.nifi.components.connector.processors.CreateDummyFlowFile", + "bundle": { + "group": "default", + "artifact": "system", + "version": "unversioned" + }, + "properties": { + "Text": "#{Text}" + }, + "propertyDescriptors": { + "Text": { + "name": "Text", + "displayName": "Text", + "identifiesControllerService": false, + "sensitive": true, + "dynamic": false + } + }, + "style": {}, + "schedulingPeriod": "1 min", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 0, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "9caa5337-2e56-3711-a344-b29bce96d3e2" + }, + { + "identifier": "efefbde7-a9e8-3864-b1d2-9649981b39d1", + "instanceIdentifier": "77e69c78-0199-1000-0000-00001e16d8fc", + "name": "Sleep", + "comments": "", + "position": { + "x": -352.0, + "y": -152.0 + }, + "type": "org.apache.nifi.components.connector.processors.Sleep", + "bundle": { + "group": "default", + "artifact": "system", + "version": "unversioned" + }, + "properties": { + "Sleep Duration": "#{Sleep Duration}", + "Sleep On Stopped": "true" + }, + "propertyDescriptors": { + }, + "style": {}, + "schedulingPeriod": "0 sec", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 25, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "9caa5337-2e56-3711-a344-b29bce96d3e2" + }, + { + "identifier": "3d651016-1570-37b6-b3a3-6575d1545cb7", + "instanceIdentifier": "81bb2507-6874-3bb7-a280-a327515aa5fe", + "name": "LogFlowFileContents", + "comments": "", + "position": { + "x": -352.0, + "y": 104.0 + }, + "type": "org.apache.nifi.components.connector.processors.LogFlowFileContents", + "bundle": { + "group": "default", + "artifact": "system", + "version": "unversioned" + }, + "properties": {}, + "propertyDescriptors": {}, + "style": {}, + "schedulingPeriod": "0 sec", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 25, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [ + "success" + ], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "9caa5337-2e56-3711-a344-b29bce96d3e2" + } + ], + "inputPorts": [], + "outputPorts": [], + "connections": [ + { + "identifier": "17cbec18-38a6-36f6-ba7a-b5cc3775ef52", + "instanceIdentifier": "d303de27-6bb1-39c3-968b-64faabb17d27", + "name": "", + "source": { + "id": "2abf1573-d8ee-38ad-a745-083fdb1a8903", + "type": "PROCESSOR", + "groupId": "9caa5337-2e56-3711-a344-b29bce96d3e2", + "name": "CreateDummyFlowFile", + "comments": "", + "instanceIdentifier": "09b902c7-c4b7-3e92-8246-ef0a0bb685e7" + }, + "destination": { + "id": "efefbde7-a9e8-3864-b1d2-9649981b39d1", + "type": "PROCESSOR", + "groupId": "9caa5337-2e56-3711-a344-b29bce96d3e2", + "name": "Sleep", + "comments": "", + "instanceIdentifier": "77e69c78-0199-1000-0000-00001e16d8fc" + }, + "labelIndex": 0, + "zIndex": 1, + "selectedRelationships": [ + "success" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "9caa5337-2e56-3711-a344-b29bce96d3e2" + }, + { + "identifier": "5b5e23d7-e6ce-3122-bfe7-3c7bb194893b", + "instanceIdentifier": "77e6bb8c-0199-1000-ffff-ffffe70db445", + "name": "", + "source": { + "id": "efefbde7-a9e8-3864-b1d2-9649981b39d1", + "type": "PROCESSOR", + "groupId": "9caa5337-2e56-3711-a344-b29bce96d3e2", + "name": "Sleep", + "comments": "", + "instanceIdentifier": "77e69c78-0199-1000-0000-00001e16d8fc" + }, + "destination": { + "id": "3d651016-1570-37b6-b3a3-6575d1545cb7", + "type": "PROCESSOR", + "groupId": "9caa5337-2e56-3711-a344-b29bce96d3e2", + "name": "LogFlowFileContents", + "comments": "", + "instanceIdentifier": "81bb2507-6874-3bb7-a280-a327515aa5fe" + }, + "labelIndex": 0, + "zIndex": 2, + "selectedRelationships": [ + "success" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "9caa5337-2e56-3711-a344-b29bce96d3e2" + } + ], + "labels": [], + "funnels": [], + "controllerServices": [], + "parameterContextName": "Generate and Log", + "defaultFlowFileExpiration": "0 sec", + "defaultBackPressureObjectThreshold": 10000, + "defaultBackPressureDataSizeThreshold": "1 GB", + "scheduledState": "ENABLED", + "executionEngine": "INHERITED", + "maxConcurrentTasks": 1, + "statelessFlowTimeout": "1 min", + "flowFileConcurrency": "UNBOUNDED", + "flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE", + "componentType": "PROCESS_GROUP" + }, + "externalControllerServices": {}, + "parameterContexts": { + "Generate and Log": { + "name": "Generate and Log", + "parameters": [ + { + "name": "Text", + "description": "", + "sensitive": false, + "provided": false, + "value": "Hello" + }, { + "name": "Sleep Duration", + "description": "", + "sensitive": false, + "provided": false, + "value": "5 sec" + } + ], + "inheritedParameterContexts": [], + "componentType": "PARAMETER_CONTEXT" + } + }, + "flowEncodingVersion": "1.0", + "parameterProviders": {}, + "latest": false +} \ No newline at end of file diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/generate-duplicate-log-flow.json b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/generate-duplicate-log-flow.json new file mode 100644 index 000000000000..b39552388e8d --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/generate-duplicate-log-flow.json @@ -0,0 +1,690 @@ +{ + "flowContents": { + "identifier": "55129302-50a3-3d2f-8abd-b2479f4e9be3", + "instanceIdentifier": "544710c8-0199-1000-0000-00002aba2366", + "name": "Connector", + "comments": "", + "position": { + "x": 33.0, + "y": -170.0 + }, + "processGroups": [ + { + "identifier": "ce570e99-1208-3f71-0000-00006b7e3901", + "instanceIdentifier": "8f439d4b-fc9a-3f25-96ce-9682838c06b0", + "name": "Duplicates", + "comments": "", + "position": { + "x": -192.0, + "y": -24.0 + }, + "processGroups": [], + "remoteProcessGroups": [], + "processors": [ + { + "identifier": "ac87c05c-fdeb-32bc-8bef-304c401331ca", + "instanceIdentifier": "5448f92d-0199-1000-0000-00005958156f", + "name": "Create Duplicates", + "comments": "", + "position": { + "x": -200.0, + "y": -152.0 + }, + "type": "org.apache.nifi.components.connector.processors.DuplicateFlowFile", + "bundle": { + "group": "default", + "artifact": "system", + "version": "unversioned" + }, + "properties": { + "Number of Duplicates": "3" + }, + "propertyDescriptors": { + "Number of Copies": { + "name": "Number of Duplicates", + "displayName": "Number of Duplicates", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + } + }, + "style": {}, + "schedulingPeriod": "0 sec", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 0, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "ce570e99-1208-3f71-0000-00006b7e3901" + } + ], + "inputPorts": [ + { + "identifier": "b94406cc-5951-398f-84eb-db602ec72a4a", + "instanceIdentifier": "5447b963-0199-1000-ffff-ffffd01a6d32", + "name": "In", + "position": { + "x": -144.0, + "y": -325.0 + }, + "type": "INPUT_PORT", + "concurrentlySchedulableTaskCount": 1, + "scheduledState": "ENABLED", + "allowRemoteAccess": false, + "portFunction": "STANDARD", + "componentType": "INPUT_PORT", + "groupIdentifier": "ce570e99-1208-3f71-0000-00006b7e3901" + } + ], + "outputPorts": [ + { + "identifier": "3aebcf02-2c3e-38aa-ab15-61c2481c3e0a", + "instanceIdentifier": "5447c88a-0199-1000-0000-00002a2f67b2", + "name": "Out", + "position": { + "x": -144.0, + "y": 104.0 + }, + "type": "OUTPUT_PORT", + "concurrentlySchedulableTaskCount": 1, + "scheduledState": "ENABLED", + "allowRemoteAccess": false, + "portFunction": "STANDARD", + "componentType": "OUTPUT_PORT", + "groupIdentifier": "ce570e99-1208-3f71-0000-00006b7e3901" + } + ], + "connections": [ + { + "identifier": "2039d770-d9ab-3e71-ae03-569214755a46", + "instanceIdentifier": "5449dab5-0199-1000-0000-000011c5a49a", + "name": "3", + "source": { + "id": "ac87c05c-fdeb-32bc-8bef-304c401331ca", + "type": "PROCESSOR", + "groupId": "ce570e99-1208-3f71-0000-00006b7e3901", + "name": "Create Duplicates", + "comments": "", + "instanceIdentifier": "5448f92d-0199-1000-0000-00005958156f" + }, + "destination": { + "id": "3aebcf02-2c3e-38aa-ab15-61c2481c3e0a", + "type": "OUTPUT_PORT", + "groupId": "ce570e99-1208-3f71-0000-00006b7e3901", + "name": "Out", + "instanceIdentifier": "5447c88a-0199-1000-0000-00002a2f67b2" + }, + "labelIndex": 0, + "zIndex": 4, + "selectedRelationships": [ + "3" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [ + { + "x": 226.0, + "y": 20.0 + } + ], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "ce570e99-1208-3f71-0000-00006b7e3901" + }, + { + "identifier": "2bdfebb9-9698-3104-95a7-acd670ecb355", + "instanceIdentifier": "54491de8-0199-1000-ffff-ffffa4414ae3", + "name": "", + "source": { + "id": "b94406cc-5951-398f-84eb-db602ec72a4a", + "type": "INPUT_PORT", + "groupId": "ce570e99-1208-3f71-0000-00006b7e3901", + "name": "In", + "instanceIdentifier": "5447b963-0199-1000-ffff-ffffd01a6d32" + }, + "destination": { + "id": "ac87c05c-fdeb-32bc-8bef-304c401331ca", + "type": "PROCESSOR", + "groupId": "ce570e99-1208-3f71-0000-00006b7e3901", + "name": "Create Duplicates", + "comments": "", + "instanceIdentifier": "5448f92d-0199-1000-0000-00005958156f" + }, + "labelIndex": 0, + "zIndex": 1, + "selectedRelationships": [ + "" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "ce570e99-1208-3f71-0000-00006b7e3901" + }, + { + "identifier": "2361063e-00eb-3090-8cee-650d9bbe340c", + "instanceIdentifier": "5449aaf6-0199-1000-ffff-ffffbf429121", + "name": "1", + "source": { + "id": "ac87c05c-fdeb-32bc-8bef-304c401331ca", + "type": "PROCESSOR", + "groupId": "ce570e99-1208-3f71-0000-00006b7e3901", + "name": "Create Duplicates", + "comments": "", + "instanceIdentifier": "5448f92d-0199-1000-0000-00005958156f" + }, + "destination": { + "id": "3aebcf02-2c3e-38aa-ab15-61c2481c3e0a", + "type": "OUTPUT_PORT", + "groupId": "ce570e99-1208-3f71-0000-00006b7e3901", + "name": "Out", + "instanceIdentifier": "5447c88a-0199-1000-0000-00002a2f67b2" + }, + "labelIndex": 0, + "zIndex": 2, + "selectedRelationships": [ + "1" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "ce570e99-1208-3f71-0000-00006b7e3901" + }, + { + "identifier": "4cecc7a5-8156-328e-b18f-54c612924b9c", + "instanceIdentifier": "5449c2b5-0199-1000-0000-000076b2bd73", + "name": "2", + "source": { + "id": "ac87c05c-fdeb-32bc-8bef-304c401331ca", + "type": "PROCESSOR", + "groupId": "ce570e99-1208-3f71-0000-00006b7e3901", + "name": "Create Duplicates", + "comments": "", + "instanceIdentifier": "5448f92d-0199-1000-0000-00005958156f" + }, + "destination": { + "id": "3aebcf02-2c3e-38aa-ab15-61c2481c3e0a", + "type": "OUTPUT_PORT", + "groupId": "ce570e99-1208-3f71-0000-00006b7e3901", + "name": "Out", + "instanceIdentifier": "5447c88a-0199-1000-0000-00002a2f67b2" + }, + "labelIndex": 0, + "zIndex": 3, + "selectedRelationships": [ + "2" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [ + { + "x": -274.0, + "y": 20.0 + } + ], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "ce570e99-1208-3f71-0000-00006b7e3901" + } + ], + "labels": [], + "funnels": [], + "controllerServices": [], + "defaultFlowFileExpiration": "0 sec", + "defaultBackPressureObjectThreshold": 10000, + "defaultBackPressureDataSizeThreshold": "1 GB", + "scheduledState": "ENABLED", + "executionEngine": "INHERITED", + "maxConcurrentTasks": 1, + "statelessFlowTimeout": "1 min", + "flowFileConcurrency": "UNBOUNDED", + "flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE", + "componentType": "PROCESS_GROUP", + "groupIdentifier": "55129302-50a3-3d2f-8abd-b2479f4e9be3" + }, + { + "identifier": "e031373e-71a2-3d7f-0000-00006cf360cf", + "instanceIdentifier": "a8a23ec0-3279-3172-939b-adb5bdaf59e1", + "name": "Destination", + "comments": "", + "position": { + "x": -192.0, + "y": 264.0 + }, + "processGroups": [], + "remoteProcessGroups": [], + "processors": [ + { + "identifier": "43814190-1b7e-3f38-8844-10c97f1d6b95", + "instanceIdentifier": "544a0c77-0199-1000-ffff-ffffd81dd5f1", + "name": "Terminate FlowFile", + "comments": "", + "position": { + "x": -216.0, + "y": -64.0 + }, + "type": "org.apache.nifi.components.connector.processors.TerminateFlowFile", + "bundle": { + "group": "default", + "artifact": "system", + "version": "unversioned" + }, + "properties": { + }, + "propertyDescriptors": { + }, + "style": {}, + "schedulingPeriod": "0 sec", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 25, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "e031373e-71a2-3d7f-0000-00006cf360cf" + } + ], + "inputPorts": [ + { + "identifier": "5daa9ba5-dec6-397d-8cd5-b3f624bda832", + "instanceIdentifier": "5447ea94-0199-1000-0000-00006ed76b5a", + "name": "In", + "position": { + "x": -160.0, + "y": -232.0 + }, + "type": "INPUT_PORT", + "concurrentlySchedulableTaskCount": 1, + "scheduledState": "ENABLED", + "allowRemoteAccess": false, + "portFunction": "STANDARD", + "componentType": "INPUT_PORT", + "groupIdentifier": "e031373e-71a2-3d7f-0000-00006cf360cf" + } + ], + "outputPorts": [], + "connections": [ + { + "identifier": "1fbf6c9c-362f-3906-a3b5-0761874b5791", + "instanceIdentifier": "544a3fb0-0199-1000-0000-000033504ad9", + "name": "", + "source": { + "id": "5daa9ba5-dec6-397d-8cd5-b3f624bda832", + "type": "INPUT_PORT", + "groupId": "e031373e-71a2-3d7f-0000-00006cf360cf", + "name": "In", + "instanceIdentifier": "5447ea94-0199-1000-0000-00006ed76b5a" + }, + "destination": { + "id": "43814190-1b7e-3f38-8844-10c97f1d6b95", + "type": "PROCESSOR", + "groupId": "e031373e-71a2-3d7f-0000-00006cf360cf", + "name": "Terminate FlowFile", + "comments": "", + "instanceIdentifier": "544a0c77-0199-1000-ffff-ffffd81dd5f1" + }, + "labelIndex": 0, + "zIndex": 1, + "selectedRelationships": [ + "" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "e031373e-71a2-3d7f-0000-00006cf360cf" + } + ], + "labels": [], + "funnels": [], + "controllerServices": [], + "defaultFlowFileExpiration": "0 sec", + "defaultBackPressureObjectThreshold": 10000, + "defaultBackPressureDataSizeThreshold": "1 GB", + "scheduledState": "ENABLED", + "executionEngine": "INHERITED", + "maxConcurrentTasks": 1, + "statelessFlowTimeout": "1 min", + "flowFileConcurrency": "UNBOUNDED", + "flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE", + "componentType": "PROCESS_GROUP", + "groupIdentifier": "55129302-50a3-3d2f-8abd-b2479f4e9be3" + }, + { + "identifier": "06847e96-c5c7-35fe-88fc-64085f82192f", + "instanceIdentifier": "54472440-0199-1000-0000-00000b009f05", + "name": "Source", + "comments": "", + "position": { + "x": -192.0, + "y": -304.0 + }, + "processGroups": [], + "remoteProcessGroups": [], + "processors": [ + { + "identifier": "ea98a6b9-3074-33a4-9d58-903767e34a7a", + "instanceIdentifier": "5448a4ad-0199-1000-0000-000078c68790", + "name": "Set Contents", + "comments": "", + "position": { + "x": -120.0, + "y": -56.0 + }, + "type": "org.apache.nifi.components.connector.processors.OverwriteFlowFile", + "bundle": { + "group": "default", + "artifact": "system", + "version": "unversioned" + }, + "properties": { + "Content": "Hi there!" + }, + "propertyDescriptors": { + "Content": { + "name": "Content", + "displayName": "Content", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + } + }, + "style": {}, + "schedulingPeriod": "0 sec", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 25, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "06847e96-c5c7-35fe-88fc-64085f82192f" + }, + { + "identifier": "53e0f8b5-e04c-34c1-906a-4a5895fa8256", + "instanceIdentifier": "5448581a-0199-1000-ffff-ffff814d4715", + "name": "Create FlowFile", + "comments": "", + "position": { + "x": -120.0, + "y": -280.0 + }, + "type": "org.apache.nifi.components.connector.processors.CreateDummyFlowFile", + "bundle": { + "group": "default", + "artifact": "system", + "version": "unversioned" + }, + "properties": { + }, + "propertyDescriptors": { + }, + "style": {}, + "schedulingPeriod": "1 min", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 0, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": [], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "06847e96-c5c7-35fe-88fc-64085f82192f" + } + ], + "inputPorts": [], + "outputPorts": [ + { + "identifier": "54ed1d21-458e-38d9-9155-53a7b959ed8e", + "instanceIdentifier": "5447a2fc-0199-1000-ffff-ffffc38ab2be", + "name": "Out", + "position": { + "x": -64.0, + "y": 184.0 + }, + "type": "OUTPUT_PORT", + "concurrentlySchedulableTaskCount": 1, + "scheduledState": "ENABLED", + "allowRemoteAccess": false, + "portFunction": "STANDARD", + "componentType": "OUTPUT_PORT", + "groupIdentifier": "06847e96-c5c7-35fe-88fc-64085f82192f" + } + ], + "connections": [ + { + "identifier": "9161e363-0085-361a-8d28-2107c3f6f8fd", + "instanceIdentifier": "5448bbc2-0199-1000-ffff-ffffbb98a878", + "name": "", + "source": { + "id": "53e0f8b5-e04c-34c1-906a-4a5895fa8256", + "type": "PROCESSOR", + "groupId": "06847e96-c5c7-35fe-88fc-64085f82192f", + "name": "Create FlowFile", + "comments": "", + "instanceIdentifier": "5448581a-0199-1000-ffff-ffff814d4715" + }, + "destination": { + "id": "ea98a6b9-3074-33a4-9d58-903767e34a7a", + "type": "PROCESSOR", + "groupId": "06847e96-c5c7-35fe-88fc-64085f82192f", + "name": "Set Contents", + "comments": "", + "instanceIdentifier": "5448a4ad-0199-1000-0000-000078c68790" + }, + "labelIndex": 0, + "zIndex": 1, + "selectedRelationships": [ + "success" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "06847e96-c5c7-35fe-88fc-64085f82192f" + }, + { + "identifier": "b2d43983-2f9a-3ffa-bb70-d633a8c5b1c8", + "instanceIdentifier": "5448c84d-0199-1000-0000-00003f3bdee2", + "name": "", + "source": { + "id": "ea98a6b9-3074-33a4-9d58-903767e34a7a", + "type": "PROCESSOR", + "groupId": "06847e96-c5c7-35fe-88fc-64085f82192f", + "name": "Set Contents", + "comments": "", + "instanceIdentifier": "5448a4ad-0199-1000-0000-000078c68790" + }, + "destination": { + "id": "54ed1d21-458e-38d9-9155-53a7b959ed8e", + "type": "OUTPUT_PORT", + "groupId": "06847e96-c5c7-35fe-88fc-64085f82192f", + "name": "Out", + "instanceIdentifier": "5447a2fc-0199-1000-ffff-ffffc38ab2be" + }, + "labelIndex": 0, + "zIndex": 2, + "selectedRelationships": [ + "success" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "06847e96-c5c7-35fe-88fc-64085f82192f" + } + ], + "labels": [], + "funnels": [], + "controllerServices": [], + "defaultFlowFileExpiration": "0 sec", + "defaultBackPressureObjectThreshold": 10000, + "defaultBackPressureDataSizeThreshold": "1 GB", + "scheduledState": "ENABLED", + "executionEngine": "INHERITED", + "maxConcurrentTasks": 1, + "statelessFlowTimeout": "1 min", + "flowFileConcurrency": "UNBOUNDED", + "flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE", + "componentType": "PROCESS_GROUP", + "groupIdentifier": "55129302-50a3-3d2f-8abd-b2479f4e9be3" + } + ], + "remoteProcessGroups": [], + "processors": [], + "inputPorts": [], + "outputPorts": [], + "connections": [ + { + "identifier": "15b78f0d-0c72-3e45-95b7-4441f8274b24", + "instanceIdentifier": "54481ee4-0199-1000-ffff-ffff9110fe5e", + "name": "", + "source": { + "id": "3aebcf02-2c3e-38aa-ab15-61c2481c3e0a", + "type": "OUTPUT_PORT", + "groupId": "ce570e99-1208-3f71-0000-00006b7e3901", + "name": "Out", + "instanceIdentifier": "5447c88a-0199-1000-0000-00002a2f67b2" + }, + "destination": { + "id": "5daa9ba5-dec6-397d-8cd5-b3f624bda832", + "type": "INPUT_PORT", + "groupId": "e031373e-71a2-3d7f-0000-00006cf360cf", + "name": "In", + "instanceIdentifier": "5447ea94-0199-1000-0000-00006ed76b5a" + }, + "labelIndex": 0, + "zIndex": 2, + "selectedRelationships": [ + "" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "55129302-50a3-3d2f-8abd-b2479f4e9be3" + }, + { + "identifier": "9a522be3-0de3-328c-af6c-33f0204bc40e", + "instanceIdentifier": "5448016d-0199-1000-0000-00002669c0b1", + "name": "", + "source": { + "id": "54ed1d21-458e-38d9-9155-53a7b959ed8e", + "type": "OUTPUT_PORT", + "groupId": "06847e96-c5c7-35fe-88fc-64085f82192f", + "name": "Out", + "instanceIdentifier": "5447a2fc-0199-1000-ffff-ffffc38ab2be" + }, + "destination": { + "id": "b94406cc-5951-398f-84eb-db602ec72a4a", + "type": "INPUT_PORT", + "groupId": "ce570e99-1208-3f71-0000-00006b7e3901", + "name": "In", + "instanceIdentifier": "5447b963-0199-1000-ffff-ffffd01a6d32" + }, + "labelIndex": 0, + "zIndex": 1, + "selectedRelationships": [ + "" + ], + "backPressureObjectThreshold": 10000, + "backPressureDataSizeThreshold": "1 GB", + "flowFileExpiration": "0 sec", + "prioritizers": [], + "bends": [], + "loadBalanceStrategy": "DO_NOT_LOAD_BALANCE", + "partitioningAttribute": "", + "loadBalanceCompression": "DO_NOT_COMPRESS", + "componentType": "CONNECTION", + "groupIdentifier": "55129302-50a3-3d2f-8abd-b2479f4e9be3" + } + ], + "labels": [], + "funnels": [], + "controllerServices": [], + "defaultFlowFileExpiration": "0 sec", + "defaultBackPressureObjectThreshold": 10000, + "defaultBackPressureDataSizeThreshold": "1 GB", + "scheduledState": "ENABLED", + "executionEngine": "INHERITED", + "maxConcurrentTasks": 1, + "statelessFlowTimeout": "1 min", + "flowFileConcurrency": "UNBOUNDED", + "flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE", + "componentType": "PROCESS_GROUP" + }, + "externalControllerServices": {}, + "parameterContexts": {}, + "flowEncodingVersion": "1.0", + "parameterProviders": {}, + "latest": false +} \ No newline at end of file diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/on-property-modified-tracker.json b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/on-property-modified-tracker.json new file mode 100644 index 000000000000..1f4f11c0fb39 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/resources/flows/on-property-modified-tracker.json @@ -0,0 +1,98 @@ +{ + "flowContents": { + "identifier": "on-property-modified-root", + "instanceIdentifier": "on-property-modified-instance", + "name": "OnPropertyModified Test Flow", + "comments": "", + "position": { + "x": 0.0, + "y": 0.0 + }, + "processGroups": [], + "remoteProcessGroups": [], + "processors": [ + { + "identifier": "on-property-modified-tracker", + "instanceIdentifier": "on-property-modified-tracker-instance", + "name": "OnPropertyModifiedTracker", + "comments": "", + "position": { + "x": 0.0, + "y": 0.0 + }, + "type": "org.apache.nifi.components.connector.processors.OnPropertyModifiedTracker", + "bundle": { + "group": "default", + "artifact": "system", + "version": "unversioned" + }, + "properties": { + "Configured Number": "#{CONFIGURED_NUMBER}" + }, + "propertyDescriptors": { + "Configured Number": { + "name": "Configured Number", + "displayName": "Configured Number", + "identifiesControllerService": false, + "sensitive": false, + "dynamic": false + } + }, + "style": {}, + "schedulingPeriod": "1 min", + "schedulingStrategy": "TIMER_DRIVEN", + "executionNode": "ALL", + "penaltyDuration": "30 sec", + "yieldDuration": "1 sec", + "bulletinLevel": "WARN", + "runDurationMillis": 0, + "concurrentlySchedulableTaskCount": 1, + "autoTerminatedRelationships": ["success"], + "scheduledState": "ENABLED", + "retryCount": 10, + "retriedRelationships": [], + "backoffMechanism": "PENALIZE_FLOWFILE", + "maxBackoffPeriod": "10 mins", + "componentType": "PROCESSOR", + "groupIdentifier": "on-property-modified-root" + } + ], + "inputPorts": [], + "outputPorts": [], + "connections": [], + "labels": [], + "funnels": [], + "controllerServices": [], + "parameterContextName": "OnPropertyModifiedContext", + "defaultFlowFileExpiration": "0 sec", + "defaultBackPressureObjectThreshold": 10000, + "defaultBackPressureDataSizeThreshold": "1 GB", + "scheduledState": "ENABLED", + "executionEngine": "INHERITED", + "maxConcurrentTasks": 1, + "statelessFlowTimeout": "1 min", + "flowFileConcurrency": "UNBOUNDED", + "flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE", + "componentType": "PROCESS_GROUP" + }, + "externalControllerServices": {}, + "parameterContexts": { + "OnPropertyModifiedContext": { + "name": "OnPropertyModifiedContext", + "parameters": [ + { + "name": "CONFIGURED_NUMBER", + "description": "", + "sensitive": false, + "provided": false, + "value": null + } + ], + "inheritedParameterContexts": [], + "componentType": "PARAMETER_CONTEXT" + } + }, + "flowEncodingVersion": "1.0", + "parameterProviders": {}, + "latest": false +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java index 5a1d3dee5202..aa47236cf173 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/main/java/org/apache/nifi/nar/StandardExtensionDiscoveringManager.java @@ -29,6 +29,7 @@ import org.apache.nifi.components.ClassloaderIsolationKeyProvider; import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.connector.Connector; import org.apache.nifi.components.state.StateProvider; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.leader.election.LeaderElectionManager; @@ -136,6 +137,7 @@ public StandardExtensionDiscoveringManager(final Collection()); definitionMap.put(FlowActionReporter.class, new HashSet<>()); definitionMap.put(ComponentMetricReporter.class, new HashSet<>()); + definitionMap.put(Connector.class, new HashSet<>()); additionalExtensionTypes.forEach(type -> definitionMap.putIfAbsent(type, new HashSet<>())); } @@ -408,9 +410,12 @@ private Set getServiceFileURLs(final Bundle bundle, final Class extensio serviceResourceUrls.add(serviceResourceUrlEnum.nextElement()); } - final Enumeration parentResourceUrlEnum = bundle.getClassLoader().getParent().getResources(servicesFile); - while (parentResourceUrlEnum.hasMoreElements()) { - serviceResourceUrls.remove(parentResourceUrlEnum.nextElement()); + final ClassLoader parentClassLoader = bundle.getClassLoader().getParent(); + if (parentClassLoader != null) { + final Enumeration parentResourceUrlEnum = parentClassLoader.getResources(servicesFile); + while (parentResourceUrlEnum.hasMoreElements()) { + serviceResourceUrls.remove(parentResourceUrlEnum.nextElement()); + } } return serviceResourceUrls; @@ -421,7 +426,7 @@ protected void loadExtension(final String extensionClassName, final Class ext } protected void registerExtensionClass(final Class extensionType, final String implementationClassName, final Bundle bundle) { - final Set registeredClasses = definitionMap.get(extensionType); + final Set registeredClasses = definitionMap.computeIfAbsent(extensionType, type -> new HashSet<>()); registerServiceClass(implementationClassName, extensionType, classNameBundleLookup, bundleCoordinateClassesLookup, bundle, registeredClasses); } @@ -507,7 +512,8 @@ public Class getClass(final ExtensionDefinition extensionDefinition) { */ private static boolean multipleVersionsAllowed(Class type) { return Processor.class.isAssignableFrom(type) || ControllerService.class.isAssignableFrom(type) || ReportingTask.class.isAssignableFrom(type) - || FlowAnalysisRule.class.isAssignableFrom(type) || ParameterProvider.class.isAssignableFrom(type) || FlowRegistryClient.class.isAssignableFrom(type); + || FlowAnalysisRule.class.isAssignableFrom(type) || ParameterProvider.class.isAssignableFrom(type) || FlowRegistryClient.class.isAssignableFrom(type) + || Connector.class.isAssignableFrom(type); } protected boolean isInstanceClassLoaderRequired(final String classType, final Bundle bundle) { @@ -859,6 +865,10 @@ public synchronized ConfigurableComponent getTempComponent(final String classTyp tempComponent = pythonBridge.createProcessor(procId, classType, bundleCoordinate.getVersion(), false, false); } else { final Class componentClass = Class.forName(classType, true, bundleClassLoader); + if (!ConfigurableComponent.class.isAssignableFrom(componentClass)) { + return null; + } + tempComponent = (ConfigurableComponent) componentClass.getDeclaredConstructor().newInstance(); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/DummyConnector.java b/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/DummyConnector.java new file mode 100644 index 000000000000..2a2b21cde1b3 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/DummyConnector.java @@ -0,0 +1,120 @@ +/* + * 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.nar; + +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.Connector; +import org.apache.nifi.components.connector.ConnectorConfigurationContext; +import org.apache.nifi.components.connector.ConnectorInitializationContext; +import org.apache.nifi.components.connector.ConnectorValidationContext; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.flow.VersionedExternalFlow; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +@Tags({"test", "connector"}) +public class DummyConnector implements Connector { + private ConnectorInitializationContext context; + + public ConnectorInitializationContext getContext() { + return context; + } + + @Override + public void initialize(final ConnectorInitializationContext connectorInitializationContext) { + this.context = connectorInitializationContext; + } + + @Override + public VersionedExternalFlow getInitialFlow() { + return null; + } + + @Override + public void start(final FlowContext flowContext) throws FlowUpdateException { + + } + + @Override + public void stop(final FlowContext flowContext) throws FlowUpdateException { + + } + + @Override + public List getConfigurationSteps() { + return List.of(); + } + + @Override + public void onConfigurationStepConfigured(final String stepName, final FlowContext flowContext) { + } + + @Override + public void prepareForUpdate(final FlowContext workingContext, final FlowContext activeContext) { + } + + @Override + public void abortUpdate(final FlowContext flowContext, final Throwable cause) { + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext flowContext) { + return List.of(); + } + + @Override + public List verify(final FlowContext flowContext) { + return List.of(); + } + + @Override + public List validate(final FlowContext flowContext, final ConnectorValidationContext validationContext) { + return List.of(); + } + + @Override + public List validateConfigurationStep(final ConfigurationStep configurationStep, final ConnectorConfigurationContext connectorConfigurationContext, + final ConnectorValidationContext connectorValidationContext) { + return List.of(); + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName, final FlowContext flowContext) { + return List.of(); + } + + @Override + public List fetchAllowableValues(final String stepName, final String propertyName, final FlowContext flowContext, final String filter) { + return List.of(); + } + + @Override + public CompletableFuture drainFlowFiles(final FlowContext flowContext) { + return CompletableFuture.completedFuture(null); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/StandardExtensionDiscoveringManagerConnectorTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/StandardExtensionDiscoveringManagerConnectorTest.java new file mode 100644 index 000000000000..7d2213ddb3b7 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/java/org/apache/nifi/nar/StandardExtensionDiscoveringManagerConnectorTest.java @@ -0,0 +1,49 @@ +/* + * 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.nar; + +import org.apache.nifi.bundle.Bundle; +import org.apache.nifi.components.connector.Connector; +import org.apache.nifi.util.NiFiProperties; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class StandardExtensionDiscoveringManagerConnectorTest { + + @Test + public void testConnectorDiscoveryFromServiceFile() { + final NiFiProperties properties = NiFiProperties.createBasicNiFiProperties("src/test/resources/nifi.properties"); + final Bundle systemBundle = SystemBundle.create(properties); + + final StandardExtensionDiscoveringManager manager = new StandardExtensionDiscoveringManager(); + manager.discoverExtensions(systemBundle, Collections.emptySet()); + + final Set connectorDefs = manager.getExtensions(Connector.class); + final ExtensionDefinition connectorDefinition = connectorDefs.stream() + .filter(def -> def.getImplementationClassName().equals("org.apache.nifi.nar.DummyConnector")) + .findFirst().orElseThrow(() -> new AssertionError("Did not find expected DummyConnector")); + + assertEquals(systemBundle, connectorDefinition.getBundle()); + assertEquals(Connector.class, connectorDefinition.getExtensionType()); + } +} + + diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/META-INF/services/org.apache.nifi.components.connector.Connector b/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/META-INF/services/org.apache.nifi.components.connector.Connector new file mode 100644 index 000000000000..89a28c784f16 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-nar-utils/src/test/resources/META-INF/services/org.apache.nifi.components.connector.Connector @@ -0,0 +1,18 @@ +# 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. +org.apache.nifi.nar.NarThreadContextClassLoaderTest$WithPropertiesConstructor +org.apache.nifi.nar.NarThreadContextClassLoaderTest$WithDefaultConstructor + +org.apache.nifi.nar.DummyConnector diff --git a/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessNiFiServer.java b/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessNiFiServer.java index a2936c840f79..e59ea12d44c8 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessNiFiServer.java +++ b/nifi-framework-bundle/nifi-framework/nifi-headless-server/src/main/java/org/apache/nifi/headless/HeadlessNiFiServer.java @@ -29,6 +29,7 @@ import org.apache.nifi.bundle.Bundle; import org.apache.nifi.cluster.ClusterDetailsFactory; import org.apache.nifi.cluster.ConnectionState; +import org.apache.nifi.components.connector.StandaloneConnectorRequestReplicator; import org.apache.nifi.components.state.StateManagerProvider; import org.apache.nifi.controller.DecommissionTask; import org.apache.nifi.controller.FlowController; @@ -150,16 +151,18 @@ public void preDestruction() throws AuthorizerDestructionException { extensionManager, statusHistoryRepository, null, - stateManagerProvider + stateManagerProvider, + new StandaloneConnectorRequestReplicator() ); flowService = StandardFlowService.createStandaloneInstance( - flowController, - props, - null, // revision manager - null, // NAR Manager - null, // Asset Synchronizer - authorizer); + flowController, + props, + null, // revision manager + null, // NAR Manager + null, // Parameter Context Asset Synchronizer + null, // Connector Asset Synchronizer + authorizer); diagnosticsFactory = new BootstrapDiagnosticsFactory(); ((BootstrapDiagnosticsFactory) diagnosticsFactory).setFlowController(flowController); diff --git a/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java b/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java index 773ad44d674a..2399c295d632 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java +++ b/nifi-framework-bundle/nifi-framework/nifi-nar-utils/src/main/java/org/apache/nifi/nar/NarUnpacker.java @@ -545,8 +545,9 @@ private static ExtensionMapping getJarExtensionMapping(final BundleCoordinate co final JarEntry controllerServiceEntry = jarFile.getJarEntry("META-INF/services/org.apache.nifi.controller.ControllerService"); final JarEntry parameterProviderEntry = jarFile.getJarEntry("META-INF/services/org.apache.nifi.parameter.ParameterProvider"); final JarEntry flowRegistryClientEntry = jarFile.getJarEntry("META-INF/services/org.apache.nifi.registry.flow.FlowRegistryClient"); + final JarEntry connectorEntry = jarFile.getJarEntry("META-INF/services/org.apache.nifi.components.connector.Connector"); - if (processorEntry == null && reportingTaskEntry == null && flowAnalysisRuleEntry == null && controllerServiceEntry == null && parameterProviderEntry == null) { + if (processorEntry == null && reportingTaskEntry == null && flowAnalysisRuleEntry == null && controllerServiceEntry == null && parameterProviderEntry == null && connectorEntry == null) { return mapping; } @@ -556,6 +557,7 @@ private static ExtensionMapping getJarExtensionMapping(final BundleCoordinate co mapping.addAllControllerServices(coordinate, detectNiFiComponents(jarFile, controllerServiceEntry)); mapping.addAllParameterProviders(coordinate, detectNiFiComponents(jarFile, parameterProviderEntry)); mapping.addAllFlowRegistryClients(coordinate, detectNiFiComponents(jarFile, flowRegistryClientEntry)); + mapping.addAllConnectors(coordinate, detectNiFiComponents(jarFile, connectorEntry)); return mapping; } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java b/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java index 966c7138711a..efd2d3d69390 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java +++ b/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java @@ -36,6 +36,7 @@ public class StandardRepositoryRecord implements RepositoryRecord { private final FlowFileRecord originalFlowFileRecord; private final FlowFileQueue originalQueue; private String swapLocation; + private String originalSwapLocation; private final Map originalAttributes; private Map updatedAttributes = null; private List transientClaims; @@ -103,11 +104,24 @@ public String getSwapLocation() { return swapLocation; } - public void setSwapLocation(final String swapLocation) { - this.swapLocation = swapLocation; - if (type != RepositoryRecordType.SWAP_OUT) { - setType(RepositoryRecordType.SWAP_IN); // we are swapping in a new record + public void setSwapLocation(final String swapLocation, final RepositoryRecordType swapType) { + if (swapType != RepositoryRecordType.SWAP_IN && swapType != RepositoryRecordType.SWAP_OUT + && swapType != RepositoryRecordType.SWAP_FILE_DELETED && swapType != RepositoryRecordType.SWAP_FILE_RENAMED) { + throw new IllegalArgumentException("swapType must be one of SWAP_IN, SWAP_OUT, SWAP_FILE_DELETED, or SWAP_FILE_RENAMED but was " + swapType); } + this.swapLocation = swapLocation; + setType(swapType); + } + + @Override + public String getOriginalSwapLocation() { + return originalSwapLocation; + } + + public void setSwapFileRenamed(final String originalSwapLocation, final String newSwapLocation) { + this.originalSwapLocation = originalSwapLocation; + this.swapLocation = newSwapLocation; + setType(RepositoryRecordType.SWAP_FILE_RENAMED); } @Override @@ -238,7 +252,7 @@ private void setType(final RepositoryRecordType newType) { return; } - if (this.type == RepositoryRecordType.CREATE) { + if (this.type == RepositoryRecordType.CREATE && getCurrent() != null) { // Because we don't copy updated attributes to `this.updatedAttributes` for CREATE records, we need to ensure // that if a record is changed from CREATE to anything else that we do properly update the `this.updatedAttributes` field. this.updatedAttributes = new HashMap<>(getCurrent().getAttributes()); diff --git a/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml b/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml index e5c7e30549b9..7468d198f903 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml +++ b/nifi-framework-bundle/nifi-framework/nifi-resources/pom.xml @@ -121,6 +121,8 @@ org.apache.nifi.asset.StandardAssetManager ./assets + org.apache.nifi.asset.StandardConnectorAssetManager + ./connector_assets diff --git a/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties b/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties index 6e536a67825a..afea4e093b25 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties +++ b/nifi-framework-bundle/nifi-framework/nifi-resources/src/main/resources/conf/nifi.properties @@ -136,6 +136,11 @@ nifi.nar.persistence.provider.properties.directory=${nifi.nar.persistence.provid # Asset Management nifi.asset.manager.implementation=${nifi.asset.manager.implementation} nifi.asset.manager.properties.directory=${nifi.asset.manager.properties.directory} +nifi.connector.asset.manager.implementation=${nifi.connector.asset.manager.implementation} +nifi.connector.asset.manager.properties.directory=${nifi.connector.asset.manager.properties.directory} + +# Connector Configuration Provider +nifi.components.connectors.configuration.provider.implementation= # Site to Site properties nifi.remote.input.host= diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java index dc0ddf33048c..706a349d0861 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/main/java/org/apache/nifi/web/server/JettyServer.java @@ -61,6 +61,7 @@ import org.apache.nifi.util.FileUtils; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.web.ContentAccess; +import org.apache.nifi.web.NiFiConnectorWebContext; import org.apache.nifi.web.NiFiWebConfigurationContext; import org.apache.nifi.web.UiExtensionType; import org.apache.nifi.web.server.filter.FilterParameter; @@ -96,6 +97,7 @@ import java.io.BufferedWriter; import java.io.File; import java.io.IOException; +import java.io.InputStream; import java.io.InputStreamReader; import java.io.OutputStreamWriter; import java.lang.reflect.InvocationTargetException; @@ -114,6 +116,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Properties; import java.util.Set; import java.util.UUID; import java.util.concurrent.BlockingQueue; @@ -194,6 +197,7 @@ public class JettyServer implements NiFiServer, ExtensionUiLoader { // component (processor, controller service, reporting task) ui extensions private UiExtensionMapping componentUiExtensions; private Collection componentUiExtensionWebContexts; + private Collection connectorUiExtensionWebContexts; private final Map> webAppsByBundleCoordinate = new ConcurrentHashMap<>(); @@ -269,6 +273,7 @@ private Handler loadInitialWars(final Set bundles) { final ExtensionUiInfo extensionUiInfo = loadWars(otherWars); componentUiExtensionWebContexts = new ArrayList<>(extensionUiInfo.componentUiExtensionWebContexts()); contentViewerWebContexts = new ArrayList<>(extensionUiInfo.contentViewerWebContexts()); + connectorUiExtensionWebContexts = new ArrayList<>(extensionUiInfo.connectorUiExtensionWebContexts()); contentViewers = new HashSet<>(extensionUiInfo.contentViewers()); componentUiExtensions = new UiExtensionMapping(extensionUiInfo.componentUiExtensionsByType()); @@ -328,15 +333,18 @@ private void processExtensionUiBundle(final Bundle bundle) { final Collection componentUiExtensionWebContexts = extensionUiInfo.componentUiExtensionWebContexts(); final Collection contentViewerWebContexts = extensionUiInfo.contentViewerWebContexts(); + final Collection connectorUiExtensionWebContexts = extensionUiInfo.connectorUiExtensionWebContexts(); // Inject the configuration context and security filter into contexts that need it final ServletContext webApiServletContext = webApiContext.getServletHandler().getServletContext(); final WebApplicationContext webApplicationContext = WebApplicationContextUtils.getRequiredWebApplicationContext(webApiServletContext); final NiFiWebConfigurationContext configurationContext = webApplicationContext.getBean("nifiWebConfigurationContext", NiFiWebConfigurationContext.class); + final NiFiConnectorWebContext connectorWebContext = webApplicationContext.getBean("nifiConnectorWebContext", NiFiConnectorWebContext.class); final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter(SPRING_SECURITY_FILTER_CHAIN); performInjectionForComponentUis(componentUiExtensionWebContexts, configurationContext, securityFilter); performInjectionForContentViewerUis(contentViewerWebContexts, webApplicationContext, securityFilter); + performInjectionForConnectorUis(connectorUiExtensionWebContexts, connectorWebContext, securityFilter); // Merge results of current loading into previously loaded results... this.componentUiExtensionWebContexts.addAll(componentUiExtensionWebContexts); @@ -417,6 +425,7 @@ private ExtensionUiInfo loadWars(final Map warToBundleLookup) { final Map mimeMappings = new HashMap<>(); final Collection componentUiExtensionWebContexts = new ArrayList<>(); final Collection contentViewerWebContexts = new ArrayList<>(); + final Collection connectorUiExtensionWebContexts = new ArrayList<>(); final Map> componentUiExtensionsByType = new HashMap<>(); final Map contentViewerServletContexts = new HashMap<>(); final Map> webAppContextsByBundleCoordinate = new HashMap<>(); @@ -433,8 +442,11 @@ private ExtensionUiInfo loadWars(final Map warToBundleLookup) { final Map> uiExtensionInWar = new HashMap<>(); identifyUiExtensionsForComponents(uiExtensionInWar, war); + // identify connector UI extensions from manifest (has route information) + final Map> connectorExtensions = readConnectorUiExtensions(war); + // only include wars that are for custom processor ui's - if (!uiExtensionInWar.isEmpty()) { + if (!uiExtensionInWar.isEmpty() || !connectorExtensions.isEmpty()) { // get the context path String warName = StringUtils.substringBeforeLast(war.getName(), "."); String warContextPath = String.format("/%s", warName); @@ -501,6 +513,35 @@ private ExtensionUiInfo loadWars(final Map warToBundleLookup) { } } + // process connector UI extensions with route information + for (final Map.Entry> connectorEntry : connectorExtensions.entrySet()) { + final String connectorType = connectorEntry.getKey(); + final Map routes = connectorEntry.getValue(); + + // build a bundle-specific key to support multiple versions of the same connector type + final BundleCoordinate coordinate = warBundle.getBundleDetails().getCoordinate(); + final String bundleSpecificKey = connectorType + ":" + coordinate.getGroup() + ":" + coordinate.getId() + ":" + coordinate.getVersion(); + + logger.info("Loading Connector UI extension [{}, {}] for {} with routes {}", UiExtensionType.Connector, warContextPath, bundleSpecificKey, routes.keySet()); + + // record the extension definition with route information + final UiExtension uiExtension = new UiExtension(UiExtensionType.Connector, warContextPath, routes); + + // create if this is the first extension for this component type (using bundle-specific key) + final List componentUiExtensionsForType = componentUiExtensionsByType.computeIfAbsent(bundleSpecificKey, k -> new ArrayList<>()); + + // see if there is already a ui extension of this same type + if (containsUiExtensionType(componentUiExtensionsForType, UiExtensionType.Connector)) { + throw new IllegalStateException(String.format("Encountered duplicate Connector UI for %s", bundleSpecificKey)); + } + + // record this extension + componentUiExtensionsForType.add(uiExtension); + + // this ui extension provides a connector custom ui + connectorUiExtensionWebContexts.add(extensionUiContext); + } + // include custom ui web context in the handlers webAppContexts.add(extensionUiContext); @@ -512,7 +553,7 @@ private ExtensionUiInfo loadWars(final Map warToBundleLookup) { } return new ExtensionUiInfo(webAppContexts, mimeMappings, contentViewers, componentUiExtensionWebContexts, contentViewerWebContexts, - componentUiExtensionsByType, contentViewerServletContexts, webAppContextsByBundleCoordinate); + connectorUiExtensionWebContexts, componentUiExtensionsByType, contentViewerServletContexts, webAppContextsByBundleCoordinate); } /** @@ -593,6 +634,58 @@ private void identifyUiExtensionsForComponents(final Map + * # connector-type.route-name=route-path + * org.apache.nifi.connectors.KafkaToS3.configuration=#/configure + * org.apache.nifi.connectors.KafkaToS3.details=#/details + * + * + * @param warFile the WAR file to inspect + * @return a map of connector type to route map (route name to route path), or empty map if no manifest found + */ + private Map> readConnectorUiExtensions(final File warFile) { + final Map> connectorExtensions = new HashMap<>(); + + try (final JarFile jarFile = new JarFile(warFile)) { + final JarEntry propertiesEntry = jarFile.getJarEntry("META-INF/nifi-connector"); + if (propertiesEntry == null) { + return connectorExtensions; + } + + try (final InputStream inputStream = jarFile.getInputStream(propertiesEntry)) { + final Properties properties = new Properties(); + properties.load(inputStream); + + for (final String propertyName : properties.stringPropertyNames()) { + final String propertyValue = properties.getProperty(propertyName); + if (propertyValue == null || propertyValue.isBlank()) { + continue; + } + + // Parse property name: connector-type.route-name + final int lastDotIndex = propertyName.lastIndexOf('.'); + if (lastDotIndex <= 0 || lastDotIndex >= propertyName.length() - 1) { + logger.warn("Invalid connector UI property format: {}. Expected format: connector-type.route-name=route-path", propertyName); + continue; + } + + final String connectorType = propertyName.substring(0, lastDotIndex); + final String routeName = propertyName.substring(lastDotIndex + 1); + + final Map routes = connectorExtensions.computeIfAbsent(connectorType, k -> new HashMap<>()); + routes.put(routeName, propertyValue); + } + } + } catch (final IOException ioe) { + logger.warn("Unable to inspect {} for Connector UI extensions.", warFile, ioe); + } + + return connectorExtensions; + } + /** * Extracts the component type. Trims the line and considers comments. * Returns null if no type was found. @@ -803,11 +896,15 @@ public void start() { // get the application context final WebApplicationContext webApplicationContext = WebApplicationContextUtils.getRequiredWebApplicationContext(webApiServletContext); final NiFiWebConfigurationContext configurationContext = webApplicationContext.getBean("nifiWebConfigurationContext", NiFiWebConfigurationContext.class); + final NiFiConnectorWebContext connectorWebContext = webApplicationContext.getBean("nifiConnectorWebContext", NiFiConnectorWebContext.class); final FilterHolder securityFilter = webApiContext.getServletHandler().getFilter(SPRING_SECURITY_FILTER_CHAIN); // component ui extensions performInjectionForComponentUis(componentUiExtensionWebContexts, configurationContext, securityFilter); + // connector ui extensions + performInjectionForConnectorUis(connectorUiExtensionWebContexts, connectorWebContext, securityFilter); + // content viewer extensions performInjectionForContentViewerUis(contentViewerWebContexts, webApplicationContext, securityFilter); webApiServletContext.setAttribute("content-viewers", contentViewers); @@ -948,6 +1045,21 @@ private void performInjectionForComponentUis(final Collection com } } + private void performInjectionForConnectorUis(final Collection connectorUiExtensionWebContexts, + final NiFiConnectorWebContext connectorWebContext, + final FilterHolder securityFilter) { + for (final WebAppContext customUiContext : connectorUiExtensionWebContexts) { + // set the connector web context in each connector custom ui servlet context + final ServletContext customUiServletContext = customUiContext.getServletHandler().getServletContext(); + customUiServletContext.setAttribute("nifi-connector-web-context", connectorWebContext); + + // add the security filter to any ui extensions wars + if (securityFilter != null) { + customUiContext.addFilter(securityFilter, "/*", EnumSet.allOf(DispatcherType.class)); + } + } + } + private void performInjectionForContentViewerUis(final Collection contentViewerWebContexts, final WebApplicationContext webApiApplicationContext, final FilterHolder securityFilter) { @@ -1017,6 +1129,7 @@ private ErrorPageErrorHandler getErrorHandler() { private record ExtensionUiInfo(Collection webAppContexts, Map mimeMappings, Collection contentViewers, Collection componentUiExtensionWebContexts, Collection contentViewerWebContexts, + Collection connectorUiExtensionWebContexts, Map> componentUiExtensionsByType, Map contentViewerServletContexts, Map> webAppContextsByBundleCoordinate) { diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/StandardServerProviderTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/StandardServerProviderTest.java index 6d7b38847cad..8b18bfeb5070 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/StandardServerProviderTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-jetty/src/test/java/org/apache/nifi/web/server/StandardServerProviderTest.java @@ -123,7 +123,7 @@ static void setConfiguration() throws Exception { void testGetServer() { final Properties applicationProperties = new Properties(); applicationProperties.setProperty(NiFiProperties.WEB_HTTP_PORT, RANDOM_PORT); - final NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, applicationProperties); + final NiFiProperties properties = NiFiProperties.createBasicNiFiProperties((String) null, applicationProperties); final StandardServerProvider provider = new StandardServerProvider(null); @@ -137,7 +137,7 @@ void testGetServer() { void testGetServerHttps() { final Properties applicationProperties = new Properties(); applicationProperties.setProperty(NiFiProperties.WEB_HTTPS_PORT, RANDOM_PORT); - final NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, applicationProperties); + final NiFiProperties properties = NiFiProperties.createBasicNiFiProperties((String) null, applicationProperties); final StandardServerProvider provider = new StandardServerProvider(sslContext); @@ -151,7 +151,7 @@ void testGetServerHttps() { void testGetServerStart() throws Exception { final Properties applicationProperties = new Properties(); applicationProperties.setProperty(NiFiProperties.WEB_HTTP_PORT, RANDOM_PORT); - final NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, applicationProperties); + final NiFiProperties properties = NiFiProperties.createBasicNiFiProperties((String) null, applicationProperties); final StandardServerProvider provider = new StandardServerProvider(null); @@ -175,7 +175,7 @@ void testGetServerHttpsRequestsCompleted() throws Exception { final Properties applicationProperties = new Properties(); applicationProperties.setProperty(NiFiProperties.WEB_HTTPS_PORT, RANDOM_PORT); applicationProperties.setProperty(NiFiProperties.WEB_PROXY_HOST, PUBLIC_HOST); - final NiFiProperties properties = NiFiProperties.createBasicNiFiProperties(null, applicationProperties); + final NiFiProperties properties = NiFiProperties.createBasicNiFiProperties((String) null, applicationProperties); final StandardServerProvider provider = new StandardServerProvider(sslContext); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtension.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtension.java index da921b4535dc..11268a99c4fc 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtension.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-ui-extension/src/main/java/org/apache/nifi/ui/extension/UiExtension.java @@ -18,6 +18,8 @@ import org.apache.nifi.web.UiExtensionType; +import java.util.Map; + /** * Information about a UI extension required to be invoked. */ @@ -25,10 +27,16 @@ public class UiExtension { private final UiExtensionType extensionType; private final String contextPath; + private final Map supportedRoutes; public UiExtension(final UiExtensionType extensionType, final String contextPath) { + this(extensionType, contextPath, null); + } + + public UiExtension(final UiExtensionType extensionType, final String contextPath, final Map supportedRoutes) { this.extensionType = extensionType; this.contextPath = contextPath; + this.supportedRoutes = supportedRoutes; } /** @@ -45,4 +53,12 @@ public String getContextPath() { return contextPath; } + /** + * @return A map of route names to route paths for this UI extension, or null if routes are not applicable. + * Route paths may include a hash prefix for hash-based routing (e.g., "#/wizard") or be path-based (e.g., "/wizard"). + */ + public Map getSupportedRoutes() { + return supportedRoutes; + } + } diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ConnectorAuditor.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ConnectorAuditor.java new file mode 100644 index 000000000000..57c762451fe3 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ConnectorAuditor.java @@ -0,0 +1,416 @@ +/* + * 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.audit; + +import org.apache.nifi.action.Action; +import org.apache.nifi.action.Component; +import org.apache.nifi.action.FlowChangeAction; +import org.apache.nifi.action.Operation; +import org.apache.nifi.action.component.details.FlowChangeExtensionDetails; +import org.apache.nifi.action.details.ActionDetails; +import org.apache.nifi.action.details.FlowChangeConfigureDetails; +import org.apache.nifi.components.connector.AssetReference; +import org.apache.nifi.components.connector.ConnectorConfiguration; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorState; +import org.apache.nifi.components.connector.ConnectorValueReference; +import org.apache.nifi.components.connector.NamedStepConfiguration; +import org.apache.nifi.components.connector.SecretReference; +import org.apache.nifi.components.connector.StepConfiguration; +import org.apache.nifi.components.connector.StringLiteralValue; +import org.apache.nifi.web.api.dto.AssetReferenceDTO; +import org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorValueReferenceDTO; +import org.apache.nifi.web.api.dto.PropertyGroupConfigurationDTO; +import org.apache.nifi.web.dao.ConnectorDAO; +import org.aspectj.lang.ProceedingJoinPoint; +import org.aspectj.lang.annotation.Around; +import org.aspectj.lang.annotation.Aspect; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.stereotype.Service; + +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Audits connector creation/removal and configuration changes. + */ +@Service +@Aspect +public class ConnectorAuditor extends NiFiAuditor { + + private static final Logger logger = LoggerFactory.getLogger(ConnectorAuditor.class); + + /** + * Audits the creation of connectors via createConnector(). + * + * @param proceedingJoinPoint join point + * @return connector node + * @throws Throwable if an error occurs + */ + @Around("within(org.apache.nifi.web.dao.ConnectorDAO+) && " + + "execution(org.apache.nifi.components.connector.ConnectorNode createConnector(..))") + public ConnectorNode createConnectorAdvice(final ProceedingJoinPoint proceedingJoinPoint) throws Throwable { + final ConnectorNode connector = (ConnectorNode) proceedingJoinPoint.proceed(); + + final Action action = generateAuditRecord(connector, Operation.Add); + if (action != null) { + saveAction(action, logger); + } + + return connector; + } + + /** + * Audits the removal of a connector via deleteConnector(). + * + * @param proceedingJoinPoint join point + * @param connectorId connector id + * @param connectorDAO connector dao + * @throws Throwable if an error occurs + */ + @Around("within(org.apache.nifi.web.dao.ConnectorDAO+) && " + + "execution(void deleteConnector(java.lang.String)) && " + + "args(connectorId) && " + + "target(connectorDAO)") + public void removeConnectorAdvice(final ProceedingJoinPoint proceedingJoinPoint, final String connectorId, final ConnectorDAO connectorDAO) throws Throwable { + final ConnectorNode connector = connectorDAO.getConnector(connectorId); + + proceedingJoinPoint.proceed(); + + final Action action = generateAuditRecord(connector, Operation.Remove); + if (action != null) { + saveAction(action, logger); + } + } + + /** + * Audits the starting of a connector via startConnector(). + * + * @param proceedingJoinPoint join point + * @param connectorId connector id + * @param connectorDAO connector dao + * @throws Throwable if an error occurs + */ + @Around("within(org.apache.nifi.web.dao.ConnectorDAO+) && " + + "execution(void startConnector(java.lang.String)) && " + + "args(connectorId) && " + + "target(connectorDAO)") + public void startConnectorAdvice(final ProceedingJoinPoint proceedingJoinPoint, final String connectorId, final ConnectorDAO connectorDAO) throws Throwable { + final ConnectorNode connector = connectorDAO.getConnector(connectorId); + final ConnectorState previousState = connector.getCurrentState(); + + proceedingJoinPoint.proceed(); + + if (isAuditable() && previousState != ConnectorState.RUNNING && previousState != ConnectorState.STARTING) { + final Action action = generateAuditRecord(connector, Operation.Start); + if (action != null) { + saveAction(action, logger); + } + } + } + + /** + * Audits the stopping of a connector via stopConnector(). + * + * @param proceedingJoinPoint join point + * @param connectorId connector id + * @param connectorDAO connector dao + * @throws Throwable if an error occurs + */ + @Around("within(org.apache.nifi.web.dao.ConnectorDAO+) && " + + "execution(void stopConnector(java.lang.String)) && " + + "args(connectorId) && " + + "target(connectorDAO)") + public void stopConnectorAdvice(final ProceedingJoinPoint proceedingJoinPoint, final String connectorId, final ConnectorDAO connectorDAO) throws Throwable { + final ConnectorNode connector = connectorDAO.getConnector(connectorId); + final ConnectorState previousState = connector.getCurrentState(); + + proceedingJoinPoint.proceed(); + + if (isAuditable() && previousState != ConnectorState.STOPPED && previousState != ConnectorState.STOPPING) { + final Action action = generateAuditRecord(connector, Operation.Stop); + if (action != null) { + saveAction(action, logger); + } + } + } + + /** + * Audits configuration step updates via updateConnectorConfigurationStep(). + * + * @param proceedingJoinPoint join point + * @param connectorId connector id + * @param configurationStepName name of the configuration step + * @param configurationStepConfiguration the configuration step configuration DTO + * @param connectorDAO connector dao + * @throws Throwable if an error occurs + */ + @Around("within(org.apache.nifi.web.dao.ConnectorDAO+) && " + + "execution(void updateConnectorConfigurationStep(java.lang.String, java.lang.String, org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO)) && " + + "args(connectorId, configurationStepName, configurationStepConfiguration) && " + + "target(connectorDAO)") + public void updateConfigurationStepAdvice(final ProceedingJoinPoint proceedingJoinPoint, final String connectorId, final String configurationStepName, + final ConfigurationStepConfigurationDTO configurationStepConfiguration, final ConnectorDAO connectorDAO) throws Throwable { + final ConnectorNode connector = connectorDAO.getConnector(connectorId); + + // Capture the current property values before the update (flat map: property name -> value) + final Map previousValues = extractCurrentPropertyValues(connector, configurationStepName); + + proceedingJoinPoint.proceed(); + + if (isAuditable()) { + // Generate audit actions for each changed property, using the DTO for group structure + final List actions = generateConfigurationChangeActions(connector, configurationStepName, previousValues, configurationStepConfiguration); + if (!actions.isEmpty()) { + saveActions(actions, logger); + } + } + } + + /** + * Extracts the current property values for a specific configuration step from the connector's working flow context. + * + * @param connector the connector node + * @param configurationStepName the name of the configuration step + * @return a map of property name to property value + */ + private Map extractCurrentPropertyValues(final ConnectorNode connector, final String configurationStepName) { + final Map result = new HashMap<>(); + + final ConnectorConfiguration configuration = connector.getWorkingFlowContext().getConfigurationContext().toConnectorConfiguration(); + for (final NamedStepConfiguration namedStepConfig : configuration.getNamedStepConfigurations()) { + if (Objects.equals(namedStepConfig.stepName(), configurationStepName)) { + final StepConfiguration stepConfig = namedStepConfig.configuration(); + for (final Map.Entry entry : stepConfig.getPropertyValues().entrySet()) { + result.put(entry.getKey(), formatValueReference(entry.getValue())); + } + break; + } + } + + return result; + } + + /** + * Formats a ConnectorValueReference as a string for audit logging. + * + * @param valueRef the value reference + * @return the formatted string representation + */ + private String formatValueReference(final ConnectorValueReference valueRef) { + if (valueRef == null) { + return null; + } + + return switch (valueRef) { + case StringLiteralValue stringLiteral -> stringLiteral.getValue(); + case AssetReference assetRef -> "assets:" + assetRef.getAssetIdentifiers(); + case SecretReference secretRef -> "secret:" + secretRef.getProviderId() + "/" + secretRef.getProviderName() + "/" + secretRef.getSecretName(); + }; + } + + /** + * Formats a ConnectorValueReferenceDTO as a string for audit logging. + * + * @param valueRefDto the value reference DTO + * @return the formatted string representation + */ + private String formatValueReferenceDto(final ConnectorValueReferenceDTO valueRefDto) { + if (valueRefDto == null) { + return null; + } + + final String valueType = valueRefDto.getValueType(); + if (valueType == null || "STRING_LITERAL".equals(valueType)) { + return valueRefDto.getValue(); + } else if ("ASSET_REFERENCE".equals(valueType)) { + return "assets:" + Stream.ofNullable(valueRefDto.getAssetReferences()) + .flatMap(List::stream) + .map(AssetReferenceDTO::getId) + .collect(Collectors.joining(",")); + } else if ("SECRET_REFERENCE".equals(valueType)) { + return "secret:" + valueRefDto.getSecretProviderId() + "/" + valueRefDto.getSecretProviderName() + "/" + valueRefDto.getSecretName(); + } + + return valueRefDto.getValue(); + } + + /** + * Generates audit actions for configuration changes by comparing previous values with the new values from the DTO. + * + * @param connector the connector node + * @param configurationStepName the configuration step name + * @param previousValues the previous property values (flat map: property name -> value) + * @param configurationStepDto the configuration step DTO containing new values with group structure + * @return list of actions for changed properties + */ + private List generateConfigurationChangeActions(final ConnectorNode connector, final String configurationStepName, + final Map previousValues, final ConfigurationStepConfigurationDTO configurationStepDto) { + final List actions = new ArrayList<>(); + final Date timestamp = new Date(); + + final FlowChangeExtensionDetails connectorDetails = new FlowChangeExtensionDetails(); + connectorDetails.setType(connector.getComponentType()); + + if (configurationStepDto.getPropertyGroupConfigurations() == null) { + return actions; + } + + // Iterate through all property groups in the DTO to preserve group names for audit logging + for (final PropertyGroupConfigurationDTO groupDto : configurationStepDto.getPropertyGroupConfigurations()) { + final String groupName = groupDto.getPropertyGroupName(); + + if (groupDto.getPropertyValues() == null) { + continue; + } + + // Check each property in this group + for (final Map.Entry propertyEntry : groupDto.getPropertyValues().entrySet()) { + final String propertyName = propertyEntry.getKey(); + final String newValue = formatValueReferenceDto(propertyEntry.getValue()); + final String previousValue = previousValues.get(propertyName); + + // Only create an action if the value changed + if (!Objects.equals(previousValue, newValue)) { + final String fullPropertyName = formatPropertyName(configurationStepName, groupName, propertyName); + + final FlowChangeConfigureDetails actionDetails = new FlowChangeConfigureDetails(); + actionDetails.setName(fullPropertyName); + actionDetails.setPreviousValue(previousValue); + actionDetails.setValue(newValue); + + final FlowChangeAction configurationAction = createFlowChangeAction(); + configurationAction.setOperation(Operation.Configure); + configurationAction.setTimestamp(timestamp); + configurationAction.setSourceId(connector.getIdentifier()); + configurationAction.setSourceName(connector.getName()); + configurationAction.setSourceType(Component.Connector); + configurationAction.setComponentDetails(connectorDetails); + configurationAction.setActionDetails(actionDetails); + + actions.add(configurationAction); + } + } + } + + return actions; + } + + /** + * Formats the property name for audit logging, including step and group context. + * + * @param stepName the configuration step name + * @param groupName the property group name + * @param propertyName the property name + * @return the formatted property name + */ + private String formatPropertyName(final String stepName, final String groupName, final String propertyName) { + if (groupName == null || groupName.isEmpty()) { + return stepName + " / " + propertyName; + } + return stepName + " / " + groupName + " / " + propertyName; + } + + /** + * Audits application of connector updates via applyConnectorUpdate(). + * + * @param proceedingJoinPoint join point + * @param connectorId connector id + * @param connectorDAO connector dao + * @throws Throwable if an error occurs + */ + @Around("within(org.apache.nifi.web.dao.ConnectorDAO+) && " + + "execution(void applyConnectorUpdate(java.lang.String)) && " + + "args(connectorId) && " + + "target(connectorDAO)") + public void applyConnectorUpdateAdvice(final ProceedingJoinPoint proceedingJoinPoint, final String connectorId, final ConnectorDAO connectorDAO) throws Throwable { + final ConnectorNode connector = connectorDAO.getConnector(connectorId); + + proceedingJoinPoint.proceed(); + + if (isAuditable()) { + final FlowChangeExtensionDetails connectorDetails = new FlowChangeExtensionDetails(); + connectorDetails.setType(connector.getComponentType()); + + final FlowChangeConfigureDetails actionDetails = new FlowChangeConfigureDetails(); + actionDetails.setName("Applied Update"); + actionDetails.setValue("true"); + actionDetails.setPreviousValue(null); + + final FlowChangeAction configurationAction = createFlowChangeAction(); + configurationAction.setOperation(Operation.Configure); + configurationAction.setTimestamp(new Date()); + configurationAction.setSourceId(connector.getIdentifier()); + configurationAction.setSourceName(connector.getName()); + configurationAction.setSourceType(Component.Connector); + configurationAction.setComponentDetails(connectorDetails); + configurationAction.setActionDetails(actionDetails); + + saveAction(configurationAction, logger); + } + } + + /** + * Generates an audit record for a connector. + * + * @param connector the connector + * @param operation the operation + * @return the action + */ + public Action generateAuditRecord(final ConnectorNode connector, final Operation operation) { + return generateAuditRecord(connector, operation, null); + } + + /** + * Generates an audit record for a connector. + * + * @param connector the connector + * @param operation the operation + * @param actionDetails the action details + * @return the action + */ + public Action generateAuditRecord(final ConnectorNode connector, final Operation operation, final ActionDetails actionDetails) { + FlowChangeAction action = null; + + if (isAuditable()) { + final FlowChangeExtensionDetails connectorDetails = new FlowChangeExtensionDetails(); + connectorDetails.setType(connector.getComponentType()); + + action = createFlowChangeAction(); + action.setOperation(operation); + action.setSourceId(connector.getIdentifier()); + action.setSourceName(connector.getName()); + action.setSourceType(Component.Connector); + action.setComponentDetails(connectorDetails); + + if (actionDetails != null) { + action.setActionDetails(actionDetails); + } + } + + return action; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java index de725ad0cacf..382126b6754e 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/AuthorizableLookup.java @@ -137,6 +137,15 @@ public interface AuthorizableLookup { */ ConnectionAuthorizable getConnection(String id); + /** + * Get the authorizable Connection, optionally including Connector-managed ProcessGroups in the search. + * + * @param id connection id + * @param includeConnectorManaged whether to search Connector-managed ProcessGroups + * @return authorizable + */ + ConnectionAuthorizable getConnection(String id, boolean includeConnectorManaged); + /** * Get the authorizable root ProcessGroup. * @@ -152,6 +161,15 @@ public interface AuthorizableLookup { */ ProcessGroupAuthorizable getProcessGroup(String id); + /** + * Get the authorizable ProcessGroup, optionally including Connector-managed ProcessGroups in the search. + * + * @param id process group id + * @param includeConnectorManaged whether to search Connector-managed ProcessGroups + * @return authorizable + */ + ProcessGroupAuthorizable getProcessGroup(String id, boolean includeConnectorManaged); + /** * Get the authorizable RemoteProcessGroup. * @@ -256,6 +274,12 @@ public interface AuthorizableLookup { */ Authorizable getParameterContexts(); + /** + * Get the authorizable for Connectors + * @return authorizable + */ + Authorizable getConnectors(); + /** * Get the authorizable connectable. Note this does not include RemoteGroupPorts. * @@ -309,6 +333,13 @@ public interface AuthorizableLookup { */ Authorizable getAuthorizableFromResource(final String resource); + /** + * Get the authorizable for the connector with the given ID + * @param connectorId the ID of the connector + * @return authorizable + */ + Authorizable getConnector(String connectorId); + /** * Get the authorizable for access to the System resource. * diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java index 15d150757216..6e59e3442ecc 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/authorization/StandardAuthorizableLookup.java @@ -58,6 +58,7 @@ import org.apache.nifi.web.controller.ControllerFacade; import org.apache.nifi.web.dao.AccessPolicyDAO; import org.apache.nifi.web.dao.ConnectionDAO; +import org.apache.nifi.web.dao.ConnectorDAO; import org.apache.nifi.web.dao.ControllerServiceDAO; import org.apache.nifi.web.dao.FlowAnalysisRuleDAO; import org.apache.nifi.web.dao.FlowRegistryDAO; @@ -123,6 +124,7 @@ public Resource getResource() { } }; + private static final Authorizable RESOURCE_AUTHORIZABLE = new Authorizable() { @Override public Authorizable getParentAuthorizable() { @@ -184,6 +186,7 @@ public Resource getResource() { private PortDAO inputPortDAO; private PortDAO outputPortDAO; private ConnectionDAO connectionDAO; + private ConnectorDAO connectorDAO; private ControllerServiceDAO controllerServiceDAO; private ReportingTaskDAO reportingTaskDAO; private FlowAnalysisRuleDAO flowAnalysisRuleDAO; @@ -291,7 +294,12 @@ public ParameterContext getParameterContext(final String id) { @Override public ConnectionAuthorizable getConnection(final String id) { - final Connection connection = connectionDAO.getConnection(id); + return getConnection(id, false); + } + + @Override + public ConnectionAuthorizable getConnection(final String id, final boolean includeConnectorManaged) { + final Connection connection = connectionDAO.getConnection(id, includeConnectorManaged); return new StandardConnectionAuthorizable(connection); } @@ -302,7 +310,12 @@ public ProcessGroupAuthorizable getRootProcessGroup() { @Override public ProcessGroupAuthorizable getProcessGroup(final String id) { - final ProcessGroup processGroup = processGroupDAO.getProcessGroup(id); + return getProcessGroup(id, false); + } + + @Override + public ProcessGroupAuthorizable getProcessGroup(final String id, final boolean includeConnectorManaged) { + final ProcessGroup processGroup = processGroupDAO.getProcessGroup(id, includeConnectorManaged); return new StandardProcessGroupAuthorizable(processGroup, controllerFacade.getExtensionManager()); } @@ -605,6 +618,11 @@ public Authorizable getAuthorizableFromResource(final String resource) { } } + @Override + public Authorizable getConnector(final String connectorId) { + return connectorDAO.getConnector(connectorId); + } + private Authorizable handleResourceTypeContainingOtherResourceType(final String resource, final ResourceType resourceType) { // get the resource type final String baseResource = StringUtils.substringAfter(resource, resourceType.getValue()); @@ -648,6 +666,7 @@ private Authorizable getAccessPolicyByResource(final ResourceType resourceType, case FlowAnalysisRule -> getFlowAnalysisRule(componentId).getAuthorizable(); case ParameterContext -> getParameterContext(componentId); case ParameterProvider -> getParameterProvider(componentId).getAuthorizable(); + case Connector -> getConnector(componentId); default -> null; }; @@ -734,6 +753,9 @@ public Resource getResource() { case ParameterContext: authorizable = getParameterContexts(); break; + case Connector: + authorizable = getConnectors(); + break; } if (authorizable == null) { @@ -826,6 +848,21 @@ public Resource getResource() { }; } + @Override + public Authorizable getConnectors() { + return new Authorizable() { + @Override + public Authorizable getParentAuthorizable() { + return null; + } + + @Override + public Resource getResource() { + return ResourceFactory.getConnectorsResource(); + } + }; + } + /** * ComponentAuthorizable for a ConfigurableComponent. This authorizable is intended only to be used when * creating new components. @@ -1407,6 +1444,11 @@ public void setConnectionDAO(ConnectionDAO connectionDAO) { this.connectionDAO = connectionDAO; } + @Autowired + public void setConnectorDAO(ConnectorDAO connectorDAO) { + this.connectorDAO = connectorDAO; + } + @Autowired public void setControllerServiceDAO(ControllerServiceDAO controllerServiceDAO) { this.controllerServiceDAO = controllerServiceDAO; diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java index 3b5babdc0692..cd7d6897a4d1 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java @@ -17,10 +17,12 @@ package org.apache.nifi.web; import io.prometheus.client.CollectorRegistry; +import org.apache.nifi.asset.Asset; import org.apache.nifi.authorization.AuthorizeAccess; import org.apache.nifi.authorization.RequestAction; import org.apache.nifi.authorization.user.NiFiUser; import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.c2.protocol.component.api.ConnectorDefinition; import org.apache.nifi.c2.protocol.component.api.ControllerServiceDefinition; import org.apache.nifi.c2.protocol.component.api.FlowAnalysisRuleDefinition; import org.apache.nifi.c2.protocol.component.api.FlowRegistryClientDefinition; @@ -57,7 +59,9 @@ import org.apache.nifi.web.api.dto.ComponentHistoryDTO; import org.apache.nifi.web.api.dto.ComponentStateDTO; import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO; +import org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ConnectorDTO; import org.apache.nifi.web.api.dto.ControllerConfigurationDTO; import org.apache.nifi.web.api.dto.ControllerDTO; import org.apache.nifi.web.api.dto.ControllerServiceDTO; @@ -106,9 +110,13 @@ import org.apache.nifi.web.api.entity.ClearBulletinsResultEntity; import org.apache.nifi.web.api.entity.ComponentValidationResultEntity; import org.apache.nifi.web.api.entity.ConfigurationAnalysisEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepNamesEntity; import org.apache.nifi.web.api.entity.ConnectionEntity; import org.apache.nifi.web.api.entity.ConnectionStatisticsEntity; import org.apache.nifi.web.api.entity.ConnectionStatusEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ConnectorPropertyAllowableValuesEntity; import org.apache.nifi.web.api.entity.ControllerBulletinsEntity; import org.apache.nifi.web.api.entity.ControllerConfigurationEntity; import org.apache.nifi.web.api.entity.ControllerServiceEntity; @@ -149,6 +157,7 @@ import org.apache.nifi.web.api.entity.RemoteProcessGroupStatusEntity; import org.apache.nifi.web.api.entity.ReportingTaskEntity; import org.apache.nifi.web.api.entity.ScheduleComponentsEntity; +import org.apache.nifi.web.api.entity.SecretsEntity; import org.apache.nifi.web.api.entity.SnippetEntity; import org.apache.nifi.web.api.entity.StartVersionControlRequestEntity; import org.apache.nifi.web.api.entity.StatusHistoryEntity; @@ -170,6 +179,7 @@ import java.util.Date; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.function.Function; import java.util.function.Supplier; @@ -178,6 +188,84 @@ * Defines the NiFiServiceFacade interface. */ public interface NiFiServiceFacade { + // ---------------------------------------- + // Connector methods + // ---------------------------------------- + + void verifyCreateConnector(ConnectorDTO connectorDTO); + + ConnectorEntity createConnector(Revision revision, ConnectorDTO connectorDTO); + + Set getConnectors(); + + ConnectorEntity getConnector(String id); + + void verifyUpdateConnector(ConnectorDTO connectorDTO); + + ConnectorEntity updateConnector(Revision revision, ConnectorDTO connectorDTO); + + void verifyDeleteConnector(String id); + + ConnectorEntity deleteConnector(Revision revision, String id); + + ConnectorEntity scheduleConnector(Revision revision, String id, ScheduledState state); + + void verifyDrainConnector(String id); + + ConnectorEntity drainConnector(Revision revision, String id); + + void verifyCancelConnectorDrain(String id); + + ConnectorEntity cancelConnectorDrain(Revision revision, String id); + + ConfigurationStepNamesEntity getConnectorConfigurationSteps(String id); + + ConfigurationStepEntity getConnectorConfigurationStep(String id, String configurationStepName); + + ConfigurationStepEntity updateConnectorConfigurationStep(Revision revision, String id, String configurationStepName, ConfigurationStepConfigurationDTO configurationStepConfiguration); + + ConnectorEntity applyConnectorUpdate(Revision revision, String connectorId); + + ConnectorEntity discardConnectorUpdate(Revision revision, String connectorId); + + ProcessGroupFlowEntity getConnectorFlow(String connectorId, String processGroupId, boolean uiOnly); + + ProcessGroupStatusEntity getConnectorProcessGroupStatus(String id, Boolean recursive); + + Set getConnectorControllerServices(String connectorId, String processGroupId, boolean includeAncestorGroups, + boolean includeDescendantGroups, boolean includeReferencingComponents); + + void verifyCanVerifyConnectorConfigurationStep(String connectorId, String configurationStepName); + + List performConnectorConfigurationStepVerification(String connectorId, String configurationStepName, ConfigurationStepConfigurationDTO configurationStepConfiguration); + + SearchResultsDTO searchConnector(String connectorId, String query); + + ConnectorPropertyAllowableValuesEntity getConnectorPropertyAllowableValues(String connectorId, String stepName, String groupName, String propertyName, String filter); + + void verifyCreateConnectorAsset(String connectorId); + + AssetEntity createConnectorAsset(String connectorId, String assetId, String assetName, InputStream content) throws IOException; + + List getConnectorAssets(String connectorId); + + Optional getConnectorAsset(String assetId); + + /** + * Verifies that the connector is in a state where FlowFiles can be purged. + * + * @param connectorId the connector ID + * @throws IllegalStateException if the connector is not in a state where FlowFiles can be purged + */ + void verifyPurgeConnectorFlowFiles(String connectorId); + + /** + * Purges all FlowFiles from the connector. + * + * @param connectorId the connector ID + * @param requestor the identity of the user requesting the purge (used for provenance events) + */ + void purgeConnectorFlowFiles(String connectorId, String requestor); // ---------------------------------------- // Synchronization methods @@ -475,6 +563,16 @@ public interface NiFiServiceFacade { */ Set getProcessorTypes(final String bundleGroupFilter, final String bundleArtifactFilter, final String typeFilter); + /** + * Returns the list of connector types. + * + * @param bundleGroupFilter if specified, must be member of bundle group + * @param bundleArtifactFilter if specified, must be member of bundle artifact + * @param typeFilter if specified, type must match + * @return The list of available connector types matching specified criteria + */ + Set getConnectorTypes(final String bundleGroupFilter, final String bundleArtifactFilter, final String typeFilter); + /** * Returns the list of controller service types. * @@ -571,6 +669,17 @@ Set getControllerServiceTypes(final String serviceType, final */ FlowAnalysisRuleDefinition getFlowAnalysisRuleDefinition(String group, String artifact, String version, String type); + /** + * Return the ConnectorDefinition for the specified Connector. + * + * @param group The bundle group + * @param artifact The bundle artifact + * @param version The bundle version + * @param type The Connector type + * @return The ConnectorDefinition + */ + ConnectorDefinition getConnectorDefinition(String group, String artifact, String version, String type); + /** * Return the additionalDetails for the specified component. * @@ -582,6 +691,18 @@ Set getControllerServiceTypes(final String serviceType, final */ String getAdditionalDetails(String group, String artifact, String version, String type); + /** + * Return the step documentation for the specified Connector configuration step. + * + * @param group The bundle group + * @param artifact The bundle artifact + * @param version The bundle version + * @param connectorType The fully qualified class name of the Connector + * @param stepName The name of the configuration step + * @return The step documentation markdown content + */ + String getStepDocumentation(String group, String artifact, String version, String connectorType, String stepName); + /** * Returns the list of parameter provider types. * @@ -1975,6 +2096,60 @@ ProcessGroupEntity updateProcessGroupContents(Revision revision, String groupId, */ ComponentStateDTO getRemoteProcessGroupState(String remoteProcessGroupId); + /** + * Gets the state for a processor within a connector's managed process group. + * + * @param connectorId the connector id + * @param processorId the processor id + * @return the component state + */ + ComponentStateDTO getConnectorProcessorState(String connectorId, String processorId); + + /** + * Verifies the processor state within a connector could be cleared. + * + * @param connectorId the connector id + * @param processorId the processor id + */ + void verifyCanClearConnectorProcessorState(String connectorId, String processorId); + + /** + * Clears the state for a processor within a connector's managed process group. + * + * @param connectorId the connector id + * @param processorId the processor id + * @param componentStateDTO the state of the processor + * @return the cleared component state + */ + ComponentStateDTO clearConnectorProcessorState(String connectorId, String processorId, ComponentStateDTO componentStateDTO); + + /** + * Gets the state for a controller service within a connector's managed process group. + * + * @param connectorId the connector id + * @param controllerServiceId the controller service id + * @return the component state + */ + ComponentStateDTO getConnectorControllerServiceState(String connectorId, String controllerServiceId); + + /** + * Verifies the controller service state within a connector could be cleared. + * + * @param connectorId the connector id + * @param controllerServiceId the controller service id + */ + void verifyCanClearConnectorControllerServiceState(String connectorId, String controllerServiceId); + + /** + * Clears the state for a controller service within a connector's managed process group. + * + * @param connectorId the connector id + * @param controllerServiceId the controller service id + * @param componentStateDTO the state of the controller service + * @return the cleared component state + */ + ComponentStateDTO clearConnectorControllerServiceState(String connectorId, String controllerServiceId, ComponentStateDTO componentStateDTO); + // ---------------------------------------- // Label methods // ---------------------------------------- @@ -3132,4 +3307,16 @@ ControllerServiceReferencingComponentsEntity updateControllerServiceReferencingC * @return the list of listen Ports accessible to the current user */ Set getListenPorts(NiFiUser user); + + // ---------------------------------------- + // Secrets methods + // ---------------------------------------- + + /** + * Gets all secrets available from all secret providers. Note: The actual secret values are not included + * in the response for security reasons; only metadata is returned. + * + * @return the secrets entity containing metadata for all available secrets + */ + SecretsEntity getSecrets(); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiResourceConfig.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiResourceConfig.java index 3baf1b987522..978977b13f9e 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiResourceConfig.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiWebApiResourceConfig.java @@ -104,6 +104,7 @@ public NiFiWebApiResourceConfig(@Context ServletContext servletContext) { register(ctx.getBean("versionsResource")); register(ctx.getBean("parameterContextResource")); register(ctx.getBean("parameterProviderResource")); + register(ctx.getBean("connectorResource")); // exception mappers register(AccessDeniedExceptionMapper.class); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java index 940310d3e8c0..bdf92b099056 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java @@ -52,6 +52,7 @@ import org.apache.nifi.authorization.user.NiFiUserUtils; import org.apache.nifi.bundle.Bundle; import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.c2.protocol.component.api.ConnectorDefinition; import org.apache.nifi.c2.protocol.component.api.ControllerServiceDefinition; import org.apache.nifi.c2.protocol.component.api.FlowAnalysisRuleDefinition; import org.apache.nifi.c2.protocol.component.api.FlowRegistryClientDefinition; @@ -71,11 +72,19 @@ import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException; import org.apache.nifi.cluster.manager.exception.UnknownNodeException; import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.components.ConfigVerificationResult; import org.apache.nifi.components.ConfigurableComponent; +import org.apache.nifi.components.DescribedValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.RequiredPermission; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.Validator; +import org.apache.nifi.components.connector.Connector; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorState; +import org.apache.nifi.components.connector.ConnectorUpdateContext; +import org.apache.nifi.components.connector.Secret; +import org.apache.nifi.components.connector.secrets.AuthorizableSecret; import org.apache.nifi.components.state.Scope; import org.apache.nifi.components.state.StateMap; import org.apache.nifi.components.validation.ValidationState; @@ -162,7 +171,7 @@ import org.apache.nifi.parameter.ParameterLookup; import org.apache.nifi.parameter.ParameterProvider; import org.apache.nifi.parameter.ParameterReferenceManager; -import org.apache.nifi.parameter.StandardParameterContext; +import org.apache.nifi.parameter.StandardParameterContext.Builder; import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.VerifiableProcessor; import org.apache.nifi.prometheusutil.AbstractMetricsRegistry; @@ -208,7 +217,7 @@ import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedPort; import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedProcessGroup; import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedRemoteGroupPort; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import org.apache.nifi.registry.flow.mapping.VersionedComponentStateLookup; import org.apache.nifi.remote.RemoteGroupPort; import org.apache.nifi.reporting.Bulletin; @@ -241,7 +250,9 @@ import org.apache.nifi.web.api.dto.ComponentValidationResultDTO; import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO; import org.apache.nifi.web.api.dto.ConfigurationAnalysisDTO; +import org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ConnectorDTO; import org.apache.nifi.web.api.dto.ControllerConfigurationDTO; import org.apache.nifi.web.api.dto.ControllerDTO; import org.apache.nifi.web.api.dto.ControllerServiceDTO; @@ -289,6 +300,7 @@ import org.apache.nifi.web.api.dto.RequiredPermissionDTO; import org.apache.nifi.web.api.dto.ResourceDTO; import org.apache.nifi.web.api.dto.RevisionDTO; +import org.apache.nifi.web.api.dto.SecretDTO; import org.apache.nifi.web.api.dto.SnippetDTO; import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO; import org.apache.nifi.web.api.dto.TenantDTO; @@ -312,6 +324,7 @@ import org.apache.nifi.web.api.dto.search.SearchResultsDTO; import org.apache.nifi.web.api.dto.status.ConnectionStatisticsDTO; import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO; +import org.apache.nifi.web.api.dto.status.ConnectorStatusDTO; import org.apache.nifi.web.api.dto.status.ControllerStatusDTO; import org.apache.nifi.web.api.dto.status.NodeProcessGroupStatusSnapshotDTO; import org.apache.nifi.web.api.dto.status.PortStatusDTO; @@ -326,6 +339,7 @@ import org.apache.nifi.web.api.entity.ActionEntity; import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity; import org.apache.nifi.web.api.entity.AffectedComponentEntity; +import org.apache.nifi.web.api.entity.AllowableValueEntity; import org.apache.nifi.web.api.entity.AssetEntity; import org.apache.nifi.web.api.entity.BulletinEntity; import org.apache.nifi.web.api.entity.ClearBulletinsForGroupResultsEntity; @@ -333,9 +347,13 @@ import org.apache.nifi.web.api.entity.ComponentReferenceEntity; import org.apache.nifi.web.api.entity.ComponentValidationResultEntity; import org.apache.nifi.web.api.entity.ConfigurationAnalysisEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepNamesEntity; import org.apache.nifi.web.api.entity.ConnectionEntity; import org.apache.nifi.web.api.entity.ConnectionStatisticsEntity; import org.apache.nifi.web.api.entity.ConnectionStatusEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ConnectorPropertyAllowableValuesEntity; import org.apache.nifi.web.api.entity.ControllerBulletinsEntity; import org.apache.nifi.web.api.entity.ControllerConfigurationEntity; import org.apache.nifi.web.api.entity.ControllerServiceEntity; @@ -381,6 +399,7 @@ import org.apache.nifi.web.api.entity.RemoteProcessGroupStatusEntity; import org.apache.nifi.web.api.entity.ReportingTaskEntity; import org.apache.nifi.web.api.entity.ScheduleComponentsEntity; +import org.apache.nifi.web.api.entity.SecretsEntity; import org.apache.nifi.web.api.entity.SnippetEntity; import org.apache.nifi.web.api.entity.StartVersionControlRequestEntity; import org.apache.nifi.web.api.entity.StatusHistoryEntity; @@ -397,7 +416,9 @@ import org.apache.nifi.web.api.request.FlowMetricsReportingStrategy; import org.apache.nifi.web.controller.ControllerFacade; import org.apache.nifi.web.dao.AccessPolicyDAO; +import org.apache.nifi.web.dao.ComponentStateDAO; import org.apache.nifi.web.dao.ConnectionDAO; +import org.apache.nifi.web.dao.ConnectorDAO; import org.apache.nifi.web.dao.ControllerServiceDAO; import org.apache.nifi.web.dao.FlowAnalysisRuleDAO; import org.apache.nifi.web.dao.FlowRegistryDAO; @@ -448,6 +469,7 @@ import java.util.List; import java.util.ListIterator; import java.util.Map; +import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -477,6 +499,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { private BulletinRepository bulletinRepository; // data access objects + private ComponentStateDAO componentStateDAO; private ProcessorDAO processorDAO; private ProcessGroupDAO processGroupDAO; private RemoteProcessGroupDAO remoteProcessGroupDAO; @@ -488,6 +511,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade { private ConnectionDAO connectionDAO; private ControllerServiceDAO controllerServiceDAO; private ReportingTaskDAO reportingTaskDAO; + private ConnectorDAO connectorDAO; private FlowAnalysisRuleDAO flowAnalysisRuleDAO; private ParameterProviderDAO parameterProviderDAO; private UserDAO userDAO; @@ -778,6 +802,11 @@ public void verifyCreateReportingTask(ReportingTaskDTO reportingTaskDTO) { reportingTaskDAO.verifyCreate(reportingTaskDTO); } + @Override + public void verifyCreateConnector(final ConnectorDTO connectorDTO) { + connectorDAO.verifyCreate(connectorDTO); + } + @Override public void verifyUpdateReportingTask(final ReportingTaskDTO reportingTaskDTO) { // if tasks does not exist, then the update request is likely creating it @@ -1398,7 +1427,7 @@ private ParameterContextEntity createParameterContextEntity(final ParameterConte public List validateComponents(final ParameterContextDTO parameterContextDto, final NiFiUser nifiUser) { final ParameterContext parameterContext = parameterContextDAO.getParameterContext(parameterContextDto.getId()); final Set boundProcessGroups = parameterContext.getParameterReferenceManager().getProcessGroupsBound(parameterContext); - final ParameterContext updatedParameterContext = new StandardParameterContext.Builder() + final ParameterContext updatedParameterContext = new Builder() .id(parameterContext.getIdentifier()) .name(parameterContext.getName()) .parameterReferenceManager(ParameterReferenceManager.EMPTY) @@ -1736,7 +1765,7 @@ private void setEffectiveParameterUpdates(final ParameterContextDTO parameterCon .collect(Collectors.toMap(entity -> entity.getParameter().getName(), Function.identity())); parameterContextDto.getParameters().clear(); - for (final Map.Entry entry : proposedParameterUpdates.entrySet()) { + for (final Entry entry : proposedParameterUpdates.entrySet()) { final String parameterName = entry.getKey(); final Parameter parameter = entry.getValue(); final ParameterEntity parameterEntity; @@ -2066,6 +2095,68 @@ public ComponentStateDTO getRemoteProcessGroupState(String remoteProcessGroupId) return dtoFactory.createComponentStateDTO(remoteProcessGroupId, remoteProcessGroup.getClass(), localState, clusterState); } + @Override + public ComponentStateDTO getConnectorProcessorState(final String connectorId, final String processorId) { + final ProcessorNode processor = locateConnectorProcessor(connectorId, processorId); + final StateMap clusterState = isClustered() ? componentStateDAO.getState(processor, Scope.CLUSTER) : null; + final StateMap localState = componentStateDAO.getState(processor, Scope.LOCAL); + return dtoFactory.createComponentStateDTO(processorId, processor.getProcessor().getClass(), localState, clusterState); + } + + @Override + public void verifyCanClearConnectorProcessorState(final String connectorId, final String processorId) { + final ProcessorNode processor = locateConnectorProcessor(connectorId, processorId); + processor.verifyCanClearState(); + } + + @Override + public ComponentStateDTO clearConnectorProcessorState(final String connectorId, final String processorId, final ComponentStateDTO componentStateDTO) { + final ProcessorNode processor = locateConnectorProcessor(connectorId, processorId); + componentStateDAO.clearState(processor, componentStateDTO); + return getConnectorProcessorState(connectorId, processorId); + } + + @Override + public ComponentStateDTO getConnectorControllerServiceState(final String connectorId, final String controllerServiceId) { + final ControllerServiceNode controllerService = locateConnectorControllerService(connectorId, controllerServiceId); + final StateMap clusterState = isClustered() ? componentStateDAO.getState(controllerService, Scope.CLUSTER) : null; + final StateMap localState = componentStateDAO.getState(controllerService, Scope.LOCAL); + return dtoFactory.createComponentStateDTO(controllerServiceId, controllerService.getControllerServiceImplementation().getClass(), localState, clusterState); + } + + @Override + public void verifyCanClearConnectorControllerServiceState(final String connectorId, final String controllerServiceId) { + final ControllerServiceNode controllerService = locateConnectorControllerService(connectorId, controllerServiceId); + controllerService.verifyCanClearState(); + } + + @Override + public ComponentStateDTO clearConnectorControllerServiceState(final String connectorId, final String controllerServiceId, final ComponentStateDTO componentStateDTO) { + final ControllerServiceNode controllerService = locateConnectorControllerService(connectorId, controllerServiceId); + componentStateDAO.clearState(controllerService, componentStateDTO); + return getConnectorControllerServiceState(connectorId, controllerServiceId); + } + + private ProcessorNode locateConnectorProcessor(final String connectorId, final String processorId) { + final ConnectorNode connectorNode = connectorDAO.getConnector(connectorId); + final ProcessGroup managedGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + final ProcessorNode processor = managedGroup.findProcessor(processorId); + if (processor == null) { + throw new ResourceNotFoundException("Unable to find processor with id '%s' within connector '%s'.".formatted(processorId, connectorId)); + } + return processor; + } + + private ControllerServiceNode locateConnectorControllerService(final String connectorId, final String controllerServiceId) { + final ConnectorNode connectorNode = connectorDAO.getConnector(connectorId); + final ProcessGroup managedGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + final ControllerServiceNode controllerService = managedGroup.findControllerService(controllerServiceId, false, true); + if (controllerService == null) { + throw new ResourceNotFoundException("Unable to find controller service with id '%s' within connector '%s'.".formatted(controllerServiceId, connectorId)); + } + return controllerService; + } + @Override public ConnectionEntity deleteConnection(final Revision revision, final String connectionId) { final Connection connection = connectionDAO.getConnection(connectionId); @@ -2087,7 +2178,7 @@ public DropRequestDTO deleteFlowFileDropRequest(final String connectionId, final @Override public ListingRequestDTO deleteFlowFileListingRequest(final String connectionId, final String listingRequestId) { - final Connection connection = connectionDAO.getConnection(connectionId); + final Connection connection = connectionDAO.getConnection(connectionId, true); final ListingRequestDTO listRequest = dtoFactory.createListingRequestDTO(connectionDAO.deleteFlowFileListingRequest(connectionId, listingRequestId)); // include whether the source and destination are running @@ -2498,7 +2589,7 @@ public DropRequestDTO createFlowFileDropRequest(final String connectionId, final @Override public ListingRequestDTO createFlowFileListingRequest(final String connectionId, final String listingRequestId) { - final Connection connection = connectionDAO.getConnection(connectionId); + final Connection connection = connectionDAO.getConnection(connectionId, true); final ListingRequestDTO listRequest = dtoFactory.createListingRequestDTO(connectionDAO.createFlowFileListingRequest(connectionId, listingRequestId)); // include whether the source and destination are running @@ -3448,6 +3539,405 @@ public ReportingTaskEntity createReportingTask(final Revision revision, final Re return entityFactory.createReportingTaskEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, bulletinEntities); } + private ConnectorStatusDTO createConnectorStatusDto(final ConnectorNode connectorNode) { + final ProcessGroup managedProcessGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + final ProcessGroupStatus managedGroupStatus = controllerFacade.getProcessGroupStatus(managedProcessGroup.getIdentifier()); + return dtoFactory.createConnectorStatusDto(connectorNode, managedGroupStatus); + } + + @Override + public ConnectorEntity createConnector(final Revision revision, final ConnectorDTO connectorDTO) { + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + final RevisionClaim claim = new StandardRevisionClaim(revision); + + final RevisionUpdate snapshot = revisionManager.updateRevision(claim, user, () -> { + final BundleCoordinate coordinate = BundleUtils.getCompatibleBundle(controllerFacade.getExtensionManager(), connectorDTO.getType(), connectorDTO.getBundle()); + final ConnectorNode connector = connectorDAO.createConnector(connectorDTO.getType(), connectorDTO.getId(), coordinate, true, true); + + controllerFacade.save(); + + final ConnectorDTO dto = dtoFactory.createConnectorDto(connector); + final FlowModification lastMod = new FlowModification(revision.incrementRevision(revision.getClientId()), user.getIdentity()); + return new StandardRevisionUpdate<>(dto, lastMod); + }); + + final ConnectorNode connector = connectorDAO.getConnector(snapshot.getComponent().getId()); + final PermissionsDTO permissions = dtoFactory.createPermissionsDto(connector); + final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(connector)); + final ConnectorStatusDTO status = createConnectorStatusDto(connector); + return entityFactory.createConnectorEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, status); + } + + @Override + public Set getConnectors() { + return connectorDAO.getConnectors().stream().map(node -> { + final ConnectorDTO dto = dtoFactory.createConnectorDto(node); + final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(node.getIdentifier())); + final PermissionsDTO permissions = dtoFactory.createPermissionsDto(node); + final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(node)); + final ConnectorStatusDTO status = createConnectorStatusDto(node); + return entityFactory.createConnectorEntity(dto, revision, permissions, operatePermissions, status); + }).collect(Collectors.toSet()); + } + + @Override + public ConnectorEntity getConnector(final String id) { + final ConnectorNode node = connectorDAO.getConnector(id); + final ConnectorDTO dto = dtoFactory.createConnectorDto(node); + final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(node.getIdentifier())); + final PermissionsDTO permissions = dtoFactory.createPermissionsDto(node); + final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(node)); + final ConnectorStatusDTO status = createConnectorStatusDto(node); + return entityFactory.createConnectorEntity(dto, revision, permissions, operatePermissions, status); + } + + @Override + public void verifyUpdateConnector(final ConnectorDTO connectorDTO) { + // No-op placeholder for future detailed verification + } + + @Override + public ConnectorEntity updateConnector(final Revision revision, final ConnectorDTO connectorDTO) { + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + final RevisionClaim claim = new StandardRevisionClaim(revision); + + final RevisionUpdate snapshot = revisionManager.updateRevision(claim, user, () -> { + connectorDAO.updateConnector(connectorDTO); + + controllerFacade.save(); + + final ConnectorNode node = connectorDAO.getConnector(connectorDTO.getId()); + final ConnectorDTO dto = dtoFactory.createConnectorDto(node); + final FlowModification lastMod = new FlowModification(revision.incrementRevision(revision.getClientId()), user.getIdentity()); + return new StandardRevisionUpdate<>(dto, lastMod); + }); + + final ConnectorNode node = connectorDAO.getConnector(snapshot.getComponent().getId()); + final PermissionsDTO permissions = dtoFactory.createPermissionsDto(node); + final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(node)); + final ConnectorStatusDTO statusDto = createConnectorStatusDto(node); + return entityFactory.createConnectorEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, statusDto); + } + + @Override + public void verifyDeleteConnector(final String id) { + // For now, DAO will enforce state; expose hook for symmetry + } + + @Override + public ConnectorEntity deleteConnector(final Revision revision, final String id) { + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + final RevisionClaim claim = new StandardRevisionClaim(revision); + + final RevisionUpdate snapshot = revisionManager.updateRevision(claim, user, () -> { + final ConnectorNode node = connectorDAO.getConnector(id); + final ConnectorDTO dto = dtoFactory.createConnectorDto(node); + connectorDAO.deleteConnector(id); + controllerFacade.save(); + + final FlowModification lastMod = new FlowModification(revision.incrementRevision(revision.getClientId()), user.getIdentity()); + return new StandardRevisionUpdate<>(dto, lastMod); + }); + + final ConnectorDTO dto = snapshot.getComponent(); + final PermissionsDTO permissions = new PermissionsDTO(); + permissions.setCanRead(Boolean.FALSE); + permissions.setCanWrite(Boolean.FALSE); + final PermissionsDTO operatePermissions = new PermissionsDTO(); + operatePermissions.setCanRead(Boolean.FALSE); + operatePermissions.setCanWrite(Boolean.FALSE); + return entityFactory.createConnectorEntity(dto, dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, null); + } + + @Override + public ConnectorEntity scheduleConnector(final Revision revision, final String id, final ScheduledState state) { + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + final RevisionClaim claim = new StandardRevisionClaim(revision); + + final RevisionUpdate snapshot = revisionManager.updateRevision(claim, user, () -> { + switch (state) { + case RUNNING -> connectorDAO.startConnector(id); + case STOPPED -> connectorDAO.stopConnector(id); + default -> throw new IllegalArgumentException("Unsupported scheduled state for Connector: " + state); + } + controllerFacade.save(); + + final ConnectorNode node = connectorDAO.getConnector(id); + final ConnectorDTO dto = dtoFactory.createConnectorDto(node); + final FlowModification lastMod = new FlowModification(revision.incrementRevision(revision.getClientId()), user.getIdentity()); + return new StandardRevisionUpdate<>(dto, lastMod); + }); + + final ConnectorNode node = connectorDAO.getConnector(snapshot.getComponent().getId()); + final PermissionsDTO permissions = dtoFactory.createPermissionsDto(node); + final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(node)); + final ConnectorStatusDTO statusDto = createConnectorStatusDto(node); + return entityFactory.createConnectorEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, statusDto); + } + + @Override + public void verifyDrainConnector(final String id) { + final ConnectorNode connector = connectorDAO.getConnector(id); + final ConnectorState currentState = connector.getCurrentState(); + if (currentState != ConnectorState.STOPPED) { + throw new IllegalStateException("Cannot drain FlowFiles for Connector " + id + " because it is not currently stopped. Current state: " + currentState); + } + } + + @Override + public ConnectorEntity drainConnector(final Revision revision, final String id) { + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + final RevisionClaim claim = new StandardRevisionClaim(revision); + + final RevisionUpdate snapshot = revisionManager.updateRevision(claim, user, () -> { + connectorDAO.drainFlowFiles(id); + controllerFacade.save(); + + final ConnectorNode node = connectorDAO.getConnector(id); + final ConnectorDTO dto = dtoFactory.createConnectorDto(node); + final FlowModification lastMod = new FlowModification(revision.incrementRevision(revision.getClientId()), user.getIdentity()); + return new StandardRevisionUpdate<>(dto, lastMod); + }); + + final ConnectorNode node = connectorDAO.getConnector(snapshot.getComponent().getId()); + final PermissionsDTO permissions = dtoFactory.createPermissionsDto(node); + final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(node)); + final ConnectorStatusDTO statusDto = createConnectorStatusDto(node); + return entityFactory.createConnectorEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, statusDto); + } + + @Override + public void verifyCancelConnectorDrain(final String id) { + connectorDAO.verifyCancelDrainFlowFile(id); + } + + @Override + public ConnectorEntity cancelConnectorDrain(final Revision revision, final String id) { + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + final RevisionClaim claim = new StandardRevisionClaim(revision); + + final RevisionUpdate snapshot = revisionManager.updateRevision(claim, user, () -> { + connectorDAO.cancelDrainFlowFiles(id); + controllerFacade.save(); + + final ConnectorNode node = connectorDAO.getConnector(id); + final ConnectorDTO dto = dtoFactory.createConnectorDto(node); + final FlowModification lastMod = new FlowModification(revision.incrementRevision(revision.getClientId()), user.getIdentity()); + return new StandardRevisionUpdate<>(dto, lastMod); + }); + + final ConnectorNode node = connectorDAO.getConnector(snapshot.getComponent().getId()); + final PermissionsDTO permissions = dtoFactory.createPermissionsDto(node); + final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(node)); + final ConnectorStatusDTO statusDto = createConnectorStatusDto(node); + return entityFactory.createConnectorEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, statusDto); + } + + @Override + public ConfigurationStepNamesEntity getConnectorConfigurationSteps(final String id) { + final ConnectorNode node = connectorDAO.getConnector(id); + final ConnectorDTO dto = dtoFactory.createConnectorDto(node); + + return entityFactory.createConfigurationStepNamesEntity(dto); + } + + @Override + public ConfigurationStepEntity getConnectorConfigurationStep(final String id, final String configurationStepName) { + final ConnectorNode node = connectorDAO.getConnector(id); + final ConnectorDTO dto = dtoFactory.createConnectorDto(node); + final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(node.getIdentifier())); + + return entityFactory.createConfigurationStepEntity(dto, configurationStepName, revision); + } + + @Override + public ConfigurationStepEntity updateConnectorConfigurationStep(final Revision revision, final String id, + final String configurationStepName, final ConfigurationStepConfigurationDTO configurationStepConfiguration) { + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + final RevisionClaim claim = new StandardRevisionClaim(revision); + + final RevisionUpdate snapshot = revisionManager.updateRevision(claim, user, () -> { + // Update the configuration step + connectorDAO.updateConnectorConfigurationStep(id, configurationStepName, configurationStepConfiguration); + controllerFacade.save(); + + // Return updated connector DTO + final ConnectorNode node = connectorDAO.getConnector(id); + final ConnectorDTO dto = dtoFactory.createConnectorDto(node); + final FlowModification lastMod = new FlowModification(revision.incrementRevision(revision.getClientId()), user.getIdentity()); + return new StandardRevisionUpdate<>(dto, lastMod); + }); + + // Create and return the configuration step entity + return entityFactory.createConfigurationStepEntity(snapshot.getComponent(), configurationStepName, dtoFactory.createRevisionDTO(snapshot.getLastModification())); + } + + @Override + public ConnectorEntity applyConnectorUpdate(final Revision revision, final String connectorId) { + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + final RevisionClaim claim = new StandardRevisionClaim(revision); + + final ConnectorUpdateContext updateContext = controllerFacade::saveImmediate; + + final RevisionUpdate snapshot = revisionManager.updateRevision(claim, user, () -> { + connectorDAO.applyConnectorUpdate(connectorId, updateContext); + + final ConnectorNode node = connectorDAO.getConnector(connectorId); + final ConnectorDTO dto = dtoFactory.createConnectorDto(node); + final FlowModification lastMod = new FlowModification(revision.incrementRevision(revision.getClientId()), user.getIdentity()); + return new StandardRevisionUpdate<>(dto, lastMod); + }); + + final ConnectorNode node = connectorDAO.getConnector(snapshot.getComponent().getId()); + final PermissionsDTO permissions = dtoFactory.createPermissionsDto(node); + final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(node)); + final ConnectorStatusDTO statusDto = createConnectorStatusDto(node); + return entityFactory.createConnectorEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, statusDto); + } + + @Override + public ConnectorEntity discardConnectorUpdate(final Revision revision, final String connectorId) { + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + final RevisionClaim claim = new StandardRevisionClaim(revision); + + final RevisionUpdate snapshot = revisionManager.updateRevision(claim, user, () -> { + connectorDAO.discardWorkingConfiguration(connectorId); + + final ConnectorNode node = connectorDAO.getConnector(connectorId); + final ConnectorDTO dto = dtoFactory.createConnectorDto(node); + final FlowModification lastMod = new FlowModification(revision.incrementRevision(revision.getClientId()), user.getIdentity()); + return new StandardRevisionUpdate<>(dto, lastMod); + }); + + final ConnectorNode node = connectorDAO.getConnector(snapshot.getComponent().getId()); + final PermissionsDTO permissions = dtoFactory.createPermissionsDto(node); + final PermissionsDTO operatePermissions = dtoFactory.createPermissionsDto(new OperationAuthorizable(node)); + final ConnectorStatusDTO statusDto = createConnectorStatusDto(node); + return entityFactory.createConnectorEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), permissions, operatePermissions, statusDto); + } + + @Override + public ProcessGroupFlowEntity getConnectorFlow(final String connectorId, final String processGroupId, final boolean uiOnly) { + final ConnectorNode connectorNode = connectorDAO.getConnector(connectorId); + final ProcessGroup managedProcessGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + final ProcessGroup targetProcessGroup = managedProcessGroup.findProcessGroup(processGroupId); + if (targetProcessGroup == null) { + throw new ResourceNotFoundException("Process Group with ID " + processGroupId + " was not found within Connector " + connectorId); + } + return createProcessGroupFlowEntity(targetProcessGroup, uiOnly); + } + + @Override + public ProcessGroupStatusEntity getConnectorProcessGroupStatus(final String id, final Boolean recursive) { + final ConnectorNode connectorNode = connectorDAO.getConnector(id); + final ProcessGroup managedProcessGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + final String processGroupId = managedProcessGroup.getIdentifier(); + + final PermissionsDTO permissions = dtoFactory.createPermissionsDto(connectorNode); + final ProcessGroupStatusDTO dto = dtoFactory.createProcessGroupStatusDto(managedProcessGroup, controllerFacade.getProcessGroupStatus(processGroupId)); + + // prune the response as necessary + if (!Boolean.TRUE.equals(recursive)) { + pruneChildGroups(dto.getAggregateSnapshot()); + if (dto.getNodeSnapshots() != null) { + for (final NodeProcessGroupStatusSnapshotDTO nodeSnapshot : dto.getNodeSnapshots()) { + pruneChildGroups(nodeSnapshot.getStatusSnapshot()); + } + } + } + + return entityFactory.createProcessGroupStatusEntity(dto, permissions); + } + + @Override + public Set getConnectorControllerServices(final String connectorId, final String processGroupId, + final boolean includeAncestorGroups, final boolean includeDescendantGroups, final boolean includeReferencingComponents) { + final ConnectorNode connectorNode = connectorDAO.getConnector(connectorId); + final ProcessGroup managedProcessGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + final ProcessGroup targetProcessGroup = managedProcessGroup.findProcessGroup(processGroupId); + if (targetProcessGroup == null) { + throw new ResourceNotFoundException("Process Group with ID " + processGroupId + " was not found within Connector " + connectorId); + } + + final Set serviceNodes = new HashSet<>(); + serviceNodes.addAll(targetProcessGroup.getControllerServices(includeAncestorGroups)); + + if (includeDescendantGroups) { + serviceNodes.addAll(targetProcessGroup.findAllControllerServices()); + } + + return serviceNodes.stream() + .map(serviceNode -> createControllerServiceEntity(serviceNode, includeReferencingComponents)) + .collect(Collectors.toSet()); + } + + @Override + public void verifyCanVerifyConnectorConfigurationStep(final String connectorId, final String configurationStepName) { + connectorDAO.verifyCanVerifyConfigurationStep(connectorId, configurationStepName); + } + + @Override + public List performConnectorConfigurationStepVerification(final String connectorId, + final String configurationStepName, final ConfigurationStepConfigurationDTO configurationStepConfiguration) { + final List verificationResults = connectorDAO.verifyConfigurationStep(connectorId, configurationStepName, configurationStepConfiguration); + return verificationResults.stream() + .map(result -> dtoFactory.createConfigVerificationResultDto(result)) + .collect(Collectors.toList()); + } + + @Override + public SearchResultsDTO searchConnector(final String connectorId, final String query) { + final ConnectorNode connectorNode = connectorDAO.getConnector(connectorId); + final ProcessGroup managedProcessGroup = connectorNode.getActiveFlowContext().getManagedProcessGroup(); + return controllerFacade.searchConnector(query, managedProcessGroup); + } + + @Override + public ConnectorPropertyAllowableValuesEntity getConnectorPropertyAllowableValues(final String connectorId, + final String stepName, final String groupName, final String propertyName, final String filter) { + // groupName is retained for REST API backward compatibility but is no longer used by the underlying API + final List allowableValues = connectorDAO.fetchAllowableValues(connectorId, stepName, propertyName, filter); + + final List allowableValueEntities = allowableValues.stream() + .map(av -> entityFactory.createAllowableValueEntity(dtoFactory.createAllowableValueDto(av), true)) + .collect(Collectors.toList()); + + return entityFactory.createConnectorPropertyAllowableValuesEntity(stepName, groupName, propertyName, allowableValueEntities); + } + + @Override + public void verifyCreateConnectorAsset(final String connectorId) { + connectorDAO.verifyCreateAsset(connectorId); + } + + @Override + public AssetEntity createConnectorAsset(final String connectorId, final String assetId, final String assetName, final InputStream content) throws IOException { + final Asset createdAsset = connectorDAO.createAsset(connectorId, assetId, assetName, content); + return dtoFactory.createAssetEntity(createdAsset); + } + + @Override + public List getConnectorAssets(final String connectorId) { + return connectorDAO.getAssets(connectorId).stream() + .map(dtoFactory::createAssetEntity) + .toList(); + } + + @Override + public Optional getConnectorAsset(final String assetId) { + return connectorDAO.getAsset(assetId); + } + + @Override + public void verifyPurgeConnectorFlowFiles(final String connectorId) { + connectorDAO.verifyPurgeFlowFiles(connectorId); + } + + @Override + public void purgeConnectorFlowFiles(final String connectorId, final String requestor) { + connectorDAO.purgeFlowFiles(connectorId, requestor); + } + @Override public ReportingTaskEntity updateReportingTask(final Revision revision, final ReportingTaskDTO reportingTaskDTO) { // get the component, ensure we have access to it, and perform the update request @@ -3616,7 +4106,7 @@ public List getParameterContextUpdatesForAppliedParamete )); final Set updatedParameterEntities = new LinkedHashSet<>(); entity.getComponent().setParameters(updatedParameterEntities); - for (final Map.Entry parameterEntry : parameterUpdates.entrySet()) { + for (final Entry parameterEntry : parameterUpdates.entrySet()) { final String parameterName = parameterEntry.getKey(); final Parameter parameter = parameterEntry.getValue(); final ParameterEntity parameterEntity; @@ -3861,7 +4351,7 @@ public DropRequestDTO getFlowFileDropRequest(final String connectionId, final St @Override public ListingRequestDTO getFlowFileListingRequest(final String connectionId, final String listingRequestId) { - final Connection connection = connectionDAO.getConnection(connectionId); + final Connection connection = connectionDAO.getConnection(connectionId, true); final ListingRequestDTO listRequest = dtoFactory.createListingRequestDTO(connectionDAO.getFlowFileListingRequest(connectionId, listingRequestId)); // include whether the source and destination are running @@ -3964,6 +4454,11 @@ public Set getProcessorTypes(final String bundleGroup, final return controllerFacade.getFlowFileProcessorTypes(bundleGroup, bundleArtifact, type); } + @Override + public Set getConnectorTypes(final String bundleGroup, final String bundleArtifact, final String type) { + return controllerFacade.getConnectorTypes(bundleGroup, bundleArtifact, type); + } + @Override public Set getControllerServiceTypes(final String serviceType, final String serviceBundleGroup, final String serviceBundleArtifact, final String serviceBundleVersion, final String bundleGroup, final String bundleArtifact, final String type) { @@ -4039,11 +4534,25 @@ public FlowAnalysisRuleDefinition getFlowAnalysisRuleDefinition(String group, St return flowAnalysisRuleDefinition; } + @Override + public ConnectorDefinition getConnectorDefinition(final String group, final String artifact, final String version, final String type) { + final ConnectorDefinition connectorDefinition = controllerFacade.getConnectorDefinition(group, artifact, version, type); + if (connectorDefinition == null) { + throw new ResourceNotFoundException("Unable to find definition for [%s:%s:%s:%s]".formatted(group, artifact, version, type)); + } + return connectorDefinition; + } + @Override public String getAdditionalDetails(String group, String artifact, String version, String type) { return controllerFacade.getAdditionalDetails(group, artifact, version, type); } + @Override + public String getStepDocumentation(final String group, final String artifact, final String version, final String connectorType, final String stepName) { + return controllerFacade.getStepDocumentation(group, artifact, version, connectorType, stepName); + } + @Override public Set getParameterProviderTypes(final String bundleGroup, final String bundleArtifact, final String type) { return controllerFacade.getParameterProviderTypes(bundleGroup, bundleArtifact, type); @@ -4103,6 +4612,7 @@ private boolean authorizeBulletin(final Bulletin bulletin) { case OUTPUT_PORT -> authorizableLookup.getOutputPort(sourceId); case REMOTE_PROCESS_GROUP -> authorizableLookup.getRemoteProcessGroup(sourceId); case PROCESS_GROUP -> authorizableLookup.getProcessGroup(sourceId).getAuthorizable(); + case CONNECTOR -> authorizableLookup.getConnector(sourceId); default -> throw new IllegalArgumentException("Unexpected ComponentType: " + type); }; } catch (final ResourceNotFoundException e) { @@ -4951,6 +5461,7 @@ public CurrentUserEntity getCurrentUser() { entity.setPoliciesPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getPolicies())); entity.setSystemPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getSystem())); entity.setParameterContextPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getParameterContexts())); + entity.setConnectorsPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getConnectors())); entity.setCanVersionFlows(CollectionUtils.isNotEmpty(flowRegistryDAO.getFlowRegistryClients().stream().map(c -> c.getIdentifier()).collect(Collectors.toSet()))); entity.setRestrictedComponentsPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getRestrictedComponents())); @@ -4977,12 +5488,15 @@ public CurrentUserEntity getCurrentUser() { @Override public ProcessGroupFlowEntity getProcessGroupFlow(final String groupId, final boolean uiOnly) { final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId); + return createProcessGroupFlowEntity(processGroup, uiOnly); + } + private ProcessGroupFlowEntity createProcessGroupFlowEntity(final ProcessGroup processGroup, final boolean uiOnly) { // Get the Process Group Status but we only need a status depth of one because for any child process group, // we ignore the status of each individual components. I.e., if Process Group A has child Group B, and child Group B // has a Processor, we don't care about the individual stats of that Processor because the ProcessGroupFlowEntity // doesn't include that anyway. So we can avoid including the information in the status that is returned. - final ProcessGroupStatus groupStatus = controllerFacade.getProcessGroupStatus(groupId, 1); + final ProcessGroupStatus groupStatus = controllerFacade.getProcessGroupStatus(processGroup.getIdentifier(), 1); final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(processGroup.getIdentifier())); final PermissionsDTO permissions = dtoFactory.createPermissionsDto(processGroup); return entityFactory.createProcessGroupFlowEntity(dtoFactory.createProcessGroupFlowDto(processGroup, groupStatus, @@ -5308,7 +5822,7 @@ public CopyResponseEntity copyComponents(final String groupId, final CopyRequest .mapAssetReferences(false) .build(); - final NiFiRegistryFlowMapper mapper = makeNiFiRegistryFlowMapper(controllerFacade.getExtensionManager(), mappingOptions); + final VersionedComponentFlowMapper mapper = makeNiFiRegistryFlowMapper(controllerFacade.getExtensionManager(), mappingOptions); final InstantiatedVersionedProcessGroup versionedProcessGroup = mapper.mapProcessGroup(processGroup, controllerFacade.getControllerServiceProvider(), controllerFacade.getFlowManager(), true); @@ -5341,7 +5855,7 @@ public CopyResponseEntity copyComponents(final String groupId, final CopyRequest final Map externalControllerServices = versionedProcessGroup.getExternalControllerServiceReferences().entrySet().stream() .filter(e -> isServiceReferenced(e.getKey(), versionedProcessors, Collections.emptySet(), versionedProcessGroups)) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + .collect(Collectors.toMap(Entry::getKey, Entry::getValue)); // move any service at the current level into external services versionedProcessGroup.getControllerServices().stream() @@ -5417,7 +5931,7 @@ private RegisteredFlowSnapshot getCurrentFlowSnapshotByGroupId(final String proc // Create a complete (include descendant flows) VersionedProcessGroup snapshot of the flow as it is // currently without any registry related fields populated, even if the flow is currently versioned. - final NiFiRegistryFlowMapper mapper = makeNiFiRegistryFlowMapper(controllerFacade.getExtensionManager()); + final VersionedComponentFlowMapper mapper = makeNiFiRegistryFlowMapper(controllerFacade.getExtensionManager()); final InstantiatedVersionedProcessGroup nonVersionedProcessGroup = mapper.mapNonVersionedProcessGroup(processGroup, controllerFacade.getControllerServiceProvider()); @@ -5592,13 +6106,13 @@ public VersionControlInformationEntity getVersionControlInformation(final String private InstantiatedVersionedProcessGroup createFlowSnapshot(final String processGroupId) { final ProcessGroup processGroup = processGroupDAO.getProcessGroup(processGroupId); - final NiFiRegistryFlowMapper mapper = makeNiFiRegistryFlowMapper(controllerFacade.getExtensionManager()); + final VersionedComponentFlowMapper mapper = makeNiFiRegistryFlowMapper(controllerFacade.getExtensionManager()); final InstantiatedVersionedProcessGroup versionedGroup = mapper.mapProcessGroup(processGroup, controllerFacade.getControllerServiceProvider(), controllerFacade.getFlowManager(), false); return versionedGroup; } private Map createVersionedParameterContexts(final ProcessGroup processGroup, final Map parameterProviderReferences) { - final NiFiRegistryFlowMapper mapper = makeNiFiRegistryFlowMapper(controllerFacade.getExtensionManager()); + final VersionedComponentFlowMapper mapper = makeNiFiRegistryFlowMapper(controllerFacade.getExtensionManager()); return mapper.mapParameterContexts(processGroup, false, parameterProviderReferences); } @@ -5632,7 +6146,7 @@ public FlowComparisonEntity getLocalModifications(final String processGroupId) { throw new NiFiCoreException("Failed to retrieve flow with Flow Registry in order to calculate local differences due to " + e.getMessage(), e); } - final NiFiRegistryFlowMapper mapper = makeNiFiRegistryFlowMapper(controllerFacade.getExtensionManager()); + final VersionedComponentFlowMapper mapper = makeNiFiRegistryFlowMapper(controllerFacade.getExtensionManager()); final VersionedProcessGroup localGroup = mapper.mapProcessGroup(processGroup, controllerFacade.getControllerServiceProvider(), controllerFacade.getFlowManager(), true); final ComparableDataFlow localFlow = new StandardComparableDataFlow("Local Flow", localGroup); @@ -5796,7 +6310,7 @@ public void verifyCanRevertLocalModifications(final String groupId, final Regist public Set getComponentsAffectedByFlowUpdate(final String processGroupId, final RegisteredFlowSnapshot updatedSnapshot) { final ProcessGroup group = processGroupDAO.getProcessGroup(processGroupId); - final NiFiRegistryFlowMapper mapper = makeNiFiRegistryFlowMapper(controllerFacade.getExtensionManager()); + final VersionedComponentFlowMapper mapper = makeNiFiRegistryFlowMapper(controllerFacade.getExtensionManager()); final VersionedProcessGroup localContents = mapper.mapProcessGroup(group, controllerFacade.getControllerServiceProvider(), controllerFacade.getFlowManager(), true); final ComparableDataFlow localFlow = new StandardComparableDataFlow("Current Flow", localContents); @@ -6029,7 +6543,7 @@ private void mapToConnectableId(final Collection connecta if (versionedIdOption.isPresent()) { versionedId = versionedIdOption.get(); } else { - versionedId = NiFiRegistryFlowMapper.generateVersionedComponentId(connectable.getIdentifier()); + versionedId = VersionedComponentFlowMapper.generateVersionedComponentId(connectable.getIdentifier()); } final List byVersionedId = destination.computeIfAbsent(versionedId, key -> new ArrayList<>()); @@ -6455,6 +6969,7 @@ private AuthorizationResult authorizeAction(final Action action) { case AccessPolicy -> authorizableLookup.getAccessPolicyById(sourceId); case User, UserGroup -> authorizableLookup.getTenant(); case Label -> authorizableLookup.getLabel(sourceId); + case Connector -> authorizableLookup.getConnector(sourceId); default -> throw new IllegalArgumentException("Unexpected Component: " + type); }; } catch (final ResourceNotFoundException e) { @@ -6522,7 +7037,7 @@ public ComponentHistoryDTO getComponentHistory(final String componentId) { final Map propertyHistoryDtos = new LinkedHashMap<>(); final Map> propertyHistory = auditService.getPreviousValues(componentId); - for (final Map.Entry> entry : propertyHistory.entrySet()) { + for (final Entry> entry : propertyHistory.entrySet()) { final List previousValueDtos = new ArrayList<>(); for (final PreviousValue previousValue : entry.getValue()) { @@ -7044,6 +7559,7 @@ public NarDetailsEntity getNarDetails(final String identifier) { componentTypesEntity.setParameterProviderTypes(dtoFactory.fromDocumentedTypes(getTypes(extensionDefinitions, ParameterProvider.class))); componentTypesEntity.setFlowRegistryClientTypes(dtoFactory.fromDocumentedTypes(getTypes(extensionDefinitions, FlowRegistryClient.class))); componentTypesEntity.setFlowAnalysisRuleTypes(dtoFactory.fromDocumentedTypes(getTypes(extensionDefinitions, FlowAnalysisRule.class))); + componentTypesEntity.setConnectorTypes(dtoFactory.fromDocumentedTypes(getTypes(extensionDefinitions, Connector.class))); return componentTypesEntity; } @@ -7242,8 +7758,8 @@ public void verifyPublicOutputPortUniqueness(final String portId, final String p * @param extensionManager the extension manager to create the flow mapper with * @return a new NiFiRegistryFlowMapper instance */ - protected NiFiRegistryFlowMapper makeNiFiRegistryFlowMapper(final ExtensionManager extensionManager) { - return new NiFiRegistryFlowMapper(extensionManager); + protected VersionedComponentFlowMapper makeNiFiRegistryFlowMapper(final ExtensionManager extensionManager) { + return new VersionedComponentFlowMapper(extensionManager); } /** @@ -7253,8 +7769,8 @@ protected NiFiRegistryFlowMapper makeNiFiRegistryFlowMapper(final ExtensionManag * @param options the flow mapping options * @return a new NiFiRegistryFlowMapper instance */ - protected NiFiRegistryFlowMapper makeNiFiRegistryFlowMapper(final ExtensionManager extensionManager, final FlowMappingOptions options) { - return new NiFiRegistryFlowMapper(extensionManager, options); + protected VersionedComponentFlowMapper makeNiFiRegistryFlowMapper(final ExtensionManager extensionManager, final FlowMappingOptions options) { + return new VersionedComponentFlowMapper(extensionManager, options); } @Override @@ -7349,6 +7865,11 @@ public void setReportingTaskDAO(final ReportingTaskDAO reportingTaskDAO) { this.reportingTaskDAO = reportingTaskDAO; } + @Autowired + public void setConnectorDAO(final ConnectorDAO connectorDAO) { + this.connectorDAO = connectorDAO; + } + @Autowired public void setFlowAnalysisRuleDAO(FlowAnalysisRuleDAO flowAnalysisRuleDAO) { this.flowAnalysisRuleDAO = flowAnalysisRuleDAO; @@ -7359,6 +7880,11 @@ public void setParameterProviderDAO(final ParameterProviderDAO parameterProvider this.parameterProviderDAO = parameterProviderDAO; } + @Autowired + public void setComponentStateDAO(final ComponentStateDAO componentStateDAO) { + this.componentStateDAO = componentStateDAO; + } + @Autowired public void setParameterContextDAO(final ParameterContextDAO parameterContextDAO) { this.parameterContextDAO = parameterContextDAO; @@ -7489,4 +8015,23 @@ public Set filterComponents(final String groupId, final Function secrets = controllerFacade.getAllSecrets(); + final List secretDtos = secrets.stream() + .filter(secret -> isSecretAuthorized(secret, user)) + .map(dtoFactory::createSecretDto) + .toList(); + return entityFactory.createSecretsEntity(secretDtos); + } + + private boolean isSecretAuthorized(final Secret secret, final NiFiUser user) { + if (secret instanceof AuthorizableSecret authorizableSecret) { + final AuthorizationResult result = authorizableSecret.checkAuthorization(authorizer, RequestAction.READ, user); + return Result.Approved.equals(result.getResult()); + } + return true; + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectorResource.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectorResource.java new file mode 100644 index 000000000000..c25cd0f450c7 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ConnectorResource.java @@ -0,0 +1,2419 @@ +/* + * 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; + +import io.swagger.v3.oas.annotations.Operation; +import io.swagger.v3.oas.annotations.Parameter; +import io.swagger.v3.oas.annotations.media.Content; +import io.swagger.v3.oas.annotations.media.Schema; +import io.swagger.v3.oas.annotations.responses.ApiResponse; +import io.swagger.v3.oas.annotations.security.SecurityRequirement; +import io.swagger.v3.oas.annotations.tags.Tag; +import jakarta.servlet.ServletContext; +import jakarta.ws.rs.Consumes; +import jakarta.ws.rs.DELETE; +import jakarta.ws.rs.DefaultValue; +import jakarta.ws.rs.GET; +import jakarta.ws.rs.HeaderParam; +import jakarta.ws.rs.HttpMethod; +import jakarta.ws.rs.POST; +import jakarta.ws.rs.PUT; +import jakarta.ws.rs.Path; +import jakarta.ws.rs.PathParam; +import jakarta.ws.rs.Produces; +import jakarta.ws.rs.QueryParam; +import jakarta.ws.rs.core.Context; +import jakarta.ws.rs.core.HttpHeaders; +import jakarta.ws.rs.core.MediaType; +import jakarta.ws.rs.core.Response; +import jakarta.ws.rs.core.StreamingOutput; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.asset.Asset; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.RequestAction; +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.resource.DataAuthorizable; +import org.apache.nifi.authorization.resource.OperationAuthorizable; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.authorization.user.NiFiUserUtils; +import org.apache.nifi.cluster.coordination.ClusterCoordinator; +import org.apache.nifi.cluster.coordination.http.replication.RequestReplicationHeader; +import org.apache.nifi.cluster.coordination.http.replication.UploadRequest; +import org.apache.nifi.cluster.coordination.http.replication.UploadRequestReplicator; +import org.apache.nifi.cluster.coordination.node.NodeConnectionState; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.processor.DataUnit; +import org.apache.nifi.stream.io.MaxLengthInputStream; +import org.apache.nifi.ui.extension.UiExtension; +import org.apache.nifi.ui.extension.UiExtensionMapping; +import org.apache.nifi.util.file.FileUtils; +import org.apache.nifi.web.NiFiServiceFacade; +import org.apache.nifi.web.ResourceNotFoundException; +import org.apache.nifi.web.Revision; +import org.apache.nifi.web.UiExtensionType; +import org.apache.nifi.web.api.concurrent.AsyncRequestManager; +import org.apache.nifi.web.api.concurrent.AsynchronousWebRequest; +import org.apache.nifi.web.api.concurrent.RequestManager; +import org.apache.nifi.web.api.concurrent.StandardAsynchronousWebRequest; +import org.apache.nifi.web.api.concurrent.StandardUpdateStep; +import org.apache.nifi.web.api.concurrent.UpdateStep; +import org.apache.nifi.web.api.dto.AssetDTO; +import org.apache.nifi.web.api.dto.BundleDTO; +import org.apache.nifi.web.api.dto.ComponentStateDTO; +import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO; +import org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorDTO; +import org.apache.nifi.web.api.dto.DropRequestDTO; +import org.apache.nifi.web.api.dto.VerifyConnectorConfigStepRequestDTO; +import org.apache.nifi.web.api.dto.search.SearchResultsDTO; +import org.apache.nifi.web.api.entity.AssetEntity; +import org.apache.nifi.web.api.entity.AssetsEntity; +import org.apache.nifi.web.api.entity.ComponentStateEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepNamesEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ConnectorPropertyAllowableValuesEntity; +import org.apache.nifi.web.api.entity.ConnectorRunStatusEntity; +import org.apache.nifi.web.api.entity.ControllerServiceEntity; +import org.apache.nifi.web.api.entity.ControllerServicesEntity; +import org.apache.nifi.web.api.entity.DropRequestEntity; +import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity; +import org.apache.nifi.web.api.entity.ProcessGroupStatusEntity; +import org.apache.nifi.web.api.entity.SearchResultsEntity; +import org.apache.nifi.web.api.entity.SecretsEntity; +import org.apache.nifi.web.api.entity.VerifyConnectorConfigStepRequestEntity; +import org.apache.nifi.web.api.request.ClientIdParameter; +import org.apache.nifi.web.api.request.LongParameter; +import org.apache.nifi.web.client.api.HttpResponseStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Controller; + +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; +import java.util.Collections; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.function.Consumer; + +/** + * RESTful endpoint for managing a Connector. + */ +@Controller +@Path("/connectors") +@Tag(name = "Connectors") +public class ConnectorResource extends ApplicationResource { + + private static final Logger logger = LoggerFactory.getLogger(ConnectorResource.class); + private static final String VERIFICATION_REQUEST_TYPE = "verification-request"; + private static final String PURGE_REQUEST_TYPE = "purge-request"; + private static final String FILENAME_HEADER = "Filename"; + private static final String CONTENT_TYPE_HEADER = "Content-Type"; + private static final String UPLOAD_CONTENT_TYPE = "application/octet-stream"; + private static final long MAX_ASSET_SIZE_BYTES = (long) DataUnit.GB.toB(1); + + private NiFiServiceFacade serviceFacade; + private Authorizer authorizer; + private FlowResource flowResource; + private ControllerServiceResource controllerServiceResource; + private UploadRequestReplicator uploadRequestReplicator; + + private final RequestManager> configVerificationRequestManager = + new AsyncRequestManager<>(100, 1L, "Connector Configuration Step Verification"); + + private final RequestManager purgeRequestManager = new AsyncRequestManager<>(100, 1L, "Connector FlowFile Purge"); + + @Context + private ServletContext servletContext; + + /** + * Populate the uri's for the specified connectors. + * + * @param connectorEntities connectors + * @return connector entities + */ + public Set populateRemainingConnectorEntitiesContent(final Set connectorEntities) { + for (ConnectorEntity connectorEntity : connectorEntities) { + populateRemainingConnectorEntityContent(connectorEntity); + } + return connectorEntities; + } + + /** + * Populate the uri's for the specified connector. + * + * @param connectorEntity connector + * @return connector entity + */ + public ConnectorEntity populateRemainingConnectorEntityContent(final ConnectorEntity connectorEntity) { + connectorEntity.setUri(generateResourceUri("connectors", connectorEntity.getId())); + + // populate the remaining content + if (connectorEntity.getComponent() != null) { + populateRemainingConnectorContent(connectorEntity.getComponent()); + } + return connectorEntity; + } + + /** + * Populates the uri for the specified connector including custom UI information. + */ + public ConnectorDTO populateRemainingConnectorContent(final ConnectorDTO connector) { + final BundleDTO bundle = connector.getBundle(); + if (bundle == null) { + return connector; + } + + // see if this connector has any ui extensions + final UiExtensionMapping uiExtensionMapping = (UiExtensionMapping) servletContext.getAttribute("nifi-ui-extensions"); + if (uiExtensionMapping.hasUiExtension(connector.getType(), bundle.getGroup(), bundle.getArtifact(), bundle.getVersion())) { + final List uiExtensions = uiExtensionMapping.getUiExtension(connector.getType(), bundle.getGroup(), bundle.getArtifact(), bundle.getVersion()); + for (final UiExtension uiExtension : uiExtensions) { + if (UiExtensionType.Connector.equals(uiExtension.getExtensionType())) { + final String contextPath = uiExtension.getContextPath(); + final Map routes = uiExtension.getSupportedRoutes(); + + if (routes != null) { + final String configurationPath = routes.get("configuration"); + if (configurationPath != null) { + connector.setConfigurationUrl(buildCustomUiUrl(contextPath, configurationPath)); + } + + final String detailsPath = routes.get("details"); + if (detailsPath != null) { + connector.setDetailsUrl(buildCustomUiUrl(contextPath, detailsPath)); + } + } + } + } + } + + return connector; + } + + /** + * Builds a custom UI URL from the context path and route path. + * Handles both hash-based routing (path starts with #) and location-based routing. + * + * @param contextPath the context path of the custom UI + * @param routePath the route path (e.g., "#/wizard" for hash-based or "/wizard" for location-based) + * @return the full URL for the custom UI route + */ + private String buildCustomUiUrl(final String contextPath, final String routePath) { + final String baseUrl = generateExternalUiUri(contextPath); + if (routePath.startsWith("#")) { + // Hash-based routing: /context-path/#/route + return baseUrl + "/" + routePath; + } else { + // Location-based routing: /context-path/route + return baseUrl + routePath; + } + } + + /** + * Creates a new connector. + * + * @param requestConnectorEntity A connectorEntity. + * @return A connectorEntity. + */ + @POST + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @Operation( + summary = "Creates a new connector", + responses = { + @ApiResponse(responseCode = "201", content = @Content(schema = @Schema(implementation = ConnectorEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Write - /connectors") + } + ) + public Response createConnector( + @Parameter(description = "The connector configuration details.", required = true) final ConnectorEntity requestConnectorEntity) { + + if (requestConnectorEntity == null || requestConnectorEntity.getComponent() == null) { + throw new IllegalArgumentException("Connector details must be specified."); + } + + if (requestConnectorEntity.getRevision() == null || (requestConnectorEntity.getRevision().getVersion() == null || requestConnectorEntity.getRevision().getVersion() != 0)) { + throw new IllegalArgumentException("A revision of 0 must be specified when creating a new Connector."); + } + + final ConnectorDTO requestConnector = requestConnectorEntity.getComponent(); + + if (StringUtils.isBlank(requestConnector.getType())) { + throw new IllegalArgumentException("The type of connector to create must be specified."); + } + + if (isReplicateRequest()) { + return replicate(HttpMethod.POST, requestConnectorEntity); + } else if (isDisconnectedFromCluster()) { + verifyDisconnectedNodeModification(requestConnectorEntity.isDisconnectedNodeAcknowledged()); + } + + return withWriteLock( + serviceFacade, + requestConnectorEntity, + lookup -> { + final Authorizable connectors = lookup.getConnectors(); + connectors.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); + }, + () -> serviceFacade.verifyCreateConnector(requestConnector), + connectorEntity -> { + final ConnectorDTO connector = connectorEntity.getComponent(); + + // set the connector id as appropriate + final String clientSpecifiedId = connector.getId(); + if (clientSpecifiedId != null) { + // validate the client-specified connector id. uniqueness is verified by verifyCreateConnector() + connector.setId(normalizeUuid(clientSpecifiedId)); + } else { + connector.setId(generateUuid()); + } + + // create the new connector + final Revision revision = getRevision(connectorEntity, connector.getId()); + final ConnectorEntity entity = serviceFacade.createConnector(revision, connector); + populateRemainingConnectorEntityContent(entity); + + // generate a 201 created response + final String uri = entity.getUri(); + return generateCreatedResponse(URI.create(uri), entity).build(); + } + ); + } + + /** + * Validates client-specified id is a UUID and normalizes its string format to the lowercase digest used by NiFi. + * + * @param clientSpecifiedId the client-passed ID, which should be a valid UUID + * @return the normalized string representation of the validated UUID + * @throws IllegalArgumentException if the clientSpecifiedId is not a valid UUID + */ + private String normalizeUuid(final String clientSpecifiedId) { + try { + final UUID uuid = UUID.fromString(clientSpecifiedId); + logger.debug("ID [{}] is a valid UUID", clientSpecifiedId); + return uuid.toString(); + } catch (final Exception e) { + logger.error("ID [{}] is not a valid UUID", clientSpecifiedId, e); + throw new IllegalArgumentException("ID [" + clientSpecifiedId + "] is not a valid UUID."); + } + } + + /** + * Retrieves the specified connector. + * + * @param id The id of the connector to retrieve + * @return A connectorEntity. + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}") + @Operation( + summary = "Gets a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConnectorEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Read - /connectors/{uuid}") + } + ) + public Response getConnector( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + // authorize access + serviceFacade.authorizeAccess(lookup -> { + final Authorizable connector = lookup.getConnector(id); + connector.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); + }); + + // get the connector + final ConnectorEntity entity = serviceFacade.getConnector(id); + populateRemainingConnectorEntityContent(entity); + + return generateOkResponse(entity).build(); + } + + /** + * Gets all available secrets from the SecretsManager for configuring a specific connector. + * + * @param id The id of the connector being configured + * @return A secretsEntity containing metadata for all available secrets. + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/secrets") + @Operation( + summary = "Gets all secrets available for configuring a connector", + description = "Returns metadata for all secrets available from all secret providers. " + + "This endpoint is used when configuring a connector to discover available secrets. " + + "Note: Actual secret values are not included in the response for security reasons.", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = SecretsEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid}") + } + ) + public Response getSecrets( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id) { + + // NOTE: fetching secrets is handled by the node that receives the request and does not need to be replicated + // Secrets are sourced from ParameterProviders which should have consistent configuration across the cluster + + // authorize access - require write permission on the specific connector since this is used for configuration + serviceFacade.authorizeAccess(lookup -> { + final Authorizable connector = lookup.getConnector(id); + connector.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); + }); + + final SecretsEntity entity = serviceFacade.getSecrets(); + return generateOkResponse(entity).build(); + } + + /** + * Updates the specified connector. + * + * @param id The id of the connector to update. + * @param requestConnectorEntity A connectorEntity. + * @return A connectorEntity. + */ + @PUT + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}") + @Operation( + summary = "Updates a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConnectorEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid}") + } + ) + public Response updateConnector( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id, + @Parameter( + description = "The connector configuration details.", + required = true + ) final ConnectorEntity requestConnectorEntity) { + + if (requestConnectorEntity == null || requestConnectorEntity.getComponent() == null) { + throw new IllegalArgumentException("Connector details must be specified."); + } + + if (requestConnectorEntity.getRevision() == null) { + throw new IllegalArgumentException("Revision must be specified."); + } + + // ensure the ids are the same + final ConnectorDTO requestConnectorDTO = requestConnectorEntity.getComponent(); + if (!id.equals(requestConnectorDTO.getId())) { + throw new IllegalArgumentException(String.format("The connector id (%s) in the request body does not equal the " + + "connector id of the requested resource (%s).", requestConnectorDTO.getId(), id)); + } + + if (isReplicateRequest()) { + return replicate(HttpMethod.PUT, requestConnectorEntity); + } else if (isDisconnectedFromCluster()) { + verifyDisconnectedNodeModification(requestConnectorEntity.isDisconnectedNodeAcknowledged()); + } + + // handle expects request (usually from the cluster manager) + final Revision requestRevision = getRevision(requestConnectorEntity, id); + return withWriteLock( + serviceFacade, + requestConnectorEntity, + requestRevision, + lookup -> { + final Authorizable connector = lookup.getConnector(id); + connector.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); + }, + () -> serviceFacade.verifyUpdateConnector(requestConnectorDTO), + (revision, connectorEntity) -> { + final ConnectorDTO connectorDTO = connectorEntity.getComponent(); + + // update the connector + final ConnectorEntity entity = serviceFacade.updateConnector(revision, connectorDTO); + populateRemainingConnectorEntityContent(entity); + + return generateOkResponse(entity).build(); + } + ); + } + + /** + * Removes the specified connector. + * + * @param version The revision is used to verify the client is working with the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response. + * @param id The id of the connector to remove. + * @return A connectorEntity. + */ + @DELETE + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}") + @Operation( + summary = "Deletes a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConnectorEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid}") + } + ) + public Response deleteConnector( + @Parameter( + description = "The revision is used to verify the client is working with the latest version of the flow." + ) + @QueryParam(VERSION) final LongParameter version, + @Parameter( + description = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response." + ) + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId, + @Parameter( + description = "Acknowledges that this node is disconnected to allow for mutable requests to proceed." + ) + @QueryParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged, + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.DELETE); + } else if (isDisconnectedFromCluster()) { + verifyDisconnectedNodeModification(disconnectedNodeAcknowledged); + } + + final ConnectorEntity requestConnectorEntity = new ConnectorEntity(); + requestConnectorEntity.setId(id); + + // handle expects request (usually from the cluster manager) + final Revision requestRevision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id); + return withWriteLock( + serviceFacade, + requestConnectorEntity, + requestRevision, + lookup -> { + final Authorizable connector = lookup.getConnector(id); + connector.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); + }, + () -> serviceFacade.verifyDeleteConnector(id), + (revision, connectorEntity) -> { + // delete the specified connector + final ConnectorEntity entity = serviceFacade.deleteConnector(revision, connectorEntity.getId()); + return generateOkResponse(entity).build(); + } + ); + } + + /** + * Updates run status of a connector. + * + * @param id The id of the connector to update. + * @param requestRunStatus A connectorRunStatusEntity. + * @return A connectorEntity. + */ + @PUT + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/run-status") + @Operation( + summary = "Updates run status of a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConnectorEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid} or /operation/connectors/{uuid}") + } + ) + public Response updateRunStatus( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id, + @Parameter( + description = "The connector run status.", + required = true + ) final ConnectorRunStatusEntity requestRunStatus) { + + if (requestRunStatus == null) { + throw new IllegalArgumentException("Connector run status must be specified."); + } + + if (requestRunStatus.getRevision() == null) { + throw new IllegalArgumentException("Revision must be specified."); + } + + requestRunStatus.validateState(); + + if (isReplicateRequest()) { + return replicate(HttpMethod.PUT, requestRunStatus); + } else if (isDisconnectedFromCluster()) { + verifyDisconnectedNodeModification(requestRunStatus.isDisconnectedNodeAcknowledged()); + } + + // handle expects request (usually from the cluster manager) + final Revision requestRevision = getRevision(requestRunStatus.getRevision(), id); + return withWriteLock( + serviceFacade, + requestRunStatus, + requestRevision, + lookup -> { + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + + final Authorizable connector = lookup.getConnector(id); + OperationAuthorizable.authorizeOperation(connector, authorizer, user); + }, + () -> serviceFacade.verifyUpdateConnector(createDTOWithDesiredRunStatus(id, requestRunStatus.getState())), + (revision, runStatusEntity) -> { + // update the connector + final ScheduledState scheduledState = ScheduledState.valueOf(runStatusEntity.getState()); + final ConnectorEntity entity = serviceFacade.scheduleConnector(revision, id, scheduledState); + populateRemainingConnectorEntityContent(entity); + + return generateOkResponse(entity).build(); + } + ); + } + + /** + * Initiates the draining of FlowFiles for the specified connector. + * + * @param id The id of the connector to drain. + * @param requestConnectorEntity A connectorEntity containing the revision. + * @return A connectorEntity. + */ + @POST + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/drain") + @Operation( + summary = "Initiates draining of FlowFiles for a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConnectorEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + description = "This will initiate draining of FlowFiles for a stopped connector. Draining allows the connector to process " + + "data that is currently in the flow but does not ingest any additional data. The connector must be in a STOPPED state " + + "before draining can begin. Once initiated, the connector will transition to a DRAINING state. Use the DELETE method " + + "on this endpoint to cancel an ongoing drain operation.", + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid} or /operation/connectors/{uuid}") + } + ) + public Response initiateDrain( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id, + @Parameter( + description = "The connector entity with revision.", + required = true + ) final ConnectorEntity requestConnectorEntity) { + + if (requestConnectorEntity == null || requestConnectorEntity.getRevision() == null) { + throw new IllegalArgumentException("Connector entity with revision must be specified."); + } + + if (requestConnectorEntity.getId() != null && !id.equals(requestConnectorEntity.getId())) { + throw new IllegalArgumentException(String.format("The connector id (%s) in the request body does not equal the " + + "connector id of the requested resource (%s).", requestConnectorEntity.getId(), id)); + } + + if (isReplicateRequest()) { + return replicate(HttpMethod.POST, requestConnectorEntity); + } else if (isDisconnectedFromCluster()) { + verifyDisconnectedNodeModification(requestConnectorEntity.isDisconnectedNodeAcknowledged()); + } + + final Revision requestRevision = getRevision(requestConnectorEntity, id); + return withWriteLock( + serviceFacade, + requestConnectorEntity, + requestRevision, + lookup -> { + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + final Authorizable connector = lookup.getConnector(id); + OperationAuthorizable.authorizeOperation(connector, authorizer, user); + }, + () -> serviceFacade.verifyDrainConnector(id), + (revision, connectorEntity) -> { + final ConnectorEntity entity = serviceFacade.drainConnector(revision, id); + populateRemainingConnectorEntityContent(entity); + + return generateOkResponse(entity).build(); + } + ); + } + + /** + * Cancels the draining of FlowFiles for the specified connector. + * + * @param version The revision is used to verify the client is working with the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response. + * @param id The id of the connector to cancel draining for. + * @return A connectorEntity. + */ + @DELETE + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/drain") + @Operation( + summary = "Cancels the draining of FlowFiles for a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConnectorEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid} or /operation/connectors/{uuid}") + } + ) + public Response cancelDrain( + @Parameter( + description = "The revision is used to verify the client is working with the latest version of the flow." + ) + @QueryParam(VERSION) final LongParameter version, + @Parameter( + description = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response." + ) + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId, + @Parameter( + description = "Acknowledges that this node is disconnected to allow for mutable requests to proceed." + ) + @QueryParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged, + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.DELETE); + } else if (isDisconnectedFromCluster()) { + verifyDisconnectedNodeModification(disconnectedNodeAcknowledged); + } + + final ConnectorEntity requestConnectorEntity = new ConnectorEntity(); + requestConnectorEntity.setId(id); + + final Revision requestRevision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id); + return withWriteLock( + serviceFacade, + requestConnectorEntity, + requestRevision, + lookup -> { + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + final Authorizable connector = lookup.getConnector(id); + OperationAuthorizable.authorizeOperation(connector, authorizer, user); + }, + () -> serviceFacade.verifyCancelConnectorDrain(id), + (revision, connectorEntity) -> { + final ConnectorEntity entity = serviceFacade.cancelConnectorDrain(revision, id); + populateRemainingConnectorEntityContent(entity); + + return generateOkResponse(entity).build(); + } + ); + } + + + @POST + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/purge-requests") + @Operation( + summary = "Creates a request to purge the FlowFiles for this connector", + responses = { + @ApiResponse( + responseCode = "202", description = "The request has been accepted. A HTTP response header will contain the URI where the response can be polled.", + content = @Content(schema = @Schema(implementation = DropRequestEntity.class)) + ), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + description = "This will create a request to purge all FlowFiles from the connector. The connector must be in a STOPPED state before purging can begin. " + + "This is an asynchronous operation. The client should poll the returned URI to get the status of the purge request.", + security = { + @SecurityRequirement(name = "Write Source Data - /data/connectors/{uuid}") + } + ) + public Response createPurgeRequest( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.POST); + } + + final ConnectorEntity requestConnectorEntity = new ConnectorEntity(); + requestConnectorEntity.setId(id); + + return withWriteLock( + serviceFacade, + requestConnectorEntity, + lookup -> { + final Authorizable dataAuthorizable = new DataAuthorizable(lookup.getConnector(id)); + dataAuthorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); + }, + () -> serviceFacade.verifyPurgeConnectorFlowFiles(id), + (connectorEntity) -> performAsyncPurge(connectorEntity, id, NiFiUserUtils.getNiFiUser()) + ); + } + + + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/purge-requests/{purge-request-id}") + @Operation( + summary = "Gets the current status of a purge request for the specified connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = DropRequestEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Only the user that submitted the request can get it") + } + ) + public Response getPurgeRequest( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String connectorId, + @Parameter( + description = "The purge request id.", + required = true + ) + @PathParam("purge-request-id") final String purgeRequestId) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + serviceFacade.authorizeAccess(lookup -> { + final Authorizable dataAuthorizable = new DataAuthorizable(lookup.getConnector(connectorId)); + dataAuthorizable.authorize(authorizer, RequestAction.WRITE, user); + }); + + final AsynchronousWebRequest asyncRequest = purgeRequestManager.getRequest(PURGE_REQUEST_TYPE, purgeRequestId, user); + final DropRequestEntity purgeRequestEntity = createPurgeRequestEntity(asyncRequest, connectorId, purgeRequestId); + return generateOkResponse(purgeRequestEntity).build(); + } + + + @DELETE + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/purge-requests/{purge-request-id}") + @Operation( + summary = "Cancels and/or removes a request to purge the FlowFiles for this connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = DropRequestEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Only the user that submitted the request can remove it") + } + ) + public Response removePurgeRequest( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String connectorId, + @Parameter( + description = "The purge request id.", + required = true + ) + @PathParam("purge-request-id") final String purgeRequestId) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.DELETE); + } + + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + + // Make sure user has write access to the connector data + serviceFacade.authorizeAccess(lookup -> { + final Authorizable dataAuthorizable = new DataAuthorizable(lookup.getConnector(connectorId)); + dataAuthorizable.authorize(authorizer, RequestAction.WRITE, user); + }); + + final boolean twoPhaseRequest = isTwoPhaseRequest(httpServletRequest); + final boolean executionPhase = isExecutionPhase(httpServletRequest); + + if (!twoPhaseRequest || executionPhase) { + final AsynchronousWebRequest asyncRequest = purgeRequestManager.removeRequest(PURGE_REQUEST_TYPE, purgeRequestId, user); + + if (!asyncRequest.isComplete()) { + asyncRequest.cancel(); + } + + final DropRequestEntity purgeRequestEntity = createPurgeRequestEntity(asyncRequest, connectorId, purgeRequestId); + return generateOkResponse(purgeRequestEntity).build(); + } + + if (isValidationPhase(httpServletRequest)) { + purgeRequestManager.getRequest(PURGE_REQUEST_TYPE, purgeRequestId, user); + return generateContinueResponse().build(); + } else if (isCancellationPhase(httpServletRequest)) { + return generateOkResponse().build(); + } else { + throw new IllegalStateException("This request does not appear to be part of the two phase commit."); + } + } + + private Response performAsyncPurge(final ConnectorEntity connectorEntity, final String connectorId, final NiFiUser user) { + final String requestId = generateUuid(); + logger.debug("Generated Purge Request with ID {} for Connector {}", requestId, connectorId); + + final List updateSteps = Collections.singletonList(new StandardUpdateStep("Purge FlowFiles")); + + final AsynchronousWebRequest request = + new StandardAsynchronousWebRequest<>(requestId, connectorEntity, connectorId, user, updateSteps); + + final Consumer> updateTask = asyncRequest -> { + try { + serviceFacade.purgeConnectorFlowFiles(connectorId, user.getIdentity()); + asyncRequest.markStepComplete(null); + } catch (final Exception e) { + logger.error("Failed to purge FlowFiles for Connector {}", connectorId, e); + asyncRequest.fail("Failed to purge FlowFiles due to " + e); + } + }; + + purgeRequestManager.submitRequest(PURGE_REQUEST_TYPE, requestId, request, updateTask); + + final DropRequestEntity purgeRequestEntity = createPurgeRequestEntity(request, connectorId, requestId); + final URI location = URI.create(purgeRequestEntity.getDropRequest().getUri()); + return Response.status(Response.Status.ACCEPTED).location(location).entity(purgeRequestEntity).build(); + } + + private DropRequestEntity createPurgeRequestEntity(final AsynchronousWebRequest asyncRequest, + final String connectorId, final String requestId) { + final DropRequestDTO dto = new DropRequestDTO(); + dto.setId(requestId); + dto.setUri(generateResourceUri("connectors", connectorId, "purge-requests", requestId)); + dto.setSubmissionTime(asyncRequest.getLastUpdated()); + dto.setLastUpdated(asyncRequest.getLastUpdated()); + dto.setPercentCompleted(asyncRequest.getPercentComplete()); + dto.setFinished(asyncRequest.isComplete()); + dto.setFailureReason(asyncRequest.getFailureReason()); + dto.setState(asyncRequest.getState()); + + final DropRequestEntity entity = new DropRequestEntity(); + entity.setDropRequest(dto); + return entity; + } + + /** + * Gets the configuration step names for the specified connector. + * + * @param id The id of the connector to retrieve configuration steps from + * @return A configurationStepNamesEntity. + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/configuration-steps") + @Operation( + summary = "Gets all configuration step names for a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConfigurationStepNamesEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Read - /connectors/{uuid}") + } + ) + public Response getConnectorConfigurationSteps( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + // authorize access + serviceFacade.authorizeAccess(lookup -> { + final Authorizable connector = lookup.getConnector(id); + connector.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); + }); + + // get the configuration step names + final ConfigurationStepNamesEntity entity = serviceFacade.getConnectorConfigurationSteps(id); + + return generateOkResponse(entity).build(); + } + + /** + * Gets a specific configuration step by name for the specified connector. + * + * @param id The id of the connector to retrieve configuration step from + * @param configurationStepName The name of the configuration step to retrieve + * @return A configurationStepEntity. + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/configuration-steps/{configurationStepName}") + @Operation( + summary = "Gets a specific configuration step by name for a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConfigurationStepEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Read - /connectors/{uuid}") + } + ) + public Response getConnectorConfigurationStep( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id, + @Parameter( + description = "The configuration step name.", + required = true + ) + @PathParam("configurationStepName") final String configurationStepName) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + // authorize access + serviceFacade.authorizeAccess(lookup -> { + final Authorizable connector = lookup.getConnector(id); + connector.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); + }); + + // get the specific configuration step + final ConfigurationStepEntity entity = serviceFacade.getConnectorConfigurationStep(id, configurationStepName); + + return generateOkResponse(entity).build(); + } + + /** + * Gets the allowable values for a specific property in a connector's configuration step. + * + * @param id The id of the connector + * @param configurationStepName The name of the configuration step + * @param propertyGroupName The name of the property group + * @param propertyName The name of the property + * @param filter Optional filter for the allowable values + * @return A ConnectorPropertyAllowableValuesEntity + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/configuration-steps/{configurationStepName}/property-groups/{propertyGroupName}/properties/{propertyName}/allowable-values") + @Operation( + summary = "Gets the allowable values for a specific property in a connector's configuration step", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConnectorPropertyAllowableValuesEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + description = "Gets the allowable values for a specific property that supports dynamic fetching of allowable values. " + + "The filter parameter can be used to narrow down the results based on the property's filtering logic.", + security = { + @SecurityRequirement(name = "Read - /connectors/{uuid}") + } + ) + public Response getConnectorPropertyAllowableValues( + @Parameter(description = "The connector id.", required = true) + @PathParam("id") final String id, + @Parameter(description = "The configuration step name.", required = true) + @PathParam("configurationStepName") final String configurationStepName, + @Parameter(description = "The property group name.", required = true) + @PathParam("propertyGroupName") final String propertyGroupName, + @Parameter(description = "The property name.", required = true) + @PathParam("propertyName") final String propertyName, + @Parameter(description = "Optional filter to narrow down the allowable values.") + @QueryParam("filter") final String filter) { + + // NOTE: fetching allowable values is handled by the node that receives the request and does not need to be replicated + + // authorize access + serviceFacade.authorizeAccess(lookup -> { + final Authorizable connector = lookup.getConnector(id); + connector.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); + }); + + // get the allowable values + final ConnectorPropertyAllowableValuesEntity entity = serviceFacade.getConnectorPropertyAllowableValues(id, configurationStepName, propertyGroupName, propertyName, filter); + + return generateOkResponse(entity).build(); + } + + /** + * Updates a specific configuration step by name for the specified connector. + * + * @param id The id of the connector to update configuration step for + * @param configurationStepName The name of the configuration step to update + * @param requestConfigurationStepEntity The configuration step configuration to update + * @return A configurationStepEntity. + */ + @PUT + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/configuration-steps/{configurationStepName}") + @Operation( + summary = "Updates a specific configuration step by name for a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConfigurationStepEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid}") + } + ) + public Response updateConnectorConfigurationStep( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id, + @Parameter( + description = "The configuration step name.", + required = true + ) + @PathParam("configurationStepName") final String configurationStepName, + @Parameter( + description = "The configuration step configuration.", + required = true + ) final ConfigurationStepEntity requestConfigurationStepEntity) { + + if (requestConfigurationStepEntity == null || requestConfigurationStepEntity.getConfigurationStep() == null) { + throw new IllegalArgumentException("Configuration step details must be specified."); + } + + if (requestConfigurationStepEntity.getParentConnectorRevision() == null) { + throw new IllegalArgumentException("Parent connector revision must be specified."); + } + + if (requestConfigurationStepEntity.getParentConnectorId() == null) { + throw new IllegalArgumentException("Parent connector ID must be specified."); + } + + if (!id.equals(requestConfigurationStepEntity.getParentConnectorId())) { + throw new IllegalArgumentException(String.format("The parent connector ID (%s) in the request body does not equal the " + + "connector ID of the requested resource (%s).", requestConfigurationStepEntity.getParentConnectorId(), id)); + } + + // ensure the configuration step names match + final ConfigurationStepConfigurationDTO requestConfigurationStep = requestConfigurationStepEntity.getConfigurationStep(); + if (!configurationStepName.equals(requestConfigurationStep.getConfigurationStepName())) { + throw new IllegalArgumentException(String.format("The configuration step name (%s) in the request body does not equal the " + + "configuration step name of the requested resource (%s).", requestConfigurationStep.getConfigurationStepName(), configurationStepName)); + } + + if (isReplicateRequest()) { + return replicate(HttpMethod.PUT, requestConfigurationStepEntity); + } else if (isDisconnectedFromCluster()) { + verifyDisconnectedNodeModification(requestConfigurationStepEntity.isDisconnectedNodeAcknowledged()); + } + + // handle expects request (usually from the cluster manager) + final Revision requestRevision = getRevision(requestConfigurationStepEntity.getParentConnectorRevision(), id); + return withWriteLock( + serviceFacade, + requestConfigurationStepEntity, + requestRevision, + lookup -> { + final Authorizable connector = lookup.getConnector(id); + connector.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); + }, + () -> { + // Verify the connector exists and the configuration step exists + serviceFacade.getConnectorConfigurationStep(id, configurationStepName); + }, + (revision, configurationStepEntity) -> { + final ConfigurationStepConfigurationDTO configurationStepConfiguration = configurationStepEntity.getConfigurationStep(); + + // update the configuration step + final ConfigurationStepEntity entity = serviceFacade.updateConnectorConfigurationStep(revision, id, configurationStepName, configurationStepConfiguration); + + return generateOkResponse(entity).build(); + } + ); + } + + /** + * Submits a request to perform verification of a specific configuration step for a connector. + * This is an asynchronous operation that will return immediately with a request ID that can be + * used to poll for the results. + * + * @param id The id of the connector + * @param configurationStepName The name of the configuration step to verify + * @param requestEntity The verify config request entity containing the properties to verify + * @return The verification request entity with the request ID + */ + @POST + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/configuration-steps/{configurationStepName}/verify-config") + @Operation( + summary = "Performs verification of a configuration step for a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = VerifyConnectorConfigStepRequestEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + description = "This will initiate the process of verifying a given Connector Configuration Step. This may be a long-running task. As a result, " + + "this endpoint will immediately return a VerifyConnectorConfigStepRequestEntity, and the process of performing the verification will occur asynchronously in the background. " + + "The client may then periodically poll the status of the request by issuing a GET request to " + + "/connectors/{connectorId}/configuration-steps/{stepName}/verify-config/{requestId}. Once the request is completed, the client is expected to issue a DELETE request to " + + "/connectors/{connectorId}/configuration-steps/{stepName}/verify-config/{requestId}.", + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid}") + } + ) + public Response submitConfigurationStepVerificationRequest( + @Parameter(description = "The connector id.", required = true) + @PathParam("id") final String id, + @Parameter(description = "The configuration step name.", required = true) + @PathParam("configurationStepName") final String configurationStepName, + @Parameter(description = "The verify config request entity containing the configuration step to verify.", required = true) + final VerifyConnectorConfigStepRequestEntity requestEntity) { + + if (requestEntity == null) { + throw new IllegalArgumentException("Connector configuration step verification request must be specified."); + } + + final VerifyConnectorConfigStepRequestDTO requestDto = requestEntity.getRequest(); + if (requestDto == null || requestDto.getConfigurationStep() == null) { + throw new IllegalArgumentException("Connector configuration step must be specified."); + } + + if (requestDto.getConnectorId() == null) { + throw new IllegalArgumentException("Connector's identifier must be specified in the request."); + } + + if (!requestDto.getConnectorId().equals(id)) { + throw new IllegalArgumentException("Connector's identifier in the request must match the identifier provided in the URL."); + } + + if (requestDto.getConfigurationStepName() == null) { + throw new IllegalArgumentException("Configuration step name must be specified in the request."); + } + + if (!requestDto.getConfigurationStepName().equals(configurationStepName)) { + throw new IllegalArgumentException("Configuration step name in the request must match the step name provided in the URL."); + } + + if (isReplicateRequest()) { + return replicate(HttpMethod.POST, requestEntity); + } + + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + + return withWriteLock( + serviceFacade, + requestEntity, + lookup -> { + final Authorizable connector = lookup.getConnector(id); + connector.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); + }, + () -> { + serviceFacade.verifyCanVerifyConnectorConfigurationStep(id, configurationStepName); + }, + entity -> performAsyncConfigStepVerification(entity, id, configurationStepName, user) + ); + } + + /** + * Returns the verification request with the given ID for a connector configuration step. + * + * @param id The id of the connector + * @param configurationStepName The name of the configuration step + * @param requestId The id of the verification request + * @return The verification request entity + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/configuration-steps/{configurationStepName}/verify-config/{requestId}") + @Operation( + summary = "Returns the Verification Request with the given ID", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = VerifyConnectorConfigStepRequestEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + description = "Returns the Verification Request with the given ID. Once a Verification Request has been created, " + + "that request can subsequently be retrieved via this endpoint, and the request that is fetched will contain the updated state, such as percent complete, the " + + "current state of the request, and any failures.", + security = { + @SecurityRequirement(name = "Only the user that submitted the request can get it") + } + ) + public Response getConfigurationStepVerificationRequest( + @Parameter(description = "The connector id.", required = true) + @PathParam("id") final String id, + @Parameter(description = "The configuration step name.", required = true) + @PathParam("configurationStepName") final String configurationStepName, + @Parameter(description = "The ID of the Verification Request", required = true) + @PathParam("requestId") final String requestId) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + + final AsynchronousWebRequest> asyncRequest = + configVerificationRequestManager.getRequest(VERIFICATION_REQUEST_TYPE, requestId, user); + final VerifyConnectorConfigStepRequestEntity updateRequestEntity = createVerifyConnectorConfigStepRequestEntity(asyncRequest, id, configurationStepName, requestId); + return generateOkResponse(updateRequestEntity).build(); + } + + /** + * Deletes the verification request with the given ID for a connector configuration step. + * + * @param id The id of the connector + * @param configurationStepName The name of the configuration step + * @param requestId The id of the verification request + * @return The verification request entity + */ + @DELETE + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/configuration-steps/{configurationStepName}/verify-config/{requestId}") + @Operation( + summary = "Deletes the Verification Request with the given ID", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = VerifyConnectorConfigStepRequestEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + description = "Deletes the Verification Request with the given ID. After a request is created, it is expected " + + "that the client will properly clean up the request by DELETE'ing it, once the Verification process has completed. If the request is deleted before the request " + + "completes, then the Verification request will finish the step that it is currently performing and then will cancel any subsequent steps.", + security = { + @SecurityRequirement(name = "Only the user that submitted the request can remove it") + } + ) + public Response deleteConfigurationStepVerificationRequest( + @Parameter(description = "The connector id.", required = true) + @PathParam("id") final String id, + @Parameter(description = "The configuration step name.", required = true) + @PathParam("configurationStepName") final String configurationStepName, + @Parameter(description = "The ID of the Verification Request", required = true) + @PathParam("requestId") final String requestId) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.DELETE); + } + + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + final boolean twoPhaseRequest = isTwoPhaseRequest(httpServletRequest); + final boolean executionPhase = isExecutionPhase(httpServletRequest); + + if (!twoPhaseRequest || executionPhase) { + final AsynchronousWebRequest> asyncRequest = + configVerificationRequestManager.removeRequest(VERIFICATION_REQUEST_TYPE, requestId, user); + + if (!asyncRequest.isComplete()) { + asyncRequest.cancel(); + } + + final VerifyConnectorConfigStepRequestEntity updateRequestEntity = createVerifyConnectorConfigStepRequestEntity(asyncRequest, id, configurationStepName, requestId); + return generateOkResponse(updateRequestEntity).build(); + } + + if (isValidationPhase(httpServletRequest)) { + configVerificationRequestManager.getRequest(VERIFICATION_REQUEST_TYPE, requestId, user); + return generateContinueResponse().build(); + } else if (isCancellationPhase(httpServletRequest)) { + return generateOkResponse().build(); + } else { + throw new IllegalStateException("This request does not appear to be part of the two phase commit."); + } + } + + private Response performAsyncConfigStepVerification(final VerifyConnectorConfigStepRequestEntity requestEntity, final String connectorId, + final String configurationStepName, final NiFiUser user) { + final String requestId = generateUuid(); + logger.debug("Generated Config Verification Request with ID {} for Connector {} Configuration Step {}", requestId, connectorId, configurationStepName); + + final VerifyConnectorConfigStepRequestDTO requestDto = requestEntity.getRequest(); + final List updateSteps = Collections.singletonList(new StandardUpdateStep("Verify Connector Configuration Step")); + + final AsynchronousWebRequest> request = + new StandardAsynchronousWebRequest<>(requestId, requestEntity, connectorId, user, updateSteps); + + final Consumer>> updateTask = asyncRequest -> { + try { + final ConfigurationStepConfigurationDTO configurationStep = requestDto.getConfigurationStep(); + final List results = serviceFacade.performConnectorConfigurationStepVerification(connectorId, configurationStepName, configurationStep); + asyncRequest.markStepComplete(results); + } catch (final Exception e) { + logger.error("Failed to verify Connector Configuration Step", e); + asyncRequest.fail("Failed to verify Connector Configuration Step due to " + e); + } + }; + + configVerificationRequestManager.submitRequest(VERIFICATION_REQUEST_TYPE, requestId, request, updateTask); + + final VerifyConnectorConfigStepRequestEntity resultsEntity = createVerifyConnectorConfigStepRequestEntity(request, connectorId, configurationStepName, requestId); + return generateOkResponse(resultsEntity).build(); + } + + private VerifyConnectorConfigStepRequestEntity createVerifyConnectorConfigStepRequestEntity( + final AsynchronousWebRequest> asyncRequest, + final String connectorId, final String configurationStepName, final String requestId) { + + final VerifyConnectorConfigStepRequestDTO requestDto = asyncRequest.getRequest().getRequest(); + final List resultsList = asyncRequest.getResults(); + + final VerifyConnectorConfigStepRequestDTO dto = new VerifyConnectorConfigStepRequestDTO(); + dto.setConnectorId(requestDto.getConnectorId()); + dto.setConfigurationStepName(requestDto.getConfigurationStepName()); + dto.setConfigurationStep(requestDto.getConfigurationStep()); + dto.setResults(resultsList); + + dto.setComplete(asyncRequest.isComplete()); + dto.setFailureReason(asyncRequest.getFailureReason()); + dto.setLastUpdated(asyncRequest.getLastUpdated()); + dto.setPercentCompleted(asyncRequest.getPercentComplete()); + dto.setRequestId(requestId); + dto.setState(asyncRequest.getState()); + dto.setUri(generateResourceUri("connectors", connectorId, "configuration-steps", configurationStepName, "verify-config", requestId)); + + final VerifyConnectorConfigStepRequestEntity entity = new VerifyConnectorConfigStepRequestEntity(); + entity.setRequest(dto); + return entity; + } + + /** + * Performs a search request within the encapsulated process group of this connector. + * + * @param id The connector id + * @param value Search string + * @return A searchResultsEntity + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/search-results") + @Operation( + summary = "Performs a search against the encapsulated process group of this connector using the specified search term", + description = "Only search results from authorized components will be returned.", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = SearchResultsEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Read - /connectors/{uuid}") + } + ) + public Response searchConnector( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id, + @Parameter( + description = "The search term.", + required = false + ) + @QueryParam("q") @DefaultValue(StringUtils.EMPTY) final String value + ) { + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + // authorize access to the connector + serviceFacade.authorizeAccess(lookup -> { + final Authorizable connector = lookup.getConnector(id); + connector.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); + }); + + // search within the connector's encapsulated process group + final SearchResultsDTO results = serviceFacade.searchConnector(id, value); + + // create the entity + final SearchResultsEntity entity = new SearchResultsEntity(); + entity.setSearchResultsDTO(results); + + // generate the response + return noCache(Response.ok(entity)).build(); + } + + private ConnectorDTO createDTOWithDesiredRunStatus(final String id, final String runStatus) { + final ConnectorDTO dto = new ConnectorDTO(); + dto.setId(id); + dto.setState(runStatus); + return dto; + } + + /** + * Applies an update to a connector. + * + * @param id The id of the connector to apply update to. + * @param requestConnectorEntity A connectorEntity containing the revision. + * @return A connectorEntity. + */ + @POST + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/apply-update") + @Operation( + summary = "Applies an update to a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConnectorEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + description = "This will apply any pending configuration changes to the connector. The client can poll the connector endpoint to check when the update is complete.", + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid}") + } + ) + public Response applyConnectorUpdate( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id, + @Parameter( + description = "The connector configuration with revision.", + required = true + ) final ConnectorEntity requestConnectorEntity) { + + if (requestConnectorEntity == null || requestConnectorEntity.getRevision() == null) { + throw new IllegalArgumentException("Connector entity with revision must be specified."); + } + + // ensure the ids are the same + if (requestConnectorEntity.getId() != null && !id.equals(requestConnectorEntity.getId())) { + throw new IllegalArgumentException(String.format("The connector id (%s) in the request body does not equal the " + + "connector id of the requested resource (%s).", requestConnectorEntity.getId(), id)); + } + + if (isReplicateRequest()) { + return replicate(HttpMethod.POST, requestConnectorEntity); + } else if (isDisconnectedFromCluster()) { + verifyDisconnectedNodeModification(requestConnectorEntity.isDisconnectedNodeAcknowledged()); + } + + // handle expects request (usually from the cluster manager) + final Revision requestRevision = getRevision(requestConnectorEntity, id); + return withWriteLock( + serviceFacade, + requestConnectorEntity, + requestRevision, + lookup -> { + final Authorizable connector = lookup.getConnector(id); + connector.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); + }, + () -> { + // Verify the connector exists + serviceFacade.getConnector(id); + }, + (revision, connectorEntity) -> { + // apply the connector update + final ConnectorEntity entity = serviceFacade.applyConnectorUpdate(revision, id); + populateRemainingConnectorEntityContent(entity); + + return generateOkResponse(entity).build(); + } + ); + } + + /** + * Discards the working configuration of a connector, reverting to the last applied configuration. + * + * @param version The revision is used to verify the client is working with the latest version of the flow. + * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response. + * @param id The id of the connector whose working configuration should be discarded. + * @return A connectorEntity. + */ + @DELETE + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/working-configuration") + @Operation( + summary = "Discards the working configuration of a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConnectorEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + description = "This will discard any pending configuration changes for the connector and revert to the last applied configuration.", + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid}") + } + ) + public Response discardConnectorUpdate( + @Parameter( + description = "The revision is used to verify the client is working with the latest version of the flow." + ) + @QueryParam(VERSION) final LongParameter version, + @Parameter( + description = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response." + ) + @QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) final ClientIdParameter clientId, + @Parameter( + description = "Acknowledges that this node is disconnected to allow for mutable requests to proceed." + ) + @QueryParam(DISCONNECTED_NODE_ACKNOWLEDGED) @DefaultValue("false") final Boolean disconnectedNodeAcknowledged, + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.DELETE); + } else if (isDisconnectedFromCluster()) { + verifyDisconnectedNodeModification(disconnectedNodeAcknowledged); + } + + final ConnectorEntity requestConnectorEntity = new ConnectorEntity(); + requestConnectorEntity.setId(id); + + final Revision requestRevision = new Revision(version == null ? null : version.getLong(), clientId.getClientId(), id); + return withWriteLock( + serviceFacade, + requestConnectorEntity, + requestRevision, + lookup -> { + final Authorizable connector = lookup.getConnector(id); + connector.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); + }, + () -> { + // Verify the connector exists + serviceFacade.getConnector(id); + }, + (revision, connectorEntity) -> { + final ConnectorEntity entity = serviceFacade.discardConnectorUpdate(revision, id); + populateRemainingConnectorEntityContent(entity); + + return generateOkResponse(entity).build(); + } + ); + } + + /** + * Retrieves the flow for a process group within the specified connector. + * + * @param connectorId The id of the connector + * @param processGroupId The process group id within the connector's hierarchy + * @param uiOnly Whether to return only UI-specific fields + * @return A processGroupFlowEntity + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{connectorId}/flow/process-groups/{processGroupId}") + @Operation( + summary = "Gets the flow for a process group within a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ProcessGroupFlowEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Read - /connectors/{uuid}") + }, + description = "Returns the flow for the specified process group within the connector's hierarchy. The processGroupId can be " + + "obtained from the managedProcessGroupId field of the ConnectorDTO for the root process group, or from child process " + + "groups within the flow. If the uiOnly query parameter is provided with a value of true, the returned entity may only " + + "contain fields that are necessary for rendering the NiFi User Interface. As such, the selected fields may change at " + + "any time, even during incremental releases, without warning. As a result, this parameter should not be provided by " + + "any client other than the UI." + ) + public Response getFlow( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("connectorId") final String connectorId, + @Parameter( + description = "The process group id.", + required = true + ) + @PathParam("processGroupId") final String processGroupId, + @Parameter( + description = "Whether to return only UI-specific fields" + ) + @QueryParam("uiOnly") @DefaultValue("false") final boolean uiOnly) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + // authorize access to the connector + serviceFacade.authorizeAccess(lookup -> { + final Authorizable connector = lookup.getConnector(connectorId); + connector.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); + }); + + // get the flow for the specified process group within the connector's hierarchy + final ProcessGroupFlowEntity entity = serviceFacade.getConnectorFlow(connectorId, processGroupId, uiOnly); + flowResource.populateRemainingFlowContent(entity.getProcessGroupFlow()); + return generateOkResponse(entity).build(); + } + + /** + * Retrieves all the controller services in the specified process group within a connector. + * + * @param connectorId The id of the connector + * @param processGroupId The process group id within the connector's hierarchy + * @param includeAncestorGroups Whether to include ancestor process groups + * @param includeDescendantGroups Whether to include descendant process groups + * @param includeReferences Whether to include services' referencing components in the response + * @return A controllerServicesEntity. + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{connectorId}/flow/process-groups/{processGroupId}/controller-services") + @Operation( + summary = "Gets all controller services for a process group within a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ControllerServicesEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Read - /connectors/{uuid}") + }, + description = "Returns the controller services for the specified process group within the connector's hierarchy. The processGroupId can be " + + "obtained from the managedProcessGroupId field of the ConnectorDTO for the root process group, or from child process " + + "groups within the flow." + ) + public Response getControllerServicesFromConnectorProcessGroup( + @Parameter(description = "The connector id.", required = true) + @PathParam("connectorId") final String connectorId, + @Parameter(description = "The process group id.", required = true) + @PathParam("processGroupId") final String processGroupId, + @Parameter(description = "Whether or not to include parent/ancestor process groups") + @QueryParam("includeAncestorGroups") + @DefaultValue("true") final boolean includeAncestorGroups, + @Parameter(description = "Whether or not to include descendant process groups") + @QueryParam("includeDescendantGroups") + @DefaultValue("false") final boolean includeDescendantGroups, + @Parameter(description = "Whether or not to include services' referencing components in the response") + @QueryParam("includeReferencingComponents") + @DefaultValue("true") final boolean includeReferences) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + // authorize access to the connector + serviceFacade.authorizeAccess(lookup -> { + final Authorizable connector = lookup.getConnector(connectorId); + connector.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); + }); + + // get the controller services for the specified process group within the connector's hierarchy + final Set controllerServices = serviceFacade.getConnectorControllerServices( + connectorId, processGroupId, includeAncestorGroups, includeDescendantGroups, includeReferences); + controllerServiceResource.populateRemainingControllerServiceEntitiesContent(controllerServices); + + // create the response entity + final ControllerServicesEntity entity = new ControllerServicesEntity(); + entity.setCurrentTime(new Date()); + entity.setControllerServices(controllerServices); + + // generate the response + return generateOkResponse(entity).build(); + } + + /** + * Retrieves the status for the process group managed by the specified connector. + * + * @param id The id of the connector + * @param recursive Optional recursive flag that defaults to false. If set to true, all descendant groups and the status of their content will be included. + * @param nodewise Whether to include breakdown per node + * @param clusterNodeId The id of a specific node to get status from + * @return A processGroupStatusEntity + * @throws InterruptedException if interrupted + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("/{id}/status") + @Operation( + summary = "Gets the status for the process group managed by a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ProcessGroupStatusEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Read - /connectors/{uuid}") + }, + description = "Returns the status for the process group managed by the specified connector. The status includes status for all descendent components. " + + "When invoked with recursive set to true, it will return the current status of every component in the connector's encapsulated flow." + ) + public Response getConnectorStatus( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String id, + @Parameter( + description = "Whether all descendant groups and the status of their content will be included. Optional, defaults to false" + ) + @QueryParam("recursive") @DefaultValue("false") final Boolean recursive, + @Parameter( + description = "Whether or not to include the breakdown per node. Optional, defaults to false" + ) + @QueryParam("nodewise") @DefaultValue("false") final Boolean nodewise, + @Parameter( + description = "The id of the node where to get the status." + ) + @QueryParam("clusterNodeId") final String clusterNodeId) throws InterruptedException { + + // ensure a valid request + if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) { + throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node."); + } + + if (isReplicateRequest()) { + // determine where this request should be sent + if (clusterNodeId == null) { + final NodeResponse nodeResponse = replicateNodeResponse(HttpMethod.GET); + final ProcessGroupStatusEntity entity = (ProcessGroupStatusEntity) nodeResponse.getUpdatedEntity(); + + // ensure there is an updated entity (result of merging) and prune the response as necessary + if (entity != null && !nodewise) { + entity.getProcessGroupStatus().setNodeSnapshots(null); + } + + return nodeResponse.getResponse(); + } else { + return replicate(HttpMethod.GET, clusterNodeId); + } + } + + // authorize access to the connector + serviceFacade.authorizeAccess(lookup -> { + final Authorizable connector = lookup.getConnector(id); + connector.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); + }); + + // get the status for the connector's managed process group + final ProcessGroupStatusEntity entity = serviceFacade.getConnectorProcessGroupStatus(id, recursive); + return generateOkResponse(entity).build(); + } + + @POST + @Consumes(MediaType.APPLICATION_OCTET_STREAM) + @Produces(MediaType.APPLICATION_JSON) + @Path("{id}/assets") + @Operation( + summary = "Creates a new Asset in the given Connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = AssetEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + description = "This endpoint will create a new Asset in the Connector. The Asset will be created with the given name and the contents of the file that is uploaded.", + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid}") + } + ) + public Response createAsset( + @PathParam("id") final String connectorId, + @HeaderParam(FILENAME_HEADER) final String assetName, + @Parameter(description = "The contents of the asset", required = true) final InputStream assetContents) throws IOException { + + // Validate input + if (StringUtils.isBlank(assetName)) { + throw new IllegalArgumentException(FILENAME_HEADER + " header is required"); + } + if (assetContents == null) { + throw new IllegalArgumentException("Asset contents must be specified."); + } + + final String sanitizedAssetName = FileUtils.getSanitizedFilename(assetName); + if (!assetName.equals(sanitizedAssetName)) { + throw new IllegalArgumentException(FILENAME_HEADER + " header contains an invalid file name"); + } + + // If clustered and not all nodes are connected, do not allow creating an asset. + // Generally, we allow the flow to be modified when nodes are disconnected, but we do not allow creating an asset because + // the cluster has no mechanism for synchronizing those assets after the upload. + final ClusterCoordinator clusterCoordinator = getClusterCoordinator(); + if (clusterCoordinator != null) { + final Set disconnectedNodes = clusterCoordinator.getNodeIdentifiers(NodeConnectionState.CONNECTING, NodeConnectionState.DISCONNECTED, NodeConnectionState.DISCONNECTING); + if (!disconnectedNodes.isEmpty()) { + throw new IllegalStateException("Cannot create an Asset because the following %s nodes are not currently connected: %s".formatted(disconnectedNodes.size(), disconnectedNodes)); + } + } + + final NiFiUser currentUser = NiFiUserUtils.getNiFiUser(); + + // Verify Connector exists + serviceFacade.verifyCreateConnectorAsset(connectorId); + + // Authorize the request + serviceFacade.authorizeAccess(lookup -> { + final Authorizable connector = lookup.getConnector(connectorId); + connector.authorize(authorizer, RequestAction.WRITE, currentUser); + }); + + // If we need to replicate the request, we do so using the Upload Request Replicator, rather than the typical replicate() method. + // This is because Upload Request Replication works differently in that it needs to be able to replicate the InputStream multiple times, + // so it must create a file on disk to do so and then use the file's content to replicate the request. It also bypasses the two-phase + // commit process that is used for other requests because doing so would result in uploading the file twice to each node or providing a + // different request for each of the two phases. + + final long startTime = System.currentTimeMillis(); + final InputStream maxLengthInputStream = new MaxLengthInputStream(assetContents, MAX_ASSET_SIZE_BYTES); + + final AssetEntity assetEntity; + if (isReplicateRequest()) { + final String uploadRequestId = UUID.randomUUID().toString(); + final UploadRequest uploadRequest = new UploadRequest.Builder() + .user(NiFiUserUtils.getNiFiUser()) + .filename(sanitizedAssetName) + .identifier(uploadRequestId) + .contents(maxLengthInputStream) + .header(FILENAME_HEADER, sanitizedAssetName) + .header(CONTENT_TYPE_HEADER, UPLOAD_CONTENT_TYPE) + .header(RequestReplicationHeader.CLUSTER_ID_GENERATION_SEED.getHeader(), uploadRequestId) + .exampleRequestUri(getAbsolutePath()) + .responseClass(AssetEntity.class) + .successfulResponseStatus(HttpResponseStatus.OK.getCode()) + .build(); + assetEntity = uploadRequestReplicator.upload(uploadRequest); + } else { + final String assetId = generateUuid(); + logger.info("Creating asset [id={},name={}] in Connector [{}]", assetId, sanitizedAssetName, connectorId); + assetEntity = serviceFacade.createConnectorAsset(connectorId, assetId, sanitizedAssetName, maxLengthInputStream); + + final AssetDTO assetDTO = assetEntity.getAsset(); + final long elapsedTime = System.currentTimeMillis() - startTime; + logger.info("Creation of asset [id={},name={}] in Connector [{}] completed in {} ms", assetDTO.getId(), assetDTO.getName(), connectorId, elapsedTime); + } + + return generateOkResponse(assetEntity).build(); + } + + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("{id}/assets") + @Operation( + summary = "Lists the assets that belong to the Connector with the given ID", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = AssetsEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + description = "Lists the assets that belong to the Connector with the given ID.", + security = { + @SecurityRequirement(name = "Read - /connectors/{uuid}") + } + ) + public Response getAssets( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String connectorId + ) { + serviceFacade.authorizeAccess(lookup -> { + final Authorizable connector = lookup.getConnector(connectorId); + connector.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); + }); + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + final List connectorAssets = serviceFacade.getConnectorAssets(connectorId); + logger.debug("Returning [{}] assets for connector [{}]", connectorAssets.size(), connectorId); + + final AssetsEntity assetsEntity = new AssetsEntity(); + assetsEntity.setAssets(connectorAssets); + + return generateOkResponse(assetsEntity).build(); + } + + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_OCTET_STREAM) + @Path("{id}/assets/{assetId}") + @Operation( + summary = "Retrieves the content of the asset with the given id for a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = byte[].class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Read - /connectors/{uuid}") + } + ) + public Response getAssetContent( + @Parameter( + description = "The connector id.", + required = true + ) + @PathParam("id") final String connectorId, + @Parameter( + description = "The asset id.", + required = true + ) + @PathParam("assetId") final String assetId + ) { + serviceFacade.authorizeAccess(lookup -> { + final Authorizable connector = lookup.getConnector(connectorId); + connector.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); + }); + + final Asset asset = serviceFacade.getConnectorAsset(assetId) + .orElseThrow(() -> new ResourceNotFoundException("Asset does not exist with id %s".formatted(assetId))); + + if (!asset.getOwnerIdentifier().equals(connectorId)) { + throw new ResourceNotFoundException("Asset does not exist with id %s".formatted(assetId)); + } + + if (!asset.getFile().exists()) { + throw new IllegalStateException("Content does not exist for asset with id %s".formatted(assetId)); + } + + final StreamingOutput streamingOutput = outputStream -> { + try (final InputStream assetInputStream = new FileInputStream(asset.getFile())) { + assetInputStream.transferTo(outputStream); + } + }; + + return generateOkResponse(streamingOutput) + .header(HttpHeaders.CONTENT_DISPOSITION, String.format("attachment; filename=\"%s\"", asset.getName())) + .build(); + } + + // ----------------- + // Processor State + // ----------------- + + /** + * Gets the state for a processor within a connector. + * + * @param connectorId the connector id + * @param processorId the processor id + * @return a ComponentStateEntity + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("{id}/processors/{processorId}/state") + @Operation( + summary = "Gets the state for a processor within a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ComponentStateEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid}") + } + ) + public Response getConnectorProcessorState( + @Parameter(description = "The connector id.", required = true) + @PathParam("id") final String connectorId, + @Parameter(description = "The processor id.", required = true) + @PathParam("processorId") final String processorId) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + serviceFacade.authorizeAccess(lookup -> { + final Authorizable connector = lookup.getConnector(connectorId); + connector.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); + }); + + final ComponentStateDTO state = serviceFacade.getConnectorProcessorState(connectorId, processorId); + + final ComponentStateEntity entity = new ComponentStateEntity(); + entity.setComponentState(state); + + return generateOkResponse(entity).build(); + } + + /** + * Clears the state for a processor within a connector. + * + * @param connectorId the connector id + * @param processorId the processor id + * @return a ComponentStateEntity + */ + @POST + @Consumes({MediaType.APPLICATION_JSON, MediaType.WILDCARD}) + @Produces(MediaType.APPLICATION_JSON) + @Path("{id}/processors/{processorId}/state/clear-requests") + @Operation( + summary = "Clears the state for a processor within a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ComponentStateEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid}") + } + ) + public Response clearConnectorProcessorState( + @Parameter(description = "The connector id.", required = true) + @PathParam("id") final String connectorId, + @Parameter(description = "The processor id.", required = true) + @PathParam("processorId") final String processorId, + @Parameter(description = "Optional component state to perform a selective key removal. If omitted, clears all state.", required = false) + final ComponentStateEntity componentStateEntity) { + + if (isReplicateRequest()) { + if (componentStateEntity == null) { + return replicate(HttpMethod.POST); + } else { + return replicate(HttpMethod.POST, componentStateEntity); + } + } + + final ConnectorEntity requestConnectorEntity = new ConnectorEntity(); + requestConnectorEntity.setId(connectorId); + + return withWriteLock( + serviceFacade, + requestConnectorEntity, + lookup -> { + final Authorizable connector = lookup.getConnector(connectorId); + connector.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); + }, + () -> serviceFacade.verifyCanClearConnectorProcessorState(connectorId, processorId), + (connectorEntity) -> { + // clear state + final ComponentStateDTO expectedState = componentStateEntity == null ? null : componentStateEntity.getComponentState(); + final ComponentStateDTO state = serviceFacade.clearConnectorProcessorState(connectorEntity.getId(), processorId, expectedState); + + // generate the response entity + final ComponentStateEntity entity = new ComponentStateEntity(); + entity.setComponentState(state); + + // generate the response + return generateOkResponse(entity).build(); + } + ); + } + + // ----------------- + // Controller Service State + // ----------------- + + /** + * Gets the state for a controller service within a connector. + * + * @param connectorId the connector id + * @param controllerServiceId the controller service id + * @return a ComponentStateEntity + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("{id}/controller-services/{controllerServiceId}/state") + @Operation( + summary = "Gets the state for a controller service within a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ComponentStateEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid}") + } + ) + public Response getConnectorControllerServiceState( + @Parameter(description = "The connector id.", required = true) + @PathParam("id") final String connectorId, + @Parameter(description = "The controller service id.", required = true) + @PathParam("controllerServiceId") final String controllerServiceId) { + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + serviceFacade.authorizeAccess(lookup -> { + final Authorizable connector = lookup.getConnector(connectorId); + connector.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); + }); + + final ComponentStateDTO state = serviceFacade.getConnectorControllerServiceState(connectorId, controllerServiceId); + + final ComponentStateEntity entity = new ComponentStateEntity(); + entity.setComponentState(state); + + return generateOkResponse(entity).build(); + } + + /** + * Clears the state for a controller service within a connector. + * + * @param connectorId the connector id + * @param controllerServiceId the controller service id + * @return a ComponentStateEntity + */ + @POST + @Consumes({MediaType.APPLICATION_JSON, MediaType.WILDCARD}) + @Produces(MediaType.APPLICATION_JSON) + @Path("{id}/controller-services/{controllerServiceId}/state/clear-requests") + @Operation( + summary = "Clears the state for a controller service within a connector", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ComponentStateEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The specified resource could not be found."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Write - /connectors/{uuid}") + } + ) + public Response clearConnectorControllerServiceState( + @Parameter(description = "The connector id.", required = true) + @PathParam("id") final String connectorId, + @Parameter(description = "The controller service id.", required = true) + @PathParam("controllerServiceId") final String controllerServiceId, + @Parameter(description = "Optional component state to perform a selective key removal. If omitted, clears all state.", required = false) + final ComponentStateEntity componentStateEntity) { + + if (isReplicateRequest()) { + if (componentStateEntity == null) { + return replicate(HttpMethod.POST); + } else { + return replicate(HttpMethod.POST, componentStateEntity); + } + } + + final ConnectorEntity requestConnectorEntity = new ConnectorEntity(); + requestConnectorEntity.setId(connectorId); + + return withWriteLock( + serviceFacade, + requestConnectorEntity, + lookup -> { + final Authorizable connector = lookup.getConnector(connectorId); + connector.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); + }, + () -> serviceFacade.verifyCanClearConnectorControllerServiceState(connectorId, controllerServiceId), + (connectorEntity) -> { + // clear state + final ComponentStateDTO expectedState = componentStateEntity == null ? null : componentStateEntity.getComponentState(); + final ComponentStateDTO state = serviceFacade.clearConnectorControllerServiceState(connectorEntity.getId(), controllerServiceId, expectedState); + + // generate the response entity + final ComponentStateEntity entity = new ComponentStateEntity(); + entity.setComponentState(state); + + // generate the response + return generateOkResponse(entity).build(); + } + ); + } + + // ----------------- + // setters + // ----------------- + + @Autowired + public void setServiceFacade(final NiFiServiceFacade serviceFacade) { + this.serviceFacade = serviceFacade; + } + + @Autowired + public void setAuthorizer(final Authorizer authorizer) { + this.authorizer = authorizer; + } + + @Autowired + public void setFlowResource(final FlowResource flowResource) { + this.flowResource = flowResource; + } + + @Autowired + public void setControllerServiceResource(final ControllerServiceResource controllerServiceResource) { + this.controllerServiceResource = controllerServiceResource; + } + + @Autowired(required = false) + public void setUploadRequestReplicator(final UploadRequestReplicator uploadRequestReplicator) { + this.uploadRequestReplicator = uploadRequestReplicator; + } + +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowFileQueueResource.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowFileQueueResource.java index c018b88a464d..399f6f064455 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowFileQueueResource.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowFileQueueResource.java @@ -307,7 +307,7 @@ public Response createFlowFileListing( serviceFacade, requestConnectionEntity, lookup -> { - final ConnectionAuthorizable connAuth = lookup.getConnection(id); + final ConnectionAuthorizable connAuth = lookup.getConnection(id, true); final Authorizable dataAuthorizable = connAuth.getSourceData(); dataAuthorizable.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); }, @@ -374,7 +374,7 @@ public Response getListingRequest( // authorize access serviceFacade.authorizeAccess(lookup -> { - final ConnectionAuthorizable connAuth = lookup.getConnection(connectionId); + final ConnectionAuthorizable connAuth = lookup.getConnection(connectionId, true); final Authorizable dataAuthorizable = connAuth.getSourceData(); dataAuthorizable.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); }); @@ -435,7 +435,7 @@ public Response deleteListingRequest( serviceFacade, new ListingEntity(connectionId, listingRequestId), lookup -> { - final ConnectionAuthorizable connAuth = lookup.getConnection(connectionId); + final ConnectionAuthorizable connAuth = lookup.getConnection(connectionId, true); final Authorizable dataAuthorizable = connAuth.getSourceData(); dataAuthorizable.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()); }, @@ -522,7 +522,7 @@ public Response createDropRequest( serviceFacade, requestConnectionEntity, lookup -> { - final ConnectionAuthorizable connAuth = lookup.getConnection(id); + final ConnectionAuthorizable connAuth = lookup.getConnection(id, true); final Authorizable dataAuthorizable = connAuth.getSourceData(); dataAuthorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); }, @@ -589,7 +589,7 @@ public Response getDropRequest( // authorize access serviceFacade.authorizeAccess(lookup -> { - final ConnectionAuthorizable connAuth = lookup.getConnection(connectionId); + final ConnectionAuthorizable connAuth = lookup.getConnection(connectionId, true); final Authorizable dataAuthorizable = connAuth.getSourceData(); dataAuthorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); }); @@ -650,7 +650,7 @@ public Response removeDropRequest( serviceFacade, new DropEntity(connectionId, dropRequestId), lookup -> { - final ConnectionAuthorizable connAuth = lookup.getConnection(connectionId); + final ConnectionAuthorizable connAuth = lookup.getConnection(connectionId, true); final Authorizable dataAuthorizable = connAuth.getSourceData(); dataAuthorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()); }, diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java index d94f8b05a4e8..959ef1e8ebd2 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java @@ -54,6 +54,7 @@ import org.apache.nifi.bundle.Bundle; import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.bundle.BundleDetails; +import org.apache.nifi.c2.protocol.component.api.ConnectorDefinition; import org.apache.nifi.c2.protocol.component.api.ControllerServiceDefinition; import org.apache.nifi.c2.protocol.component.api.FlowAnalysisRuleDefinition; import org.apache.nifi.c2.protocol.component.api.FlowRegistryClientDefinition; @@ -115,6 +116,9 @@ import org.apache.nifi.web.api.entity.ComponentHistoryEntity; import org.apache.nifi.web.api.entity.ConnectionStatisticsEntity; import org.apache.nifi.web.api.entity.ConnectionStatusEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ConnectorTypesEntity; +import org.apache.nifi.web.api.entity.ConnectorsEntity; import org.apache.nifi.web.api.entity.ContentViewerEntity; import org.apache.nifi.web.api.entity.ControllerBulletinsEntity; import org.apache.nifi.web.api.entity.ControllerServiceEntity; @@ -155,6 +159,7 @@ import org.apache.nifi.web.api.entity.ScheduleComponentsEntity; import org.apache.nifi.web.api.entity.SearchResultsEntity; import org.apache.nifi.web.api.entity.StatusHistoryEntity; +import org.apache.nifi.web.api.entity.StepDocumentationEntity; import org.apache.nifi.web.api.entity.VersionedFlowEntity; import org.apache.nifi.web.api.entity.VersionedFlowSnapshotMetadataEntity; import org.apache.nifi.web.api.entity.VersionedFlowSnapshotMetadataSetEntity; @@ -231,6 +236,7 @@ public class FlowResource extends ApplicationResource { private ControllerServiceResource controllerServiceResource; private ReportingTaskResource reportingTaskResource; private ParameterProviderResource parameterProviderResource; + private ConnectorResource connectorResource; @Context private ServletContext servletContext; @@ -240,7 +246,7 @@ public class FlowResource extends ApplicationResource { * * @param flow group */ - private void populateRemainingFlowContent(ProcessGroupFlowDTO flow) { + public void populateRemainingFlowContent(final ProcessGroupFlowDTO flow) { FlowDTO flowStructure = flow.getFlow(); // populate the remaining fields for the processors, connections, process group refs, remote process groups, and labels if appropriate @@ -776,6 +782,105 @@ public Response getParameterProviders() { return generateOkResponse(entity).build(); } + // ---------- + // connectors + // ---------- + + /** + * Retrieves all the of connectors in this NiFi. + * + * @return A connectorsEntity. + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("connectors") + @Operation( + summary = "Gets all connectors", + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConnectorsEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Read - /flow") + } + ) + public Response getConnectors() { + + authorizeFlow(); + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + // get all the connectors + final Set connectors = serviceFacade.getConnectors(); + connectorResource.populateRemainingConnectorEntitiesContent(connectors); + + // create the response entity + final ConnectorsEntity entity = new ConnectorsEntity(); + entity.setCurrentTime(new Date()); + entity.setConnectors(connectors); + + // generate the response + return generateOkResponse(entity).build(); + } + + /** + * Retrieves the types of connectors that this NiFi supports. + * + * @return A connectorTypesEntity. + * @throws InterruptedException if interrupted + */ + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("connector-types") + @Operation( + summary = "Retrieves the types of connectors that this NiFi supports", + description = NON_GUARANTEED_ENDPOINT, + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConnectorTypesEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "409", description = "The request was valid but NiFi was not in the appropriate state to process it.") + }, + security = { + @SecurityRequirement(name = "Read - /flow") + } + ) + public Response getConnectorTypes( + @Parameter( + description = "If specified, will only return types that are a member of this bundle group." + ) + @QueryParam("bundleGroupFilter") String bundleGroupFilter, + @Parameter( + description = "If specified, will only return types that are a member of this bundle artifact." + ) + @QueryParam("bundleArtifactFilter") String bundleArtifactFilter, + @Parameter( + description = "If specified, will only return types whose fully qualified classname matches." + ) + @QueryParam("type") String typeFilter) throws InterruptedException { + + authorizeFlow(); + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + // create response entity + final ConnectorTypesEntity entity = new ConnectorTypesEntity(); + entity.setConnectorTypes(serviceFacade.getConnectorTypes(bundleGroupFilter, bundleArtifactFilter, typeFilter)); + + // generate the response + return generateOkResponse(entity).build(); + } + // --------------- // reporting-tasks // --------------- @@ -2180,6 +2285,60 @@ public Response getFlowAnalysisRuleDefinition( return generateOkResponse(entity).build(); } + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("connector-definition/{group}/{artifact}/{version}/{type}") + @Operation( + summary = "Retrieves the Connector Definition for the specified component type.", + description = NON_GUARANTEED_ENDPOINT, + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = ConnectorDefinition.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The connector definition for the coordinates could not be located.") + }, + security = { + @SecurityRequirement(name = "Read - /flow") + } + ) + public Response getConnectorDefinition( + @Parameter( + description = "The bundle group", + required = true + ) + @PathParam("group") final String group, + @Parameter( + description = "The bundle artifact", + required = true + ) + @PathParam("artifact") final String artifact, + @Parameter( + description = "The bundle version", + required = true + ) + @PathParam("version") final String version, + @Parameter( + description = "The connector type", + required = true + ) + @PathParam("type") final String type + ) throws InterruptedException { + + authorizeFlow(); + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + // create response entity + final ConnectorDefinition entity = serviceFacade.getConnectorDefinition(group, artifact, version, type); + + // generate the response + return generateOkResponse(entity).build(); + } + @GET @Consumes(MediaType.WILDCARD) @Produces(MediaType.APPLICATION_JSON) @@ -2236,6 +2395,50 @@ public Response getAdditionalDetails( return generateOkResponse(entity).build(); } + @GET + @Consumes(MediaType.WILDCARD) + @Produces(MediaType.APPLICATION_JSON) + @Path("steps/{group}/{artifact}/{version}/{connectorType}/{stepName}") + @Operation( + summary = "Retrieves the step documentation for the specified Connector configuration step.", + description = NON_GUARANTEED_ENDPOINT, + responses = { + @ApiResponse(responseCode = "200", content = @Content(schema = @Schema(implementation = StepDocumentationEntity.class))), + @ApiResponse(responseCode = "400", description = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."), + @ApiResponse(responseCode = "401", description = "Client could not be authenticated."), + @ApiResponse(responseCode = "403", description = "Client is not authorized to make this request."), + @ApiResponse(responseCode = "404", description = "The step documentation for the coordinates could not be located.") + }, + security = { + @SecurityRequirement(name = "Read - /flow") + } + ) + public Response getStepDocumentation( + @Parameter(description = "The bundle group", required = true) + @PathParam("group") final String group, + @Parameter(description = "The bundle artifact", required = true) + @PathParam("artifact") final String artifact, + @Parameter(description = "The bundle version", required = true) + @PathParam("version") final String version, + @Parameter(description = "The fully qualified Connector type", required = true) + @PathParam("connectorType") final String connectorType, + @Parameter(description = "The configuration step name", required = true) + @PathParam("stepName") final String stepName + ) throws InterruptedException { + + authorizeFlow(); + + if (isReplicateRequest()) { + return replicate(HttpMethod.GET); + } + + final String stepDocumentation = serviceFacade.getStepDocumentation(group, artifact, version, connectorType, stepName); + final StepDocumentationEntity entity = new StepDocumentationEntity(); + entity.setStepDocumentation(stepDocumentation); + + return generateOkResponse(entity).build(); + } + /** * Retrieves the types of parameter providers that this NiFi supports. * @@ -4151,6 +4354,11 @@ public void setParameterProviderResource(final ParameterProviderResource paramet this.parameterProviderResource = parameterProviderResource; } + @Autowired + public void setConnectorResource(final ConnectorResource connectorResource) { + this.connectorResource = connectorResource; + } + @Autowired public void setAuthorizer(Authorizer authorizer) { this.authorizer = authorizer; diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ParameterContextResource.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ParameterContextResource.java index 51ece06c1f5f..522fd359741a 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ParameterContextResource.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ParameterContextResource.java @@ -537,7 +537,7 @@ public Response getAssetContent( final Asset asset = assetManager.getAsset(assetId) .orElseThrow(() -> new ResourceNotFoundException("Asset does not exist with the id %s".formatted(assetId))); - if (!asset.getParameterContextIdentifier().equals(parameterContextId)) { + if (!asset.getOwnerIdentifier().equals(parameterContextId)) { throw new ResourceNotFoundException("Asset does not exist with id %s".formatted(assetId)); } @@ -811,7 +811,7 @@ private void validateAssetReferences(final ParameterContextDTO parameterContextD } final Asset asset = assetManager.getAsset(referencedAsset.getId()) .orElseThrow(() -> new IllegalArgumentException("Request contains a reference to an Asset (%s) that does not exist".formatted(referencedAsset))); - if (!asset.getParameterContextIdentifier().equals(parameterContextDto.getId())) { + if (!asset.getOwnerIdentifier().equals(parameterContextDto.getId())) { throw new IllegalArgumentException("Request contains a reference to an Asset (%s) that does not exist in Parameter Context (%s)" .formatted(referencedAsset, parameterContextDto.getId())); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java index 6d55f9d0651a..435d6846956d 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java @@ -870,7 +870,7 @@ public Response removeDropRequest( } private void authorizeHandleDropAllFlowFilesRequest(String processGroupId, AuthorizableLookup lookup) { - final ProcessGroupAuthorizable processGroup = lookup.getProcessGroup(processGroupId); + final ProcessGroupAuthorizable processGroup = lookup.getProcessGroup(processGroupId, true); authorizeProcessGroup(processGroup, authorizer, lookup, RequestAction.READ, false, false, false, false, false); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java index 6954dc6607a9..945847b6398d 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java @@ -62,9 +62,27 @@ import org.apache.nifi.cluster.manager.StatusMerger; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.DescribedValue; import org.apache.nifi.components.PropertyDependency; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.AssetReference; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.ConfigurationStepDependency; +import org.apache.nifi.components.connector.ConnectorAction; +import org.apache.nifi.components.connector.ConnectorConfiguration; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorPropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorPropertyGroup; +import org.apache.nifi.components.connector.ConnectorRepository; +import org.apache.nifi.components.connector.ConnectorValueReference; +import org.apache.nifi.components.connector.FrameworkFlowContext; +import org.apache.nifi.components.connector.NamedStepConfiguration; +import org.apache.nifi.components.connector.Secret; +import org.apache.nifi.components.connector.SecretReference; +import org.apache.nifi.components.connector.StepConfiguration; +import org.apache.nifi.components.connector.StringLiteralValue; import org.apache.nifi.components.state.Scope; import org.apache.nifi.components.state.StateMap; import org.apache.nifi.components.validation.ValidationState; @@ -129,6 +147,7 @@ import org.apache.nifi.groups.RemoteProcessGroup; import org.apache.nifi.groups.RemoteProcessGroupCounts; import org.apache.nifi.history.History; +import org.apache.nifi.manifest.RuntimeManifestService; import org.apache.nifi.nar.ExtensionDefinition; import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.nar.NarClassLoadersHolder; @@ -221,6 +240,8 @@ import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO; import org.apache.nifi.web.api.dto.status.ConnectionStatusPredictionsSnapshotDTO; import org.apache.nifi.web.api.dto.status.ConnectionStatusSnapshotDTO; +import org.apache.nifi.web.api.dto.status.ConnectorStatusDTO; +import org.apache.nifi.web.api.dto.status.ConnectorStatusSnapshotDTO; import org.apache.nifi.web.api.dto.status.PortStatusDTO; import org.apache.nifi.web.api.dto.status.PortStatusSnapshotDTO; import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO; @@ -263,6 +284,7 @@ import java.nio.charset.StandardCharsets; import java.text.Collator; import java.text.NumberFormat; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -302,6 +324,8 @@ public final class DtoFactory { private EntityFactory entityFactory; private Authorizer authorizer; private ExtensionManager extensionManager; + private ConnectorRepository connectorRepository; + private RuntimeManifestService runtimeManifestService; public ControllerConfigurationDTO createControllerConfigurationDto(final ControllerFacade controllerFacade) { final ControllerConfigurationDTO dto = new ControllerConfigurationDTO(); @@ -5200,14 +5224,22 @@ public void setEntityFactory(final EntityFactory entityFactory) { this.entityFactory = entityFactory; } - public void setBulletinRepository(BulletinRepository bulletinRepository) { + public void setBulletinRepository(final BulletinRepository bulletinRepository) { this.bulletinRepository = bulletinRepository; } - public void setExtensionManager(ExtensionManager extensionManager) { + public void setExtensionManager(final ExtensionManager extensionManager) { this.extensionManager = extensionManager; } + public void setConnectorRepository(final ConnectorRepository connectorRepository) { + this.connectorRepository = connectorRepository; + } + + public void setRuntimeManifestService(RuntimeManifestService runtimeManifestService) { + this.runtimeManifestService = runtimeManifestService; + } + private ProcessingPerformanceStatusDTO createProcessingPerformanceStatusDTO(final ProcessingPerformanceStatus performanceStatus) { final ProcessingPerformanceStatusDTO performanceStatusDTO = new ProcessingPerformanceStatusDTO(); @@ -5227,4 +5259,352 @@ private String getDigest(final String name) { final byte[] digest = MessageDigestUtils.getDigest(bytes); return HexFormat.of().formatHex(digest); } + + public ConnectorDTO createConnectorDto(final ConnectorNode connector) { + if (connector == null) { + return null; + } + + final ConnectorDTO dto = new ConnectorDTO(); + dto.setId(connector.getIdentifier()); + dto.setName(connector.getName()); + + final ValidationState validationState = connector.getValidationState(); + dto.setValidationStatus(validationState.getStatus().name()); + dto.setValidationErrors(convertValidationErrors(validationState.getValidationErrors())); + dto.setType(connector.getCanonicalClassName()); + dto.setExtensionMissing(connector.isExtensionMissing()); + + dto.setBundle(createBundleDto(connector.getBundleCoordinate())); + dto.setState(connector.getCurrentState().name()); + + final FrameworkFlowContext activeFlowContext = connector.getActiveFlowContext(); + dto.setManagedProcessGroupId(activeFlowContext.getManagedProcessGroup().getIdentifier()); + dto.setActiveConfiguration(createConnectorConfigurationDtoFromFlowContext(connector, activeFlowContext)); + dto.setWorkingConfiguration(createConnectorConfigurationDtoFromFlowContext(connector, connector.getWorkingFlowContext())); + dto.setAvailableActions(createConnectorActionDtos(connector)); + + return dto; + } + + public ConnectorStatusDTO createConnectorStatusDto(final ConnectorNode connectorNode, final ProcessGroupStatus managedGroupStatus) { + if (connectorNode == null) { + return null; + } + + final ConnectorStatusDTO statusDto = new ConnectorStatusDTO(); + statusDto.setId(connectorNode.getIdentifier()); + statusDto.setName(connectorNode.getName()); + statusDto.setType(connectorNode.getCanonicalClassName()); + statusDto.setRunStatus(connectorNode.getCurrentState().name()); + statusDto.setValidationStatus(connectorNode.getValidationStatus().name()); + statusDto.setStatsLastRefreshed(new Date()); + + final ConnectorStatusSnapshotDTO snapshot = new ConnectorStatusSnapshotDTO(); + statusDto.setAggregateSnapshot(snapshot); + + snapshot.setId(connectorNode.getIdentifier()); + snapshot.setName(connectorNode.getName()); + snapshot.setType(connectorNode.getCanonicalClassName()); + snapshot.setRunStatus(connectorNode.getCurrentState().name()); + + // Populate all status metrics from the managed process group + if (managedGroupStatus != null) { + snapshot.setFlowFilesSent(managedGroupStatus.getFlowFilesSent()); + snapshot.setBytesSent(managedGroupStatus.getBytesSent()); + snapshot.setFlowFilesReceived(managedGroupStatus.getFlowFilesReceived()); + snapshot.setBytesReceived(managedGroupStatus.getBytesReceived()); + snapshot.setBytesRead(managedGroupStatus.getBytesRead()); + snapshot.setBytesWritten(managedGroupStatus.getBytesWritten()); + snapshot.setFlowFilesQueued(managedGroupStatus.getQueuedCount()); + snapshot.setBytesQueued(managedGroupStatus.getQueuedContentSize()); + snapshot.setActiveThreadCount(managedGroupStatus.getActiveThreadCount()); + + final ProcessingPerformanceStatus performanceStatus = managedGroupStatus.getProcessingPerformanceStatus(); + if (performanceStatus != null) { + snapshot.setProcessingPerformanceStatus(createProcessingPerformanceStatusDTO(performanceStatus)); + } + } + + // Populate idle status + final Optional idleDuration = connectorNode.getIdleDuration(); + if (idleDuration.isPresent()) { + snapshot.setIdle(true); + final long idleMillis = idleDuration.get().toMillis(); + snapshot.setIdleDurationMillis(idleMillis); + snapshot.setIdleDuration(FormatUtils.formatHoursMinutesSeconds(idleMillis, TimeUnit.MILLISECONDS)); + } else { + snapshot.setIdle(false); + } + + StatusMerger.updatePrettyPrintedFields(snapshot); + + return statusDto; + } + + private List createConnectorActionDtos(final ConnectorNode connector) { + return connector.getAvailableActions().stream() + .map(this::createConnectorActionDto) + .collect(Collectors.toList()); + } + + private ConnectorActionDTO createConnectorActionDto(final ConnectorAction action) { + final ConnectorActionDTO dto = new ConnectorActionDTO(); + dto.setName(action.getName()); + dto.setDescription(action.getDescription()); + dto.setAllowed(action.isAllowed()); + dto.setReasonNotAllowed(action.getReasonNotAllowed()); + return dto; + } + + private ConnectorConfigurationDTO createConnectorConfigurationDtoFromFlowContext(final ConnectorNode connector, final FrameworkFlowContext flowContext) { + final List configurationSteps = connector.getConfigurationSteps(); + if (configurationSteps == null || configurationSteps.isEmpty()) { + return null; + } + + if (flowContext == null || flowContext.getConfigurationContext() == null) { + return null; + } + + final BundleCoordinate bundleCoordinate = connector.getBundleCoordinate(); + final String connectorType = connector.getCanonicalClassName(); + + final Set stepsWithDocumentation = discoverStepsWithDocumentation(bundleCoordinate, connectorType); + + final ConnectorConfiguration configuration = flowContext.getConfigurationContext().toConnectorConfiguration(); + final ConnectorConfigurationDTO dto = new ConnectorConfigurationDTO(); + final List configurationStepDtos = configurationSteps.stream() + .map(step -> createConfigurationStepConfigurationDtoFromStep(step, configuration, stepsWithDocumentation)) + .collect(Collectors.toList()); + dto.setConfigurationStepConfigurations(configurationStepDtos); + return dto; + } + + private Set discoverStepsWithDocumentation(final BundleCoordinate bundleCoordinate, final String connectorType) { + if (runtimeManifestService == null) { + return Collections.emptySet(); + } + + try { + final Map stepDocs = runtimeManifestService.discoverStepDocumentation( + bundleCoordinate.getGroup(), bundleCoordinate.getId(), bundleCoordinate.getVersion(), connectorType); + return stepDocs.keySet(); + } catch (final Exception e) { + logger.debug("Unable to discover step documentation for connector [{}]: {}", connectorType, e.getMessage()); + return Collections.emptySet(); + } + } + + private ConfigurationStepConfigurationDTO createConfigurationStepConfigurationDtoFromStep(final ConfigurationStep step, + final ConnectorConfiguration configuration, + final Set stepsWithDocumentation) { + if (step == null) { + return null; + } + + final ConfigurationStepConfigurationDTO dto = new ConfigurationStepConfigurationDTO(); + dto.setConfigurationStepName(step.getName()); + dto.setConfigurationStepDescription(step.getDescription()); + dto.setDocumented(stepsWithDocumentation.contains(step.getName())); + + // Get the current configuration values for this step from the flat StepConfiguration + final StepConfiguration stepConfig = configuration.getNamedStepConfigurations().stream() + .filter(c -> step.getName().equals(c.stepName())) + .map(NamedStepConfiguration::configuration) + .findFirst() + .orElse(null); + + // Convert property groups from the schema, merging in current values from flat stepConfig + final List propertyGroupDtos = step.getPropertyGroups().stream() + .map(propertyGroup -> createPropertyGroupConfigurationDtoFromGroup(propertyGroup, stepConfig)) + .collect(Collectors.toList()); + dto.setPropertyGroupConfigurations(propertyGroupDtos); + + // Convert step dependencies + final Set dependencyDtos = step.getDependencies().stream() + .map(this::createConfigurationStepDependencyDto) + .collect(Collectors.toSet()); + dto.setDependencies(dependencyDtos); + + return dto; + } + + private ConfigurationStepDependencyDTO createConfigurationStepDependencyDto(final ConfigurationStepDependency dependency) { + final ConfigurationStepDependencyDTO dto = new ConfigurationStepDependencyDTO(); + dto.setStepName(dependency.getStepName()); + dto.setPropertyName(dependency.getPropertyName()); + dto.setDependentValues(dependency.getDependentValues()); + return dto; + } + + private PropertyGroupConfigurationDTO createPropertyGroupConfigurationDtoFromGroup(final ConnectorPropertyGroup propertyGroup, final StepConfiguration stepConfig) { + if (propertyGroup == null) { + return null; + } + + final PropertyGroupConfigurationDTO dto = new PropertyGroupConfigurationDTO(); + dto.setPropertyGroupName(propertyGroup.getName()); + dto.setPropertyGroupDescription(propertyGroup.getDescription()); + + // Convert property descriptors from the schema, keyed by property name + // Use LinkedHashMap to preserve the order from the connector + final Map propertyDescriptorMap = new LinkedHashMap<>(); + for (final ConnectorPropertyDescriptor propertyDescriptor : propertyGroup.getProperties()) { + final ConnectorPropertyDescriptorDTO descriptorDto = createConnectorPropertyDescriptorDto(propertyDescriptor); + propertyDescriptorMap.put(descriptorDto.getName(), descriptorDto); + } + dto.setPropertyDescriptors(propertyDescriptorMap); + + // Get the current property values for this group's properties from flat stepConfig + // Use LinkedHashMap to preserve the order from the connector + final Map propertyValues = new LinkedHashMap<>(); + if (stepConfig != null) { + // For each property in this group, check if there's a value in the flat stepConfig + for (final ConnectorPropertyDescriptor propertyDescriptor : propertyGroup.getProperties()) { + final ConnectorValueReference valueRef = stepConfig.getPropertyValue(propertyDescriptor.getName()); + if (valueRef != null) { + propertyValues.put(propertyDescriptor.getName(), createConnectorValueReferenceDto(valueRef)); + } + } + } + + dto.setPropertyValues(propertyValues); + return dto; + } + + private ConnectorValueReferenceDTO createConnectorValueReferenceDto(final ConnectorValueReference valueReference) { + if (valueReference == null) { + return null; + } + + final ConnectorValueReferenceDTO dto = new ConnectorValueReferenceDTO(); + dto.setValueType(valueReference.getValueType() != null ? valueReference.getValueType().name() : null); + + switch (valueReference) { + case StringLiteralValue stringLiteral -> dto.setValue(stringLiteral.getValue()); + case AssetReference assetRef -> { + final List assetReferences = assetRef.getAssetIdentifiers().stream() + .map(this::createConnectorAssetReferenceDto) + .toList(); + dto.setAssetReferences(assetReferences); + } + case SecretReference secretRef -> { + dto.setSecretProviderId(secretRef.getProviderId()); + dto.setSecretProviderName(secretRef.getProviderName()); + dto.setSecretName(secretRef.getSecretName()); + dto.setFullyQualifiedSecretName(secretRef.getFullyQualifiedName()); + } + } + + return dto; + } + + private AssetReferenceDTO createConnectorAssetReferenceDto(final String assetId) { + final String assetName = connectorRepository.getAsset(assetId).map(Asset::getName).orElse(assetId); + return new AssetReferenceDTO(assetId, assetName); + } + + private ConnectorPropertyDescriptorDTO createConnectorPropertyDescriptorDto(final ConnectorPropertyDescriptor propertyDescriptor) { + if (propertyDescriptor == null) { + return null; + } + + final ConnectorPropertyDescriptorDTO dto = new ConnectorPropertyDescriptorDTO(); + dto.setName(propertyDescriptor.getName()); + dto.setDescription(propertyDescriptor.getDescription()); + dto.setDefaultValue(propertyDescriptor.getDefaultValue()); + dto.setRequired(propertyDescriptor.isRequired()); + dto.setType(propertyDescriptor.getType() != null ? propertyDescriptor.getType().name() : null); + dto.setAllowableValuesFetchable(propertyDescriptor.isAllowableValuesFetchable()); + + // Convert allowable values if present + if (propertyDescriptor.getAllowableValues() != null && !propertyDescriptor.getAllowableValues().isEmpty()) { + final List allowableValueEntities = propertyDescriptor.getAllowableValues().stream() + .map(describedValue -> { + final AllowableValueDTO allowableValueDto = new AllowableValueDTO(); + allowableValueDto.setValue(describedValue.getValue()); + allowableValueDto.setDisplayName(describedValue.getDisplayName()); + allowableValueDto.setDescription(describedValue.getDescription()); + + final AllowableValueEntity entity = new AllowableValueEntity(); + entity.setAllowableValue(allowableValueDto); + entity.setCanRead(true); + return entity; + }) + .collect(Collectors.toList()); + dto.setAllowableValues(allowableValueEntities); + } + + // Convert dependencies if present + if (propertyDescriptor.getDependencies() != null && !propertyDescriptor.getDependencies().isEmpty()) { + final Set dependencyDtos = propertyDescriptor.getDependencies().stream() + .map(dependency -> { + final ConnectorPropertyDependencyDTO dependencyDto = new ConnectorPropertyDependencyDTO(); + dependencyDto.setPropertyName(dependency.getPropertyName()); + dependencyDto.setDependentValues(dependency.getDependentValues()); + return dependencyDto; + }) + .collect(Collectors.toSet()); + dto.setDependencies(dependencyDtos); + } + + return dto; + } + + /** + * Creates an AllowableValueDTO from the specified AllowableValue. + * + * @param allowableValue the allowable value + * @return the DTO + */ + public AllowableValueDTO createAllowableValueDto(final DescribedValue allowableValue) { + if (allowableValue == null) { + return null; + } + + final AllowableValueDTO dto = new AllowableValueDTO(); + dto.setValue(allowableValue.getValue()); + dto.setDisplayName(allowableValue.getDisplayName()); + dto.setDescription(allowableValue.getDescription()); + return dto; + } + + /** + * Creates a ConfigVerificationResultDTO from the specified ConfigVerificationResult. + * + * @param result the verification result + * @return the DTO + */ + public ConfigVerificationResultDTO createConfigVerificationResultDto(final ConfigVerificationResult result) { + final ConfigVerificationResultDTO dto = new ConfigVerificationResultDTO(); + dto.setExplanation(result.getExplanation()); + dto.setOutcome(result.getOutcome().name()); + dto.setVerificationStepName(result.getVerificationStepName()); + dto.setSubject(result.getSubject()); + return dto; + } + + /** + * Creates a SecretDTO from the specified Secret. Note: The secret value is intentionally not included + * in the DTO for security reasons. + * + * @param secret the secret + * @return the DTO containing only the secret's metadata + */ + public SecretDTO createSecretDto(final Secret secret) { + if (secret == null) { + return null; + } + + final SecretDTO dto = new SecretDTO(); + dto.setProviderId(secret.getProviderId()); + dto.setProviderName(secret.getProviderName()); + dto.setGroupName(secret.getGroupName()); + dto.setName(secret.getName()); + dto.setFullyQualifiedName(secret.getFullyQualifiedName()); + dto.setDescription(secret.getDescription()); + return dto; + } + } diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java index f17f2781440d..be52686de488 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java @@ -24,6 +24,7 @@ import org.apache.nifi.web.api.dto.status.ConnectionStatisticsSnapshotDTO; 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.ConnectorStatusDTO; import org.apache.nifi.web.api.dto.status.ControllerServiceStatusDTO; import org.apache.nifi.web.api.dto.status.FlowAnalysisRuleStatusDTO; import org.apache.nifi.web.api.dto.status.PortStatusDTO; @@ -44,11 +45,15 @@ import org.apache.nifi.web.api.entity.BulletinEntity; import org.apache.nifi.web.api.entity.ComponentReferenceEntity; import org.apache.nifi.web.api.entity.ComponentValidationResultEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepNamesEntity; import org.apache.nifi.web.api.entity.ConnectionEntity; import org.apache.nifi.web.api.entity.ConnectionStatisticsEntity; import org.apache.nifi.web.api.entity.ConnectionStatisticsSnapshotEntity; import org.apache.nifi.web.api.entity.ConnectionStatusEntity; import org.apache.nifi.web.api.entity.ConnectionStatusSnapshotEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ConnectorPropertyAllowableValuesEntity; import org.apache.nifi.web.api.entity.ControllerConfigurationEntity; import org.apache.nifi.web.api.entity.ControllerServiceEntity; import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentEntity; @@ -80,6 +85,7 @@ import org.apache.nifi.web.api.entity.RemoteProcessGroupStatusEntity; import org.apache.nifi.web.api.entity.RemoteProcessGroupStatusSnapshotEntity; import org.apache.nifi.web.api.entity.ReportingTaskEntity; +import org.apache.nifi.web.api.entity.SecretsEntity; import org.apache.nifi.web.api.entity.SnippetEntity; import org.apache.nifi.web.api.entity.StatusHistoryEntity; import org.apache.nifi.web.api.entity.TenantEntity; @@ -124,6 +130,25 @@ public ProcessorStatusEntity createProcessorStatusEntity(final ProcessorStatusDT return entity; } + public ConnectorEntity createConnectorEntity(final ConnectorDTO dto, + final RevisionDTO revision, + final PermissionsDTO permissions, + final PermissionsDTO operatePermissions, + final ConnectorStatusDTO status) { + final ConnectorEntity entity = new ConnectorEntity(); + entity.setRevision(revision); + if (dto != null) { + entity.setPermissions(permissions); + entity.setOperatePermissions(operatePermissions); + entity.setStatus(status); + entity.setId(dto.getId()); + if (permissions != null && permissions.getCanRead()) { + entity.setComponent(dto); + } + } + return entity; + } + public ProcessorStatusSnapshotEntity createProcessorStatusSnapshotEntity(final ProcessorStatusSnapshotDTO status, final PermissionsDTO permissions) { final ProcessorStatusSnapshotEntity entity = new ProcessorStatusSnapshotEntity(); entity.setId(status.getId()); @@ -842,4 +867,68 @@ public FlowRegistryBranchEntity createBranchEntity(final FlowRegistryBranchDTO d public NarSummaryEntity createNarSummaryEntity(final NarSummaryDTO narSummaryDTO) { return new NarSummaryEntity(narSummaryDTO); } + + public ConfigurationStepNamesEntity createConfigurationStepNamesEntity(final ConnectorDTO connectorDto) { + final ConfigurationStepNamesEntity entity = new ConfigurationStepNamesEntity(); + + final List configurationStepNames; + if (connectorDto != null + && connectorDto.getWorkingConfiguration() != null + && connectorDto.getWorkingConfiguration().getConfigurationStepConfigurations() != null) { + + configurationStepNames = connectorDto.getWorkingConfiguration().getConfigurationStepConfigurations() + .stream() + .map(ConfigurationStepConfigurationDTO::getConfigurationStepName) + .collect(Collectors.toList()); + } else { + configurationStepNames = List.of(); + } + + entity.setConfigurationStepNames(configurationStepNames); + return entity; + } + + public ConfigurationStepEntity createConfigurationStepEntity(final ConnectorDTO connectorDto, + final String configurationStepName, final RevisionDTO parentConnectorRevision) { + ConfigurationStepConfigurationDTO foundConfigurationStep = null; + + if (connectorDto != null + && connectorDto.getWorkingConfiguration() != null + && connectorDto.getWorkingConfiguration().getConfigurationStepConfigurations() != null) { + + foundConfigurationStep = connectorDto.getWorkingConfiguration().getConfigurationStepConfigurations() + .stream() + .filter(step -> configurationStepName.equals(step.getConfigurationStepName())) + .findFirst() + .orElse(null); + } + + if (foundConfigurationStep == null) { + final String connectorId = connectorDto != null ? connectorDto.getId() : "null"; + throw new IllegalArgumentException( + String.format("Configuration step '%s' not found for connector '%s'.", configurationStepName, connectorId)); + } + + final ConfigurationStepEntity entity = new ConfigurationStepEntity(); + entity.setConfigurationStep(foundConfigurationStep); + entity.setParentConnectorId(connectorDto != null ? connectorDto.getId() : null); + entity.setParentConnectorRevision(parentConnectorRevision); + return entity; + } + + public ConnectorPropertyAllowableValuesEntity createConnectorPropertyAllowableValuesEntity( + final String configurationStepName, final String propertyGroupName, final String propertyName, final List allowableValues) { + final ConnectorPropertyAllowableValuesEntity entity = new ConnectorPropertyAllowableValuesEntity(); + entity.setConfigurationStepName(configurationStepName); + entity.setPropertyGroupName(propertyGroupName); + entity.setPropertyName(propertyName); + entity.setAllowableValues(allowableValues); + return entity; + } + + public SecretsEntity createSecretsEntity(final List secrets) { + final SecretsEntity entity = new SecretsEntity(); + entity.setSecrets(secrets); + return entity; + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/configuration/WebApplicationConfiguration.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/configuration/WebApplicationConfiguration.java index 02e23dbe2121..7ed4627cfb78 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/configuration/WebApplicationConfiguration.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/configuration/WebApplicationConfiguration.java @@ -18,6 +18,7 @@ import org.apache.nifi.admin.service.AuditService; import org.apache.nifi.audit.NiFiAuditor; +import org.apache.nifi.authorization.AuthorizableLookup; import org.apache.nifi.authorization.Authorizer; import org.apache.nifi.authorization.StandardAuthorizableLookup; import org.apache.nifi.cluster.coordination.ClusterCoordinator; @@ -31,6 +32,7 @@ import org.apache.nifi.services.FlowService; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.web.ContentAccess; +import org.apache.nifi.web.NiFiConnectorWebContext; import org.apache.nifi.web.NiFiServiceFacade; import org.apache.nifi.web.NiFiServiceFacadeLock; import org.apache.nifi.web.NiFiWebConfigurationContext; @@ -44,9 +46,11 @@ import org.apache.nifi.web.api.metrics.jmx.JmxMetricsResultConverter; import org.apache.nifi.web.api.metrics.jmx.JmxMetricsService; import org.apache.nifi.web.api.metrics.jmx.StandardJmxMetricsService; +import org.apache.nifi.web.connector.StandardNiFiConnectorWebContext; import org.apache.nifi.web.controller.ControllerFacade; import org.apache.nifi.web.controller.ControllerSearchService; import org.apache.nifi.web.dao.AccessPolicyDAO; +import org.apache.nifi.web.dao.ConnectorDAO; import org.apache.nifi.web.dao.impl.ComponentDAO; import org.apache.nifi.web.revision.RevisionManager; import org.apache.nifi.web.search.query.RegexSearchQueryParser; @@ -104,6 +108,10 @@ public class WebApplicationConfiguration { private NiFiServiceFacade serviceFacade; + private ConnectorDAO connectorDAO; + + private AuthorizableLookup authorizableLookup; + public WebApplicationConfiguration( final Authorizer authorizer, final AccessPolicyDAO accessPolicyDao, @@ -145,6 +153,16 @@ public void setServiceFacade(final NiFiServiceFacade serviceFacade) { this.serviceFacade = serviceFacade; } + @Autowired + public void setConnectorDAO(final ConnectorDAO connectorDAO) { + this.connectorDAO = connectorDAO; + } + + @Autowired + public void setAuthorizableLookup(final AuthorizableLookup authorizableLookup) { + this.authorizableLookup = authorizableLookup; + } + @Bean public EntityFactory entityFactory() { return new EntityFactory(); @@ -168,6 +186,8 @@ public DtoFactory dtoFactory() { dtoFactory.setControllerServiceProvider(flowController.getControllerServiceProvider()); dtoFactory.setEntityFactory(entityFactory()); dtoFactory.setExtensionManager(extensionManager); + dtoFactory.setConnectorRepository(flowController.getConnectorRepository()); + dtoFactory.setRuntimeManifestService(runtimeManifestService); return dtoFactory; } @@ -254,6 +274,15 @@ public NiFiWebConfigurationContext nifiWebConfigurationContext() { return context; } + @Bean + public NiFiConnectorWebContext nifiConnectorWebContext() { + final StandardNiFiConnectorWebContext context = new StandardNiFiConnectorWebContext(); + context.setConnectorDAO(connectorDAO); + context.setAuthorizer(authorizer); + context.setAuthorizableLookup(authorizableLookup); + return context; + } + @Bean public ControllerFacade controllerFacade() { final ControllerFacade controllerFacade = new ControllerFacade(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/StandardNiFiConnectorWebContext.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/StandardNiFiConnectorWebContext.java new file mode 100644 index 000000000000..5b5018174c47 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/StandardNiFiConnectorWebContext.java @@ -0,0 +1,99 @@ +/* + * 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.connector; + +import org.apache.nifi.authorization.AuthorizableLookup; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.web.NiFiConnectorWebContext; +import org.apache.nifi.web.connector.authorization.AuthorizingConnectorInvocationHandler; +import org.apache.nifi.web.connector.authorization.AuthorizingFlowContext; +import org.apache.nifi.web.connector.authorization.ConnectorAuthorizationContext; +import org.apache.nifi.web.dao.ConnectorDAO; + +import java.lang.reflect.Proxy; + +/** + * Implements the NiFiConnectorWebContext interface to provide + * Connector instances to connector custom UIs. + * + *

The returned Connector instance is wrapped in an authorization proxy that + * enforces permissions based on the {@link ConnectorWebMethod} annotation on + * the connector interface methods. Methods without this annotation cannot be + * invoked through the proxy.

+ * + *

The returned FlowContext instances are also wrapped in authorization wrappers + * that enforce read/write permissions on all operations.

+ */ +public class StandardNiFiConnectorWebContext implements NiFiConnectorWebContext { + + private ConnectorDAO connectorDAO; + private Authorizer authorizer; + private AuthorizableLookup authorizableLookup; + + @Override + @SuppressWarnings("unchecked") + public ConnectorWebContext getConnectorWebContext(final String connectorId) throws IllegalArgumentException { + final ConnectorNode connectorNode = connectorDAO.getConnector(connectorId); + if (connectorNode == null) { + throw new IllegalArgumentException("Unable to find connector with id: " + connectorId); + } + + final ConnectorAuthorizationContext authContext = new ConnectorAuthorizationContext(connectorId, authorizer, authorizableLookup); + + final T connector = (T) connectorNode.getConnector(); + final T authorizedConnectorProxy = createAuthorizingProxy(connector, connectorId); + + final FlowContext workingFlowContext = new AuthorizingFlowContext(connectorNode.getWorkingFlowContext(), authContext); + final FlowContext activeFlowContext = new AuthorizingFlowContext(connectorNode.getActiveFlowContext(), authContext); + + return new ConnectorWebContext<>(authorizedConnectorProxy, workingFlowContext, activeFlowContext); + } + + /** + * Creates a proxy around the given connector that enforces authorization + * based on {@link ConnectorWebMethod} annotations. + * + * @param the type of the connector + * @param connector the connector instance to wrap + * @param connectorId the ID of the connector + * @return a proxy that enforces authorization on method invocations + */ + @SuppressWarnings("unchecked") + private T createAuthorizingProxy(final T connector, final String connectorId) { + final AuthorizingConnectorInvocationHandler handler = new AuthorizingConnectorInvocationHandler<>( + connector, connectorId, authorizer, authorizableLookup); + + return (T) Proxy.newProxyInstance( + connector.getClass().getClassLoader(), + connector.getClass().getInterfaces(), + handler); + } + + public void setConnectorDAO(final ConnectorDAO connectorDAO) { + this.connectorDAO = connectorDAO; + } + + public void setAuthorizer(final Authorizer authorizer) { + this.authorizer = authorizer; + } + + public void setAuthorizableLookup(final AuthorizableLookup authorizableLookup) { + this.authorizableLookup = authorizableLookup; + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingConnectionFacade.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingConnectionFacade.java new file mode 100644 index 000000000000..ea72b197f6b6 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingConnectionFacade.java @@ -0,0 +1,89 @@ +/* + * 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.connector.authorization; + +import org.apache.nifi.components.connector.DropFlowFileSummary; +import org.apache.nifi.components.connector.components.ConnectionFacade; +import org.apache.nifi.controller.queue.QueueSize; +import org.apache.nifi.flow.VersionedConnection; +import org.apache.nifi.flowfile.FlowFile; + +import java.io.IOException; +import java.util.Objects; +import java.util.function.Predicate; + +/** + * A wrapper around {@link ConnectionFacade} that enforces authorization before delegating + * to the underlying implementation. + */ +public class AuthorizingConnectionFacade implements ConnectionFacade { + + private final ConnectionFacade delegate; + private final ConnectorAuthorizationContext authContext; + + public AuthorizingConnectionFacade(final ConnectionFacade delegate, final ConnectorAuthorizationContext authContext) { + this.delegate = delegate; + this.authContext = authContext; + } + + @Override + public VersionedConnection getDefinition() { + authContext.authorizeRead(); + return delegate.getDefinition(); + } + + @Override + public QueueSize getQueueSize() { + authContext.authorizeRead(); + return delegate.getQueueSize(); + } + + @Override + public void purge() { + authContext.authorizeWrite(); + delegate.purge(); + } + + @Override + public DropFlowFileSummary dropFlowFiles(final Predicate predicate) throws IOException { + authContext.authorizeWrite(); + return delegate.dropFlowFiles(predicate); + } + + @Override + public String toString() { + return "AuthorizingConnectionFacade[delegate=" + delegate + "]"; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final AuthorizingConnectionFacade that = (AuthorizingConnectionFacade) o; + return Objects.equals(delegate, that.delegate); + } + + @Override + public int hashCode() { + return Objects.hashCode(delegate); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingConnectorInvocationHandler.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingConnectorInvocationHandler.java new file mode 100644 index 000000000000..130fa4324852 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingConnectorInvocationHandler.java @@ -0,0 +1,130 @@ +/* + * 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.connector.authorization; + +import org.apache.nifi.authorization.AuthorizableLookup; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.RequestAction; +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.authorization.user.NiFiUserUtils; +import org.apache.nifi.web.ConnectorWebMethod; + +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; + +/** + * An InvocationHandler that wraps a Connector instance and enforces authorization + * based on the {@link ConnectorWebMethod} annotation present on the invoked method. + * + *

Methods must be annotated with {@link ConnectorWebMethod} to be invokable through + * this handler. The annotation specifies whether READ or WRITE access is required. + * Methods without the annotation will result in an {@link IllegalStateException}.

+ * + * @param the type of the Connector being proxied + */ +public class AuthorizingConnectorInvocationHandler implements InvocationHandler { + + private final T delegate; + private final String connectorId; + private final Authorizer authorizer; + private final AuthorizableLookup authorizableLookup; + + /** + * Constructs an AuthorizingConnectorInvocationHandler. + * + * @param delegate the actual Connector instance to delegate method calls to + * @param connectorId the ID of the connector, used for authorization lookups + * @param authorizer the Authorizer to use for authorization checks + * @param authorizableLookup the lookup service to obtain the Authorizable for the connector + */ + public AuthorizingConnectorInvocationHandler(final T delegate, final String connectorId, + final Authorizer authorizer, final AuthorizableLookup authorizableLookup) { + this.delegate = delegate; + this.connectorId = connectorId; + this.authorizer = authorizer; + this.authorizableLookup = authorizableLookup; + } + + @Override + public Object invoke(final Object proxy, final Method method, final Object[] args) throws Throwable { + final ConnectorWebMethod annotation = findConnectorWebMethodAnnotation(method); + + if (annotation == null) { + throw new IllegalStateException(String.format( + "Method [%s] on connector [%s] is not annotated with @ConnectorWebMethod and cannot be invoked through the Connector Web Context", + method.getName(), connectorId)); + } + + final RequestAction requiredAction = mapAccessTypeToRequestAction(annotation.value()); + final Authorizable connector = authorizableLookup.getConnector(connectorId); + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + + connector.authorize(authorizer, requiredAction, user); + + try { + return method.invoke(delegate, args); + } catch (final InvocationTargetException e) { + throw e.getCause(); + } + } + + /** + * Maps the ConnectorWebMethod.AccessType to the corresponding RequestAction. + * + * @param accessType the access type from the annotation + * @return the corresponding RequestAction + */ + private RequestAction mapAccessTypeToRequestAction(final ConnectorWebMethod.AccessType accessType) { + return switch (accessType) { + case READ -> RequestAction.READ; + case WRITE -> RequestAction.WRITE; + }; + } + + /** + * Finds the ConnectorWebMethod annotation on the given method. This method searches + * the declaring class's interfaces to find the annotation, as the method parameter + * may be from the proxy class rather than the interface. + * + * @param method the method to search for the annotation + * @return the ConnectorWebMethod annotation, or null if not found + */ + private ConnectorWebMethod findConnectorWebMethodAnnotation(final Method method) { + final ConnectorWebMethod directAnnotation = method.getAnnotation(ConnectorWebMethod.class); + if (directAnnotation != null) { + return directAnnotation; + } + + for (final Class iface : delegate.getClass().getInterfaces()) { + try { + final Method interfaceMethod = iface.getMethod(method.getName(), method.getParameterTypes()); + final ConnectorWebMethod interfaceAnnotation = interfaceMethod.getAnnotation(ConnectorWebMethod.class); + if (interfaceAnnotation != null) { + return interfaceAnnotation; + } + } catch (final NoSuchMethodException ignored) { + // Method not found on this interface; continue searching other interfaces + continue; + } + } + + return null; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingControllerServiceFacade.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingControllerServiceFacade.java new file mode 100644 index 000000000000..a85d924079f8 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingControllerServiceFacade.java @@ -0,0 +1,122 @@ +/* + * 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.connector.authorization; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.InvocationFailedException; +import org.apache.nifi.components.connector.components.ControllerServiceFacade; +import org.apache.nifi.components.connector.components.ControllerServiceLifecycle; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedParameterContext; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * A wrapper around {@link ControllerServiceFacade} that enforces authorization before delegating + * to the underlying implementation. + */ +public class AuthorizingControllerServiceFacade implements ControllerServiceFacade { + + private final ControllerServiceFacade delegate; + private final ConnectorAuthorizationContext authContext; + + public AuthorizingControllerServiceFacade(final ControllerServiceFacade delegate, final ConnectorAuthorizationContext authContext) { + this.delegate = delegate; + this.authContext = authContext; + } + + @Override + public VersionedControllerService getDefinition() { + authContext.authorizeRead(); + return delegate.getDefinition(); + } + + @Override + public ControllerServiceLifecycle getLifecycle() { + authContext.authorizeRead(); + return new AuthorizingControllerServiceLifecycle(delegate.getLifecycle(), authContext); + } + + @Override + public List validate() { + authContext.authorizeRead(); + return delegate.validate(); + } + + @Override + public List validate(final Map propertyValues) { + authContext.authorizeRead(); + return delegate.validate(propertyValues); + } + + @Override + public List verify(final Map propertyValues, final Map variables) { + authContext.authorizeRead(); + return delegate.verify(propertyValues, variables); + } + + @Override + public List verify(final Map propertyValues, final VersionedParameterContext parameterContext, final Map variables) { + authContext.authorizeRead(); + return delegate.verify(propertyValues, parameterContext, variables); + } + + @Override + public List verify(final VersionedExternalFlow versionedExternalFlow, final Map variables) { + authContext.authorizeRead(); + return delegate.verify(versionedExternalFlow, variables); + } + + @Override + public Object invokeConnectorMethod(final String methodName, final Map arguments) throws InvocationFailedException { + authContext.authorizeWrite(); + return delegate.invokeConnectorMethod(methodName, arguments); + } + + @Override + public T invokeConnectorMethod(final String methodName, final Map arguments, final Class returnType) throws InvocationFailedException { + authContext.authorizeWrite(); + return delegate.invokeConnectorMethod(methodName, arguments, returnType); + } + + @Override + public String toString() { + return "AuthorizingControllerServiceFacade[delegate=" + delegate + "]"; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final AuthorizingControllerServiceFacade that = (AuthorizingControllerServiceFacade) o; + return Objects.equals(delegate, that.delegate); + } + + @Override + public int hashCode() { + return Objects.hashCode(delegate); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingControllerServiceLifecycle.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingControllerServiceLifecycle.java new file mode 100644 index 000000000000..9c4332a45a4b --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingControllerServiceLifecycle.java @@ -0,0 +1,63 @@ +/* + * 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.connector.authorization; + +import org.apache.nifi.components.connector.components.ControllerServiceLifecycle; +import org.apache.nifi.components.connector.components.ControllerServiceState; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** + * A wrapper around {@link ControllerServiceLifecycle} that enforces authorization before delegating + * to the underlying implementation. + */ +public class AuthorizingControllerServiceLifecycle implements ControllerServiceLifecycle { + + private final ControllerServiceLifecycle delegate; + private final ConnectorAuthorizationContext authContext; + + public AuthorizingControllerServiceLifecycle(final ControllerServiceLifecycle delegate, final ConnectorAuthorizationContext authContext) { + this.delegate = delegate; + this.authContext = authContext; + } + + @Override + public ControllerServiceState getState() { + authContext.authorizeRead(); + return delegate.getState(); + } + + @Override + public CompletableFuture enable() { + authContext.authorizeWrite(); + return delegate.enable(); + } + + @Override + public CompletableFuture enable(final Map propertyValueOverrides) { + authContext.authorizeWrite(); + return delegate.enable(propertyValueOverrides); + } + + @Override + public CompletableFuture disable() { + authContext.authorizeWrite(); + return delegate.disable(); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingFlowContext.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingFlowContext.java new file mode 100644 index 000000000000..d03b5e71fb3e --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingFlowContext.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.connector.authorization; + +import org.apache.nifi.components.connector.ConnectorConfigurationContext; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.components.FlowContextType; +import org.apache.nifi.components.connector.components.ParameterContextFacade; +import org.apache.nifi.components.connector.components.ProcessGroupFacade; +import org.apache.nifi.flow.Bundle; + +/** + * A wrapper around {@link FlowContext} that enforces authorization before delegating + * to the underlying implementation. + */ +public class AuthorizingFlowContext implements FlowContext { + + private final FlowContext delegate; + private final ConnectorAuthorizationContext authContext; + + public AuthorizingFlowContext(final FlowContext delegate, final ConnectorAuthorizationContext authContext) { + this.delegate = delegate; + this.authContext = authContext; + } + + @Override + public ProcessGroupFacade getRootGroup() { + authContext.authorizeRead(); + return new AuthorizingProcessGroupFacade(delegate.getRootGroup(), authContext); + } + + @Override + public ParameterContextFacade getParameterContext() { + authContext.authorizeRead(); + return new AuthorizingParameterContextFacade(delegate.getParameterContext(), authContext); + } + + @Override + public ConnectorConfigurationContext getConfigurationContext() { + authContext.authorizeRead(); + return delegate.getConfigurationContext(); + } + + @Override + public FlowContextType getType() { + authContext.authorizeRead(); + return delegate.getType(); + } + + @Override + public Bundle getBundle() { + authContext.authorizeRead(); + return delegate.getBundle(); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingParameterContextFacade.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingParameterContextFacade.java new file mode 100644 index 000000000000..25eb8fb117c1 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingParameterContextFacade.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.connector.authorization; + +import org.apache.nifi.asset.Asset; +import org.apache.nifi.components.connector.components.ParameterContextFacade; +import org.apache.nifi.components.connector.components.ParameterValue; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Collection; +import java.util.Objects; +import java.util.Set; + +/** + * A wrapper around {@link ParameterContextFacade} that enforces authorization before delegating + * to the underlying implementation. + */ +public class AuthorizingParameterContextFacade implements ParameterContextFacade { + + private final ParameterContextFacade delegate; + private final ConnectorAuthorizationContext authContext; + + public AuthorizingParameterContextFacade(final ParameterContextFacade delegate, final ConnectorAuthorizationContext authContext) { + this.delegate = delegate; + this.authContext = authContext; + } + + @Override + public void updateParameters(final Collection parameterValues) { + authContext.authorizeWrite(); + delegate.updateParameters(parameterValues); + } + + @Override + public String getValue(final String parameterName) { + authContext.authorizeRead(); + return delegate.getValue(parameterName); + } + + @Override + public Set getDefinedParameterNames() { + authContext.authorizeRead(); + return delegate.getDefinedParameterNames(); + } + + @Override + public boolean isSensitive(final String parameterName) { + authContext.authorizeRead(); + return delegate.isSensitive(parameterName); + } + + @Override + public Asset createAsset(final InputStream inputStream) throws IOException { + authContext.authorizeWrite(); + return delegate.createAsset(inputStream); + } + + @Override + public String toString() { + return "AuthorizingParameterContextFacade[delegate=" + delegate + "]"; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final AuthorizingParameterContextFacade that = (AuthorizingParameterContextFacade) o; + return Objects.equals(delegate, that.delegate); + } + + @Override + public int hashCode() { + return Objects.hashCode(delegate); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessGroupFacade.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessGroupFacade.java new file mode 100644 index 000000000000..f20e97a9becc --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessGroupFacade.java @@ -0,0 +1,178 @@ +/* + * 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.connector.authorization; + +import org.apache.nifi.components.connector.DropFlowFileSummary; +import org.apache.nifi.components.connector.components.ComponentHierarchyScope; +import org.apache.nifi.components.connector.components.ConnectionFacade; +import org.apache.nifi.components.connector.components.ControllerServiceFacade; +import org.apache.nifi.components.connector.components.ControllerServiceReferenceScope; +import org.apache.nifi.components.connector.components.ProcessGroupFacade; +import org.apache.nifi.components.connector.components.ProcessGroupLifecycle; +import org.apache.nifi.components.connector.components.ProcessorFacade; +import org.apache.nifi.components.connector.components.StatelessGroupLifecycle; +import org.apache.nifi.controller.queue.QueueSize; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flowfile.FlowFile; + +import java.io.IOException; +import java.util.Objects; +import java.util.Set; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +/** + * A wrapper around {@link ProcessGroupFacade} that enforces authorization before delegating + * to the underlying implementation. + */ +public class AuthorizingProcessGroupFacade implements ProcessGroupFacade { + + private final ProcessGroupFacade delegate; + private final ConnectorAuthorizationContext authContext; + + public AuthorizingProcessGroupFacade(final ProcessGroupFacade delegate, final ConnectorAuthorizationContext authContext) { + this.delegate = delegate; + this.authContext = authContext; + } + + @Override + public VersionedProcessGroup getDefinition() { + authContext.authorizeRead(); + return delegate.getDefinition(); + } + + @Override + public ProcessorFacade getProcessor(final String id) { + authContext.authorizeRead(); + final ProcessorFacade processor = delegate.getProcessor(id); + return processor == null ? null : new AuthorizingProcessorFacade(processor, authContext); + } + + @Override + public Set getProcessors() { + authContext.authorizeRead(); + return delegate.getProcessors().stream() + .map(p -> new AuthorizingProcessorFacade(p, authContext)) + .collect(Collectors.toSet()); + } + + @Override + public ControllerServiceFacade getControllerService(final String id) { + authContext.authorizeRead(); + final ControllerServiceFacade service = delegate.getControllerService(id); + return service == null ? null : new AuthorizingControllerServiceFacade(service, authContext); + } + + @Override + public Set getControllerServices() { + authContext.authorizeRead(); + return delegate.getControllerServices().stream() + .map(s -> new AuthorizingControllerServiceFacade(s, authContext)) + .collect(Collectors.toSet()); + } + + @Override + public Set getControllerServices(final ControllerServiceReferenceScope referenceScope, final ComponentHierarchyScope hierarchyScope) { + authContext.authorizeRead(); + return delegate.getControllerServices(referenceScope, hierarchyScope).stream() + .map(s -> new AuthorizingControllerServiceFacade(s, authContext)) + .collect(Collectors.toSet()); + } + + @Override + public ConnectionFacade getConnection(final String id) { + authContext.authorizeRead(); + final ConnectionFacade connection = delegate.getConnection(id); + return connection == null ? null : new AuthorizingConnectionFacade(connection, authContext); + } + + @Override + public Set getConnections() { + authContext.authorizeRead(); + return delegate.getConnections().stream() + .map(c -> new AuthorizingConnectionFacade(c, authContext)) + .collect(Collectors.toSet()); + } + + @Override + public ProcessGroupFacade getProcessGroup(final String id) { + authContext.authorizeRead(); + final ProcessGroupFacade group = delegate.getProcessGroup(id); + return group == null ? null : new AuthorizingProcessGroupFacade(group, authContext); + } + + @Override + public Set getProcessGroups() { + authContext.authorizeRead(); + return delegate.getProcessGroups().stream() + .map(g -> new AuthorizingProcessGroupFacade(g, authContext)) + .collect(Collectors.toSet()); + } + + @Override + public QueueSize getQueueSize() { + authContext.authorizeRead(); + return delegate.getQueueSize(); + } + + @Override + public boolean isFlowEmpty() { + authContext.authorizeRead(); + return delegate.isFlowEmpty(); + } + + @Override + public StatelessGroupLifecycle getStatelessLifecycle() { + authContext.authorizeRead(); + return new AuthorizingStatelessGroupLifecycle(delegate.getStatelessLifecycle(), authContext); + } + + @Override + public ProcessGroupLifecycle getLifecycle() { + authContext.authorizeRead(); + return new AuthorizingProcessGroupLifecycle(delegate.getLifecycle(), authContext); + } + + @Override + public DropFlowFileSummary dropFlowFiles(final Predicate predicate) throws IOException { + authContext.authorizeWrite(); + return delegate.dropFlowFiles(predicate); + } + + @Override + public String toString() { + return "AuthorizingProcessGroupFacade[delegate=" + delegate + "]"; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final AuthorizingProcessGroupFacade that = (AuthorizingProcessGroupFacade) o; + return Objects.equals(delegate, that.delegate); + } + + @Override + public int hashCode() { + return Objects.hashCode(delegate); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessGroupLifecycle.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessGroupLifecycle.java new file mode 100644 index 000000000000..1ac3267c0111 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessGroupLifecycle.java @@ -0,0 +1,129 @@ +/* + * 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.connector.authorization; + +import org.apache.nifi.components.connector.components.ComponentHierarchyScope; +import org.apache.nifi.components.connector.components.ControllerServiceReferenceScope; +import org.apache.nifi.components.connector.components.ProcessGroupLifecycle; + +import java.util.Collection; +import java.util.concurrent.CompletableFuture; + +/** + * A wrapper around {@link ProcessGroupLifecycle} that enforces authorization before delegating + * to the underlying implementation. All lifecycle operations require WRITE authorization. + */ +public class AuthorizingProcessGroupLifecycle implements ProcessGroupLifecycle { + + private final ProcessGroupLifecycle delegate; + private final ConnectorAuthorizationContext authContext; + + public AuthorizingProcessGroupLifecycle(final ProcessGroupLifecycle delegate, final ConnectorAuthorizationContext authContext) { + this.delegate = delegate; + this.authContext = authContext; + } + + @Override + public CompletableFuture enableControllerServices(final ControllerServiceReferenceScope referenceScope, final ComponentHierarchyScope hierarchyScope) { + authContext.authorizeWrite(); + return delegate.enableControllerServices(referenceScope, hierarchyScope); + } + + @Override + public CompletableFuture enableControllerServices(final Collection serviceIdentifiers) { + authContext.authorizeWrite(); + return delegate.enableControllerServices(serviceIdentifiers); + } + + @Override + public CompletableFuture disableControllerServices(final ComponentHierarchyScope scope) { + authContext.authorizeWrite(); + return delegate.disableControllerServices(scope); + } + + @Override + public CompletableFuture disableControllerServices(final Collection serviceIdentifiers) { + authContext.authorizeWrite(); + return delegate.disableControllerServices(serviceIdentifiers); + } + + @Override + public CompletableFuture startProcessors(final ComponentHierarchyScope scope) { + authContext.authorizeWrite(); + return delegate.startProcessors(scope); + } + + @Override + public CompletableFuture start(final ControllerServiceReferenceScope serviceReferenceScope) { + authContext.authorizeWrite(); + return delegate.start(serviceReferenceScope); + } + + @Override + public CompletableFuture stop() { + authContext.authorizeWrite(); + return delegate.stop(); + } + + @Override + public CompletableFuture stopProcessors(final ComponentHierarchyScope scope) { + authContext.authorizeWrite(); + return delegate.stopProcessors(scope); + } + + @Override + public CompletableFuture startPorts(final ComponentHierarchyScope scope) { + authContext.authorizeWrite(); + return delegate.startPorts(scope); + } + + @Override + public CompletableFuture stopPorts(final ComponentHierarchyScope scope) { + authContext.authorizeWrite(); + return delegate.stopPorts(scope); + } + + @Override + public CompletableFuture startRemoteProcessGroups(final ComponentHierarchyScope scope) { + authContext.authorizeWrite(); + return delegate.startRemoteProcessGroups(scope); + } + + @Override + public CompletableFuture stopRemoteProcessGroups(final ComponentHierarchyScope scope) { + authContext.authorizeWrite(); + return delegate.stopRemoteProcessGroups(scope); + } + + @Override + public CompletableFuture startStatelessGroups(final ComponentHierarchyScope scope) { + authContext.authorizeWrite(); + return delegate.startStatelessGroups(scope); + } + + @Override + public CompletableFuture stopStatelessGroups(final ComponentHierarchyScope scope) { + authContext.authorizeWrite(); + return delegate.stopStatelessGroups(scope); + } + + @Override + public int getActiveThreadCount() { + authContext.authorizeRead(); + return delegate.getActiveThreadCount(); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessorFacade.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessorFacade.java new file mode 100644 index 000000000000..d392d2c2b9f2 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessorFacade.java @@ -0,0 +1,122 @@ +/* + * 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.connector.authorization; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.connector.InvocationFailedException; +import org.apache.nifi.components.connector.components.ProcessorFacade; +import org.apache.nifi.components.connector.components.ProcessorLifecycle; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedParameterContext; +import org.apache.nifi.flow.VersionedProcessor; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * A wrapper around {@link ProcessorFacade} that enforces authorization before delegating + * to the underlying implementation. + */ +public class AuthorizingProcessorFacade implements ProcessorFacade { + + private final ProcessorFacade delegate; + private final ConnectorAuthorizationContext authContext; + + public AuthorizingProcessorFacade(final ProcessorFacade delegate, final ConnectorAuthorizationContext authContext) { + this.delegate = delegate; + this.authContext = authContext; + } + + @Override + public VersionedProcessor getDefinition() { + authContext.authorizeRead(); + return delegate.getDefinition(); + } + + @Override + public ProcessorLifecycle getLifecycle() { + authContext.authorizeRead(); + return new AuthorizingProcessorLifecycle(delegate.getLifecycle(), authContext); + } + + @Override + public List validate() { + authContext.authorizeRead(); + return delegate.validate(); + } + + @Override + public List validate(final Map propertyValues) { + authContext.authorizeRead(); + return delegate.validate(propertyValues); + } + + @Override + public List verify(final Map propertyValues, final Map attributes) { + authContext.authorizeRead(); + return delegate.verify(propertyValues, attributes); + } + + @Override + public List verify(final Map propertyValues, final VersionedParameterContext parameterContext, final Map attributes) { + authContext.authorizeRead(); + return delegate.verify(propertyValues, parameterContext, attributes); + } + + @Override + public List verify(final VersionedExternalFlow versionedExternalFlow, final Map attributes) { + authContext.authorizeRead(); + return delegate.verify(versionedExternalFlow, attributes); + } + + @Override + public Object invokeConnectorMethod(final String methodName, final Map arguments) throws InvocationFailedException { + authContext.authorizeWrite(); + return delegate.invokeConnectorMethod(methodName, arguments); + } + + @Override + public T invokeConnectorMethod(final String methodName, final Map arguments, final Class returnType) throws InvocationFailedException { + authContext.authorizeWrite(); + return delegate.invokeConnectorMethod(methodName, arguments, returnType); + } + + @Override + public String toString() { + return "AuthorizingProcessorFacade[delegate=" + delegate + "]"; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final AuthorizingProcessorFacade that = (AuthorizingProcessorFacade) o; + return Objects.equals(delegate, that.delegate); + } + + @Override + public int hashCode() { + return Objects.hashCode(delegate); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessorLifecycle.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessorLifecycle.java new file mode 100644 index 000000000000..562551def84f --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingProcessorLifecycle.java @@ -0,0 +1,80 @@ +/* + * 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.connector.authorization; + +import org.apache.nifi.components.connector.components.ProcessorLifecycle; +import org.apache.nifi.components.connector.components.ProcessorState; + +import java.util.concurrent.CompletableFuture; + +/** + * A wrapper around {@link ProcessorLifecycle} that enforces authorization before delegating + * to the underlying implementation. + */ +public class AuthorizingProcessorLifecycle implements ProcessorLifecycle { + + private final ProcessorLifecycle delegate; + private final ConnectorAuthorizationContext authContext; + + public AuthorizingProcessorLifecycle(final ProcessorLifecycle delegate, final ConnectorAuthorizationContext authContext) { + this.delegate = delegate; + this.authContext = authContext; + } + + @Override + public ProcessorState getState() { + authContext.authorizeRead(); + return delegate.getState(); + } + + @Override + public int getActiveThreadCount() { + authContext.authorizeRead(); + return delegate.getActiveThreadCount(); + } + + @Override + public void terminate() { + authContext.authorizeWrite(); + delegate.terminate(); + } + + @Override + public CompletableFuture stop() { + authContext.authorizeWrite(); + return delegate.stop(); + } + + @Override + public CompletableFuture start() { + authContext.authorizeWrite(); + return delegate.start(); + } + + @Override + public void disable() { + authContext.authorizeWrite(); + delegate.disable(); + } + + @Override + public void enable() { + authContext.authorizeWrite(); + delegate.enable(); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingStatelessGroupLifecycle.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingStatelessGroupLifecycle.java new file mode 100644 index 000000000000..bf822669a6cf --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/AuthorizingStatelessGroupLifecycle.java @@ -0,0 +1,55 @@ +/* + * 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.connector.authorization; + +import org.apache.nifi.components.connector.components.StatelessGroupLifecycle; + +import java.util.concurrent.CompletableFuture; + +/** + * A wrapper around {@link StatelessGroupLifecycle} that enforces authorization before delegating + * to the underlying implementation. All lifecycle operations require WRITE authorization. + */ +public class AuthorizingStatelessGroupLifecycle implements StatelessGroupLifecycle { + + private final StatelessGroupLifecycle delegate; + private final ConnectorAuthorizationContext authContext; + + public AuthorizingStatelessGroupLifecycle(final StatelessGroupLifecycle delegate, final ConnectorAuthorizationContext authContext) { + this.delegate = delegate; + this.authContext = authContext; + } + + @Override + public CompletableFuture start() { + authContext.authorizeWrite(); + return delegate.start(); + } + + @Override + public CompletableFuture stop() { + authContext.authorizeWrite(); + return delegate.stop(); + } + + @Override + public CompletableFuture terminate() { + authContext.authorizeWrite(); + return delegate.terminate(); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/ConnectorAuthorizationContext.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/ConnectorAuthorizationContext.java new file mode 100644 index 000000000000..00f993004736 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/connector/authorization/ConnectorAuthorizationContext.java @@ -0,0 +1,74 @@ +/* + * 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.connector.authorization; + +import org.apache.nifi.authorization.AuthorizableLookup; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.RequestAction; +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.authorization.user.NiFiUserUtils; + +/** + * Holds the authorization context needed to authorize operations on a Connector's FlowContext + * and its associated facades. + */ +public class ConnectorAuthorizationContext { + + private final String connectorId; + private final Authorizer authorizer; + private final AuthorizableLookup authorizableLookup; + + public ConnectorAuthorizationContext(final String connectorId, final Authorizer authorizer, final AuthorizableLookup authorizableLookup) { + this.connectorId = connectorId; + this.authorizer = authorizer; + this.authorizableLookup = authorizableLookup; + } + + /** + * Authorizes the current user for read access to the connector. + */ + public void authorizeRead() { + authorize(RequestAction.READ); + } + + /** + * Authorizes the current user for write access to the connector. + */ + public void authorizeWrite() { + authorize(RequestAction.WRITE); + } + + private void authorize(final RequestAction action) { + final Authorizable connector = authorizableLookup.getConnector(connectorId); + final NiFiUser user = NiFiUserUtils.getNiFiUser(); + connector.authorize(authorizer, action, user); + } + + public String getConnectorId() { + return connectorId; + } + + public Authorizer getAuthorizer() { + return authorizer; + } + + public AuthorizableLookup getAuthorizableLookup() { + return authorizableLookup; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java index 728eec6f7956..feb9a8099308 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java @@ -33,6 +33,7 @@ import org.apache.nifi.bundle.Bundle; import org.apache.nifi.bundle.BundleCoordinate; import org.apache.nifi.c2.protocol.component.api.ComponentManifest; +import org.apache.nifi.c2.protocol.component.api.ConnectorDefinition; import org.apache.nifi.c2.protocol.component.api.ControllerServiceDefinition; import org.apache.nifi.c2.protocol.component.api.FlowAnalysisRuleDefinition; import org.apache.nifi.c2.protocol.component.api.FlowRegistryClientDefinition; @@ -43,6 +44,9 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.components.RequiredPermission; +import org.apache.nifi.components.connector.Connector; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.Secret; import org.apache.nifi.components.listen.ListenComponent; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Connection; @@ -493,6 +497,18 @@ public Set getFlowFileProcessorTypes(final String bundleGroup return dtoFactory.fromDocumentedTypes(getExtensionManager().getExtensions(Processor.class), bundleGroupFilter, bundleArtifactFilter, typeFilter); } + /** + * Gets the Connector types that this controller supports. + * + * @param bundleGroupFilter if specified, must be member of bundle group + * @param bundleArtifactFilter if specified, must be member of bundle artifact + * @param typeFilter if specified, type must match + * @return types + */ + public Set getConnectorTypes(final String bundleGroupFilter, final String bundleArtifactFilter, final String typeFilter) { + return dtoFactory.fromDocumentedTypes(getExtensionManager().getExtensions(Connector.class), bundleGroupFilter, bundleArtifactFilter, typeFilter); + } + /** * Gets the FlowFileComparator types that this controller supports. * @@ -664,6 +680,29 @@ public FlowAnalysisRuleDefinition getFlowAnalysisRuleDefinition(String group, St return componentManifest.getFlowAnalysisRules().stream().filter(flowAnalysisRuleDefinition -> type.equals(flowAnalysisRuleDefinition.getType())).findFirst().orElse(null); } + public ConnectorDefinition getConnectorDefinition(final String group, final String artifact, final String version, final String type) { + final ComponentManifest componentManifest = getComponentManifest(group, artifact, version); + final List connectorDefinitions = componentManifest.getConnectors(); + if (connectorDefinitions == null) { + return null; + } + final ConnectorDefinition connectorDefinition = connectorDefinitions.stream() + .filter(definition -> group.equals(definition.getGroup()) + && artifact.equals(definition.getArtifact()) + && version.equals(definition.getVersion()) + && type.equals(definition.getType())) + .findFirst() + .orElse(null); + + if (connectorDefinition != null && connectorDefinition.getConfigurationSteps() != null) { + final Map stepDocumentation = runtimeManifestService.discoverStepDocumentation(group, artifact, version, type); + final Set documentedSteps = stepDocumentation.keySet(); + connectorDefinition.getConfigurationSteps().forEach(step -> step.setDocumented(documentedSteps.contains(step.getName()))); + } + + return connectorDefinition; + } + public String getAdditionalDetails(String group, String artifact, String version, String type) { final Map additionalDetailsMap = runtimeManifestService.discoverAdditionalDetails(group, artifact, version); final File additionalDetailsFile = additionalDetailsMap.get(type); @@ -680,6 +719,22 @@ public String getAdditionalDetails(String group, String artifact, String version } } + public String getStepDocumentation(final String group, final String artifact, final String version, final String connectorType, final String stepName) { + final Map stepDocsMap = runtimeManifestService.discoverStepDocumentation(group, artifact, version, connectorType); + final File stepDocFile = stepDocsMap.get(stepName); + + if (stepDocFile == null) { + throw new ResourceNotFoundException("Unable to find step documentation for step [%s] in connector [%s]".formatted(stepName, connectorType)); + } + + try (final Stream stepDocLines = Files.lines(stepDocFile.toPath())) { + return stepDocLines.collect(Collectors.joining("\n")); + } catch (final IOException e) { + throw new RuntimeException("Unable to load step documentation content for " + + stepDocFile.getAbsolutePath() + " due to: " + e.getMessage(), e); + } + } + /** * Gets the ParameterProvider types that this controller supports. * @@ -958,6 +1013,10 @@ public void save() throws NiFiCoreException { flowService.saveFlowChanges(TimeUnit.SECONDS, writeDelaySeconds); } + public void saveImmediate() throws IOException { + flowService.saveFlowChanges(); + } + /** * Returns the socket port that the local instance is listening on for * Site-to-Site communications @@ -1013,6 +1072,9 @@ public List getResources() { resources.add(ResourceFactory.getResourceResource()); resources.add(ResourceFactory.getSiteToSiteResource()); resources.add(ResourceFactory.getParameterContextsResource()); + resources.add(ResourceFactory.getConnectorsResource()); + resources.add(ResourceFactory.getDataResource(ResourceFactory.getConnectorsResource())); + resources.add(ResourceFactory.getProvenanceDataResource(ResourceFactory.getConnectorsResource())); // add each parameter context flowController.getFlowManager().getParameterContextManager().getParameterContexts().forEach(parameterContext -> resources.add(parameterContext.getResource())); @@ -1128,6 +1190,16 @@ public List getResources() { resources.add(ResourceFactory.getOperationResource(flowRegistryResource)); } + // add each connector + for (final ConnectorNode connector : flowController.getFlowManager().getAllConnectors()) { + final Resource connectorResource = connector.getResource(); + resources.add(connectorResource); + resources.add(ResourceFactory.getDataResource(connectorResource)); + resources.add(ResourceFactory.getProvenanceDataResource(connectorResource)); + resources.add(ResourceFactory.getPolicyResource(connectorResource)); + resources.add(ResourceFactory.getOperationResource(connectorResource)); + } + return resources; } @@ -1791,7 +1863,9 @@ private void setComponentDetails(final ProvenanceEventDTO dto) { final Connectable connectable = findLocalConnectable(dto.getComponentId()); if (connectable != null) { - dto.setGroupId(connectable.getProcessGroup().getIdentifier()); + final ProcessGroup connectableGroup = connectable.getProcessGroup(); + dto.setGroupId(connectableGroup.getIdentifier()); + connectableGroup.getConnectorIdentifier().ifPresent(dto::setConnectorId); // if the user is approved for this component policy, provide additional details, otherwise override/redact as necessary if (Result.Approved.equals(connectable.checkAuthorization(authorizer, RequestAction.READ, user).getResult())) { @@ -1807,7 +1881,13 @@ private void setComponentDetails(final ProvenanceEventDTO dto) { final RemoteGroupPort remoteGroupPort = root.findRemoteGroupPort(dto.getComponentId()); if (remoteGroupPort != null) { - dto.setGroupId(remoteGroupPort.getProcessGroupIdentifier()); + final String remoteGroupPortGroupId = remoteGroupPort.getProcessGroupIdentifier(); + dto.setGroupId(remoteGroupPortGroupId); + + final ProcessGroup remotePortGroup = root.findProcessGroup(remoteGroupPortGroupId); + if (remotePortGroup != null) { + remotePortGroup.getConnectorIdentifier().ifPresent(dto::setConnectorId); + } // if the user is approved for this component policy, provide additional details, otherwise override/redact as necessary if (Result.Approved.equals(remoteGroupPort.checkAuthorization(authorizer, RequestAction.READ, user).getResult())) { @@ -1822,7 +1902,9 @@ private void setComponentDetails(final ProvenanceEventDTO dto) { final Connection connection = root.findConnection(dto.getComponentId()); if (connection != null) { - dto.setGroupId(connection.getProcessGroup().getIdentifier()); + final ProcessGroup connectionGroup = connection.getProcessGroup(); + dto.setGroupId(connectionGroup.getIdentifier()); + connectionGroup.getConnectorIdentifier().ifPresent(dto::setConnectorId); // if the user is approved for this component policy, provide additional details, otherwise override/redact as necessary if (Result.Approved.equals(connection.checkAuthorization(authorizer, RequestAction.READ, user).getResult())) { @@ -1917,6 +1999,24 @@ public Set getListenPorts(final NiFiUser user) { return listenPorts; } + /** + * Searches within a connector's encapsulated process group for the specified term. + * + * @param searchLiteral search string specified by the user + * @param connectorProcessGroup the connector's managed process group to search within + * @return result + */ + public SearchResultsDTO searchConnector(final String searchLiteral, final ProcessGroup connectorProcessGroup) { + final SearchResultsDTO results = new SearchResultsDTO(); + final SearchQuery searchQuery = searchQueryParser.parse(searchLiteral, NiFiUserUtils.getNiFiUser(), connectorProcessGroup, connectorProcessGroup); + + if (!StringUtils.isEmpty(searchQuery.getTerm())) { + controllerSearchService.search(searchQuery, results); + } + + return results; + } + public void verifyComponentTypes(VersionedProcessGroup versionedFlow) { flowController.verifyComponentTypesInSnippet(versionedFlow); } @@ -1961,6 +2061,15 @@ public VersionedReportingTaskImporter createReportingTaskImporter() { return new StandardVersionedReportingTaskImporter(flowController); } + /** + * Gets all secrets from the SecretsManager. + * + * @return list of all secrets available from all secret providers + */ + public List getAllSecrets() { + return flowController.getConnectorRepository().getSecretsManager().getAllSecrets(); + } + /* * setters */ diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ConnectionDAO.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ConnectionDAO.java index 98caa34453ff..8785d670a6c8 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ConnectionDAO.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ConnectionDAO.java @@ -43,6 +43,15 @@ public interface ConnectionDAO { */ Connection getConnection(String id); + /** + * Gets the specified Connection, optionally including Connector-managed ProcessGroups in the search. + * + * @param id The connection id + * @param includeConnectorManaged Whether to search Connector-managed ProcessGroups + * @return The connection + */ + Connection getConnection(String id, boolean includeConnectorManaged); + /** * Gets the specified flow file drop request. * diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ConnectorDAO.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ConnectorDAO.java new file mode 100644 index 000000000000..70801f1dcdc9 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ConnectorDAO.java @@ -0,0 +1,84 @@ +/* + * 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.dao; + +import org.apache.nifi.asset.Asset; +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorUpdateContext; +import org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorDTO; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import java.util.Optional; + +public interface ConnectorDAO { + + void verifyCreate(ConnectorDTO connectorDTO); + + boolean hasConnector(String id); + + ConnectorNode getConnector(String id); + + List getConnectors(); + + ConnectorNode createConnector(String type, String id, BundleCoordinate bundleCoordinate, boolean firstTimeAdded, boolean registerLogObserver); + + void updateConnector(ConnectorDTO connectorDTO); + + void deleteConnector(String id); + + void startConnector(String id); + + void stopConnector(String id); + + void drainFlowFiles(String id); + + void cancelDrainFlowFiles(String id); + + void verifyCancelDrainFlowFile(String id); + + void verifyPurgeFlowFiles(String id); + + void purgeFlowFiles(String id, String requestor); + + void updateConnectorConfigurationStep(String id, String configurationStepName, ConfigurationStepConfigurationDTO configurationStepConfiguration); + + void applyConnectorUpdate(String id, ConnectorUpdateContext updateContext); + + void discardWorkingConfiguration(String id); + + void verifyCanVerifyConfigurationStep(String id, String configurationStepName); + + List verifyConfigurationStep(String id, String configurationStepName, ConfigurationStepConfigurationDTO configurationStepConfiguration); + + List fetchAllowableValues(String id, String stepName, String propertyName, String filter); + + void verifyCreateAsset(String id); + + Asset createAsset(String id, String assetId, String assetName, InputStream content) throws IOException; + + List getAssets(String id); + + Optional getAsset(String assetId); + +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java index df546810a6c9..1dd36a46b88b 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java @@ -58,6 +58,15 @@ public interface ProcessGroupDAO { */ ProcessGroup getProcessGroup(String groupId); + /** + * Gets the specified process group, optionally including Connector-managed ProcessGroups in the search. + * + * @param groupId The process group id + * @param includeConnectorManaged Whether to search Connector-managed ProcessGroups + * @return The process group + */ + ProcessGroup getProcessGroup(String groupId, boolean includeConnectorManaged); + /** * Gets all of the process groups. * diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/ComponentDAO.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/ComponentDAO.java index 078644a24a2a..26f0de302520 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/ComponentDAO.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/ComponentDAO.java @@ -63,14 +63,35 @@ protected boolean isAnyNotNull(T... objects) { * @param groupId id * @return group */ - protected ProcessGroup locateProcessGroup(FlowController flowController, String groupId) { - ProcessGroup group = flowController.getFlowManager().getGroup(groupId); + protected ProcessGroup locateProcessGroup(final FlowController flowController, final String groupId) { + return locateProcessGroup(flowController, groupId, false); + } - if (group == null) { - throw new ResourceNotFoundException(String.format("Unable to locate group with id '%s'.", groupId)); + /** + * Locates the specified ProcessGroup, optionally including Connector-managed ProcessGroups. + * + * @param flowController controller + * @param groupId id + * @param includeConnectorManaged whether to include Connector-managed ProcessGroups in the search + * @return group + */ + protected ProcessGroup locateProcessGroup(final FlowController flowController, final String groupId, final boolean includeConnectorManaged) { + // First, try to find the group in the main flow hierarchy (non-Connector groups) + ProcessGroup group = flowController.getFlowManager().getGroup(groupId, null); + + if (group != null) { + return group; + } + + // Optionally search Connector-managed ProcessGroups + if (includeConnectorManaged) { + group = flowController.getFlowManager().getGroup(groupId); + if (group != null) { + return group; + } } - return group; + throw new ResourceNotFoundException(String.format("Unable to locate group with id '%s'.", groupId)); } protected void verifyCreate(final ExtensionManager extensionManager, final String type, final BundleDTO bundle) { diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java index 696d8c11cf8e..47aa7a14dc8b 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectionDAO.java @@ -23,6 +23,8 @@ import org.apache.nifi.authorization.resource.DataAuthorizable; import org.apache.nifi.authorization.user.NiFiUser; import org.apache.nifi.authorization.user.NiFiUserUtils; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.FrameworkFlowContext; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; import org.apache.nifi.connectable.Connection; @@ -76,14 +78,33 @@ public class StandardConnectionDAO extends ComponentDAO implements ConnectionDAO private Authorizer authorizer; private Connection locateConnection(final String connectionId) { + return locateConnection(connectionId, false); + } + + private Connection locateConnection(final String connectionId, final boolean includeConnectorManaged) { + // First, search the main flow hierarchy final ProcessGroup rootGroup = flowController.getFlowManager().getRootGroup(); - final Connection connection = rootGroup.findConnection(connectionId); + Connection connection = rootGroup.findConnection(connectionId); - if (connection == null) { - throw new ResourceNotFoundException(String.format("Unable to find connection with id '%s'.", connectionId)); - } else { + if (connection != null) { return connection; } + + // Optionally search Connector-managed ProcessGroups + if (includeConnectorManaged) { + for (final ConnectorNode connector : flowController.getConnectorRepository().getConnectors()) { + final FrameworkFlowContext flowContext = connector.getActiveFlowContext(); + if (flowContext != null) { + final ProcessGroup managedGroup = flowContext.getManagedProcessGroup(); + connection = managedGroup.findConnection(connectionId); + if (connection != null) { + return connection; + } + } + } + } + + throw new ResourceNotFoundException(String.format("Unable to find connection with id '%s'.", connectionId)); } @Override @@ -97,6 +118,11 @@ public Connection getConnection(final String id) { return locateConnection(id); } + @Override + public Connection getConnection(final String id, final boolean includeConnectorManaged) { + return locateConnection(id, includeConnectorManaged); + } + @Override public Set getConnections(final String groupId) { final ProcessGroup group = locateProcessGroup(flowController, groupId); @@ -105,7 +131,7 @@ public Set getConnections(final String groupId) { @Override public DropFlowFileStatus getFlowFileDropRequest(String connectionId, String dropRequestId) { - final Connection connection = locateConnection(connectionId); + final Connection connection = locateConnection(connectionId, true); final FlowFileQueue queue = connection.getFlowFileQueue(); final DropFlowFileStatus dropRequest = queue.getDropFlowFileStatus(dropRequestId); @@ -118,7 +144,7 @@ public DropFlowFileStatus getFlowFileDropRequest(String connectionId, String dro @Override public ListFlowFileStatus getFlowFileListingRequest(String connectionId, String listingRequestId) { - final Connection connection = locateConnection(connectionId); + final Connection connection = locateConnection(connectionId, true); final FlowFileQueue queue = connection.getFlowFileQueue(); final ListFlowFileStatus listRequest = queue.getListFlowFileStatus(listingRequestId); @@ -132,7 +158,7 @@ public ListFlowFileStatus getFlowFileListingRequest(String connectionId, String @Override public FlowFileRecord getFlowFile(String id, String flowFileUuid) { try { - final Connection connection = locateConnection(id); + final Connection connection = locateConnection(id, true); final FlowFileQueue queue = connection.getFlowFileQueue(); final FlowFileRecord flowFile = queue.getFlowFile(flowFileUuid); @@ -342,7 +368,7 @@ public Connection createConnection(final String groupId, final ConnectionDTO con @Override public DropFlowFileStatus createFlowFileDropRequest(String id, String dropRequestId) { - final Connection connection = locateConnection(id); + final Connection connection = locateConnection(id, true); final FlowFileQueue queue = connection.getFlowFileQueue(); final NiFiUser user = NiFiUserUtils.getNiFiUser(); @@ -355,7 +381,7 @@ public DropFlowFileStatus createFlowFileDropRequest(String id, String dropReques @Override public ListFlowFileStatus createFlowFileListingRequest(String id, String listingRequestId) { - final Connection connection = locateConnection(id); + final Connection connection = locateConnection(id, true); final FlowFileQueue queue = connection.getFlowFileQueue(); // ensure we can list @@ -450,7 +476,7 @@ private void verifyList(final FlowFileQueue queue) { @Override public void verifyList(String id) { - final Connection connection = locateConnection(id); + final Connection connection = locateConnection(id, true); final FlowFileQueue queue = connection.getFlowFileQueue(); verifyList(queue); } @@ -623,7 +649,7 @@ public void deleteConnection(final String id) { @Override public DropFlowFileStatus deleteFlowFileDropRequest(String connectionId, String dropRequestId) { - final Connection connection = locateConnection(connectionId); + final Connection connection = locateConnection(connectionId, true); final FlowFileQueue queue = connection.getFlowFileQueue(); final DropFlowFileStatus dropFlowFileStatus = queue.cancelDropFlowFileRequest(dropRequestId); @@ -636,7 +662,7 @@ public DropFlowFileStatus deleteFlowFileDropRequest(String connectionId, String @Override public ListFlowFileStatus deleteFlowFileListingRequest(String connectionId, String listingRequestId) { - final Connection connection = locateConnection(connectionId); + final Connection connection = locateConnection(connectionId, true); final FlowFileQueue queue = connection.getFlowFileQueue(); final ListFlowFileStatus listFlowFileStatus = queue.cancelListFlowFileRequest(listingRequestId); @@ -652,7 +678,7 @@ public DownloadableContent getContent(String id, String flowFileUuid, String req try { final NiFiUser user = NiFiUserUtils.getNiFiUser(); - final Connection connection = locateConnection(id); + final Connection connection = locateConnection(id, true); final FlowFileQueue queue = connection.getFlowFileQueue(); final FlowFileRecord flowFile = queue.getFlowFile(flowFileUuid); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectorDAO.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectorDAO.java new file mode 100644 index 000000000000..697e8c5de6c7 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardConnectorDAO.java @@ -0,0 +1,284 @@ +/* + * 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.dao.impl; + +import org.apache.nifi.asset.Asset; +import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.connector.AssetReference; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorRepository; +import org.apache.nifi.components.connector.ConnectorUpdateContext; +import org.apache.nifi.components.connector.ConnectorValueReference; +import org.apache.nifi.components.connector.ConnectorValueType; +import org.apache.nifi.components.connector.SecretReference; +import org.apache.nifi.components.connector.StepConfiguration; +import org.apache.nifi.components.connector.StringLiteralValue; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.flow.FlowManager; +import org.apache.nifi.web.NiFiCoreException; +import org.apache.nifi.web.ResourceNotFoundException; +import org.apache.nifi.web.api.dto.AssetReferenceDTO; +import org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorDTO; +import org.apache.nifi.web.api.dto.ConnectorValueReferenceDTO; +import org.apache.nifi.web.api.dto.PropertyGroupConfigurationDTO; +import org.apache.nifi.web.dao.ConnectorDAO; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.stereotype.Repository; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +@Repository +public class StandardConnectorDAO implements ConnectorDAO { + + private static final Logger logger = LoggerFactory.getLogger(StandardConnectorDAO.class); + + private FlowController flowController; + + @Autowired + public void setFlowController(final FlowController flowController) { + this.flowController = flowController; + } + + private FlowManager getFlowManager() { + return flowController.getFlowManager(); + } + + private ConnectorRepository getConnectorRepository() { + return flowController.getConnectorRepository(); + } + + @Override + public void verifyCreate(final ConnectorDTO connectorDTO) { + final String id = connectorDTO.getId(); + if (id != null) { + getConnectorRepository().verifyCreate(id); + } + } + + @Override + public boolean hasConnector(final String id) { + return getConnectorRepository().getConnector(id) != null; + } + + @Override + public ConnectorNode getConnector(final String id) { + final ConnectorNode connector = getConnectorRepository().getConnector(id); + if (connector == null) { + throw new ResourceNotFoundException("Could not find Connector with ID " + id); + } + return connector; + } + + @Override + public List getConnectors() { + return getConnectorRepository().getConnectors(); + } + + @Override + public ConnectorNode createConnector(final String type, final String id, final BundleCoordinate bundleCoordinate, final boolean firstTimeAdded, final boolean registerLogObserver) { + final FlowManager flowManager = getFlowManager(); + final ConnectorNode connector = flowManager.createConnector(type, id, bundleCoordinate, firstTimeAdded, registerLogObserver); + return connector; + } + + @Override + public void updateConnector(final ConnectorDTO connectorDTO) { + final ConnectorNode connector = getConnector(connectorDTO.getId()); + if (connectorDTO.getName() != null) { + getConnectorRepository().updateConnector(connector, connectorDTO.getName()); + } + } + + @Override + public void deleteConnector(final String id) { + getConnectorRepository().deleteAssets(id); + getConnectorRepository().removeConnector(id); + } + + @Override + public void startConnector(final String id) { + final ConnectorNode connector = getConnector(id); + getConnectorRepository().startConnector(connector); + } + + @Override + public void stopConnector(final String id) { + final ConnectorNode connector = getConnector(id); + getConnectorRepository().stopConnector(connector); + } + + @Override + public void drainFlowFiles(final String id) { + final ConnectorNode connector = getConnector(id); + connector.drainFlowFiles(); + } + + @Override + public void cancelDrainFlowFiles(final String id) { + final ConnectorNode connector = getConnector(id); + connector.cancelDrainFlowFiles(); + } + + @Override + public void verifyCancelDrainFlowFile(final String id) { + final ConnectorNode connector = getConnector(id); + connector.verifyCancelDrainFlowFiles(); + } + + @Override + public void verifyPurgeFlowFiles(final String id) { + final ConnectorNode connector = getConnector(id); + connector.verifyCanPurgeFlowFiles(); + } + + @Override + public void purgeFlowFiles(final String id, final String requestor) { + final ConnectorNode connector = getConnector(id); + try { + connector.purgeFlowFiles(requestor).get(); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException("Thread was interrupted while purging FlowFiles for Connector " + id, e); + } catch (final ExecutionException e) { + throw new IllegalStateException("Failed to purge FlowFiles for Connector " + id, e.getCause()); + } + } + + @Override + public void updateConnectorConfigurationStep(final String id, final String configurationStepName, final ConfigurationStepConfigurationDTO configurationStepDto) { + final ConnectorNode connector = getConnector(id); + + // Convert DTO to domain object - flatten all property groups into a single StepConfiguration + final StepConfiguration stepConfiguration = convertToStepConfiguration(configurationStepDto); + + // Update the connector configuration through the repository + try { + getConnectorRepository().configureConnector(connector, configurationStepName, stepConfiguration); + } catch (final Exception e) { + throw new IllegalStateException("Failed to update connector configuration: " + e, e); + } + } + + private StepConfiguration convertToStepConfiguration(final ConfigurationStepConfigurationDTO dto) { + final Map propertyValues = new HashMap<>(); + if (dto.getPropertyGroupConfigurations() != null) { + for (final PropertyGroupConfigurationDTO groupDto : dto.getPropertyGroupConfigurations()) { + if (groupDto.getPropertyValues() != null) { + for (final Map.Entry entry : groupDto.getPropertyValues().entrySet()) { + propertyValues.put(entry.getKey(), convertToConnectorValueReference(entry.getValue())); + } + } + } + } + return new StepConfiguration(propertyValues); + } + + private ConnectorValueReference convertToConnectorValueReference(final ConnectorValueReferenceDTO dto) { + if (dto == null) { + return null; + } + final ConnectorValueType valueType = dto.getValueType() != null ? ConnectorValueType.valueOf(dto.getValueType()) : ConnectorValueType.STRING_LITERAL; + return switch (valueType) { + case STRING_LITERAL -> new StringLiteralValue(dto.getValue()); + case ASSET_REFERENCE -> new AssetReference(convertToAssetIdentifiers(dto.getAssetReferences())); + case SECRET_REFERENCE -> new SecretReference(dto.getSecretProviderId(), dto.getSecretProviderName(), dto.getSecretName(), dto.getFullyQualifiedSecretName()); + }; + } + + private Set convertToAssetIdentifiers(final List assetReferenceDTOs) { + if (assetReferenceDTOs == null || assetReferenceDTOs.isEmpty()) { + return Collections.emptySet(); + } + return assetReferenceDTOs.stream().map(AssetReferenceDTO::getId).collect(Collectors.toSet()); + } + + @Override + public void applyConnectorUpdate(final String id, final ConnectorUpdateContext updateContext) { + final ConnectorNode connector = getConnector(id); + try { + getConnectorRepository().applyUpdate(connector, updateContext); + } catch (final Exception e) { + throw new NiFiCoreException("Failed to apply connector update: " + e, e); + } + } + + @Override + public void discardWorkingConfiguration(final String id) { + final ConnectorNode connector = getConnector(id); + getConnectorRepository().discardWorkingConfiguration(connector); + } + + @Override + public void verifyCanVerifyConfigurationStep(final String id, final String configurationStepName) { + // Verify that the connector exists + getConnector(id); + } + + @Override + public List verifyConfigurationStep(final String id, final String configurationStepName, final ConfigurationStepConfigurationDTO configurationStepDto) { + final ConnectorNode connector = getConnector(id); + getConnectorRepository().syncAssetsFromProvider(connector); + final StepConfiguration stepConfiguration = convertToStepConfiguration(configurationStepDto); + return connector.verifyConfigurationStep(configurationStepName, stepConfiguration); + } + + @Override + public List fetchAllowableValues(final String id, final String stepName, final String propertyName, final String filter) { + final ConnectorNode connector = getConnector(id); + if (filter == null || filter.isEmpty()) { + return connector.fetchAllowableValues(stepName, propertyName); + } else { + return connector.fetchAllowableValues(stepName, propertyName, filter); + } + } + + @Override + public void verifyCreateAsset(final String id) { + getConnector(id); + } + + @Override + public Asset createAsset(final String id, final String assetId, final String assetName, final InputStream content) throws IOException { + return getConnectorRepository().storeAsset(id, assetId, assetName, content); + } + + @Override + public List getAssets(final String id) { + return getConnectorRepository().getAssets(id); + } + + @Override + public Optional getAsset(final String assetId) { + return getConnectorRepository().getAsset(assetId); + } +} + + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java index d7d4d51a66ea..6c13a29d0b3e 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java @@ -473,7 +473,7 @@ public List verifyConfiguration(final String contro final ConfigurationContext configurationContext = new StandardConfigurationContext(serviceNode, properties, serviceNode.getAnnotationData(), parameterLookup, flowController.getControllerServiceProvider(), null); - final List verificationResults = serviceNode.verifyConfiguration(configurationContext, configVerificationLog, variables, extensionManager); + final List verificationResults = serviceNode.verifyConfiguration(configurationContext, configVerificationLog, variables, extensionManager, parameterLookup); final List resultsDtos = verificationResults.stream() .map(this::createConfigVerificationResultDto) .collect(Collectors.toList()); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardParameterContextDAO.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardParameterContextDAO.java index 840c8014795e..090368245eb4 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardParameterContextDAO.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardParameterContextDAO.java @@ -350,7 +350,7 @@ public void verifyAssets(final ParameterContextDTO parameterContextDto, final Ma if (parameter != null) { final List assets = parameter.getReferencedAssets() == null ? Collections.emptyList() : parameter.getReferencedAssets(); for (final Asset asset : assets) { - if (!asset.getParameterContextIdentifier().equals(parameterContextDto.getId())) { + if (!asset.getOwnerIdentifier().equals(parameterContextDto.getId())) { throw new IllegalArgumentException(String.format("Parameter [%s] is not allowed to reference asset [%s] which does not belong to parameter context [%s]", parameter.getDescriptor().getName(), asset.getName(), parameterContextDto.getId())); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java index abfcb07b316c..b6612c3126d4 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java @@ -44,7 +44,7 @@ import org.apache.nifi.registry.flow.StandardVersionControlInformation; import org.apache.nifi.registry.flow.VersionControlInformation; import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedProcessGroup; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import org.apache.nifi.remote.RemoteGroupPort; import org.apache.nifi.web.ResourceNotFoundException; import org.apache.nifi.web.api.dto.ProcessGroupDTO; @@ -120,7 +120,7 @@ public ProcessGroup createProcessGroup(String parentGroupId, ProcessGroupDTO pro @Override public boolean hasProcessGroup(String groupId) { - return flowController.getFlowManager().getGroup(groupId) != null; + return flowController.getFlowManager().getGroup(groupId, null) != null; } @Override @@ -187,6 +187,11 @@ public ProcessGroup getProcessGroup(String groupId) { return locateProcessGroup(flowController, groupId); } + @Override + public ProcessGroup getProcessGroup(String groupId, boolean includeConnectorManaged) { + return locateProcessGroup(flowController, groupId, includeConnectorManaged); + } + @Override public Set getProcessGroups(final String parentGroupId, final ProcessGroupRecursivity processGroupRecursivity) { ProcessGroup group = locateProcessGroup(flowController, parentGroupId); @@ -411,7 +416,7 @@ public void activateControllerServices(final String groupId, final ControllerSer .map(flowManager::getControllerServiceNode) .collect(Collectors.toList()); - final ProcessGroup group = flowManager.getGroup(groupId); + final ProcessGroup group = flowManager.getGroup(groupId, null); if (group == null) { throw new IllegalArgumentException("Cannot activate Controller Services with IDs " + serviceIds + " because the associated Process Group (id=" + groupId + ") could not be found"); } @@ -524,7 +529,7 @@ public ProcessGroup updateVersionControlInformation(final VersionControlInformat final String groupId = versionControlInformation.getGroupId(); final ProcessGroup group = locateProcessGroup(flowController, groupId); - final NiFiRegistryFlowMapper mapper = new NiFiRegistryFlowMapper(flowController.getExtensionManager()); + final VersionedComponentFlowMapper mapper = new VersionedComponentFlowMapper(flowController.getExtensionManager()); final InstantiatedVersionedProcessGroup flowSnapshot = mapper.mapProcessGroup(group, flowController.getControllerServiceProvider(), flowController.getFlowManager(), false); updateVersionControlInformation(group, flowSnapshot, versionControlInformation, versionedComponentMapping); @@ -616,7 +621,7 @@ public void verifyDeleteFlowRegistry(String registryId) { @Override public DropFlowFileStatus createDropAllFlowFilesRequest(String processGroupId, String dropRequestId) { - ProcessGroup processGroup = locateProcessGroup(flowController, processGroupId); + ProcessGroup processGroup = locateProcessGroup(flowController, processGroupId, true); final NiFiUser user = NiFiUserUtils.getNiFiUser(); if (user == null) { @@ -628,14 +633,14 @@ public DropFlowFileStatus createDropAllFlowFilesRequest(String processGroupId, S @Override public DropFlowFileStatus getDropAllFlowFilesRequest(String processGroupId, String dropRequestId) { - ProcessGroup processGroup = locateProcessGroup(flowController, processGroupId); + ProcessGroup processGroup = locateProcessGroup(flowController, processGroupId, true); return processGroup.getDropAllFlowFilesStatus(dropRequestId); } @Override public DropFlowFileStatus deleteDropAllFlowFilesRequest(String processGroupId, String dropRequestId) { - ProcessGroup processGroup = locateProcessGroup(flowController, processGroupId); + ProcessGroup processGroup = locateProcessGroup(flowController, processGroupId, true); return processGroup.cancelDropAllFlowFiles(dropRequestId); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java index 97856e9ad8b2..ebdccd1c15c6 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessorDAO.java @@ -490,7 +490,8 @@ public List verifyProcessorConfiguration(final Stri final LogRepository logRepository = new NopLogRepository(); final ComponentLog configVerificationLog = new SimpleProcessLogger(processor, logRepository, new StandardLoggingContext(processor)); final ExtensionManager extensionManager = flowController.getExtensionManager(); - final List verificationResults = processor.verifyConfiguration(processContext, configVerificationLog, attributes, extensionManager); + final List verificationResults = processor.verifyConfiguration(processContext, configVerificationLog, attributes, extensionManager, + processor.getProcessGroup().getParameterContext()); final List resultsDtos = verificationResults.stream() .map(this::createConfigVerificationResultDto) diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/filter/ConnectorRequestContextFilter.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/filter/ConnectorRequestContextFilter.java new file mode 100644 index 000000000000..75d76d0a8c29 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/filter/ConnectorRequestContextFilter.java @@ -0,0 +1,89 @@ +/* + * 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.filter; + +import jakarta.servlet.Filter; +import jakarta.servlet.FilterChain; +import jakarta.servlet.FilterConfig; +import jakarta.servlet.ServletException; +import jakarta.servlet.ServletRequest; +import jakarta.servlet.ServletResponse; +import jakarta.servlet.http.HttpServletRequest; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.authorization.user.NiFiUserUtils; +import org.apache.nifi.components.connector.ConnectorRequestContext; +import org.apache.nifi.components.connector.ConnectorRequestContextHolder; +import org.apache.nifi.components.connector.StandardConnectorRequestContext; + +import java.io.IOException; +import java.util.Collections; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Servlet filter that populates the {@link ConnectorRequestContextHolder} thread-local + * with the current HTTP request headers and authenticated {@link NiFiUser}. + * + *

This filter must be registered after the Spring Security filter chain so that the + * authenticated {@link NiFiUser} is available via the security context. The context is + * always cleared in the {@code finally} block to prevent thread-local memory leaks.

+ */ +public class ConnectorRequestContextFilter implements Filter { + + @Override + public void doFilter(final ServletRequest request, final ServletResponse response, final FilterChain chain) + throws IOException, ServletException { + try { + if (request instanceof HttpServletRequest httpServletRequest) { + final ConnectorRequestContext context = createContext(httpServletRequest); + ConnectorRequestContextHolder.setContext(context); + } + chain.doFilter(request, response); + } finally { + ConnectorRequestContextHolder.clearContext(); + } + } + + private ConnectorRequestContext createContext(final HttpServletRequest request) { + final NiFiUser nifiUser = NiFiUserUtils.getNiFiUser(); + final Map> headers = captureHeaders(request); + return new StandardConnectorRequestContext(nifiUser, headers); + } + + private Map> captureHeaders(final HttpServletRequest request) { + final Map> headers = new HashMap<>(); + final Enumeration headerNames = request.getHeaderNames(); + if (headerNames != null) { + while (headerNames.hasMoreElements()) { + final String name = headerNames.nextElement(); + final List values = Collections.list(request.getHeaders(name)); + headers.put(name, Collections.unmodifiableList(values)); + } + } + return headers; + } + + @Override + public void init(final FilterConfig filterConfig) { + } + + @Override + public void destroy() { + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/webapp/WEB-INF/web.xml b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/webapp/WEB-INF/web.xml index af90238f98f5..1025a301d541 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/webapp/WEB-INF/web.xml +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/webapp/WEB-INF/web.xml @@ -61,4 +61,12 @@ springSecurityFilterChain /* + + connectorRequestContextFilter + org.apache.nifi.web.filter.ConnectorRequestContextFilter + + + connectorRequestContextFilter + /* + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/audit/TestLabelAuditor.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/audit/TestLabelAuditor.java index af03389ec3c2..48605cd64fe9 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/audit/TestLabelAuditor.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/audit/TestLabelAuditor.java @@ -119,7 +119,7 @@ void setAuditor() { @Test void testCreateLabelAdvice() { final LabelDTO labelDto = getLabelDto(); - when(flowManager.getGroup(eq(GROUP_ID))).thenReturn(processGroup); + when(flowManager.getGroup(eq(GROUP_ID), eq(null))).thenReturn(processGroup); when(flowManager.createLabel(eq(LABEL_ID), eq(LABEL))).thenReturn(new StandardLabel(LABEL_ID, LABEL)); final Label label = labelDao.createLabel(GROUP_ID, labelDto); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/audit/TestProcessGroupAuditor.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/audit/TestProcessGroupAuditor.java index 0a12bae9bbbb..48c3a57b0e1f 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/audit/TestProcessGroupAuditor.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/audit/TestProcessGroupAuditor.java @@ -176,7 +176,7 @@ void testVerifyStartProcessGroupAuditing() { when(processGroup.findRemoteGroupPort(REMOTE_GROUP_INPUT_PORT)).thenReturn(remoteGroupInputPort); when(remoteGroupInputPort.getRemoteProcessGroup()).thenReturn(remoteProcessGroup); - when(flowManager.getGroup(eq(PG_1))).thenReturn(processGroup); + when(flowManager.getGroup(eq(PG_1), eq(null))).thenReturn(processGroup); when(flowManager.findConnectable(eq(PROC_1))).thenReturn(processor1); when(flowManager.findConnectable(eq(PROC_2))).thenReturn(processor2); when(flowManager.findConnectable(eq(INPUT_PORT))).thenReturn(inputPort); @@ -286,7 +286,7 @@ void testVerifyEnableProcessGroupAuditing() { when(processGroup.findInputPort(OUTPUT_PORT)).thenReturn(null); when(processGroup.findOutputPort(OUTPUT_PORT)).thenReturn(outputPort); - when(flowManager.getGroup(eq(PG_1))).thenReturn(processGroup); + when(flowManager.getGroup(eq(PG_1), eq(null))).thenReturn(processGroup); when(flowManager.findConnectable(eq(PROC_1))).thenReturn(processor1); when(flowManager.findConnectable(eq(PROC_2))).thenReturn(processor2); when(flowManager.findConnectable(eq(INPUT_PORT))).thenReturn(inputPort); @@ -354,7 +354,7 @@ void testVerifyEnableControllerServicesAuditing() { when(cs.getName()).thenReturn(CS_1); when(processGroup.findControllerService(eq(CS_1), eq(true), eq(true))).thenReturn(cs); - when(flowManager.getGroup(eq(PG_1))).thenReturn(processGroup); + when(flowManager.getGroup(eq(PG_1), eq(null))).thenReturn(processGroup); when(flowManager.getControllerServiceNode(eq(CS_1))).thenReturn(cs); when(flowController.getFlowManager()).thenReturn(flowManager); when(flowController.getControllerServiceProvider()).thenReturn(csProvider); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/audit/TestProcessorAuditor.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/audit/TestProcessorAuditor.java index 307d0daedf05..0603361eca1d 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/audit/TestProcessorAuditor.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/audit/TestProcessorAuditor.java @@ -147,7 +147,7 @@ void testCreateProcessorAdvice() { when(flowController.getControllerServiceProvider()).thenReturn(mock(ControllerServiceProvider.class)); when(flowController.getStateManagerProvider()).thenReturn(mockStateManagerProvider); - when(flowManager.getGroup(GROUP_ID)).thenReturn(processGroup); + when(flowManager.getGroup(GROUP_ID, null)).thenReturn(processGroup); when(flowManager.createProcessor(anyString(), anyString(), any())).thenReturn(mockProcessorNode); final Bundle bundle = getBundle(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/authorization/StandardAuthorizableLookupTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/authorization/StandardAuthorizableLookupTest.java index 116f55c72703..9a7bfc6cdced 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/authorization/StandardAuthorizableLookupTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/authorization/StandardAuthorizableLookupTest.java @@ -22,21 +22,28 @@ import org.apache.nifi.authorization.resource.DataTransferAuthorizable; import org.apache.nifi.authorization.resource.OperationAuthorizable; import org.apache.nifi.authorization.resource.ProvenanceDataAuthorizable; +import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.connectable.Connection; import org.apache.nifi.controller.FlowAnalysisRuleNode; import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.groups.ProcessGroup; import org.apache.nifi.nar.ExtensionDiscoveringManager; import org.apache.nifi.nar.ExtensionManager; import org.apache.nifi.registry.flow.FlowRegistryClientNode; import org.apache.nifi.web.controller.ControllerFacade; +import org.apache.nifi.web.dao.ConnectionDAO; import org.apache.nifi.web.dao.FlowAnalysisRuleDAO; import org.apache.nifi.web.dao.FlowRegistryDAO; +import org.apache.nifi.web.dao.ProcessGroupDAO; import org.apache.nifi.web.dao.ProcessorDAO; import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class StandardAuthorizableLookupTest { @@ -103,6 +110,108 @@ void testGetAuthorizableFromResourceFlowAnalysisRule() { assertEquals(flowAnalysisRuleNode, authorizable); } + @Test + void testGetConnectionWithoutIncludeConnectorManaged() { + final StandardAuthorizableLookup lookup = getLookup(); + final ConnectionDAO connectionDAO = mock(ConnectionDAO.class); + final Connection connection = mock(Connection.class); + final Connectable sourceConnectable = mock(Connectable.class); + + when(connectionDAO.getConnection(eq(COMPONENT_ID), eq(false))).thenReturn(connection); + when(connection.getSource()).thenReturn(sourceConnectable); + when(connection.getDestination()).thenReturn(sourceConnectable); + when(connection.getSourceAuthorizable()).thenReturn(sourceConnectable); + lookup.setConnectionDAO(connectionDAO); + + final ConnectionAuthorizable result = lookup.getConnection(COMPONENT_ID); + + assertNotNull(result); + verify(connectionDAO).getConnection(eq(COMPONENT_ID), eq(false)); + } + + @Test + void testGetConnectionWithIncludeConnectorManagedTrue() { + final StandardAuthorizableLookup lookup = getLookup(); + final ConnectionDAO connectionDAO = mock(ConnectionDAO.class); + final Connection connection = mock(Connection.class); + final Connectable sourceConnectable = mock(Connectable.class); + + when(connectionDAO.getConnection(eq(COMPONENT_ID), eq(true))).thenReturn(connection); + when(connection.getSource()).thenReturn(sourceConnectable); + when(connection.getDestination()).thenReturn(sourceConnectable); + when(connection.getSourceAuthorizable()).thenReturn(sourceConnectable); + lookup.setConnectionDAO(connectionDAO); + + final ConnectionAuthorizable result = lookup.getConnection(COMPONENT_ID, true); + + assertNotNull(result); + verify(connectionDAO).getConnection(eq(COMPONENT_ID), eq(true)); + } + + @Test + void testGetConnectionWithIncludeConnectorManagedFalse() { + final StandardAuthorizableLookup lookup = getLookup(); + final ConnectionDAO connectionDAO = mock(ConnectionDAO.class); + final Connection connection = mock(Connection.class); + final Connectable sourceConnectable = mock(Connectable.class); + + when(connectionDAO.getConnection(eq(COMPONENT_ID), eq(false))).thenReturn(connection); + when(connection.getSource()).thenReturn(sourceConnectable); + when(connection.getDestination()).thenReturn(sourceConnectable); + when(connection.getSourceAuthorizable()).thenReturn(sourceConnectable); + lookup.setConnectionDAO(connectionDAO); + + final ConnectionAuthorizable result = lookup.getConnection(COMPONENT_ID, false); + + assertNotNull(result); + verify(connectionDAO).getConnection(eq(COMPONENT_ID), eq(false)); + } + + @Test + void testGetProcessGroupWithoutIncludeConnectorManaged() { + final StandardAuthorizableLookup lookup = getLookup(); + final ProcessGroupDAO processGroupDAO = mock(ProcessGroupDAO.class); + final ProcessGroup processGroup = mock(ProcessGroup.class); + + when(processGroupDAO.getProcessGroup(eq(COMPONENT_ID), eq(false))).thenReturn(processGroup); + lookup.setProcessGroupDAO(processGroupDAO); + + final ProcessGroupAuthorizable result = lookup.getProcessGroup(COMPONENT_ID); + + assertNotNull(result); + verify(processGroupDAO).getProcessGroup(eq(COMPONENT_ID), eq(false)); + } + + @Test + void testGetProcessGroupWithIncludeConnectorManagedTrue() { + final StandardAuthorizableLookup lookup = getLookup(); + final ProcessGroupDAO processGroupDAO = mock(ProcessGroupDAO.class); + final ProcessGroup processGroup = mock(ProcessGroup.class); + + when(processGroupDAO.getProcessGroup(eq(COMPONENT_ID), eq(true))).thenReturn(processGroup); + lookup.setProcessGroupDAO(processGroupDAO); + + final ProcessGroupAuthorizable result = lookup.getProcessGroup(COMPONENT_ID, true); + + assertNotNull(result); + verify(processGroupDAO).getProcessGroup(eq(COMPONENT_ID), eq(true)); + } + + @Test + void testGetProcessGroupWithIncludeConnectorManagedFalse() { + final StandardAuthorizableLookup lookup = getLookup(); + final ProcessGroupDAO processGroupDAO = mock(ProcessGroupDAO.class); + final ProcessGroup processGroup = mock(ProcessGroup.class); + + when(processGroupDAO.getProcessGroup(eq(COMPONENT_ID), eq(false))).thenReturn(processGroup); + lookup.setProcessGroupDAO(processGroupDAO); + + final ProcessGroupAuthorizable result = lookup.getProcessGroup(COMPONENT_ID, false); + + assertNotNull(result); + verify(processGroupDAO).getProcessGroup(eq(COMPONENT_ID), eq(false)); + } + private StandardAuthorizableLookup getLookup() { final ExtensionManager extensionManager = mock(ExtensionDiscoveringManager.class); final ControllerFacade controllerFacade = mock(ControllerFacade.class); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/StandardNiFiServiceFacadeTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/StandardNiFiServiceFacadeTest.java index 5416a41a9f31..7186baf3425b 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/StandardNiFiServiceFacadeTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/StandardNiFiServiceFacadeTest.java @@ -36,6 +36,13 @@ import org.apache.nifi.authorization.user.NiFiUserDetails; import org.apache.nifi.authorization.user.StandardNiFiUser.Builder; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.FrameworkFlowContext; +import org.apache.nifi.components.connector.Secret; +import org.apache.nifi.components.connector.secrets.AuthorizableSecret; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.controller.ControllerService; import org.apache.nifi.controller.Counter; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.ProcessorNode; @@ -63,6 +70,7 @@ import org.apache.nifi.history.History; import org.apache.nifi.history.HistoryQuery; import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.processor.Processor; import org.apache.nifi.registry.flow.FlowRegistryUtil; import org.apache.nifi.registry.flow.RegisteredFlowSnapshot; import org.apache.nifi.registry.flow.VersionControlInformation; @@ -75,7 +83,7 @@ import org.apache.nifi.registry.flow.diff.StaticDifferenceDescriptor; import org.apache.nifi.registry.flow.mapping.FlowMappingOptions; import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedProcessGroup; -import org.apache.nifi.registry.flow.mapping.NiFiRegistryFlowMapper; +import org.apache.nifi.registry.flow.mapping.VersionedComponentFlowMapper; import org.apache.nifi.reporting.Bulletin; import org.apache.nifi.reporting.BulletinFactory; import org.apache.nifi.reporting.ComponentType; @@ -85,6 +93,7 @@ import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.validation.RuleViolation; import org.apache.nifi.validation.RuleViolationsManager; +import org.apache.nifi.web.api.dto.ComponentStateDTO; import org.apache.nifi.web.api.dto.CounterDTO; import org.apache.nifi.web.api.dto.CountersDTO; import org.apache.nifi.web.api.dto.CountersSnapshotDTO; @@ -94,6 +103,7 @@ import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; 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.search.SearchResultsDTO; import org.apache.nifi.web.api.dto.status.StatusHistoryDTO; import org.apache.nifi.web.api.entity.ActionEntity; import org.apache.nifi.web.api.entity.AffectedComponentEntity; @@ -102,10 +112,13 @@ import org.apache.nifi.web.api.entity.CopyRequestEntity; import org.apache.nifi.web.api.entity.CopyResponseEntity; import org.apache.nifi.web.api.entity.ProcessGroupEntity; +import org.apache.nifi.web.api.entity.SecretsEntity; import org.apache.nifi.web.api.entity.StatusHistoryEntity; import org.apache.nifi.web.api.entity.TenantEntity; import org.apache.nifi.web.api.entity.TenantsEntity; import org.apache.nifi.web.controller.ControllerFacade; +import org.apache.nifi.web.dao.ComponentStateDAO; +import org.apache.nifi.web.dao.ConnectorDAO; import org.apache.nifi.web.dao.ProcessGroupDAO; import org.apache.nifi.web.dao.RemoteProcessGroupDAO; import org.apache.nifi.web.dao.UserDAO; @@ -325,7 +338,7 @@ public void testGetComponentsAffectedByFlowUpdate_WithNewStatelessProcessGroup_R when(flowController.getExtensionManager()).thenReturn(extensionManager); final StandardNiFiServiceFacade serviceFacadeSpy = spy(serviceFacade); - final NiFiRegistryFlowMapper flowMapper = mock(NiFiRegistryFlowMapper.class); + final VersionedComponentFlowMapper flowMapper = mock(VersionedComponentFlowMapper.class); doReturn(flowMapper).when(serviceFacadeSpy).makeNiFiRegistryFlowMapper(extensionManager); final InstantiatedVersionedProcessGroup localRoot = new InstantiatedVersionedProcessGroup("local-root-instance", groupId); @@ -511,7 +524,7 @@ public void testCopyComponents() { // use spy to mock the make() method for generating a new flow mapper to make this testable final StandardNiFiServiceFacade serviceFacadeSpy = spy(serviceFacade); - final NiFiRegistryFlowMapper flowMapper = mock(NiFiRegistryFlowMapper.class); + final VersionedComponentFlowMapper flowMapper = mock(VersionedComponentFlowMapper.class); doReturn(flowMapper).when(serviceFacadeSpy).makeNiFiRegistryFlowMapper(eq(extensionManager), any(FlowMappingOptions.class)); final InstantiatedVersionedProcessGroup nonVersionedProcessGroup = mock(InstantiatedVersionedProcessGroup.class); @@ -641,7 +654,7 @@ public void testGetCurrentFlowSnapshotByGroupId() { // use spy to mock the make() method for generating a new flow mapper to make this testable final StandardNiFiServiceFacade serviceFacadeSpy = spy(serviceFacade); - final NiFiRegistryFlowMapper flowMapper = mock(NiFiRegistryFlowMapper.class); + final VersionedComponentFlowMapper flowMapper = mock(VersionedComponentFlowMapper.class); when(serviceFacadeSpy.makeNiFiRegistryFlowMapper(extensionManager)).thenReturn(flowMapper); final InstantiatedVersionedProcessGroup nonVersionedProcessGroup = mock(InstantiatedVersionedProcessGroup.class); @@ -701,7 +714,7 @@ public void testGetCurrentFlowSnapshotByGroupIdWithReferencedControllerServices( // use spy to mock the make() method for generating a new flow mapper to make this testable final StandardNiFiServiceFacade serviceFacadeSpy = spy(serviceFacade); - final NiFiRegistryFlowMapper flowMapper = mock(NiFiRegistryFlowMapper.class); + final VersionedComponentFlowMapper flowMapper = mock(VersionedComponentFlowMapper.class); when(serviceFacadeSpy.makeNiFiRegistryFlowMapper(extensionManager)).thenReturn(flowMapper); final InstantiatedVersionedProcessGroup nonVersionedProcessGroup = spy(new InstantiatedVersionedProcessGroup(UUID.randomUUID().toString(), UUID.randomUUID().toString())); @@ -1547,6 +1560,38 @@ public void testUpdateAllCounters() { verify(dtoFactory, times(1)).createCountersDto(any()); } + @Test + public void testSearchConnector() { + final String connectorId = "connector-id"; + final String searchQuery = "test-search"; + final String managedGroupId = "managed-group-id"; + + final ConnectorDAO connectorDAO = mock(ConnectorDAO.class); + serviceFacade.setConnectorDAO(connectorDAO); + + final ConnectorNode connectorNode = mock(ConnectorNode.class); + final FrameworkFlowContext flowContext = mock(FrameworkFlowContext.class); + final ProcessGroup managedProcessGroup = mock(ProcessGroup.class); + + when(connectorDAO.getConnector(connectorId)).thenReturn(connectorNode); + when(connectorNode.getActiveFlowContext()).thenReturn(flowContext); + when(flowContext.getManagedProcessGroup()).thenReturn(managedProcessGroup); + when(managedProcessGroup.getIdentifier()).thenReturn(managedGroupId); + + final ControllerFacade controllerFacade = mock(ControllerFacade.class); + final SearchResultsDTO expectedResults = new SearchResultsDTO(); + when(controllerFacade.searchConnector(searchQuery, managedProcessGroup)).thenReturn(expectedResults); + serviceFacade.setControllerFacade(controllerFacade); + + final SearchResultsDTO results = serviceFacade.searchConnector(connectorId, searchQuery); + + assertNotNull(results); + verify(connectorDAO).getConnector(connectorId); + verify(connectorNode).getActiveFlowContext(); + verify(flowContext).getManagedProcessGroup(); + verify(controllerFacade).searchConnector(searchQuery, managedProcessGroup); + } + @Test public void testUpdateAllCountersWithEmptyCounters() { // Mock ControllerFacade to return empty list @@ -1664,4 +1709,321 @@ public void testClearBulletinsForComponentsWithEmptySet() { assertNotNull(result); assertEquals(0, result.getBulletinsCleared()); } + + @Test + public void testGetSecretsFiltersUnauthorizedSecrets() { + final Authentication authentication = new NiFiAuthenticationToken(new NiFiUserDetails(new Builder().identity(USER_1).build())); + SecurityContextHolder.getContext().setAuthentication(authentication); + + final ControllerFacade controllerFacade = mock(ControllerFacade.class); + serviceFacade.setControllerFacade(controllerFacade); + + final AuthorizableSecret authorizedSecret = mock(AuthorizableSecret.class); + when(authorizedSecret.getProviderName()).thenReturn("provider1"); + when(authorizedSecret.getGroupName()).thenReturn("group1"); + when(authorizedSecret.getName()).thenReturn("authorized-secret"); + when(authorizedSecret.getDescription()).thenReturn("An authorized secret"); + when(authorizedSecret.checkAuthorization(any(Authorizer.class), any(), any())).thenReturn(AuthorizationResult.approved()); + + final AuthorizableSecret unauthorizedSecret = mock(AuthorizableSecret.class); + when(unauthorizedSecret.getProviderName()).thenReturn("provider2"); + when(unauthorizedSecret.getGroupName()).thenReturn("group2"); + when(unauthorizedSecret.getName()).thenReturn("unauthorized-secret"); + when(unauthorizedSecret.getDescription()).thenReturn("An unauthorized secret"); + when(unauthorizedSecret.checkAuthorization(any(Authorizer.class), any(), any())).thenReturn(AuthorizationResult.denied()); + + when(controllerFacade.getAllSecrets()).thenReturn(List.of(authorizedSecret, unauthorizedSecret)); + + final SecretsEntity result = serviceFacade.getSecrets(); + + assertNotNull(result); + assertNotNull(result.getSecrets()); + assertEquals(1, result.getSecrets().size()); + assertEquals("authorized-secret", result.getSecrets().get(0).getName()); + } + + @Test + public void testGetSecretsWithNonAuthorizableSecrets() { + final Authentication authentication = new NiFiAuthenticationToken(new NiFiUserDetails(new Builder().identity(USER_1).build())); + SecurityContextHolder.getContext().setAuthentication(authentication); + + final ControllerFacade controllerFacade = mock(ControllerFacade.class); + serviceFacade.setControllerFacade(controllerFacade); + + final Secret nonAuthorizableSecret = mock(Secret.class); + when(nonAuthorizableSecret.getProviderName()).thenReturn("provider1"); + when(nonAuthorizableSecret.getGroupName()).thenReturn("group1"); + when(nonAuthorizableSecret.getName()).thenReturn("non-authorizable-secret"); + when(nonAuthorizableSecret.getDescription()).thenReturn("A non-authorizable secret"); + + when(controllerFacade.getAllSecrets()).thenReturn(List.of(nonAuthorizableSecret)); + + final SecretsEntity result = serviceFacade.getSecrets(); + + assertNotNull(result); + assertNotNull(result.getSecrets()); + assertEquals(1, result.getSecrets().size()); + assertEquals("non-authorizable-secret", result.getSecrets().get(0).getName()); + } + + @Test + public void testGetSecretsWithEmptyList() { + final Authentication authentication = new NiFiAuthenticationToken(new NiFiUserDetails(new Builder().identity(USER_1).build())); + SecurityContextHolder.getContext().setAuthentication(authentication); + + final ControllerFacade controllerFacade = mock(ControllerFacade.class); + serviceFacade.setControllerFacade(controllerFacade); + + when(controllerFacade.getAllSecrets()).thenReturn(List.of()); + + final SecretsEntity result = serviceFacade.getSecrets(); + + assertNotNull(result); + assertNotNull(result.getSecrets()); + assertTrue(result.getSecrets().isEmpty()); + } + + // ----------------- + // Connector State Tests + // ----------------- + + @Test + public void testGetConnectorProcessorState() { + final String connectorId = "connector-id"; + final String processorId = "processor-id"; + + final ConnectorDAO connectorDAO = mock(ConnectorDAO.class); + final ComponentStateDAO componentStateDAO = mock(ComponentStateDAO.class); + final DtoFactory dtoFactory = mock(DtoFactory.class); + serviceFacade.setConnectorDAO(connectorDAO); + serviceFacade.setComponentStateDAO(componentStateDAO); + serviceFacade.setDtoFactory(dtoFactory); + + final ConnectorNode connectorNode = mock(ConnectorNode.class); + final FrameworkFlowContext flowContext = mock(FrameworkFlowContext.class); + final ProcessGroup managedProcessGroup = mock(ProcessGroup.class); + final ProcessorNode processorNode = mock(ProcessorNode.class); + final Processor processor = mock(Processor.class); + final StateMap localStateMap = mock(StateMap.class); + + when(connectorDAO.getConnector(connectorId)).thenReturn(connectorNode); + when(connectorNode.getActiveFlowContext()).thenReturn(flowContext); + when(flowContext.getManagedProcessGroup()).thenReturn(managedProcessGroup); + when(managedProcessGroup.findProcessor(processorId)).thenReturn(processorNode); + when(processorNode.getProcessor()).thenReturn(processor); + when(componentStateDAO.getState(processorNode, Scope.LOCAL)).thenReturn(localStateMap); + + final ComponentStateDTO expectedDto = new ComponentStateDTO(); + expectedDto.setComponentId(processorId); + when(dtoFactory.createComponentStateDTO(eq(processorId), eq(processor.getClass()), eq(localStateMap), any())).thenReturn(expectedDto); + + final ComponentStateDTO result = serviceFacade.getConnectorProcessorState(connectorId, processorId); + + assertNotNull(result); + assertEquals(processorId, result.getComponentId()); + verify(connectorDAO).getConnector(connectorId); + verify(managedProcessGroup).findProcessor(processorId); + verify(componentStateDAO).getState(processorNode, Scope.LOCAL); + } + + @Test + public void testGetConnectorProcessorStateNotFound() { + final String connectorId = "connector-id"; + final String processorId = "non-existent-processor-id"; + + final ConnectorDAO connectorDAO = mock(ConnectorDAO.class); + serviceFacade.setConnectorDAO(connectorDAO); + + final ConnectorNode connectorNode = mock(ConnectorNode.class); + final FrameworkFlowContext flowContext = mock(FrameworkFlowContext.class); + final ProcessGroup managedProcessGroup = mock(ProcessGroup.class); + + when(connectorDAO.getConnector(connectorId)).thenReturn(connectorNode); + when(connectorNode.getActiveFlowContext()).thenReturn(flowContext); + when(flowContext.getManagedProcessGroup()).thenReturn(managedProcessGroup); + when(managedProcessGroup.findProcessor(processorId)).thenReturn(null); + + assertThrows(ResourceNotFoundException.class, () -> serviceFacade.getConnectorProcessorState(connectorId, processorId)); + } + + @Test + public void testVerifyCanClearConnectorProcessorState() { + final String connectorId = "connector-id"; + final String processorId = "processor-id"; + + final ConnectorDAO connectorDAO = mock(ConnectorDAO.class); + serviceFacade.setConnectorDAO(connectorDAO); + + final ConnectorNode connectorNode = mock(ConnectorNode.class); + final FrameworkFlowContext flowContext = mock(FrameworkFlowContext.class); + final ProcessGroup managedProcessGroup = mock(ProcessGroup.class); + final ProcessorNode processorNode = mock(ProcessorNode.class); + + when(connectorDAO.getConnector(connectorId)).thenReturn(connectorNode); + when(connectorNode.getActiveFlowContext()).thenReturn(flowContext); + when(flowContext.getManagedProcessGroup()).thenReturn(managedProcessGroup); + when(managedProcessGroup.findProcessor(processorId)).thenReturn(processorNode); + + // Should not throw + serviceFacade.verifyCanClearConnectorProcessorState(connectorId, processorId); + + verify(processorNode).verifyCanClearState(); + } + + @Test + public void testClearConnectorProcessorState() { + final String connectorId = "connector-id"; + final String processorId = "processor-id"; + + final ConnectorDAO connectorDAO = mock(ConnectorDAO.class); + final ComponentStateDAO componentStateDAO = mock(ComponentStateDAO.class); + final DtoFactory dtoFactory = mock(DtoFactory.class); + serviceFacade.setConnectorDAO(connectorDAO); + serviceFacade.setComponentStateDAO(componentStateDAO); + serviceFacade.setDtoFactory(dtoFactory); + + final ConnectorNode connectorNode = mock(ConnectorNode.class); + final FrameworkFlowContext flowContext = mock(FrameworkFlowContext.class); + final ProcessGroup managedProcessGroup = mock(ProcessGroup.class); + final ProcessorNode processorNode = mock(ProcessorNode.class); + final Processor processor = mock(Processor.class); + final StateMap localStateMap = mock(StateMap.class); + + when(connectorDAO.getConnector(connectorId)).thenReturn(connectorNode); + when(connectorNode.getActiveFlowContext()).thenReturn(flowContext); + when(flowContext.getManagedProcessGroup()).thenReturn(managedProcessGroup); + when(managedProcessGroup.findProcessor(processorId)).thenReturn(processorNode); + when(processorNode.getProcessor()).thenReturn(processor); + when(componentStateDAO.getState(processorNode, Scope.LOCAL)).thenReturn(localStateMap); + + final ComponentStateDTO expectedDto = new ComponentStateDTO(); + expectedDto.setComponentId(processorId); + when(dtoFactory.createComponentStateDTO(eq(processorId), eq(processor.getClass()), eq(localStateMap), any())).thenReturn(expectedDto); + + final ComponentStateDTO result = serviceFacade.clearConnectorProcessorState(connectorId, processorId, null); + + assertNotNull(result); + assertEquals(processorId, result.getComponentId()); + verify(componentStateDAO).clearState(processorNode, null); + } + + @Test + public void testGetConnectorControllerServiceState() { + final String connectorId = "connector-id"; + final String controllerServiceId = "controller-service-id"; + + final ConnectorDAO connectorDAO = mock(ConnectorDAO.class); + final ComponentStateDAO componentStateDAO = mock(ComponentStateDAO.class); + final DtoFactory dtoFactory = mock(DtoFactory.class); + serviceFacade.setConnectorDAO(connectorDAO); + serviceFacade.setComponentStateDAO(componentStateDAO); + serviceFacade.setDtoFactory(dtoFactory); + + final ConnectorNode connectorNode = mock(ConnectorNode.class); + final FrameworkFlowContext flowContext = mock(FrameworkFlowContext.class); + final ProcessGroup managedProcessGroup = mock(ProcessGroup.class); + final ControllerServiceNode controllerServiceNode = mock(ControllerServiceNode.class); + final ControllerService controllerService = mock(ControllerService.class); + final StateMap localStateMap = mock(StateMap.class); + + when(connectorDAO.getConnector(connectorId)).thenReturn(connectorNode); + when(connectorNode.getActiveFlowContext()).thenReturn(flowContext); + when(flowContext.getManagedProcessGroup()).thenReturn(managedProcessGroup); + when(managedProcessGroup.findControllerService(controllerServiceId, false, true)).thenReturn(controllerServiceNode); + when(controllerServiceNode.getControllerServiceImplementation()).thenReturn(controllerService); + when(componentStateDAO.getState(controllerServiceNode, Scope.LOCAL)).thenReturn(localStateMap); + + final ComponentStateDTO expectedDto = new ComponentStateDTO(); + expectedDto.setComponentId(controllerServiceId); + when(dtoFactory.createComponentStateDTO(eq(controllerServiceId), eq(controllerService.getClass()), eq(localStateMap), any())).thenReturn(expectedDto); + + final ComponentStateDTO result = serviceFacade.getConnectorControllerServiceState(connectorId, controllerServiceId); + + assertNotNull(result); + assertEquals(controllerServiceId, result.getComponentId()); + verify(connectorDAO).getConnector(connectorId); + verify(managedProcessGroup).findControllerService(controllerServiceId, false, true); + verify(componentStateDAO).getState(controllerServiceNode, Scope.LOCAL); + } + + @Test + public void testGetConnectorControllerServiceStateNotFound() { + final String connectorId = "connector-id"; + final String controllerServiceId = "non-existent-controller-service-id"; + + final ConnectorDAO connectorDAO = mock(ConnectorDAO.class); + serviceFacade.setConnectorDAO(connectorDAO); + + final ConnectorNode connectorNode = mock(ConnectorNode.class); + final FrameworkFlowContext flowContext = mock(FrameworkFlowContext.class); + final ProcessGroup managedProcessGroup = mock(ProcessGroup.class); + + when(connectorDAO.getConnector(connectorId)).thenReturn(connectorNode); + when(connectorNode.getActiveFlowContext()).thenReturn(flowContext); + when(flowContext.getManagedProcessGroup()).thenReturn(managedProcessGroup); + when(managedProcessGroup.findControllerService(controllerServiceId, false, true)).thenReturn(null); + + assertThrows(ResourceNotFoundException.class, () -> serviceFacade.getConnectorControllerServiceState(connectorId, controllerServiceId)); + } + + @Test + public void testVerifyCanClearConnectorControllerServiceState() { + final String connectorId = "connector-id"; + final String controllerServiceId = "controller-service-id"; + + final ConnectorDAO connectorDAO = mock(ConnectorDAO.class); + serviceFacade.setConnectorDAO(connectorDAO); + + final ConnectorNode connectorNode = mock(ConnectorNode.class); + final FrameworkFlowContext flowContext = mock(FrameworkFlowContext.class); + final ProcessGroup managedProcessGroup = mock(ProcessGroup.class); + final ControllerServiceNode controllerServiceNode = mock(ControllerServiceNode.class); + + when(connectorDAO.getConnector(connectorId)).thenReturn(connectorNode); + when(connectorNode.getActiveFlowContext()).thenReturn(flowContext); + when(flowContext.getManagedProcessGroup()).thenReturn(managedProcessGroup); + when(managedProcessGroup.findControllerService(controllerServiceId, false, true)).thenReturn(controllerServiceNode); + + // Should not throw + serviceFacade.verifyCanClearConnectorControllerServiceState(connectorId, controllerServiceId); + + verify(controllerServiceNode).verifyCanClearState(); + } + + @Test + public void testClearConnectorControllerServiceState() { + final String connectorId = "connector-id"; + final String controllerServiceId = "controller-service-id"; + + final ConnectorDAO connectorDAO = mock(ConnectorDAO.class); + final ComponentStateDAO componentStateDAO = mock(ComponentStateDAO.class); + final DtoFactory dtoFactory = mock(DtoFactory.class); + serviceFacade.setConnectorDAO(connectorDAO); + serviceFacade.setComponentStateDAO(componentStateDAO); + serviceFacade.setDtoFactory(dtoFactory); + + final ConnectorNode connectorNode = mock(ConnectorNode.class); + final FrameworkFlowContext flowContext = mock(FrameworkFlowContext.class); + final ProcessGroup managedProcessGroup = mock(ProcessGroup.class); + final ControllerServiceNode controllerServiceNode = mock(ControllerServiceNode.class); + final ControllerService controllerService = mock(ControllerService.class); + final StateMap localStateMap = mock(StateMap.class); + + when(connectorDAO.getConnector(connectorId)).thenReturn(connectorNode); + when(connectorNode.getActiveFlowContext()).thenReturn(flowContext); + when(flowContext.getManagedProcessGroup()).thenReturn(managedProcessGroup); + when(managedProcessGroup.findControllerService(controllerServiceId, false, true)).thenReturn(controllerServiceNode); + when(controllerServiceNode.getControllerServiceImplementation()).thenReturn(controllerService); + when(componentStateDAO.getState(controllerServiceNode, Scope.LOCAL)).thenReturn(localStateMap); + + final ComponentStateDTO expectedDto = new ComponentStateDTO(); + expectedDto.setComponentId(controllerServiceId); + when(dtoFactory.createComponentStateDTO(eq(controllerServiceId), eq(controllerService.getClass()), eq(localStateMap), any())).thenReturn(expectedDto); + + final ComponentStateDTO result = serviceFacade.clearConnectorControllerServiceState(connectorId, controllerServiceId, null); + + assertNotNull(result); + assertEquals(controllerServiceId, result.getComponentId()); + verify(componentStateDAO).clearState(controllerServiceNode, null); + } } diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestConnectorResource.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestConnectorResource.java new file mode 100644 index 000000000000..5d50471cf961 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestConnectorResource.java @@ -0,0 +1,748 @@ +/* + * 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; + +import jakarta.servlet.ServletContext; +import jakarta.servlet.http.HttpServletRequest; +import jakarta.ws.rs.core.Response; +import jakarta.ws.rs.core.UriBuilder; +import jakarta.ws.rs.core.UriInfo; +import org.apache.nifi.authorization.AccessDeniedException; +import org.apache.nifi.authorization.AuthorizeAccess; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.NiFiServiceFacade; +import org.apache.nifi.web.Revision; +import org.apache.nifi.web.api.dto.AllowableValueDTO; +import org.apache.nifi.web.api.dto.ComponentStateDTO; +import org.apache.nifi.web.api.dto.ConnectorDTO; +import org.apache.nifi.web.api.dto.RevisionDTO; +import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO; +import org.apache.nifi.web.api.entity.AllowableValueEntity; +import org.apache.nifi.web.api.entity.ComponentStateEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ConnectorPropertyAllowableValuesEntity; +import org.apache.nifi.web.api.entity.ConnectorRunStatusEntity; +import org.apache.nifi.web.api.entity.ControllerServiceEntity; +import org.apache.nifi.web.api.entity.ControllerServicesEntity; +import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity; +import org.apache.nifi.web.api.entity.SecretsEntity; +import org.apache.nifi.web.api.request.ClientIdParameter; +import org.apache.nifi.web.api.request.LongParameter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.net.URI; +import java.util.List; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class TestConnectorResource { + + @InjectMocks + private ConnectorResource connectorResource; + + @Mock + private NiFiServiceFacade serviceFacade; + + @Mock + private Authorizer authorizer; + + @Mock + private HttpServletRequest httpServletRequest; + + @Mock + private ServletContext servletContext; + + @Mock + private NiFiProperties properties; + + @Mock + private UriInfo uriInfo; + + @Mock + private UriBuilder uriBuilder; + + @Mock + private FlowResource flowResource; + + @Mock + private ControllerServiceResource controllerServiceResource; + + private static final String CONNECTOR_ID = "test-connector-id"; + private static final String CONNECTOR_NAME = "Test Connector"; + private static final String CONNECTOR_TYPE = "TestConnectorType"; + private static final String CONFIGURATION_STEP_NAME = "test-step"; + private static final String PROPERTY_GROUP_NAME = "test-group"; + private static final String PROPERTY_NAME = "test-property"; + private static final String PROCESS_GROUP_ID = "test-process-group-id"; + private static final String PROCESSOR_ID = "test-processor-id"; + private static final String CONTROLLER_SERVICE_ID = "test-controller-service-id"; + + @BeforeEach + public void setUp() throws Exception { + lenient().when(httpServletRequest.getHeader(any())).thenReturn(null); + lenient().when(httpServletRequest.getServletContext()).thenReturn(servletContext); + lenient().when(httpServletRequest.getContextPath()).thenReturn("/nifi-api"); + lenient().when(httpServletRequest.getScheme()).thenReturn("http"); + lenient().when(httpServletRequest.getServerName()).thenReturn("localhost"); + lenient().when(httpServletRequest.getServerPort()).thenReturn(8080); + lenient().when(servletContext.getInitParameter(any())).thenReturn(null); + lenient().when(properties.isNode()).thenReturn(Boolean.FALSE); + + lenient().when(uriInfo.getBaseUriBuilder()).thenReturn(uriBuilder); + lenient().when(uriBuilder.segment(any(String[].class))).thenReturn(uriBuilder); + lenient().when(uriBuilder.build()).thenReturn(new URI("http://localhost:8080/nifi-api/connectors/" + CONNECTOR_ID)); + + connectorResource.setServiceFacade(serviceFacade); + connectorResource.setFlowResource(flowResource); + connectorResource.setControllerServiceResource(controllerServiceResource); + connectorResource.httpServletRequest = httpServletRequest; + connectorResource.properties = properties; + connectorResource.uriInfo = uriInfo; + } + + @Test + public void testGetConnector() { + final ConnectorEntity connectorEntity = createConnectorEntity(); + + when(serviceFacade.getConnector(CONNECTOR_ID)).thenReturn(connectorEntity); + + try (Response response = connectorResource.getConnector(CONNECTOR_ID)) { + assertEquals(200, response.getStatus()); + assertEquals(connectorEntity, response.getEntity()); + } + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade).getConnector(CONNECTOR_ID); + } + + @Test + public void testGetConnectorNotAuthorized() { + doThrow(AccessDeniedException.class).when(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + + assertThrows(AccessDeniedException.class, () -> connectorResource.getConnector(CONNECTOR_ID)); + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade, never()).getConnector(anyString()); + } + + @Test + public void testUpdateConnector() { + final ConnectorEntity requestEntity = createConnectorEntity(); + final ConnectorEntity responseEntity = createConnectorEntity(); + + when(serviceFacade.updateConnector(any(Revision.class), any(ConnectorDTO.class))).thenReturn(responseEntity); + + try (Response response = connectorResource.updateConnector(CONNECTOR_ID, requestEntity)) { + assertEquals(200, response.getStatus()); + assertEquals(responseEntity, response.getEntity()); + } + + verify(serviceFacade).verifyUpdateConnector(any(ConnectorDTO.class)); + verify(serviceFacade).updateConnector(any(Revision.class), any(ConnectorDTO.class)); + } + + @Test + public void testUpdateConnectorNotAuthorized() { + final ConnectorEntity requestEntity = createConnectorEntity(); + + doThrow(AccessDeniedException.class).when(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + + assertThrows(AccessDeniedException.class, () -> connectorResource.updateConnector(CONNECTOR_ID, requestEntity)); + + verify(serviceFacade, never()).verifyUpdateConnector(any(ConnectorDTO.class)); + verify(serviceFacade, never()).updateConnector(any(Revision.class), any(ConnectorDTO.class)); + } + + @Test + public void testUpdateConnectorWithMismatchedId() { + final ConnectorEntity requestEntity = createConnectorEntity(); + requestEntity.getComponent().setId("different-id"); + + assertThrows(IllegalArgumentException.class, () -> + connectorResource.updateConnector(CONNECTOR_ID, requestEntity)); + + verify(serviceFacade, never()).updateConnector(any(Revision.class), any(ConnectorDTO.class)); + } + + @Test + public void testUpdateConnectorWithNullEntity() { + assertThrows(IllegalArgumentException.class, () -> + connectorResource.updateConnector(CONNECTOR_ID, null)); + + verify(serviceFacade, never()).updateConnector(any(Revision.class), any(ConnectorDTO.class)); + } + + @Test + public void testUpdateConnectorWithNullComponent() { + final ConnectorEntity requestEntity = new ConnectorEntity(); + requestEntity.setComponent(null); + + assertThrows(IllegalArgumentException.class, () -> + connectorResource.updateConnector(CONNECTOR_ID, requestEntity)); + + verify(serviceFacade, never()).updateConnector(any(Revision.class), any(ConnectorDTO.class)); + } + + @Test + public void testUpdateConnectorWithNullRevision() { + final ConnectorEntity requestEntity = createConnectorEntity(); + requestEntity.setRevision(null); + + assertThrows(IllegalArgumentException.class, () -> + connectorResource.updateConnector(CONNECTOR_ID, requestEntity)); + + verify(serviceFacade, never()).updateConnector(any(Revision.class), any(ConnectorDTO.class)); + } + + @Test + public void testDeleteConnector() { + final ConnectorEntity responseEntity = createConnectorEntity(); + + when(serviceFacade.deleteConnector(any(Revision.class), eq(CONNECTOR_ID))).thenReturn(responseEntity); + + try (Response response = connectorResource.deleteConnector(new LongParameter("1"), new ClientIdParameter("client-id"), false, CONNECTOR_ID)) { + assertEquals(200, response.getStatus()); + assertEquals(responseEntity, response.getEntity()); + } + + verify(serviceFacade).verifyDeleteConnector(CONNECTOR_ID); + verify(serviceFacade).deleteConnector(any(Revision.class), eq(CONNECTOR_ID)); + } + + @Test + public void testDeleteConnectorNotAuthorized() { + doThrow(AccessDeniedException.class).when(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + + assertThrows(AccessDeniedException.class, () -> + connectorResource.deleteConnector(new LongParameter("1"), new ClientIdParameter("client-id"), false, CONNECTOR_ID)); + + verify(serviceFacade, never()).verifyDeleteConnector(anyString()); + verify(serviceFacade, never()).deleteConnector(any(Revision.class), anyString()); + } + + @Test + public void testUpdateRunStatus() { + final ConnectorRunStatusEntity requestEntity = createConnectorRunStatusEntity(); + final ConnectorEntity responseEntity = createConnectorEntity(); + + when(serviceFacade.scheduleConnector(any(Revision.class), eq(CONNECTOR_ID), eq(ScheduledState.RUNNING))) + .thenReturn(responseEntity); + + try (Response response = connectorResource.updateRunStatus(CONNECTOR_ID, requestEntity)) { + assertEquals(200, response.getStatus()); + assertEquals(responseEntity, response.getEntity()); + } + + verify(serviceFacade).verifyUpdateConnector(any(ConnectorDTO.class)); + verify(serviceFacade).scheduleConnector(any(Revision.class), eq(CONNECTOR_ID), eq(ScheduledState.RUNNING)); + } + + @Test + public void testUpdateRunStatusNotAuthorized() { + final ConnectorRunStatusEntity requestEntity = createConnectorRunStatusEntity(); + + doThrow(AccessDeniedException.class).when(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + + assertThrows(AccessDeniedException.class, () -> + connectorResource.updateRunStatus(CONNECTOR_ID, requestEntity)); + + verify(serviceFacade, never()).verifyUpdateConnector(any(ConnectorDTO.class)); + verify(serviceFacade, never()).scheduleConnector(any(Revision.class), anyString(), any(ScheduledState.class)); + } + + @Test + public void testUpdateRunStatusWithNullEntity() { + assertThrows(IllegalArgumentException.class, () -> + connectorResource.updateRunStatus(CONNECTOR_ID, null)); + + verify(serviceFacade, never()).scheduleConnector(any(Revision.class), anyString(), any(ScheduledState.class)); + } + + @Test + public void testUpdateRunStatusWithNullRevision() { + final ConnectorRunStatusEntity requestEntity = createConnectorRunStatusEntity(); + requestEntity.setRevision(null); + + assertThrows(IllegalArgumentException.class, () -> + connectorResource.updateRunStatus(CONNECTOR_ID, requestEntity)); + + verify(serviceFacade, never()).scheduleConnector(any(Revision.class), anyString(), any(ScheduledState.class)); + } + + @Test + public void testGetConnectorPropertyAllowableValues() { + final ConnectorPropertyAllowableValuesEntity responseEntity = createConnectorPropertyAllowableValuesEntity(); + + when(serviceFacade.getConnectorPropertyAllowableValues(CONNECTOR_ID, CONFIGURATION_STEP_NAME, PROPERTY_GROUP_NAME, PROPERTY_NAME, null)) + .thenReturn(responseEntity); + + try (Response response = connectorResource.getConnectorPropertyAllowableValues(CONNECTOR_ID, CONFIGURATION_STEP_NAME, PROPERTY_GROUP_NAME, PROPERTY_NAME, null)) { + assertEquals(200, response.getStatus()); + assertEquals(responseEntity, response.getEntity()); + } + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade).getConnectorPropertyAllowableValues(CONNECTOR_ID, CONFIGURATION_STEP_NAME, PROPERTY_GROUP_NAME, PROPERTY_NAME, null); + } + + @Test + public void testGetConnectorPropertyAllowableValuesWithFilter() { + final String filter = "test-filter"; + final ConnectorPropertyAllowableValuesEntity responseEntity = createConnectorPropertyAllowableValuesEntity(); + + when(serviceFacade.getConnectorPropertyAllowableValues(CONNECTOR_ID, CONFIGURATION_STEP_NAME, PROPERTY_GROUP_NAME, PROPERTY_NAME, filter)) + .thenReturn(responseEntity); + + try (Response response = connectorResource.getConnectorPropertyAllowableValues(CONNECTOR_ID, CONFIGURATION_STEP_NAME, PROPERTY_GROUP_NAME, PROPERTY_NAME, filter)) { + assertEquals(200, response.getStatus()); + assertEquals(responseEntity, response.getEntity()); + } + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade).getConnectorPropertyAllowableValues(CONNECTOR_ID, CONFIGURATION_STEP_NAME, PROPERTY_GROUP_NAME, PROPERTY_NAME, filter); + } + + @Test + public void testGetConnectorPropertyAllowableValuesNotAuthorized() { + doThrow(AccessDeniedException.class).when(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + + assertThrows(AccessDeniedException.class, () -> + connectorResource.getConnectorPropertyAllowableValues(CONNECTOR_ID, CONFIGURATION_STEP_NAME, PROPERTY_GROUP_NAME, PROPERTY_NAME, null)); + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade, never()).getConnectorPropertyAllowableValues(anyString(), anyString(), anyString(), anyString(), any()); + } + + @Test + public void testGetSecrets() { + final SecretsEntity responseEntity = new SecretsEntity(); + responseEntity.setSecrets(List.of()); + + when(serviceFacade.getSecrets()).thenReturn(responseEntity); + + try (Response response = connectorResource.getSecrets(CONNECTOR_ID)) { + assertEquals(200, response.getStatus()); + assertEquals(responseEntity, response.getEntity()); + } + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade).getSecrets(); + } + + @Test + public void testGetSecretsNotAuthorized() { + doThrow(AccessDeniedException.class).when(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + + assertThrows(AccessDeniedException.class, () -> connectorResource.getSecrets(CONNECTOR_ID)); + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade, never()).getSecrets(); + } + + @Test + public void testGetFlow() { + final ProcessGroupFlowEntity responseEntity = createProcessGroupFlowEntity(); + when(serviceFacade.getConnectorFlow(CONNECTOR_ID, PROCESS_GROUP_ID, false)).thenReturn(responseEntity); + + try (Response response = connectorResource.getFlow(CONNECTOR_ID, PROCESS_GROUP_ID, false)) { + assertEquals(200, response.getStatus()); + assertEquals(responseEntity, response.getEntity()); + } + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade).getConnectorFlow(CONNECTOR_ID, PROCESS_GROUP_ID, false); + } + + @Test + public void testGetFlowNotAuthorized() { + doThrow(AccessDeniedException.class).when(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + + assertThrows(AccessDeniedException.class, () -> connectorResource.getFlow(CONNECTOR_ID, PROCESS_GROUP_ID, false)); + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade, never()).getConnectorFlow(anyString(), anyString(), eq(false)); + } + + @Test + public void testGetControllerServicesFromConnectorProcessGroup() { + final Set controllerServices = Set.of(); + when(serviceFacade.getConnectorControllerServices(CONNECTOR_ID, PROCESS_GROUP_ID, true, false, true)).thenReturn(controllerServices); + + try (Response response = connectorResource.getControllerServicesFromConnectorProcessGroup(CONNECTOR_ID, PROCESS_GROUP_ID, true, false, true)) { + assertEquals(200, response.getStatus()); + final ControllerServicesEntity entity = (ControllerServicesEntity) response.getEntity(); + assertEquals(controllerServices, entity.getControllerServices()); + } + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade).getConnectorControllerServices(CONNECTOR_ID, PROCESS_GROUP_ID, true, false, true); + verify(controllerServiceResource).populateRemainingControllerServiceEntitiesContent(controllerServices); + } + + @Test + public void testGetControllerServicesFromConnectorProcessGroupNotAuthorized() { + doThrow(AccessDeniedException.class).when(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + + assertThrows(AccessDeniedException.class, () -> + connectorResource.getControllerServicesFromConnectorProcessGroup(CONNECTOR_ID, PROCESS_GROUP_ID, true, false, true)); + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade, never()).getConnectorControllerServices(anyString(), anyString(), eq(true), eq(false), eq(true)); + } + + @Test + public void testInitiateDrain() { + final ConnectorEntity requestEntity = createConnectorEntity(); + final ConnectorEntity responseEntity = createConnectorEntity(); + responseEntity.getComponent().setState("DRAINING"); + + when(serviceFacade.drainConnector(any(Revision.class), eq(CONNECTOR_ID))).thenReturn(responseEntity); + + try (Response response = connectorResource.initiateDrain(CONNECTOR_ID, requestEntity)) { + assertEquals(200, response.getStatus()); + assertEquals(responseEntity, response.getEntity()); + } + + verify(serviceFacade).verifyDrainConnector(CONNECTOR_ID); + verify(serviceFacade).drainConnector(any(Revision.class), eq(CONNECTOR_ID)); + } + + @Test + public void testInitiateDrainNotAuthorized() { + final ConnectorEntity requestEntity = createConnectorEntity(); + + doThrow(AccessDeniedException.class).when(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + + assertThrows(AccessDeniedException.class, () -> + connectorResource.initiateDrain(CONNECTOR_ID, requestEntity)); + + verify(serviceFacade, never()).verifyDrainConnector(anyString()); + verify(serviceFacade, never()).drainConnector(any(Revision.class), anyString()); + } + + @Test + public void testInitiateDrainWithNullEntity() { + assertThrows(IllegalArgumentException.class, () -> + connectorResource.initiateDrain(CONNECTOR_ID, null)); + + verify(serviceFacade, never()).drainConnector(any(Revision.class), anyString()); + } + + @Test + public void testInitiateDrainWithNullRevision() { + final ConnectorEntity requestEntity = createConnectorEntity(); + requestEntity.setRevision(null); + + assertThrows(IllegalArgumentException.class, () -> + connectorResource.initiateDrain(CONNECTOR_ID, requestEntity)); + + verify(serviceFacade, never()).drainConnector(any(Revision.class), anyString()); + } + + @Test + public void testInitiateDrainWithMismatchedId() { + final ConnectorEntity requestEntity = createConnectorEntity(); + requestEntity.setId("different-id"); + + assertThrows(IllegalArgumentException.class, () -> + connectorResource.initiateDrain(CONNECTOR_ID, requestEntity)); + + verify(serviceFacade, never()).drainConnector(any(Revision.class), anyString()); + } + + private ConnectorEntity createConnectorEntity() { + final ConnectorEntity entity = new ConnectorEntity(); + + final ConnectorDTO dto = new ConnectorDTO(); + dto.setId(CONNECTOR_ID); + dto.setName(CONNECTOR_NAME); + dto.setType(CONNECTOR_TYPE); + dto.setState("STOPPED"); + entity.setComponent(dto); + + final RevisionDTO revision = new RevisionDTO(); + revision.setVersion(1L); + revision.setClientId("client-id"); + entity.setRevision(revision); + + return entity; + } + + private ConnectorRunStatusEntity createConnectorRunStatusEntity() { + final ConnectorRunStatusEntity entity = new ConnectorRunStatusEntity(); + entity.setState("RUNNING"); + + final RevisionDTO revision = new RevisionDTO(); + revision.setVersion(1L); + revision.setClientId("client-id"); + entity.setRevision(revision); + + return entity; + } + + private ConnectorPropertyAllowableValuesEntity createConnectorPropertyAllowableValuesEntity() { + final ConnectorPropertyAllowableValuesEntity entity = new ConnectorPropertyAllowableValuesEntity(); + entity.setConfigurationStepName(CONFIGURATION_STEP_NAME); + entity.setPropertyGroupName(PROPERTY_GROUP_NAME); + entity.setPropertyName(PROPERTY_NAME); + + final AllowableValueDTO allowableValueDto1 = new AllowableValueDTO(); + allowableValueDto1.setValue("value1"); + allowableValueDto1.setDisplayName("Value 1"); + allowableValueDto1.setDescription("First allowable value"); + + final AllowableValueEntity allowableValueEntity1 = new AllowableValueEntity(); + allowableValueEntity1.setAllowableValue(allowableValueDto1); + allowableValueEntity1.setCanRead(true); + + final AllowableValueDTO allowableValueDto2 = new AllowableValueDTO(); + allowableValueDto2.setValue("value2"); + allowableValueDto2.setDisplayName("Value 2"); + allowableValueDto2.setDescription("Second allowable value"); + + final AllowableValueEntity allowableValueEntity2 = new AllowableValueEntity(); + allowableValueEntity2.setAllowableValue(allowableValueDto2); + allowableValueEntity2.setCanRead(true); + + entity.setAllowableValues(List.of(allowableValueEntity1, allowableValueEntity2)); + + return entity; + } + + private ProcessGroupFlowEntity createProcessGroupFlowEntity() { + final ProcessGroupFlowEntity entity = new ProcessGroupFlowEntity(); + final ProcessGroupFlowDTO flowDTO = new ProcessGroupFlowDTO(); + flowDTO.setId("root-process-group-id"); + entity.setProcessGroupFlow(flowDTO); + return entity; + } + + @Test + public void testCreateConnectorWithValidClientSpecifiedUuid() { + final String uppercaseUuid = "A1B2C3D4-E5F6-7890-ABCD-EF1234567890"; + final String normalizedUuid = uppercaseUuid.toLowerCase(); + + final ConnectorEntity requestEntity = createConnectorEntity(); + requestEntity.getComponent().setId(uppercaseUuid); + requestEntity.getComponent().setType(CONNECTOR_TYPE); + requestEntity.getRevision().setVersion(0L); + + final ConnectorEntity responseEntity = createConnectorEntity(); + responseEntity.getComponent().setId(normalizedUuid); + + when(serviceFacade.createConnector(any(Revision.class), any(ConnectorDTO.class))).thenReturn(responseEntity); + + try (Response response = connectorResource.createConnector(requestEntity)) { + assertEquals(201, response.getStatus()); + final ConnectorEntity entity = (ConnectorEntity) response.getEntity(); + assertEquals(normalizedUuid, entity.getComponent().getId()); + } + + verify(serviceFacade).verifyCreateConnector(any(ConnectorDTO.class)); + verify(serviceFacade).createConnector(any(Revision.class), any(ConnectorDTO.class)); + } + + @Test + public void testCreateConnectorWithInvalidClientSpecifiedUuid() { + final String invalidId = "not-a-valid-uuid"; + + final ConnectorEntity requestEntity = createConnectorEntity(); + requestEntity.getComponent().setId(invalidId); + requestEntity.getComponent().setType(CONNECTOR_TYPE); + requestEntity.getRevision().setVersion(0L); + + final IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> + connectorResource.createConnector(requestEntity)); + + assertEquals("ID [" + invalidId + "] is not a valid UUID.", exception.getMessage()); + + verify(serviceFacade, never()).createConnector(any(Revision.class), any(ConnectorDTO.class)); + } + + @Test + public void testCreateConnectorWithNullEntity() { + assertThrows(IllegalArgumentException.class, () -> + connectorResource.createConnector(null)); + + verify(serviceFacade, never()).createConnector(any(Revision.class), any(ConnectorDTO.class)); + } + + @Test + public void testCreateConnectorWithNullComponent() { + final ConnectorEntity requestEntity = new ConnectorEntity(); + requestEntity.setComponent(null); + + assertThrows(IllegalArgumentException.class, () -> + connectorResource.createConnector(requestEntity)); + + verify(serviceFacade, never()).createConnector(any(Revision.class), any(ConnectorDTO.class)); + } + + @Test + public void testCreateConnectorWithInvalidRevision() { + final ConnectorEntity requestEntity = createConnectorEntity(); + requestEntity.getComponent().setType(CONNECTOR_TYPE); + requestEntity.getRevision().setVersion(1L); + + assertThrows(IllegalArgumentException.class, () -> + connectorResource.createConnector(requestEntity)); + + verify(serviceFacade, never()).createConnector(any(Revision.class), any(ConnectorDTO.class)); + } + + @Test + public void testCreateConnectorWithBlankType() { + final ConnectorEntity requestEntity = createConnectorEntity(); + requestEntity.getComponent().setType(""); + requestEntity.getRevision().setVersion(0L); + + assertThrows(IllegalArgumentException.class, () -> + connectorResource.createConnector(requestEntity)); + + verify(serviceFacade, never()).createConnector(any(Revision.class), any(ConnectorDTO.class)); + } + + @Test + public void testGetConnectorProcessorState() { + final ComponentStateDTO stateDTO = new ComponentStateDTO(); + stateDTO.setComponentId(PROCESSOR_ID); + when(serviceFacade.getConnectorProcessorState(CONNECTOR_ID, PROCESSOR_ID)).thenReturn(stateDTO); + + try (Response response = connectorResource.getConnectorProcessorState(CONNECTOR_ID, PROCESSOR_ID)) { + assertEquals(200, response.getStatus()); + final ComponentStateEntity entity = (ComponentStateEntity) response.getEntity(); + assertEquals(PROCESSOR_ID, entity.getComponentState().getComponentId()); + } + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade).getConnectorProcessorState(CONNECTOR_ID, PROCESSOR_ID); + } + + @Test + public void testGetConnectorProcessorStateNotAuthorized() { + doThrow(AccessDeniedException.class).when(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + + assertThrows(AccessDeniedException.class, () -> + connectorResource.getConnectorProcessorState(CONNECTOR_ID, PROCESSOR_ID)); + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade, never()).getConnectorProcessorState(anyString(), anyString()); + } + + @Test + public void testClearConnectorProcessorState() { + final ComponentStateDTO stateDTO = new ComponentStateDTO(); + stateDTO.setComponentId(PROCESSOR_ID); + when(serviceFacade.clearConnectorProcessorState(eq(CONNECTOR_ID), eq(PROCESSOR_ID), any())).thenReturn(stateDTO); + + try (Response response = connectorResource.clearConnectorProcessorState(CONNECTOR_ID, PROCESSOR_ID, null)) { + assertEquals(200, response.getStatus()); + final ComponentStateEntity entity = (ComponentStateEntity) response.getEntity(); + assertEquals(PROCESSOR_ID, entity.getComponentState().getComponentId()); + } + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade).verifyCanClearConnectorProcessorState(CONNECTOR_ID, PROCESSOR_ID); + verify(serviceFacade).clearConnectorProcessorState(eq(CONNECTOR_ID), eq(PROCESSOR_ID), any()); + } + + @Test + public void testClearConnectorProcessorStateNotAuthorized() { + doThrow(AccessDeniedException.class).when(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + + assertThrows(AccessDeniedException.class, () -> + connectorResource.clearConnectorProcessorState(CONNECTOR_ID, PROCESSOR_ID, null)); + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade, never()).verifyCanClearConnectorProcessorState(anyString(), anyString()); + verify(serviceFacade, never()).clearConnectorProcessorState(anyString(), anyString(), any()); + } + + @Test + public void testGetConnectorControllerServiceState() { + final ComponentStateDTO stateDTO = new ComponentStateDTO(); + stateDTO.setComponentId(CONTROLLER_SERVICE_ID); + when(serviceFacade.getConnectorControllerServiceState(CONNECTOR_ID, CONTROLLER_SERVICE_ID)).thenReturn(stateDTO); + + try (Response response = connectorResource.getConnectorControllerServiceState(CONNECTOR_ID, CONTROLLER_SERVICE_ID)) { + assertEquals(200, response.getStatus()); + final ComponentStateEntity entity = (ComponentStateEntity) response.getEntity(); + assertEquals(CONTROLLER_SERVICE_ID, entity.getComponentState().getComponentId()); + } + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade).getConnectorControllerServiceState(CONNECTOR_ID, CONTROLLER_SERVICE_ID); + } + + @Test + public void testGetConnectorControllerServiceStateNotAuthorized() { + doThrow(AccessDeniedException.class).when(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + + assertThrows(AccessDeniedException.class, () -> + connectorResource.getConnectorControllerServiceState(CONNECTOR_ID, CONTROLLER_SERVICE_ID)); + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade, never()).getConnectorControllerServiceState(anyString(), anyString()); + } + + @Test + public void testClearConnectorControllerServiceState() { + final ComponentStateDTO stateDTO = new ComponentStateDTO(); + stateDTO.setComponentId(CONTROLLER_SERVICE_ID); + when(serviceFacade.clearConnectorControllerServiceState(eq(CONNECTOR_ID), eq(CONTROLLER_SERVICE_ID), any())).thenReturn(stateDTO); + + try (Response response = connectorResource.clearConnectorControllerServiceState(CONNECTOR_ID, CONTROLLER_SERVICE_ID, null)) { + assertEquals(200, response.getStatus()); + final ComponentStateEntity entity = (ComponentStateEntity) response.getEntity(); + assertEquals(CONTROLLER_SERVICE_ID, entity.getComponentState().getComponentId()); + } + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade).verifyCanClearConnectorControllerServiceState(CONNECTOR_ID, CONTROLLER_SERVICE_ID); + verify(serviceFacade).clearConnectorControllerServiceState(eq(CONNECTOR_ID), eq(CONTROLLER_SERVICE_ID), any()); + } + + @Test + public void testClearConnectorControllerServiceStateNotAuthorized() { + doThrow(AccessDeniedException.class).when(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + + assertThrows(AccessDeniedException.class, () -> + connectorResource.clearConnectorControllerServiceState(CONNECTOR_ID, CONTROLLER_SERVICE_ID, null)); + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade, never()).verifyCanClearConnectorControllerServiceState(anyString(), anyString()); + verify(serviceFacade, never()).clearConnectorControllerServiceState(anyString(), anyString(), any()); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java index b82cbd73977b..1b68a19f7982 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestDataTransferResource.java @@ -242,7 +242,7 @@ public void testExtendTransaction() throws Exception { final UriInfo uriInfo = mockUriInfo(locationUriStr); final InputStream inputStream = null; - final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null)); + final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties((String) null)); final String transactionId = transactionManager.createTransaction(); final Response response = resource.extendPortTransactionTTL("input-ports", "port-id", transactionId, req, res, context, uriInfo, inputStream); @@ -273,7 +273,7 @@ public void testReceiveFlowFiles() throws Exception { final ServletContext context = null; final InputStream inputStream = null; - final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null)); + final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties((String) null)); final String transactionId = transactionManager.createTransaction(); final Response response = resource.receiveFlowFiles("port-id", transactionId, req, context, inputStream); @@ -300,7 +300,7 @@ public void testReceiveZeroFlowFiles() throws Exception { final ServletContext context = null; final InputStream inputStream = null; - final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null)); + final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties((String) null)); final String transactionId = transactionManager.createTransaction(); final Response response = resource.receiveFlowFiles("port-id", transactionId, req, context, inputStream); @@ -319,7 +319,7 @@ public void testCommitInputPortTransaction() { final ServletContext context = null; final InputStream inputStream = null; - final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null)); + final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties((String) null)); final String transactionId = transactionManager.createTransaction(); final Response response = resource.commitInputPortTransaction(ResponseCode.CONFIRM_TRANSACTION.getCode(), "port-id", transactionId, req, context, inputStream); @@ -342,7 +342,7 @@ public void testTransferFlowFiles() { final HttpServletResponse res = null; final InputStream inputStream = null; - final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null)); + final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties((String) null)); final String transactionId = transactionManager.createTransaction(); final Response response = resource.transferFlowFiles("port-id", transactionId, req, res, context, inputStream); @@ -364,7 +364,7 @@ public void testCommitOutputPortTransaction() { final ServletContext context = null; final InputStream inputStream = null; - final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null)); + final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties((String) null)); final String transactionId = transactionManager.createTransaction(); final Response response = resource.commitOutputPortTransaction(ResponseCode.CONFIRM_TRANSACTION.getCode(), @@ -390,7 +390,7 @@ public void testCommitOutputPortTransactionBadChecksum() throws Exception { final ServletContext context = null; final InputStream inputStream = null; - final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties(null)); + final HttpRemoteSiteListener transactionManager = HttpRemoteSiteListener.getInstance(NiFiProperties.createBasicNiFiProperties((String) null)); final String transactionId = transactionManager.createTransaction(); final Response response = resource.commitOutputPortTransaction(ResponseCode.CONFIRM_TRANSACTION.getCode(), @@ -408,7 +408,7 @@ private DataTransferResource getDataTransferResource() { final NiFiServiceFacade serviceFacade = mock(NiFiServiceFacade.class); final HttpFlowFileServerProtocol serverProtocol = mock(HttpFlowFileServerProtocol.class); - final DataTransferResource resource = new DataTransferResource(NiFiProperties.createBasicNiFiProperties(null)) { + final DataTransferResource resource = new DataTransferResource(NiFiProperties.createBasicNiFiProperties((String) null)) { @Override protected void authorizeDataTransfer(AuthorizableLookup lookup, ResourceType resourceType, String identifier) { } @@ -418,7 +418,7 @@ HttpFlowFileServerProtocol getHttpFlowFileServerProtocol(VersionNegotiator versi return serverProtocol; } }; - resource.setProperties(NiFiProperties.createBasicNiFiProperties(null)); + resource.setProperties(NiFiProperties.createBasicNiFiProperties((String) null)); resource.setServiceFacade(serviceFacade); return resource; } diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestFlowResource.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestFlowResource.java index 5bb5447d8dfa..de1cf5c48527 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestFlowResource.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/api/TestFlowResource.java @@ -29,6 +29,8 @@ import jakarta.ws.rs.core.MediaType; import jakarta.ws.rs.core.Response; import jakarta.ws.rs.core.StreamingOutput; +import org.apache.nifi.authorization.AccessDeniedException; +import org.apache.nifi.authorization.AuthorizeAccess; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.validation.DisabledServiceValidationResult; import org.apache.nifi.connectable.Port; @@ -54,10 +56,12 @@ import org.apache.nifi.web.api.dto.DifferenceDTO; import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity; import org.apache.nifi.web.api.entity.ClearBulletinsForGroupRequestEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; import org.apache.nifi.web.api.entity.FlowComparisonEntity; import org.apache.nifi.web.api.request.FlowMetricsProducer; import org.apache.nifi.web.api.request.FlowMetricsReportingStrategy; import org.jetbrains.annotations.NotNull; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.ArgumentCaptor; @@ -94,6 +98,7 @@ import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -138,7 +143,17 @@ public class TestFlowResource { private NiFiServiceFacade serviceFacade; @Mock - private NiFiProperties niFiProperties; + private NiFiProperties properties; + + @Mock + private ConnectorResource connectorResource; + + @BeforeEach + public void setUp() { + lenient().when(properties.isNode()).thenReturn(Boolean.FALSE); + resource.properties = properties; + resource.setConnectorResource(connectorResource); + } @Test public void testGetFlowMetricsProducerInvalid() { @@ -450,7 +465,7 @@ public void testActivateControllerServicesStateEnabled() { entity.setId(PROCESS_GROUP_ID); entity.setState("ENABLED"); - when(niFiProperties.isNode()).thenReturn(false); + when(properties.isNode()).thenReturn(false); resource.httpServletRequest = new MockHttpServletRequest(); final ArgumentCaptor>> revisionsCaptor = ArgumentCaptor.captor(); @@ -530,7 +545,7 @@ public void testClearBulletinsIncludesControllerServices() { entity.setFromTimestamp(Instant.now()); // components is null, so clearBulletins will gather all authorized components - when(niFiProperties.isNode()).thenReturn(false); + when(properties.isNode()).thenReturn(false); resource.httpServletRequest = new MockHttpServletRequest(); // Capture the function passed to filterComponents (called 3 times) @@ -601,6 +616,31 @@ public void testClearBulletinsIncludesControllerServices() { assertEquals(5, componentIds.size(), "Should have exactly 5 authorized components"); } + @Test + public void testGetConnectors() { + final ConnectorEntity connectorEntity = new ConnectorEntity(); + + when(serviceFacade.getConnectors()).thenReturn(Set.of(connectorEntity)); + + try (Response response = resource.getConnectors()) { + assertEquals(200, response.getStatus()); + assertNotNull(response.getEntity()); + } + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade).getConnectors(); + } + + @Test + public void testGetConnectorsNotAuthorized() { + doThrow(AccessDeniedException.class).when(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + + assertThrows(AccessDeniedException.class, () -> resource.getConnectors()); + + verify(serviceFacade).authorizeAccess(any(AuthorizeAccess.class)); + verify(serviceFacade, never()).getConnectors(); + } + private void setUpGetVersionDifference() { doReturn(getDifferences()).when(serviceFacade).getVersionDifference(anyString(), any(FlowVersionLocation.class), any(FlowVersionLocation.class)); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/StandardNiFiConnectorWebContextTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/StandardNiFiConnectorWebContextTest.java new file mode 100644 index 000000000000..a10a15a76196 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/StandardNiFiConnectorWebContextTest.java @@ -0,0 +1,219 @@ +/* + * 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.connector; + +import org.apache.nifi.authorization.AccessDeniedException; +import org.apache.nifi.authorization.AuthorizableLookup; +import org.apache.nifi.authorization.AuthorizationRequest; +import org.apache.nifi.authorization.AuthorizationResult; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.RequestAction; +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.authorization.user.NiFiUserDetails; +import org.apache.nifi.authorization.user.StandardNiFiUser; +import org.apache.nifi.components.connector.Connector; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.FrameworkFlowContext; +import org.apache.nifi.web.ConnectorWebMethod; +import org.apache.nifi.web.ConnectorWebMethod.AccessType; +import org.apache.nifi.web.NiFiConnectorWebContext.ConnectorWebContext; +import org.apache.nifi.web.connector.authorization.AuthorizingFlowContext; +import org.apache.nifi.web.dao.ConnectorDAO; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.springframework.security.core.Authentication; +import org.springframework.security.core.context.SecurityContext; +import org.springframework.security.core.context.SecurityContextHolder; + +import java.lang.reflect.Proxy; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNotSame; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.withSettings; + +@ExtendWith(MockitoExtension.class) +public class StandardNiFiConnectorWebContextTest { + + private static final String CONNECTOR_ID = "test-connector-id"; + private static final String USER_IDENTITY = "test-user"; + + @Mock + private ConnectorDAO connectorDAO; + + @Mock + private Authorizer authorizer; + + @Mock + private AuthorizableLookup authorizableLookup; + + @Mock + private ConnectorNode connectorNode; + + @Mock + private FrameworkFlowContext workingFlowContext; + + @Mock + private FrameworkFlowContext activeFlowContext; + + @Mock + private Authorizable connectorAuthorizable; + + @Mock + private SecurityContext securityContext; + + @Mock + private Authentication authentication; + + private StandardNiFiConnectorWebContext context; + + private TestConnector testConnectorMock; + private String lastWrittenValue; + + @BeforeEach + void setUp() { + context = new StandardNiFiConnectorWebContext(); + context.setConnectorDAO(connectorDAO); + context.setAuthorizer(authorizer); + context.setAuthorizableLookup(authorizableLookup); + + testConnectorMock = mock(TestConnector.class, withSettings().extraInterfaces(Connector.class).lenient()); + lenient().when(testConnectorMock.readData()).thenReturn("read-result"); + lenient().when(testConnectorMock.writeData(any())).thenAnswer(invocation -> { + lastWrittenValue = invocation.getArgument(0); + return null; + }); + + lenient().when(connectorDAO.getConnector(CONNECTOR_ID)).thenReturn(connectorNode); + lenient().when(connectorNode.getConnector()).thenReturn((Connector) testConnectorMock); + lenient().when(connectorNode.getWorkingFlowContext()).thenReturn(workingFlowContext); + lenient().when(connectorNode.getActiveFlowContext()).thenReturn(activeFlowContext); + lenient().when(authorizableLookup.getConnector(CONNECTOR_ID)).thenReturn(connectorAuthorizable); + lenient().when(authorizer.authorize(any(AuthorizationRequest.class))).thenReturn(AuthorizationResult.approved()); + + final NiFiUser user = new StandardNiFiUser.Builder().identity(USER_IDENTITY).build(); + final NiFiUserDetails userDetails = new NiFiUserDetails(user); + lenient().when(authentication.getPrincipal()).thenReturn(userDetails); + lenient().when(securityContext.getAuthentication()).thenReturn(authentication); + SecurityContextHolder.setContext(securityContext); + } + + @AfterEach + void tearDown() { + SecurityContextHolder.clearContext(); + } + + @Test + void testGetConnectorWebContextReturnsProxiedConnector() { + final ConnectorWebContext webContext = context.getConnectorWebContext(CONNECTOR_ID); + + assertNotNull(webContext); + assertNotNull(webContext.connector()); + assertTrue(Proxy.isProxyClass(webContext.connector().getClass())); + assertNotSame(testConnectorMock, webContext.connector()); + } + + @Test + void testGetConnectorWebContextReturnsWrappedFlowContexts() { + final ConnectorWebContext webContext = context.getConnectorWebContext(CONNECTOR_ID); + + assertNotNull(webContext.workingFlowContext()); + assertNotNull(webContext.activeFlowContext()); + assertTrue(webContext.workingFlowContext() instanceof AuthorizingFlowContext); + assertTrue(webContext.activeFlowContext() instanceof AuthorizingFlowContext); + } + + @Test + void testGetConnectorWebContextThrowsForNonExistentConnector() { + when(connectorDAO.getConnector("non-existent-id")).thenReturn(null); + + final IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, + () -> context.getConnectorWebContext("non-existent-id")); + assertEquals("Unable to find connector with id: non-existent-id", exception.getMessage()); + } + + @Test + void testProxiedConnectorEnforcesReadAuthorization() { + final ConnectorWebContext webContext = context.getConnectorWebContext(CONNECTOR_ID); + final TestConnector proxy = webContext.connector(); + + final String result = proxy.readData(); + + assertEquals("read-result", result); + verify(connectorAuthorizable).authorize(any(Authorizer.class), any(RequestAction.class), any(NiFiUser.class)); + } + + @Test + void testProxiedConnectorEnforcesWriteAuthorization() { + final ConnectorWebContext webContext = context.getConnectorWebContext(CONNECTOR_ID); + final TestConnector proxy = webContext.connector(); + + proxy.writeData("test-value"); + + assertEquals("test-value", lastWrittenValue); + verify(connectorAuthorizable).authorize(any(Authorizer.class), any(RequestAction.class), any(NiFiUser.class)); + } + + @Test + void testProxiedConnectorBlocksUnannotatedMethods() { + final ConnectorWebContext webContext = context.getConnectorWebContext(CONNECTOR_ID); + final TestConnector proxy = webContext.connector(); + + assertThrows(IllegalStateException.class, proxy::unannotatedMethod); + } + + @Test + void testProxiedConnectorPropagatesAuthorizationFailure() { + doThrow(new AccessDeniedException("Access denied")) + .when(connectorAuthorizable).authorize(any(Authorizer.class), any(RequestAction.class), any(NiFiUser.class)); + + final ConnectorWebContext webContext = context.getConnectorWebContext(CONNECTOR_ID); + final TestConnector proxy = webContext.connector(); + + assertThrows(AccessDeniedException.class, proxy::readData); + } + + /** + * Test interface representing a Connector with annotated methods. + * This interface is used with Mockito's extraInterfaces to create a mock + * that implements both this interface and Connector. + */ + public interface TestConnector { + + @ConnectorWebMethod(AccessType.READ) + String readData(); + + @ConnectorWebMethod(AccessType.WRITE) + Void writeData(String value); + + void unannotatedMethod(); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/authorization/AuthorizingConnectorInvocationHandlerTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/authorization/AuthorizingConnectorInvocationHandlerTest.java new file mode 100644 index 000000000000..e12b7366f9ed --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/authorization/AuthorizingConnectorInvocationHandlerTest.java @@ -0,0 +1,239 @@ +/* + * 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.connector.authorization; + +import org.apache.nifi.authorization.AccessDeniedException; +import org.apache.nifi.authorization.AuthorizableLookup; +import org.apache.nifi.authorization.AuthorizationRequest; +import org.apache.nifi.authorization.AuthorizationResult; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.RequestAction; +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.authorization.user.NiFiUserDetails; +import org.apache.nifi.authorization.user.StandardNiFiUser; +import org.apache.nifi.web.ConnectorWebMethod; +import org.apache.nifi.web.ConnectorWebMethod.AccessType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.springframework.security.core.Authentication; +import org.springframework.security.core.context.SecurityContext; +import org.springframework.security.core.context.SecurityContextHolder; + +import java.lang.reflect.Proxy; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.verify; + +@ExtendWith(MockitoExtension.class) +public class AuthorizingConnectorInvocationHandlerTest { + + private static final String CONNECTOR_ID = "test-connector-id"; + private static final String USER_IDENTITY = "test-user"; + + @Mock + private Authorizer authorizer; + + @Mock + private AuthorizableLookup authorizableLookup; + + @Mock + private Authorizable connectorAuthorizable; + + @Mock + private SecurityContext securityContext; + + @Mock + private Authentication authentication; + + private TestConnectorImpl connectorImpl; + + @BeforeEach + void setUp() { + connectorImpl = new TestConnectorImpl(); + lenient().when(authorizableLookup.getConnector(CONNECTOR_ID)).thenReturn(connectorAuthorizable); + + final NiFiUser user = new StandardNiFiUser.Builder().identity(USER_IDENTITY).build(); + final NiFiUserDetails userDetails = new NiFiUserDetails(user); + lenient().when(authentication.getPrincipal()).thenReturn(userDetails); + lenient().when(securityContext.getAuthentication()).thenReturn(authentication); + SecurityContextHolder.setContext(securityContext); + + lenient().when(authorizer.authorize(any(AuthorizationRequest.class))).thenReturn(AuthorizationResult.approved()); + } + + @AfterEach + void tearDown() { + SecurityContextHolder.clearContext(); + } + + @Test + void testReadMethodAuthorizesWithReadAction() { + final TestConnector proxy = createProxy(); + + final String result = proxy.readData(); + + assertEquals("read-result", result); + final ArgumentCaptor actionCaptor = ArgumentCaptor.forClass(RequestAction.class); + verify(connectorAuthorizable).authorize(eq(authorizer), actionCaptor.capture(), any(NiFiUser.class)); + assertEquals(RequestAction.READ, actionCaptor.getValue()); + } + + @Test + void testWriteMethodAuthorizesWithWriteAction() { + final TestConnector proxy = createProxy(); + + proxy.writeData("test-value"); + + assertEquals("test-value", connectorImpl.getLastWrittenValue()); + final ArgumentCaptor actionCaptor = ArgumentCaptor.forClass(RequestAction.class); + verify(connectorAuthorizable).authorize(eq(authorizer), actionCaptor.capture(), any(NiFiUser.class)); + assertEquals(RequestAction.WRITE, actionCaptor.getValue()); + } + + @Test + void testMethodWithoutAnnotationThrowsException() { + final TestConnector proxy = createProxy(); + + final IllegalStateException exception = assertThrows(IllegalStateException.class, proxy::unannotatedMethod); + assertEquals(String.format("Method [unannotatedMethod] on connector [%s] is not annotated with " + + "@ConnectorWebMethod and cannot be invoked through the Connector Web Context", CONNECTOR_ID), exception.getMessage()); + } + + @Test + void testAuthorizationFailurePropagatesAccessDeniedException() { + doThrow(new AccessDeniedException("Access denied for testing")) + .when(connectorAuthorizable).authorize(any(Authorizer.class), any(RequestAction.class), any(NiFiUser.class)); + + final TestConnector proxy = createProxy(); + + assertThrows(AccessDeniedException.class, proxy::readData); + } + + @Test + void testMethodArgumentsPassedCorrectly() { + final TestConnector proxy = createProxy(); + + final int result = proxy.processItems(List.of("a", "b", "c"), 10); + + assertEquals(13, result); + } + + @Test + void testDelegateExceptionUnwrapped() { + final TestConnector proxy = createProxy(); + + final RuntimeException exception = assertThrows(RuntimeException.class, proxy::throwingMethod); + assertEquals("Intentional test exception", exception.getMessage()); + } + + @Test + void testReadMethodWithReturnValue() { + final TestConnector proxy = createProxy(); + + final List result = proxy.getItems(); + + assertEquals(List.of("item1", "item2", "item3"), result); + } + + private TestConnector createProxy() { + final AuthorizingConnectorInvocationHandler handler = new AuthorizingConnectorInvocationHandler<>( + connectorImpl, CONNECTOR_ID, authorizer, authorizableLookup); + + return (TestConnector) Proxy.newProxyInstance( + TestConnector.class.getClassLoader(), + new Class[]{TestConnector.class}, + handler); + } + + /** + * Test interface representing a Connector with annotated methods. + */ + public interface TestConnector { + + @ConnectorWebMethod(AccessType.READ) + String readData(); + + @ConnectorWebMethod(AccessType.WRITE) + void writeData(String value); + + @ConnectorWebMethod(AccessType.READ) + List getItems(); + + @ConnectorWebMethod(AccessType.WRITE) + int processItems(List items, int multiplier); + + @ConnectorWebMethod(AccessType.READ) + void throwingMethod(); + + void unannotatedMethod(); + } + + /** + * Test implementation of the TestConnector interface. + */ + public static class TestConnectorImpl implements TestConnector { + + private String lastWrittenValue; + + @Override + public String readData() { + return "read-result"; + } + + @Override + public void writeData(final String value) { + this.lastWrittenValue = value; + } + + @Override + public List getItems() { + return List.of("item1", "item2", "item3"); + } + + @Override + public int processItems(final List items, final int multiplier) { + return items.size() + multiplier; + } + + @Override + public void throwingMethod() { + throw new RuntimeException("Intentional test exception"); + } + + @Override + public void unannotatedMethod() { + // This method is intentionally not annotated + } + + public String getLastWrittenValue() { + return lastWrittenValue; + } + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/authorization/AuthorizingFlowContextTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/authorization/AuthorizingFlowContextTest.java new file mode 100644 index 000000000000..ddc4f277ba56 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/authorization/AuthorizingFlowContextTest.java @@ -0,0 +1,167 @@ +/* + * 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.connector.authorization; + +import org.apache.nifi.authorization.AccessDeniedException; +import org.apache.nifi.authorization.AuthorizableLookup; +import org.apache.nifi.authorization.AuthorizationRequest; +import org.apache.nifi.authorization.AuthorizationResult; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.RequestAction; +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.authorization.user.NiFiUserDetails; +import org.apache.nifi.authorization.user.StandardNiFiUser; +import org.apache.nifi.components.connector.ConnectorConfigurationContext; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.components.FlowContextType; +import org.apache.nifi.components.connector.components.ParameterContextFacade; +import org.apache.nifi.components.connector.components.ProcessGroupFacade; +import org.apache.nifi.flow.Bundle; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.springframework.security.core.Authentication; +import org.springframework.security.core.context.SecurityContext; +import org.springframework.security.core.context.SecurityContextHolder; + +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class AuthorizingFlowContextTest { + + private static final String CONNECTOR_ID = "test-connector-id"; + private static final String USER_IDENTITY = "test-user"; + + @Mock + private FlowContext delegate; + + @Mock + private Authorizer authorizer; + + @Mock + private AuthorizableLookup authorizableLookup; + + @Mock + private Authorizable connectorAuthorizable; + + @Mock + private ProcessGroupFacade processGroupFacade; + + @Mock + private ParameterContextFacade parameterContextFacade; + + @Mock + private ConnectorConfigurationContext configurationContext; + + @Mock + private SecurityContext securityContext; + + @Mock + private Authentication authentication; + + private AuthorizingFlowContext authorizingFlowContext; + + @BeforeEach + void setUp() { + when(authorizableLookup.getConnector(CONNECTOR_ID)).thenReturn(connectorAuthorizable); + lenient().when(authorizer.authorize(any(AuthorizationRequest.class))).thenReturn(AuthorizationResult.approved()); + + final NiFiUser user = new StandardNiFiUser.Builder().identity(USER_IDENTITY).build(); + final NiFiUserDetails userDetails = new NiFiUserDetails(user); + when(authentication.getPrincipal()).thenReturn(userDetails); + when(securityContext.getAuthentication()).thenReturn(authentication); + SecurityContextHolder.setContext(securityContext); + + final ConnectorAuthorizationContext authContext = new ConnectorAuthorizationContext(CONNECTOR_ID, authorizer, authorizableLookup); + authorizingFlowContext = new AuthorizingFlowContext(delegate, authContext); + } + + @AfterEach + void tearDown() { + SecurityContextHolder.clearContext(); + } + + @Test + void testGetRootGroupAuthorizesReadAndReturnsWrappedFacade() { + when(delegate.getRootGroup()).thenReturn(processGroupFacade); + + final ProcessGroupFacade result = authorizingFlowContext.getRootGroup(); + + assertNotNull(result); + assertTrue(result instanceof AuthorizingProcessGroupFacade); + verify(connectorAuthorizable).authorize(any(Authorizer.class), any(RequestAction.class), any(NiFiUser.class)); + } + + @Test + void testGetParameterContextAuthorizesReadAndReturnsWrappedFacade() { + when(delegate.getParameterContext()).thenReturn(parameterContextFacade); + + final ParameterContextFacade result = authorizingFlowContext.getParameterContext(); + + assertNotNull(result); + assertTrue(result instanceof AuthorizingParameterContextFacade); + verify(connectorAuthorizable).authorize(any(Authorizer.class), any(RequestAction.class), any(NiFiUser.class)); + } + + @Test + void testGetConfigurationContextAuthorizesRead() { + when(delegate.getConfigurationContext()).thenReturn(configurationContext); + + final ConnectorConfigurationContext result = authorizingFlowContext.getConfigurationContext(); + + assertNotNull(result); + verify(connectorAuthorizable).authorize(any(Authorizer.class), any(RequestAction.class), any(NiFiUser.class)); + } + + @Test + void testGetTypeAuthorizesRead() { + when(delegate.getType()).thenReturn(FlowContextType.WORKING); + + authorizingFlowContext.getType(); + + verify(connectorAuthorizable).authorize(any(Authorizer.class), any(RequestAction.class), any(NiFiUser.class)); + } + + @Test + void testGetBundleAuthorizesRead() { + when(delegate.getBundle()).thenReturn(new Bundle("group", "artifact", "version")); + + authorizingFlowContext.getBundle(); + + verify(connectorAuthorizable).authorize(any(Authorizer.class), any(RequestAction.class), any(NiFiUser.class)); + } + + @Test + void testAuthorizationFailurePropagates() { + doThrow(new AccessDeniedException("Access denied")) + .when(connectorAuthorizable).authorize(any(Authorizer.class), any(RequestAction.class), any(NiFiUser.class)); + + assertThrows(AccessDeniedException.class, () -> authorizingFlowContext.getRootGroup()); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/authorization/AuthorizingParameterContextFacadeTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/authorization/AuthorizingParameterContextFacadeTest.java new file mode 100644 index 000000000000..6fe3cdfeb332 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/connector/authorization/AuthorizingParameterContextFacadeTest.java @@ -0,0 +1,162 @@ +/* + * 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.connector.authorization; + +import org.apache.nifi.asset.Asset; +import org.apache.nifi.authorization.AuthorizableLookup; +import org.apache.nifi.authorization.AuthorizationRequest; +import org.apache.nifi.authorization.AuthorizationResult; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.RequestAction; +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.authorization.user.NiFiUserDetails; +import org.apache.nifi.authorization.user.StandardNiFiUser; +import org.apache.nifi.components.connector.components.ParameterContextFacade; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.springframework.security.core.Authentication; +import org.springframework.security.core.context.SecurityContext; +import org.springframework.security.core.context.SecurityContextHolder; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.util.Collections; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class AuthorizingParameterContextFacadeTest { + + private static final String CONNECTOR_ID = "test-connector-id"; + private static final String USER_IDENTITY = "test-user"; + + @Mock + private ParameterContextFacade delegate; + + @Mock + private Authorizer authorizer; + + @Mock + private AuthorizableLookup authorizableLookup; + + @Mock + private Authorizable connectorAuthorizable; + + @Mock + private SecurityContext securityContext; + + @Mock + private Authentication authentication; + + @Mock + private Asset asset; + + private AuthorizingParameterContextFacade authorizingFacade; + + @BeforeEach + void setUp() { + when(authorizableLookup.getConnector(CONNECTOR_ID)).thenReturn(connectorAuthorizable); + lenient().when(authorizer.authorize(any(AuthorizationRequest.class))).thenReturn(AuthorizationResult.approved()); + + final NiFiUser user = new StandardNiFiUser.Builder().identity(USER_IDENTITY).build(); + final NiFiUserDetails userDetails = new NiFiUserDetails(user); + when(authentication.getPrincipal()).thenReturn(userDetails); + when(securityContext.getAuthentication()).thenReturn(authentication); + SecurityContextHolder.setContext(securityContext); + + final ConnectorAuthorizationContext authContext = new ConnectorAuthorizationContext(CONNECTOR_ID, authorizer, authorizableLookup); + authorizingFacade = new AuthorizingParameterContextFacade(delegate, authContext); + } + + @AfterEach + void tearDown() { + SecurityContextHolder.clearContext(); + } + + @Test + void testGetValueAuthorizesWithReadAction() { + when(delegate.getValue("param1")).thenReturn("value1"); + + final String result = authorizingFacade.getValue("param1"); + + assertEquals("value1", result); + final ArgumentCaptor actionCaptor = ArgumentCaptor.forClass(RequestAction.class); + verify(connectorAuthorizable).authorize(eq(authorizer), actionCaptor.capture(), any(NiFiUser.class)); + assertEquals(RequestAction.READ, actionCaptor.getValue()); + } + + @Test + void testGetDefinedParameterNamesAuthorizesWithReadAction() { + when(delegate.getDefinedParameterNames()).thenReturn(Set.of("param1", "param2")); + + final Set result = authorizingFacade.getDefinedParameterNames(); + + assertEquals(Set.of("param1", "param2"), result); + final ArgumentCaptor actionCaptor = ArgumentCaptor.forClass(RequestAction.class); + verify(connectorAuthorizable).authorize(eq(authorizer), actionCaptor.capture(), any(NiFiUser.class)); + assertEquals(RequestAction.READ, actionCaptor.getValue()); + } + + @Test + void testIsSensitiveAuthorizesWithReadAction() { + when(delegate.isSensitive("param1")).thenReturn(true); + + final boolean result = authorizingFacade.isSensitive("param1"); + + assertTrue(result); + final ArgumentCaptor actionCaptor = ArgumentCaptor.forClass(RequestAction.class); + verify(connectorAuthorizable).authorize(eq(authorizer), actionCaptor.capture(), any(NiFiUser.class)); + assertEquals(RequestAction.READ, actionCaptor.getValue()); + } + + @Test + void testUpdateParametersAuthorizesWithWriteAction() { + authorizingFacade.updateParameters(Collections.emptyList()); + + final ArgumentCaptor actionCaptor = ArgumentCaptor.forClass(RequestAction.class); + verify(connectorAuthorizable).authorize(eq(authorizer), actionCaptor.capture(), any(NiFiUser.class)); + assertEquals(RequestAction.WRITE, actionCaptor.getValue()); + verify(delegate).updateParameters(Collections.emptyList()); + } + + @Test + void testCreateAssetAuthorizesWithWriteAction() throws IOException { + final ByteArrayInputStream inputStream = new ByteArrayInputStream(new byte[0]); + when(delegate.createAsset(inputStream)).thenReturn(asset); + + final Asset result = authorizingFacade.createAsset(inputStream); + + assertEquals(asset, result); + final ArgumentCaptor actionCaptor = ArgumentCaptor.forClass(RequestAction.class); + verify(connectorAuthorizable).authorize(eq(authorizer), actionCaptor.capture(), any(NiFiUser.class)); + assertEquals(RequestAction.WRITE, actionCaptor.getValue()); + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerFacadeTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerFacadeTest.java new file mode 100644 index 000000000000..86a231d92c79 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerFacadeTest.java @@ -0,0 +1,306 @@ +/* + * 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.controller; + +import org.apache.nifi.authorization.AuthorizationResult; +import org.apache.nifi.authorization.Authorizer; +import org.apache.nifi.authorization.RequestAction; +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.authorization.user.NiFiUserDetails; +import org.apache.nifi.authorization.user.StandardNiFiUser; +import org.apache.nifi.c2.protocol.component.api.Bundle; +import org.apache.nifi.c2.protocol.component.api.ComponentManifest; +import org.apache.nifi.c2.protocol.component.api.ConnectorDefinition; +import org.apache.nifi.c2.protocol.component.api.RuntimeManifest; +import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.flow.FlowManager; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.manifest.RuntimeManifestService; +import org.apache.nifi.provenance.ProvenanceAuthorizableFactory; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.ProvenanceEventType; +import org.apache.nifi.provenance.ProvenanceRepository; +import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO; +import org.apache.nifi.web.api.dto.search.SearchResultsDTO; +import org.apache.nifi.web.search.query.SearchQuery; +import org.apache.nifi.web.search.query.SearchQueryParser; +import org.apache.nifi.web.security.token.NiFiAuthenticationToken; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.springframework.security.core.context.SecurityContextHolder; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class ControllerFacadeTest { + + private static final String SEARCH_TERM = "test-search"; + + @Mock + private SearchQueryParser searchQueryParser; + + @Mock + private ControllerSearchService controllerSearchService; + + @Mock + private ProcessGroup connectorProcessGroup; + + @Mock + private SearchQuery searchQuery; + + @Mock + private RuntimeManifestService runtimeManifestService; + + @Mock + private RuntimeManifest runtimeManifest; + + @Mock + private Bundle manifestBundle; + + @Mock + private ComponentManifest componentManifest; + + private ControllerFacade controllerFacade; + + private static final String TEST_GROUP = "org.apache.nifi"; + private static final String TEST_ARTIFACT = "nifi-test-nar"; + private static final String TEST_VERSION = "2.0.0"; + private static final String TEST_CONNECTOR_TYPE = "org.apache.nifi.connectors.TestConnector"; + + @BeforeEach + public void setUp() { + controllerFacade = new ControllerFacade(); + controllerFacade.setSearchQueryParser(searchQueryParser); + controllerFacade.setControllerSearchService(controllerSearchService); + controllerFacade.setRuntimeManifestService(runtimeManifestService); + + final NiFiUser user = new StandardNiFiUser.Builder().identity("test-user").build(); + SecurityContextHolder.getContext().setAuthentication(new NiFiAuthenticationToken(new NiFiUserDetails(user))); + } + + @Test + public void testSearchConnectorDelegatesSearchQueryParserWithConnectorProcessGroupAsRootAndActive() { + when(searchQueryParser.parse(eq(SEARCH_TERM), any(NiFiUser.class), eq(connectorProcessGroup), eq(connectorProcessGroup))).thenReturn(searchQuery); + when(searchQuery.getTerm()).thenReturn(SEARCH_TERM); + + final SearchResultsDTO results = controllerFacade.searchConnector(SEARCH_TERM, connectorProcessGroup); + + assertNotNull(results); + verify(searchQueryParser).parse(eq(SEARCH_TERM), any(NiFiUser.class), eq(connectorProcessGroup), eq(connectorProcessGroup)); + verify(controllerSearchService).search(eq(searchQuery), any(SearchResultsDTO.class)); + verify(controllerSearchService, never()).searchParameters(any(), any()); + } + + @Test + public void testSearchConnectorWithEmptyTermDoesNotInvokeSearchService() { + when(searchQueryParser.parse(eq(""), any(NiFiUser.class), eq(connectorProcessGroup), eq(connectorProcessGroup))).thenReturn(searchQuery); + when(searchQuery.getTerm()).thenReturn(""); + + final SearchResultsDTO results = controllerFacade.searchConnector("", connectorProcessGroup); + + assertNotNull(results); + verify(searchQueryParser).parse(eq(""), any(NiFiUser.class), eq(connectorProcessGroup), eq(connectorProcessGroup)); + } + + @Test + public void testSearchConnectorWithNullTermDoesNotInvokeSearchService() { + when(searchQueryParser.parse(eq(null), any(NiFiUser.class), eq(connectorProcessGroup), eq(connectorProcessGroup))).thenReturn(searchQuery); + when(searchQuery.getTerm()).thenReturn(null); + + final SearchResultsDTO results = controllerFacade.searchConnector(null, connectorProcessGroup); + + assertNotNull(results); + verify(searchQueryParser).parse(eq(null), any(NiFiUser.class), eq(connectorProcessGroup), eq(connectorProcessGroup)); + } + + @Test + public void testGetConnectorDefinitionReturnsDefinitionWhenFound() { + final ConnectorDefinition expectedDefinition = new ConnectorDefinition(); + expectedDefinition.setGroup(TEST_GROUP); + expectedDefinition.setArtifact(TEST_ARTIFACT); + expectedDefinition.setVersion(TEST_VERSION); + expectedDefinition.setType(TEST_CONNECTOR_TYPE); + + when(runtimeManifestService.getManifestForBundle(TEST_GROUP, TEST_ARTIFACT, TEST_VERSION)).thenReturn(runtimeManifest); + when(runtimeManifest.getBundles()).thenReturn(List.of(manifestBundle)); + when(manifestBundle.getComponentManifest()).thenReturn(componentManifest); + when(componentManifest.getConnectors()).thenReturn(List.of(expectedDefinition)); + + final ConnectorDefinition result = controllerFacade.getConnectorDefinition(TEST_GROUP, TEST_ARTIFACT, TEST_VERSION, TEST_CONNECTOR_TYPE); + + assertNotNull(result); + assertEquals(TEST_CONNECTOR_TYPE, result.getType()); + } + + @Test + public void testGetConnectorDefinitionReturnsNullWhenConnectorsListIsNull() { + when(runtimeManifestService.getManifestForBundle(TEST_GROUP, TEST_ARTIFACT, TEST_VERSION)).thenReturn(runtimeManifest); + when(runtimeManifest.getBundles()).thenReturn(List.of(manifestBundle)); + when(manifestBundle.getComponentManifest()).thenReturn(componentManifest); + when(componentManifest.getConnectors()).thenReturn(null); + + final ConnectorDefinition result = controllerFacade.getConnectorDefinition(TEST_GROUP, TEST_ARTIFACT, TEST_VERSION, TEST_CONNECTOR_TYPE); + + assertNull(result); + } + + @Test + public void testGetConnectorDefinitionReturnsNullWhenConnectorTypeNotFound() { + final ConnectorDefinition otherDefinition = new ConnectorDefinition(); + otherDefinition.setType("org.apache.nifi.connectors.OtherConnector"); + + when(runtimeManifestService.getManifestForBundle(TEST_GROUP, TEST_ARTIFACT, TEST_VERSION)).thenReturn(runtimeManifest); + when(runtimeManifest.getBundles()).thenReturn(List.of(manifestBundle)); + when(manifestBundle.getComponentManifest()).thenReturn(componentManifest); + when(componentManifest.getConnectors()).thenReturn(List.of(otherDefinition)); + + final ConnectorDefinition result = controllerFacade.getConnectorDefinition(TEST_GROUP, TEST_ARTIFACT, TEST_VERSION, TEST_CONNECTOR_TYPE); + + assertNull(result); + } + + @Test + public void testGetConnectorDefinitionReturnsNullWhenConnectorsListIsEmpty() { + when(runtimeManifestService.getManifestForBundle(TEST_GROUP, TEST_ARTIFACT, TEST_VERSION)).thenReturn(runtimeManifest); + when(runtimeManifest.getBundles()).thenReturn(List.of(manifestBundle)); + when(manifestBundle.getComponentManifest()).thenReturn(componentManifest); + when(componentManifest.getConnectors()).thenReturn(Collections.emptyList()); + + final ConnectorDefinition result = controllerFacade.getConnectorDefinition(TEST_GROUP, TEST_ARTIFACT, TEST_VERSION, TEST_CONNECTOR_TYPE); + + assertNull(result); + } + + @Test + public void testGetProvenanceEventSetsConnectorIdWhenComponentBelongsToConnector() throws IOException { + final String componentId = "test-component-id"; + final String groupId = "test-group-id"; + final String connectorId = "test-connector-id"; + + final ControllerFacade facade = createProvenanceFacade(componentId, groupId, Optional.of(connectorId)); + + final ProvenanceEventDTO result = facade.getProvenanceEvent(1L); + + assertNotNull(result); + assertEquals(connectorId, result.getConnectorId()); + assertEquals(groupId, result.getGroupId()); + assertEquals(componentId, result.getComponentId()); + } + + @Test + public void testGetProvenanceEventConnectorIdIsNullWhenComponentNotManagedByConnector() throws IOException { + final String componentId = "test-component-id"; + final String groupId = "test-group-id"; + + final ControllerFacade facade = createProvenanceFacade(componentId, groupId, Optional.empty()); + + final ProvenanceEventDTO result = facade.getProvenanceEvent(1L); + + assertNotNull(result); + assertNull(result.getConnectorId()); + assertEquals(groupId, result.getGroupId()); + assertEquals(componentId, result.getComponentId()); + } + + /** + * Creates a ControllerFacade wired with mocks sufficient to test provenance event creation + * through the connectable branch of setComponentDetails. + */ + private ControllerFacade createProvenanceFacade(final String componentId, final String groupId, + final Optional connectorIdentifier) throws IOException { + // Mock the ProvenanceEventRecord + final ProvenanceEventRecord event = mock(ProvenanceEventRecord.class); + when(event.getEventId()).thenReturn(1L); + when(event.getEventTime()).thenReturn(System.currentTimeMillis()); + when(event.getEventType()).thenReturn(ProvenanceEventType.CREATE); + when(event.getFlowFileUuid()).thenReturn("test-flowfile-uuid"); + when(event.getFileSize()).thenReturn(1024L); + when(event.getComponentId()).thenReturn(componentId); + when(event.getComponentType()).thenReturn("TestProcessor"); + when(event.getUpdatedAttributes()).thenReturn(Map.of()); + when(event.getPreviousAttributes()).thenReturn(Map.of()); + when(event.getParentUuids()).thenReturn(List.of()); + when(event.getChildUuids()).thenReturn(List.of()); + when(event.getEventDuration()).thenReturn(-1L); + when(event.getLineageStartDate()).thenReturn(0L); + + // Mock the ProcessGroup that the connectable belongs to + final ProcessGroup processGroup = mock(ProcessGroup.class); + when(processGroup.getIdentifier()).thenReturn(groupId); + when(processGroup.getConnectorIdentifier()).thenReturn(connectorIdentifier); + + // Mock a Connectable found in the flow + final Connectable connectable = mock(Connectable.class); + when(connectable.getProcessGroup()).thenReturn(processGroup); + when(connectable.getName()).thenReturn("Test Component"); + when(connectable.checkAuthorization(any(), any(), any())).thenReturn(AuthorizationResult.approved()); + + // Mock FlowManager + final FlowManager flowManager = mock(FlowManager.class); + when(flowManager.findConnectable(componentId)).thenReturn(connectable); + when(flowManager.getRootGroup()).thenReturn(mock(ProcessGroup.class)); + + // Mock ProvenanceRepository + final ProvenanceRepository provenanceRepository = mock(ProvenanceRepository.class); + when(provenanceRepository.getEvent(eq(1L), any(NiFiUser.class))).thenReturn(event); + + // Mock data authorization as not approved so we skip content availability logic + final Authorizable dataAuthorizable = mock(Authorizable.class); + when(dataAuthorizable.checkAuthorization(any(), eq(RequestAction.READ), any(NiFiUser.class), any())) + .thenReturn(AuthorizationResult.denied("test")); + + // Mock ProvenanceAuthorizableFactory + final ProvenanceAuthorizableFactory provenanceAuthorizableFactory = mock(ProvenanceAuthorizableFactory.class); + when(provenanceAuthorizableFactory.createLocalDataAuthorizable(componentId)).thenReturn(dataAuthorizable); + + // Mock FlowController + final FlowController flowController = mock(FlowController.class); + when(flowController.getFlowManager()).thenReturn(flowManager); + when(flowController.getProvenanceRepository()).thenReturn(provenanceRepository); + when(flowController.getProvenanceAuthorizableFactory()).thenReturn(provenanceAuthorizableFactory); + + // Mock Authorizer + final Authorizer authorizer = mock(Authorizer.class); + + // Build the ControllerFacade + final ControllerFacade facade = new ControllerFacade(); + facade.setFlowController(flowController); + facade.setAuthorizer(authorizer); + + return facade; + } +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardConnectionDAOTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardConnectionDAOTest.java new file mode 100644 index 000000000000..7aa6dc19d224 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardConnectionDAOTest.java @@ -0,0 +1,196 @@ +/* + * 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.dao.impl; + +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorRepository; +import org.apache.nifi.components.connector.FrameworkFlowContext; +import org.apache.nifi.connectable.Connection; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.flow.FlowManager; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.web.ResourceNotFoundException; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; + +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) +class StandardConnectionDAOTest { + + private StandardConnectionDAO connectionDAO; + + @Mock + private FlowController flowController; + + @Mock + private FlowManager flowManager; + + @Mock + private ProcessGroup rootGroup; + + @Mock + private ConnectorRepository connectorRepository; + + @Mock + private ConnectorNode connectorNode; + + @Mock + private FrameworkFlowContext frameworkFlowContext; + + @Mock + private ProcessGroup connectorManagedGroup; + + @Mock + private Connection rootConnection; + + @Mock + private Connection connectorConnection; + + private static final String ROOT_CONNECTION_ID = "root-connection-id"; + private static final String CONNECTOR_CONNECTION_ID = "connector-connection-id"; + private static final String NON_EXISTENT_ID = "non-existent-id"; + + @BeforeEach + void setUp() { + connectionDAO = new StandardConnectionDAO(); + connectionDAO.setFlowController(flowController); + + when(flowController.getFlowManager()).thenReturn(flowManager); + when(flowManager.getRootGroup()).thenReturn(rootGroup); + when(flowController.getConnectorRepository()).thenReturn(connectorRepository); + + // Setup root group connection + when(rootGroup.findConnection(ROOT_CONNECTION_ID)).thenReturn(rootConnection); + when(rootGroup.findConnection(CONNECTOR_CONNECTION_ID)).thenReturn(null); + when(rootGroup.findConnection(NON_EXISTENT_ID)).thenReturn(null); + + // Setup connector managed group + when(connectorRepository.getConnectors()).thenReturn(List.of(connectorNode)); + when(connectorNode.getActiveFlowContext()).thenReturn(frameworkFlowContext); + when(frameworkFlowContext.getManagedProcessGroup()).thenReturn(connectorManagedGroup); + when(connectorManagedGroup.findConnection(CONNECTOR_CONNECTION_ID)).thenReturn(connectorConnection); + when(connectorManagedGroup.findConnection(ROOT_CONNECTION_ID)).thenReturn(null); + when(connectorManagedGroup.findConnection(NON_EXISTENT_ID)).thenReturn(null); + } + + @Test + void testGetConnectionFromRootGroup() { + final Connection result = connectionDAO.getConnection(ROOT_CONNECTION_ID); + + assertEquals(rootConnection, result); + } + + @Test + void testGetConnectionFromRootGroupWithIncludeConnectorManagedFalse() { + final Connection result = connectionDAO.getConnection(ROOT_CONNECTION_ID, false); + + assertEquals(rootConnection, result); + } + + @Test + void testGetConnectionFromRootGroupWithIncludeConnectorManagedTrue() { + final Connection result = connectionDAO.getConnection(ROOT_CONNECTION_ID, true); + + assertEquals(rootConnection, result); + } + + @Test + void testGetConnectionFromConnectorManagedGroupWithIncludeConnectorManagedTrue() { + final Connection result = connectionDAO.getConnection(CONNECTOR_CONNECTION_ID, true); + + assertEquals(connectorConnection, result); + } + + @Test + void testGetConnectionFromConnectorManagedGroupWithIncludeConnectorManagedFalseThrows() { + assertThrows(ResourceNotFoundException.class, () -> + connectionDAO.getConnection(CONNECTOR_CONNECTION_ID, false) + ); + } + + @Test + void testGetConnectionWithDefaultDoesNotFindConnectorManagedConnection() { + assertThrows(ResourceNotFoundException.class, () -> + connectionDAO.getConnection(CONNECTOR_CONNECTION_ID) + ); + } + + @Test + void testGetConnectionWithNonExistentIdThrows() { + assertThrows(ResourceNotFoundException.class, () -> + connectionDAO.getConnection(NON_EXISTENT_ID, true) + ); + } + + @Test + void testHasConnectionInRootGroup() { + assertTrue(connectionDAO.hasConnection(ROOT_CONNECTION_ID)); + } + + @Test + void testHasConnectionNotInRootGroup() { + // hasConnection only checks the root group, not connector-managed groups + assertFalse(connectionDAO.hasConnection(CONNECTOR_CONNECTION_ID)); + } + + @Test + void testHasConnectionWithNonExistentId() { + assertFalse(connectionDAO.hasConnection(NON_EXISTENT_ID)); + } + + @Test + void testGetConnectionFromConnectorWithNullActiveFlowContext() { + // Simulate connector with no active flow context + when(connectorNode.getActiveFlowContext()).thenReturn(null); + + assertThrows(ResourceNotFoundException.class, () -> + connectionDAO.getConnection(CONNECTOR_CONNECTION_ID, true) + ); + } + + @Test + void testGetConnectionWithMultipleConnectors() { + // Setup a second connector + final ConnectorNode connectorNode2 = org.mockito.Mockito.mock(ConnectorNode.class); + final FrameworkFlowContext flowContext2 = org.mockito.Mockito.mock(FrameworkFlowContext.class); + final ProcessGroup managedGroup2 = org.mockito.Mockito.mock(ProcessGroup.class); + final Connection connectionInSecondConnector = org.mockito.Mockito.mock(Connection.class); + final String secondConnectorConnectionId = "second-connector-connection-id"; + + when(connectorRepository.getConnectors()).thenReturn(List.of(connectorNode, connectorNode2)); + when(connectorNode2.getActiveFlowContext()).thenReturn(flowContext2); + when(flowContext2.getManagedProcessGroup()).thenReturn(managedGroup2); + when(managedGroup2.findConnection(secondConnectorConnectionId)).thenReturn(connectionInSecondConnector); + + final Connection result = connectionDAO.getConnection(secondConnectorConnectionId, true); + + assertEquals(connectionInSecondConnector, result); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardConnectorDAOTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardConnectorDAOTest.java new file mode 100644 index 000000000000..d1c589fd2296 --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardConnectorDAOTest.java @@ -0,0 +1,311 @@ +/* + * 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.dao.impl; + +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.DescribedValue; +import org.apache.nifi.components.connector.ConnectorConfiguration; +import org.apache.nifi.components.connector.ConnectorNode; +import org.apache.nifi.components.connector.ConnectorRepository; +import org.apache.nifi.components.connector.ConnectorUpdateContext; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.FrameworkFlowContext; +import org.apache.nifi.components.connector.MutableConnectorConfigurationContext; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.web.NiFiCoreException; +import org.apache.nifi.web.ResourceNotFoundException; +import org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorDTO; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.InOrder; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; + +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.inOrder; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) +class StandardConnectorDAOTest { + + private StandardConnectorDAO connectorDAO; + + @Mock + private FlowController flowController; + + @Mock + private ConnectorRepository connectorRepository; + + @Mock + private ConnectorNode connectorNode; + + @Mock + private ConnectorUpdateContext connectorUpdateContext; + + @Mock + private FrameworkFlowContext frameworkFlowContext; + + @Mock + private MutableConnectorConfigurationContext configurationContext; + + @Mock + private ConnectorConfiguration connectorConfiguration; + + private static final String CONNECTOR_ID = "test-connector-id"; + private static final String STEP_NAME = "test-step"; + private static final String PROPERTY_NAME = "test-property"; + + @BeforeEach + void setUp() { + connectorDAO = new StandardConnectorDAO(); + connectorDAO.setFlowController(flowController); + + when(flowController.getConnectorRepository()).thenReturn(connectorRepository); + + final MutableConnectorConfigurationContext configContext = mock(MutableConnectorConfigurationContext.class); + when(configContext.toConnectorConfiguration()).thenReturn(mock(ConnectorConfiguration.class)); + final FrameworkFlowContext activeContext = mock(FrameworkFlowContext.class); + when(activeContext.getConfigurationContext()).thenReturn(configContext); + when(connectorNode.getActiveFlowContext()).thenReturn(activeContext); + } + + @Test + void testApplyConnectorUpdate() throws Exception { + when(connectorRepository.getConnector(CONNECTOR_ID)).thenReturn(connectorNode); + + connectorDAO.applyConnectorUpdate(CONNECTOR_ID, connectorUpdateContext); + + verify(connectorRepository).getConnector(CONNECTOR_ID); + verify(connectorRepository).applyUpdate(connectorNode, connectorUpdateContext); + } + + @Test + void testApplyConnectorUpdateWithNonExistentConnector() throws Exception { + when(connectorRepository.getConnector(CONNECTOR_ID)).thenReturn(null); + + final ResourceNotFoundException exception = assertThrows(ResourceNotFoundException.class, () -> + connectorDAO.applyConnectorUpdate(CONNECTOR_ID, connectorUpdateContext) + ); + + assertEquals("Could not find Connector with ID " + CONNECTOR_ID, exception.getMessage()); + verify(connectorRepository).getConnector(CONNECTOR_ID); + verify(connectorRepository, never()).applyUpdate(any(ConnectorNode.class), any(ConnectorUpdateContext.class)); + } + + @Test + void testApplyConnectorUpdateWithFlowUpdateException() throws Exception { + when(connectorRepository.getConnector(CONNECTOR_ID)).thenReturn(connectorNode); + doThrow(new FlowUpdateException("Flow update failed")).when(connectorRepository).applyUpdate(connectorNode, connectorUpdateContext); + + final NiFiCoreException exception = assertThrows(NiFiCoreException.class, () -> + connectorDAO.applyConnectorUpdate(CONNECTOR_ID, connectorUpdateContext) + ); + + assertEquals("Failed to apply connector update: org.apache.nifi.components.connector.FlowUpdateException: Flow update failed", exception.getMessage()); + verify(connectorRepository).getConnector(CONNECTOR_ID); + verify(connectorRepository).applyUpdate(connectorNode, connectorUpdateContext); + } + + @Test + void testApplyConnectorUpdateWithRuntimeException() throws Exception { + when(connectorRepository.getConnector(CONNECTOR_ID)).thenReturn(connectorNode); + doThrow(new RuntimeException("Test exception")).when(connectorRepository).applyUpdate(connectorNode, connectorUpdateContext); + + final NiFiCoreException exception = assertThrows(NiFiCoreException.class, () -> + connectorDAO.applyConnectorUpdate(CONNECTOR_ID, connectorUpdateContext) + ); + + assertEquals("Failed to apply connector update: java.lang.RuntimeException: Test exception", exception.getMessage()); + verify(connectorRepository).getConnector(CONNECTOR_ID); + verify(connectorRepository).applyUpdate(connectorNode, connectorUpdateContext); + } + + @Test + void testApplyConnectorUpdateWithNullException() throws Exception { + when(connectorRepository.getConnector(CONNECTOR_ID)).thenReturn(connectorNode); + doThrow(new RuntimeException()).when(connectorRepository).applyUpdate(connectorNode, connectorUpdateContext); + + final NiFiCoreException exception = assertThrows(NiFiCoreException.class, () -> + connectorDAO.applyConnectorUpdate(CONNECTOR_ID, connectorUpdateContext) + ); + + assertEquals("Failed to apply connector update: java.lang.RuntimeException", exception.getMessage()); + verify(connectorRepository).getConnector(CONNECTOR_ID); + verify(connectorRepository).applyUpdate(connectorNode, connectorUpdateContext); + } + + @Test + void testGetConnectorWithNonExistentId() { + when(connectorRepository.getConnector(CONNECTOR_ID)).thenReturn(null); + + assertThrows(ResourceNotFoundException.class, () -> + connectorDAO.getConnector(CONNECTOR_ID) + ); + + verify(connectorRepository).getConnector(CONNECTOR_ID); + } + + @Test + void testGetConnectorSuccess() { + when(connectorRepository.getConnector(CONNECTOR_ID)).thenReturn(connectorNode); + + final ConnectorNode result = connectorDAO.getConnector(CONNECTOR_ID); + + assertEquals(connectorNode, result); + verify(connectorRepository).getConnector(CONNECTOR_ID); + } + + @Test + void testFetchAllowableValuesWithoutFilter() { + final List expectedValues = List.of( + new AllowableValue("value1", "Value 1", "First value"), + new AllowableValue("value2", "Value 2", "Second value") + ); + when(connectorRepository.getConnector(CONNECTOR_ID)).thenReturn(connectorNode); + when(connectorNode.fetchAllowableValues(STEP_NAME, PROPERTY_NAME)).thenReturn(expectedValues); + + final List result = connectorDAO.fetchAllowableValues(CONNECTOR_ID, STEP_NAME, PROPERTY_NAME, null); + + assertEquals(expectedValues, result); + verify(connectorNode).fetchAllowableValues(STEP_NAME, PROPERTY_NAME); + verify(connectorNode, never()).fetchAllowableValues(any(), any(), any()); + } + + @Test + void testFetchAllowableValuesWithEmptyFilter() { + final List expectedValues = List.of( + new AllowableValue("value1", "Value 1", "First value") + ); + when(connectorRepository.getConnector(CONNECTOR_ID)).thenReturn(connectorNode); + when(connectorNode.fetchAllowableValues(STEP_NAME, PROPERTY_NAME)).thenReturn(expectedValues); + + final List result = connectorDAO.fetchAllowableValues(CONNECTOR_ID, STEP_NAME, PROPERTY_NAME, ""); + + assertEquals(expectedValues, result); + verify(connectorNode).fetchAllowableValues(STEP_NAME, PROPERTY_NAME); + verify(connectorNode, never()).fetchAllowableValues(any(), any(), any()); + } + + @Test + void testFetchAllowableValuesWithFilter() { + final String filter = "test-filter"; + final List expectedValues = List.of( + new AllowableValue("filtered-value", "Filtered Value", "Filtered result") + ); + when(connectorRepository.getConnector(CONNECTOR_ID)).thenReturn(connectorNode); + when(connectorNode.fetchAllowableValues(STEP_NAME, PROPERTY_NAME, filter)).thenReturn(expectedValues); + + final List result = connectorDAO.fetchAllowableValues(CONNECTOR_ID, STEP_NAME, PROPERTY_NAME, filter); + + assertEquals(expectedValues, result); + verify(connectorNode).fetchAllowableValues(STEP_NAME, PROPERTY_NAME, filter); + verify(connectorNode, never()).fetchAllowableValues(STEP_NAME, PROPERTY_NAME); + } + + @Test + void testFetchAllowableValuesWithNonExistentConnector() { + when(connectorRepository.getConnector(CONNECTOR_ID)).thenReturn(null); + + assertThrows(ResourceNotFoundException.class, () -> + connectorDAO.fetchAllowableValues(CONNECTOR_ID, STEP_NAME, PROPERTY_NAME, null) + ); + + verify(connectorRepository).getConnector(CONNECTOR_ID); + } + + @Test + void testVerifyConfigurationStepSyncsAssetsBeforeVerification() { + when(connectorRepository.getConnector(CONNECTOR_ID)).thenReturn(connectorNode); + final ConfigurationStepConfigurationDTO stepConfigDto = new ConfigurationStepConfigurationDTO(); + + connectorDAO.verifyConfigurationStep(CONNECTOR_ID, STEP_NAME, stepConfigDto); + + final InOrder inOrder = inOrder(connectorRepository, connectorNode); + inOrder.verify(connectorRepository).syncAssetsFromProvider(connectorNode); + inOrder.verify(connectorNode).verifyConfigurationStep(any(), any()); + } + + @Test + void testDeleteConnectorRemovesAssetsAndConnector() { + connectorDAO.deleteConnector(CONNECTOR_ID); + + verify(connectorRepository).deleteAssets(CONNECTOR_ID); + verify(connectorRepository).removeConnector(CONNECTOR_ID); + } + + @Test + void testDeleteConnectorDoesNotRemoveConnectorWhenAssetDeletionFails() { + doThrow(new RuntimeException("Asset deletion failed")).when(connectorRepository).deleteAssets(CONNECTOR_ID); + + assertThrows(RuntimeException.class, () -> + connectorDAO.deleteConnector(CONNECTOR_ID) + ); + + verify(connectorRepository).deleteAssets(CONNECTOR_ID); + verify(connectorRepository, never()).removeConnector(any()); + } + + @Test + void testVerifyCreateDelegatesToRepository() { + final ConnectorDTO connectorDTO = new ConnectorDTO(); + connectorDTO.setId(CONNECTOR_ID); + + connectorDAO.verifyCreate(connectorDTO); + + verify(connectorRepository).verifyCreate(CONNECTOR_ID); + } + + @Test + void testVerifyCreateWithRepositoryThrowingException() { + final ConnectorDTO connectorDTO = new ConnectorDTO(); + connectorDTO.setId(CONNECTOR_ID); + + doThrow(new IllegalStateException("A Connector already exists with ID %s".formatted(CONNECTOR_ID))) + .when(connectorRepository).verifyCreate(CONNECTOR_ID); + + final IllegalStateException exception = assertThrows(IllegalStateException.class, () -> + connectorDAO.verifyCreate(connectorDTO) + ); + + assertEquals("A Connector already exists with ID %s".formatted(CONNECTOR_ID), exception.getMessage()); + } + + @Test + void testVerifyCreateWithNullId() { + final ConnectorDTO connectorDTO = new ConnectorDTO(); + connectorDTO.setId(null); + + connectorDAO.verifyCreate(connectorDTO); + + verify(connectorRepository, never()).verifyCreate(any()); + } + +} + diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAOTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAOTest.java new file mode 100644 index 000000000000..cf5441f199ab --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAOTest.java @@ -0,0 +1,130 @@ +/* + * 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.dao.impl; + +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.flow.FlowManager; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.web.ResourceNotFoundException; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) +class StandardProcessGroupDAOTest { + + private StandardProcessGroupDAO processGroupDAO; + + @Mock + private FlowController flowController; + + @Mock + private FlowManager flowManager; + + @Mock + private ProcessGroup rootGroup; + + @Mock + private ProcessGroup connectorManagedGroup; + + private static final String ROOT_GROUP_ID = "root-group-id"; + private static final String CONNECTOR_GROUP_ID = "connector-group-id"; + private static final String NON_EXISTENT_ID = "non-existent-id"; + + @BeforeEach + void setUp() { + processGroupDAO = new StandardProcessGroupDAO(); + processGroupDAO.setFlowController(flowController); + + when(flowController.getFlowManager()).thenReturn(flowManager); + + // Setup root group lookup (non-connector managed) + when(flowManager.getGroup(ROOT_GROUP_ID, null)).thenReturn(rootGroup); + when(flowManager.getGroup(CONNECTOR_GROUP_ID, null)).thenReturn(null); + when(flowManager.getGroup(NON_EXISTENT_ID, null)).thenReturn(null); + + // Setup connector-managed group lookup (includes all groups) + when(flowManager.getGroup(ROOT_GROUP_ID)).thenReturn(rootGroup); + when(flowManager.getGroup(CONNECTOR_GROUP_ID)).thenReturn(connectorManagedGroup); + when(flowManager.getGroup(NON_EXISTENT_ID)).thenReturn(null); + } + + @Test + void testGetProcessGroupFromRootHierarchy() { + final ProcessGroup result = processGroupDAO.getProcessGroup(ROOT_GROUP_ID); + + assertEquals(rootGroup, result); + } + + @Test + void testGetProcessGroupFromRootHierarchyWithIncludeConnectorManagedFalse() { + final ProcessGroup result = processGroupDAO.getProcessGroup(ROOT_GROUP_ID, false); + + assertEquals(rootGroup, result); + } + + @Test + void testGetProcessGroupFromRootHierarchyWithIncludeConnectorManagedTrue() { + final ProcessGroup result = processGroupDAO.getProcessGroup(ROOT_GROUP_ID, true); + + assertEquals(rootGroup, result); + } + + @Test + void testGetProcessGroupFromConnectorManagedWithIncludeConnectorManagedTrue() { + final ProcessGroup result = processGroupDAO.getProcessGroup(CONNECTOR_GROUP_ID, true); + + assertEquals(connectorManagedGroup, result); + } + + @Test + void testGetProcessGroupFromConnectorManagedWithIncludeConnectorManagedFalseThrows() { + assertThrows(ResourceNotFoundException.class, () -> + processGroupDAO.getProcessGroup(CONNECTOR_GROUP_ID, false) + ); + } + + @Test + void testGetProcessGroupWithDefaultDoesNotFindConnectorManagedGroup() { + assertThrows(ResourceNotFoundException.class, () -> + processGroupDAO.getProcessGroup(CONNECTOR_GROUP_ID) + ); + } + + @Test + void testGetProcessGroupWithNonExistentIdThrows() { + assertThrows(ResourceNotFoundException.class, () -> + processGroupDAO.getProcessGroup(NON_EXISTENT_ID, true) + ); + } + + @Test + void testGetProcessGroupWithNonExistentIdAndIncludeConnectorManagedFalseThrows() { + assertThrows(ResourceNotFoundException.class, () -> + processGroupDAO.getProcessGroup(NON_EXISTENT_ID, false) + ); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardProcessorDAOTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardProcessorDAOTest.java index 30c9fdfdc979..1ae61d3dce6c 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardProcessorDAOTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/StandardProcessorDAOTest.java @@ -210,7 +210,7 @@ void testCreateProcessor(@TempDir final File tempDir) { final Bundle bundle = new Bundle(bundleDetails, getClass().getClassLoader()); final List bundles = List.of(bundle); - when(flowManager.getGroup(eq(groupId))).thenReturn(processGroup); + when(flowManager.getGroup(eq(groupId), eq(null))).thenReturn(processGroup); when(flowController.getExtensionManager()).thenReturn(extensionManager); when(flowManager.createProcessor(eq(processorType), eq(id), eq(bundleCoordinate))).thenReturn(processorNode); when(extensionManager.getBundles(eq(processorType))).thenReturn(bundles); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/TestStandardParameterContextDAO.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/TestStandardParameterContextDAO.java index bbba23b38bfa..60531f909d70 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/TestStandardParameterContextDAO.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/TestStandardParameterContextDAO.java @@ -84,7 +84,7 @@ void setUp() { final NiFiUserDetails userDetail = new NiFiUserDetails(user); when(authentication.getPrincipal()).thenReturn(userDetail); - final ParameterReferenceManager parameterReferenceManager = new StandardParameterReferenceManager(flowController.getFlowManager()); + final ParameterReferenceManager parameterReferenceManager = new StandardParameterReferenceManager(() -> flowController.getFlowManager().getRootGroup()); final FlowManager flowManager = flowController.getFlowManager(); final StandardParameterContextManager parameterContextLookup = new StandardParameterContextManager(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/TestStandardProcessGroupDAO.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/TestStandardProcessGroupDAO.java index 9ca3f0a837f0..85c55f88f9f7 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/TestStandardProcessGroupDAO.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/dao/impl/TestStandardProcessGroupDAO.java @@ -59,21 +59,12 @@ void setUp() { testSubject = new StandardProcessGroupDAO(); testSubject.setFlowController(flowController); - when(flowController - .getFlowManager() - .getGroup(PARENT_PROCESS_GROUP_ID) - ).thenReturn(parentProcessGroup); - - when(flowController - .getFlowManager() - .getParameterContextManager() - .getParameterContext(PARAMETER_CONTEXT_ID) - ).thenReturn(parameterContext); + when(flowController.getFlowManager().getGroup(PARENT_PROCESS_GROUP_ID, null)).thenReturn(parentProcessGroup); + when(flowController.getFlowManager().getParameterContextManager().getParameterContext(PARAMETER_CONTEXT_ID)).thenReturn(parameterContext); } @Test public void testCreateProcessGroup() { - //GIVEN ParameterContextReferenceEntity parameterContextReferenceEntity = new ParameterContextReferenceEntity(); parameterContextReferenceEntity.setId(PARAMETER_CONTEXT_ID); @@ -84,10 +75,8 @@ public void testCreateProcessGroup() { processGroupDTO.setPosition(new PositionDTO(10.0, 20.0)); processGroupDTO.setParameterContext(parameterContextReferenceEntity); - //WHEN ProcessGroup createdProcessGroup = testSubject.createProcessGroup(PARENT_PROCESS_GROUP_ID, processGroupDTO); - //THEN verify(createdProcessGroup).setParent(parentProcessGroup); verify(createdProcessGroup).setParameterContext(parameterContext); verify(createdProcessGroup).setName(PROCESS_GROUP_NAME); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/filter/ConnectorRequestContextFilterTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/filter/ConnectorRequestContextFilterTest.java new file mode 100644 index 000000000000..1ceef675efbf --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/filter/ConnectorRequestContextFilterTest.java @@ -0,0 +1,178 @@ +/* + * 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.filter; + +import jakarta.servlet.FilterChain; +import jakarta.servlet.ServletResponse; +import org.apache.nifi.authorization.user.NiFiUser; +import org.apache.nifi.authorization.user.NiFiUserDetails; +import org.apache.nifi.components.connector.ConnectorRequestContext; +import org.apache.nifi.components.connector.ConnectorRequestContextHolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.quality.Strictness; +import org.springframework.mock.web.MockHttpServletRequest; +import org.springframework.security.authentication.TestingAuthenticationToken; +import org.springframework.security.core.context.SecurityContextHolder; + +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.withSettings; + +@ExtendWith(MockitoExtension.class) +class ConnectorRequestContextFilterTest { + + private static final String TEST_USER = "test.user@apache.org"; + private static final String TOKEN_HEADER = "API-Key"; + private static final String TOKEN_VALUE = "test-api-key-123"; + private static final String ROLE_HEADER = "API-Role"; + private static final String ROLE_VALUE = "SERVICE_ROLE"; + + @Mock + private FilterChain filterChain; + + @Mock + private ServletResponse response; + + @AfterEach + void cleanup() { + ConnectorRequestContextHolder.clearContext(); + SecurityContextHolder.clearContext(); + } + + @Test + void testContextIsSetDuringFilterChain() throws Exception { + final MockHttpServletRequest request = new MockHttpServletRequest(); + request.addHeader(TOKEN_HEADER, TOKEN_VALUE); + request.addHeader(ROLE_HEADER, ROLE_VALUE); + setUpSecurityContext(); + + final ConnectorRequestContext[] capturedContext = new ConnectorRequestContext[1]; + doAnswer(invocation -> { + capturedContext[0] = ConnectorRequestContextHolder.getContext(); + return null; + }).when(filterChain).doFilter(any(), any()); + + final ConnectorRequestContextFilter filter = new ConnectorRequestContextFilter(); + filter.doFilter(request, response, filterChain); + + assertNotNull(capturedContext[0]); + assertNotNull(capturedContext[0].getAuthenticatedUser()); + assertEquals(TEST_USER, capturedContext[0].getAuthenticatedUser().getIdentity()); + + assertTrue(capturedContext[0].hasRequestHeader(TOKEN_HEADER)); + assertTrue(capturedContext[0].hasRequestHeader(ROLE_HEADER)); + assertEquals(TOKEN_VALUE, capturedContext[0].getFirstRequestHeaderValue(TOKEN_HEADER)); + assertEquals(ROLE_VALUE, capturedContext[0].getFirstRequestHeaderValue(ROLE_HEADER)); + assertEquals(List.of(TOKEN_VALUE), capturedContext[0].getRequestHeaderValues(TOKEN_HEADER)); + assertEquals(List.of(ROLE_VALUE), capturedContext[0].getRequestHeaderValues(ROLE_HEADER)); + } + + @Test + void testContextIsClearedAfterFilterChain() throws Exception { + final MockHttpServletRequest request = new MockHttpServletRequest(); + request.addHeader(TOKEN_HEADER, TOKEN_VALUE); + setUpSecurityContext(); + + final ConnectorRequestContextFilter filter = new ConnectorRequestContextFilter(); + filter.doFilter(request, response, filterChain); + + assertNull(ConnectorRequestContextHolder.getContext()); + } + + @Test + void testContextIsClearedAfterFilterChainException() throws Exception { + final MockHttpServletRequest request = new MockHttpServletRequest(); + setUpSecurityContext(); + + doAnswer(invocation -> { + throw new RuntimeException("simulated error"); + }).when(filterChain).doFilter(any(), any()); + + final ConnectorRequestContextFilter filter = new ConnectorRequestContextFilter(); + try { + filter.doFilter(request, response, filterChain); + } catch (final RuntimeException ignored) { + // expected + } + + assertNull(ConnectorRequestContextHolder.getContext()); + } + + @Test + void testContextWithNoAuthenticatedUser() throws Exception { + final MockHttpServletRequest request = new MockHttpServletRequest(); + request.addHeader(TOKEN_HEADER, TOKEN_VALUE); + + final ConnectorRequestContext[] capturedContext = new ConnectorRequestContext[1]; + doAnswer(invocation -> { + capturedContext[0] = ConnectorRequestContextHolder.getContext(); + return null; + }).when(filterChain).doFilter(any(), any()); + + final ConnectorRequestContextFilter filter = new ConnectorRequestContextFilter(); + filter.doFilter(request, response, filterChain); + + assertNotNull(capturedContext[0]); + assertNull(capturedContext[0].getAuthenticatedUser()); + assertEquals(TOKEN_VALUE, capturedContext[0].getFirstRequestHeaderValue(TOKEN_HEADER)); + } + + @Test + void testHeaderMapIsCaseInsensitive() throws Exception { + final MockHttpServletRequest request = new MockHttpServletRequest(); + request.addHeader(TOKEN_HEADER, TOKEN_VALUE); + + final ConnectorRequestContext[] capturedContext = new ConnectorRequestContext[1]; + doAnswer(invocation -> { + capturedContext[0] = ConnectorRequestContextHolder.getContext(); + return null; + }).when(filterChain).doFilter(any(), any()); + + final ConnectorRequestContextFilter filter = new ConnectorRequestContextFilter(); + filter.doFilter(request, response, filterChain); + + assertNotNull(capturedContext[0]); + assertEquals(TOKEN_VALUE, capturedContext[0].getFirstRequestHeaderValue(TOKEN_HEADER.toLowerCase())); + assertEquals(TOKEN_VALUE, capturedContext[0].getFirstRequestHeaderValue(TOKEN_HEADER.toUpperCase())); + assertTrue(capturedContext[0].hasRequestHeader(TOKEN_HEADER.toLowerCase())); + assertTrue(capturedContext[0].hasRequestHeader(TOKEN_HEADER.toUpperCase())); + assertFalse(capturedContext[0].hasRequestHeader("Non-Existent-Header")); + assertEquals(List.of(), capturedContext[0].getRequestHeaderValues("Non-Existent-Header")); + assertNull(capturedContext[0].getFirstRequestHeaderValue("Non-Existent-Header")); + } + + private void setUpSecurityContext() { + final NiFiUser user = mock(NiFiUser.class, withSettings().strictness(Strictness.LENIENT)); + when(user.getIdentity()).thenReturn(TEST_USER); + final NiFiUserDetails userDetails = new NiFiUserDetails(user); + final TestingAuthenticationToken authentication = new TestingAuthenticationToken(userDetails, ""); + SecurityContextHolder.getContext().setAuthentication(authentication); + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/oidc/registration/StandardClientRegistrationProviderTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/oidc/registration/StandardClientRegistrationProviderTest.java index 3e22dc432f88..f48057a6f15f 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/oidc/registration/StandardClientRegistrationProviderTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/oidc/registration/StandardClientRegistrationProviderTest.java @@ -186,7 +186,7 @@ private NiFiProperties getProperties(final String discoveryUrl) { properties.put(NiFiProperties.SECURITY_USER_OIDC_CLIENT_SECRET, CLIENT_SECRET); properties.put(NiFiProperties.SECURITY_USER_OIDC_CLAIM_IDENTIFYING_USER, USER_NAME_ATTRIBUTE_NAME); properties.put(NiFiProperties.SECURITY_USER_OIDC_ADDITIONAL_SCOPES, OidcScopes.PROFILE); - return NiFiProperties.createBasicNiFiProperties(null, properties); + return NiFiProperties.createBasicNiFiProperties((String) null, properties); } private OIDCProviderMetadata getProviderMetadata() { diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/saml2/registration/StandardRegistrationBuilderProviderTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/saml2/registration/StandardRegistrationBuilderProviderTest.java index c682f316357b..5e1f5bd8b902 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/saml2/registration/StandardRegistrationBuilderProviderTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/saml2/registration/StandardRegistrationBuilderProviderTest.java @@ -167,7 +167,7 @@ private KeyStore getKeyStore() throws GeneralSecurityException { private NiFiProperties getProperties(final String metadataUrl) { final Properties properties = new Properties(); properties.setProperty(NiFiProperties.SECURITY_USER_SAML_IDP_METADATA_URL, metadataUrl); - return NiFiProperties.createBasicNiFiProperties(null, properties); + return NiFiProperties.createBasicNiFiProperties((String) null, properties); } private NiFiProperties getPropertiesTrustStoreStrategy(final String metadataUrl) { @@ -175,7 +175,7 @@ private NiFiProperties getPropertiesTrustStoreStrategy(final String metadataUrl) properties.setProperty(NiFiProperties.SECURITY_USER_SAML_IDP_METADATA_URL, metadataUrl); properties.setProperty(NiFiProperties.SECURITY_USER_SAML_HTTP_CLIENT_TRUSTSTORE_STRATEGY, StandardRegistrationBuilderProvider.NIFI_TRUST_STORE_STRATEGY); - return NiFiProperties.createBasicNiFiProperties(null, properties); + return NiFiProperties.createBasicNiFiProperties((String) null, properties); } final String getMetadata() throws IOException { diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/saml2/registration/StandardRelyingPartyRegistrationRepositoryTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/saml2/registration/StandardRelyingPartyRegistrationRepositoryTest.java index 667f8209ebd7..f3ddd48338cf 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/saml2/registration/StandardRelyingPartyRegistrationRepositoryTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/saml2/registration/StandardRelyingPartyRegistrationRepositoryTest.java @@ -132,7 +132,7 @@ private void assertRegistrationPropertiesFound(final RelyingPartyRegistration re private NiFiProperties getProperties() { final Properties properties = getStandardProperties(); - return NiFiProperties.createBasicNiFiProperties(null, properties); + return NiFiProperties.createBasicNiFiProperties((String) null, properties); } private NiFiProperties getSingleLogoutProperties() { @@ -140,7 +140,7 @@ private NiFiProperties getSingleLogoutProperties() { properties.setProperty(NiFiProperties.SECURITY_USER_SAML_SINGLE_LOGOUT_ENABLED, Boolean.TRUE.toString()); properties.setProperty(NiFiProperties.SECURITY_USER_SAML_SIGNATURE_ALGORITHM, SignatureConstants.ALGO_ID_SIGNATURE_RSA_SHA512); - return NiFiProperties.createBasicNiFiProperties(null, properties); + return NiFiProperties.createBasicNiFiProperties((String) null, properties); } private Properties getStandardProperties() { diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/x509/X509AuthenticationProviderTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/x509/X509AuthenticationProviderTest.java index ac157418948e..328eab1525a0 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/x509/X509AuthenticationProviderTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-security/src/test/java/org/apache/nifi/web/security/x509/X509AuthenticationProviderTest.java @@ -80,7 +80,7 @@ public void setup() { return AuthorizationResult.approved(); }); - x509AuthenticationProvider = new X509AuthenticationProvider(authorizer, NiFiProperties.createBasicNiFiProperties(null)); + x509AuthenticationProvider = new X509AuthenticationProvider(authorizer, NiFiProperties.createBasicNiFiProperties((String) null)); } @Test diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/access-policies/ui/component-access-policies/component-access-policies.component.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/access-policies/ui/component-access-policies/component-access-policies.component.ts index 8486e1882d00..742ba8dccc97 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/access-policies/ui/component-access-policies/component-access-policies.component.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/access-policies/ui/component-access-policies/component-access-policies.component.ts @@ -268,7 +268,8 @@ export class ComponentAccessPolicies implements OnInit, OnDestroy { } } else if ( policyComponentState.resource === 'parameter-contexts' || - policyComponentState.resource === 'parameter-providers' + policyComponentState.resource === 'parameter-providers' || + policyComponentState.resource === 'connectors' ) { switch (option.value) { case 'read-data': @@ -331,6 +332,7 @@ export class ComponentAccessPolicies implements OnInit, OnDestroy { case 'parameter-providers': case 'parameter-contexts': case 'reporting-tasks': + case 'connectors': return 'icon-drop'; } @@ -357,6 +359,8 @@ export class ComponentAccessPolicies implements OnInit, OnDestroy { return 'Reporting Task'; case 'parameter-providers': return ComponentType.ParameterProvider; + case 'connectors': + return 'Connector'; } return ComponentType.ProcessGroup; diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/access-policies/ui/global-access-policies/global-access-policies.component.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/access-policies/ui/global-access-policies/global-access-policies.component.ts index 74226feb6834..d84692cf9d47 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/access-policies/ui/global-access-policies/global-access-policies.component.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/access-policies/ui/global-access-policies/global-access-policies.component.ts @@ -201,6 +201,7 @@ export class GlobalAccessPolicies implements OnInit, OnDestroy { return ( resource === 'controller' || resource === 'parameter-contexts' || + resource === 'connectors' || resource === 'counters' || resource === 'policies' || resource === 'tenants' diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation-routing.module.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation-routing.module.ts index 544dd8106c5d..61abbbff12a3 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation-routing.module.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation-routing.module.ts @@ -26,6 +26,7 @@ import { ParameterProviderDefinition } from '../ui/parameter-provider-definition import { FlowAnalysisRuleDefinition } from '../ui/flow-analysis-rule-definition/flow-analysis-rule-definition.component'; import { Overview } from '../ui/overview/overview.component'; import { FlowRegistryClientDefinition } from '../ui/flow-registry-client-definition/flow-registry-client-definition.component'; +import { ConnectorDefinitionComponent } from '../ui/connector-definition/connector-definition.component'; const routes: Routes = [ { @@ -57,6 +58,10 @@ const routes: Routes = [ path: `${ComponentType.FlowAnalysisRule}/:group/:artifact/:version/:type`, component: FlowAnalysisRuleDefinition }, + { + path: `${ComponentType.Connector}/:group/:artifact/:version/:type`, + component: ConnectorDefinitionComponent + }, { path: 'overview', component: Overview diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation.component.html b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation.component.html index 22109056a224..0174272d21d5 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation.component.html +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation.component.html @@ -125,6 +125,19 @@ } "> + + + Connectors + + + @if (extensionTypes.length === 0) { diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation.component.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation.component.ts index ff14eb6379eb..332d9d74eb0d 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation.component.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation.component.ts @@ -29,6 +29,7 @@ import { NiFiState } from '../../../state'; import { Store } from '@ngrx/store'; import { loadExtensionTypesForDocumentation } from '../../../state/extension-types/extension-types.actions'; import { + selectConnectorTypes, selectControllerServiceTypes, selectFlowAnalysisRuleTypes, selectParameterProviderTypes, @@ -82,6 +83,9 @@ export class Documentation implements OnInit, AfterViewInit { flowAnalysisRuleTypes$ = this.store .select(selectFlowAnalysisRuleTypes) .pipe(map((extensionTypes) => this.sortExtensions(extensionTypes))); + connectorTypes$ = this.store + .select(selectConnectorTypes) + .pipe(map((extensionTypes) => this.sortExtensions(extensionTypes))); accordion = viewChild.required(MatAccordion); @@ -98,6 +102,7 @@ export class Documentation implements OnInit, AfterViewInit { reportingTasksExpanded = false; parameterProvidersExpanded = false; registryClientsExpanded = false; + connectorsExpanded = false; constructor() { this.store @@ -135,6 +140,9 @@ export class Documentation implements OnInit, AfterViewInit { case ComponentType.FlowRegistryClient: this.registryClientsExpanded = true; break; + case ComponentType.Connector: + this.connectorsExpanded = true; + break; } }); diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation.module.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation.module.ts index 0766ac00a88c..5c3f0460905c 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation.module.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/feature/documentation.module.ts @@ -36,6 +36,7 @@ import { ReportingTaskDefinitionEffects } from '../state/reporting-task-definiti import { ParameterProviderDefinitionEffects } from '../state/parameter-provider-definition/parameter-provider-definition.effects'; import { FlowAnalysisRuleDefinitionEffects } from '../state/flow-analysis-rule-definition/flow-analysis-rule-definition.effects'; import { FlowRegistryClientDefinitionEffects } from '../state/flow-registry-client-definition/flow-registry-client-definition.effects'; +import { ConnectorDefinitionEffects } from '../state/connector-definition/connector-definition.effects'; @NgModule({ declarations: [Documentation], @@ -51,6 +52,7 @@ import { FlowRegistryClientDefinitionEffects } from '../state/flow-registry-clie FlowRegistryClientDefinitionEffects, ParameterProviderDefinitionEffects, FlowAnalysisRuleDefinitionEffects, + ConnectorDefinitionEffects, AdditionalDetailsEffects, DocumentationEffects ), diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/service/documentation.service.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/service/documentation.service.ts index a9955bf6bda1..2b7c12f2ab24 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/service/documentation.service.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/service/documentation.service.ts @@ -26,6 +26,7 @@ import { ReportingTaskDefinition } from '../state/reporting-task-definition'; import { ParameterProviderDefinition } from '../state/parameter-provider-definition'; import { FlowAnalysisRuleDefinition } from '../state/flow-analysis-rule-definition'; import { FlowRegistryClientDefinition } from '../state/flow-registry-client-definition'; +import { ConnectorDefinition } from '../state/connector-definition'; @Injectable({ providedIn: 'root' }) export class DocumentationService { @@ -74,4 +75,20 @@ export class DocumentationService { `${DocumentationService.API}/flow/additional-details/${coordinates.group}/${coordinates.artifact}/${coordinates.version}/${coordinates.type}` ); } + + getConnectorDefinition(coordinates: DefinitionCoordinates): Observable { + return this.httpClient.get( + `${DocumentationService.API}/flow/connector-definition/${coordinates.group}/${coordinates.artifact}/${coordinates.version}/${coordinates.type}` + ); + } + + getStepDocumentation(coordinates: DefinitionCoordinates, stepName: string): Observable { + return this.httpClient.get( + `${DocumentationService.API}/flow/steps/${coordinates.group}/${coordinates.artifact}/${coordinates.version}/${coordinates.type}/${stepName}` + ); + } +} + +export interface StepDocumentationEntity { + stepDocumentation: string; } diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.actions.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.actions.ts new file mode 100644 index 000000000000..aaa80ac1b1a1 --- /dev/null +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.actions.ts @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import { createAction, props } from '@ngrx/store'; +import { ConnectorDefinition } from './index'; +import { DefinitionCoordinates } from '../index'; + +const CONNECTOR_DEFINITION_PREFIX = '[Connector Definition]'; + +export const loadConnectorDefinition = createAction( + `${CONNECTOR_DEFINITION_PREFIX} Load Connector Definition`, + props<{ coordinates: DefinitionCoordinates }>() +); + +export const loadConnectorDefinitionSuccess = createAction( + `${CONNECTOR_DEFINITION_PREFIX} Load Connector Definition Success`, + props<{ connectorDefinition: ConnectorDefinition }>() +); + +export const connectorDefinitionApiError = createAction( + `${CONNECTOR_DEFINITION_PREFIX} Load Connector Definition Error`, + props<{ error: string }>() +); + +export const resetConnectorDefinitionState = createAction( + `${CONNECTOR_DEFINITION_PREFIX} Reset Connector Definition State` +); + +export const loadStepDocumentation = createAction( + `${CONNECTOR_DEFINITION_PREFIX} Load Step Documentation`, + props<{ coordinates: DefinitionCoordinates; stepName: string }>() +); + +export const loadStepDocumentationSuccess = createAction( + `${CONNECTOR_DEFINITION_PREFIX} Load Step Documentation Success`, + props<{ stepName: string; documentation: string }>() +); + +export const stepDocumentationApiError = createAction( + `${CONNECTOR_DEFINITION_PREFIX} Load Step Documentation Error`, + props<{ stepName: string; error: string }>() +); diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.effects.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.effects.ts new file mode 100644 index 000000000000..e4ebac8ffcd2 --- /dev/null +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.effects.ts @@ -0,0 +1,79 @@ +/* + * 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. + */ + +import { Injectable, inject } from '@angular/core'; +import { Actions, createEffect, ofType } from '@ngrx/effects'; +import * as ConnectorDefinitionActions from './connector-definition.actions'; +import { catchError, from, map, mergeMap, of, switchMap } from 'rxjs'; +import { DocumentationService } from '../../service/documentation.service'; +import { ConnectorDefinition } from './index'; +import { ErrorHelper } from '../../../../service/error-helper.service'; +import { HttpErrorResponse } from '@angular/common/http'; + +@Injectable() +export class ConnectorDefinitionEffects { + private actions$ = inject(Actions); + private documentationService = inject(DocumentationService); + private errorHelper = inject(ErrorHelper); + + loadConnectorDefinition$ = createEffect(() => + this.actions$.pipe( + ofType(ConnectorDefinitionActions.loadConnectorDefinition), + map((action) => action.coordinates), + switchMap((coordinates) => + from(this.documentationService.getConnectorDefinition(coordinates)).pipe( + map((connectorDefinition: ConnectorDefinition) => + ConnectorDefinitionActions.loadConnectorDefinitionSuccess({ + connectorDefinition + }) + ), + catchError((errorResponse: HttpErrorResponse) => + of( + ConnectorDefinitionActions.connectorDefinitionApiError({ + error: this.errorHelper.getErrorString(errorResponse) + }) + ) + ) + ) + ) + ) + ); + + loadStepDocumentation$ = createEffect(() => + this.actions$.pipe( + ofType(ConnectorDefinitionActions.loadStepDocumentation), + mergeMap((action) => + from(this.documentationService.getStepDocumentation(action.coordinates, action.stepName)).pipe( + map((response) => + ConnectorDefinitionActions.loadStepDocumentationSuccess({ + stepName: action.stepName, + documentation: response.stepDocumentation + }) + ), + catchError((errorResponse: HttpErrorResponse) => + of( + ConnectorDefinitionActions.stepDocumentationApiError({ + stepName: action.stepName, + error: this.errorHelper.getErrorString(errorResponse) + }) + ) + ) + ) + ) + ) + ); +} diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.reducer.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.reducer.ts new file mode 100644 index 000000000000..e2953573dd51 --- /dev/null +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.reducer.ts @@ -0,0 +1,92 @@ +/* + * 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. + */ + +import { createReducer, on } from '@ngrx/store'; +import { ConnectorDefinitionState } from './index'; +import { + connectorDefinitionApiError, + loadConnectorDefinition, + loadConnectorDefinitionSuccess, + loadStepDocumentation, + loadStepDocumentationSuccess, + resetConnectorDefinitionState, + stepDocumentationApiError +} from './connector-definition.actions'; + +export const initialConnectorDefinitionState: ConnectorDefinitionState = { + connectorDefinition: null, + error: null, + status: 'pending', + stepDocumentation: {} +}; + +export const connectorDefinitionReducer = createReducer( + initialConnectorDefinitionState, + on(loadConnectorDefinition, (state) => ({ + ...state, + connectorDefinition: null, + error: null, + status: 'loading' as const + })), + on(loadConnectorDefinitionSuccess, (state, { connectorDefinition }) => ({ + ...state, + connectorDefinition, + error: null, + status: 'success' as const + })), + on(connectorDefinitionApiError, (state, { error }) => ({ + ...state, + error, + status: 'error' as const + })), + on(resetConnectorDefinitionState, () => ({ + ...initialConnectorDefinitionState + })), + on(loadStepDocumentation, (state, { stepName }) => ({ + ...state, + stepDocumentation: { + ...state.stepDocumentation, + [stepName]: { + documentation: null, + error: null, + status: 'loading' as const + } + } + })), + on(loadStepDocumentationSuccess, (state, { stepName, documentation }) => ({ + ...state, + stepDocumentation: { + ...state.stepDocumentation, + [stepName]: { + documentation, + error: null, + status: 'success' as const + } + } + })), + on(stepDocumentationApiError, (state, { stepName, error }) => ({ + ...state, + stepDocumentation: { + ...state.stepDocumentation, + [stepName]: { + documentation: null, + error, + status: 'error' as const + } + } + })) +); diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.selectors.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.selectors.ts new file mode 100644 index 000000000000..ad0adc5060cb --- /dev/null +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/connector-definition.selectors.ts @@ -0,0 +1,31 @@ +/* + * 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. + */ + +import { createSelector } from '@ngrx/store'; +import { DocumentationState, selectDocumentationState } from '../index'; +import { connectorDefinitionFeatureKey, StepDocumentationState } from './index'; + +export const selectConnectorDefinitionState = createSelector( + selectDocumentationState, + (state: DocumentationState) => state[connectorDefinitionFeatureKey] +); + +export const selectStepDocumentation = (stepName: string) => + createSelector( + selectConnectorDefinitionState, + (state): StepDocumentationState | undefined => state.stepDocumentation[stepName] + ); diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/index.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/index.ts new file mode 100644 index 000000000000..fda3d93c8b3f --- /dev/null +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/connector-definition/index.ts @@ -0,0 +1,73 @@ +/* + * 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. + */ + +import { ExtensionComponent, PropertyAllowableValue } from '../index'; + +export const connectorDefinitionFeatureKey = 'connectorDefinition'; + +export interface ConnectorPropertyDependency { + propertyName: string; + dependentValues?: string[]; +} + +export interface ConnectorPropertyDescriptor { + name: string; + description: string; + defaultValue?: string; + required: boolean; + propertyType?: string; + allowableValuesFetchable: boolean; + allowableValues?: PropertyAllowableValue[]; + dependencies?: ConnectorPropertyDependency[]; +} + +export interface ConnectorPropertyGroup { + name: string; + description: string; + properties?: ConnectorPropertyDescriptor[]; +} + +export interface ConfigurationStepDependency { + stepName: string; + propertyName: string; + dependentValues?: string[]; +} + +export interface ConfigurationStep { + name: string; + description: string; + documented: boolean; + stepDependencies?: ConfigurationStepDependency[]; + propertyGroups?: ConnectorPropertyGroup[]; +} + +export interface ConnectorDefinition extends ExtensionComponent { + configurationSteps?: ConfigurationStep[]; +} + +export interface StepDocumentationState { + documentation: string | null; + error: string | null; + status: 'pending' | 'loading' | 'success' | 'error'; +} + +export interface ConnectorDefinitionState { + connectorDefinition: ConnectorDefinition | null; + error: string | null; + status: 'pending' | 'loading' | 'success' | 'error'; + stepDocumentation: { [stepName: string]: StepDocumentationState }; +} diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/index.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/index.ts index 0e81eab6441c..65d380edb785 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/index.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/state/index.ts @@ -41,6 +41,8 @@ import { import { parameterProviderDefinitionReducer } from './parameter-provider-definition/parameter-provider-definition.reducer'; import { flowAnalysisRuleDefinitionFeatureKey, FlowAnalysisRuleDefinitionState } from './flow-analysis-rule-definition'; import { flowAnalysisRuleDefinitionReducer } from './flow-analysis-rule-definition/flow-analysis-rule-definition.reducer'; +import { connectorDefinitionFeatureKey, ConnectorDefinitionState } from './connector-definition'; +import { connectorDefinitionReducer } from './connector-definition/connector-definition.reducer'; import { ComponentType } from '@nifi/shared'; import { DocumentedType } from '../../../state/shared'; @@ -196,6 +198,7 @@ export interface DocumentationState { [flowRegistryClientDefinitionFeatureKey]: FlowRegistryClientDefinitionState; [parameterProviderDefinitionFeatureKey]: ParameterProviderDefinitionState; [flowAnalysisRuleDefinitionFeatureKey]: FlowAnalysisRuleDefinitionState; + [connectorDefinitionFeatureKey]: ConnectorDefinitionState; [additionalDetailsFeatureKey]: AdditionalDetailsState; [externalDocumentationFeatureKey]: ExternalDocumentationState; } @@ -208,6 +211,7 @@ export function reducers(state: DocumentationState | undefined, action: Action) [flowRegistryClientDefinitionFeatureKey]: flowRegistryClientDefinitionReducer, [parameterProviderDefinitionFeatureKey]: parameterProviderDefinitionReducer, [flowAnalysisRuleDefinitionFeatureKey]: flowAnalysisRuleDefinitionReducer, + [connectorDefinitionFeatureKey]: connectorDefinitionReducer, [additionalDetailsFeatureKey]: additionalDetailsReducer, [externalDocumentationFeatureKey]: externalDocumentationReducer })(state, action); diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.html b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.html new file mode 100644 index 000000000000..395c7fff9df8 --- /dev/null +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.html @@ -0,0 +1,95 @@ + + +@if (propertyDescriptor) { +
+
+
Name
+
{{ propertyDescriptor.name }}
+
+
+
Description
+
{{ propertyDescriptor.description }}
+
+
+
Type
+
{{ formatPropertyType() }}
+
+
+
Required
+
{{ propertyDescriptor.required ? 'Yes' : 'No' }}
+
+ @if (formatDefaultValue(); as defaultValue) { +
+
Default Value
+
{{ defaultValue }}
+
+ } + @if (propertyDescriptor.allowableValues) { +
+
Allowable Values
+
+
    + @for (allowableValue of propertyDescriptor.allowableValues; track allowableValue.value) { +
  • + {{ allowableValue.displayName }} + @if (allowableValue.description) { + + } +
  • + } +
+
+
+ } + @if (propertyDescriptor.dependencies) { +
+
+ Dependencies + +
+
+
    + @for ( + propertyDependency of sortDependencies(propertyDescriptor.dependencies); + track propertyDependency.propertyName + ) { +
  • + @if (propertyDependency.dependentValues) { + {{ propertyDependency.propertyName }} is set to any of [{{ + formatDependentValue(propertyDependency) + }}] + } @else { + {{ propertyDependency.propertyName }} has any value specified + } +
  • + } +
+
+
+ } +
+} diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.scss b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.scss new file mode 100644 index 000000000000..eb0ceabcbddf --- /dev/null +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.scss @@ -0,0 +1,20 @@ +/* + * 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. + */ + +:host { + display: block; +} diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.spec.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.spec.ts new file mode 100644 index 000000000000..45143688eb52 --- /dev/null +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.spec.ts @@ -0,0 +1,176 @@ +/* + * 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. + */ + +import { TestBed } from '@angular/core/testing'; + +import { ConnectorPropertyDefinitionComponent } from './connector-property-definition.component'; +import { ConnectorPropertyDescriptor } from '../../../state/connector-definition'; + +describe('ConnectorPropertyDefinitionComponent', () => { + function createMockProperty(overrides?: Partial): ConnectorPropertyDescriptor { + return { + name: 'Test Property', + description: 'A test property description', + required: false, + allowableValuesFetchable: false, + ...overrides + }; + } + + async function setup(propertyOverrides?: Partial) { + await TestBed.configureTestingModule({ + imports: [ConnectorPropertyDefinitionComponent] + }).compileComponents(); + + const fixture = TestBed.createComponent(ConnectorPropertyDefinitionComponent); + const component = fixture.componentInstance; + component.propertyDescriptor = createMockProperty(propertyOverrides); + fixture.detectChanges(); + + return { component, fixture }; + } + + describe('Component initialization', () => { + it('should create', async () => { + const { component } = await setup(); + expect(component).toBeTruthy(); + }); + + it('should have property descriptor set', async () => { + const { component } = await setup({ name: 'Custom Property' }); + expect(component.propertyDescriptor.name).toBe('Custom Property'); + }); + }); + + describe('Default value formatting', () => { + it('should return undefined when no default value is set', async () => { + const { component } = await setup({ defaultValue: undefined }); + expect(component.formatDefaultValue()).toBeUndefined(); + }); + + it('should return the default value as-is when no allowable values exist', async () => { + const { component } = await setup({ defaultValue: 'my-default' }); + expect(component.formatDefaultValue()).toBe('my-default'); + }); + + it('should return the display name when default value matches an allowable value', async () => { + const { component } = await setup({ + defaultValue: 'val1', + allowableValues: [ + { value: 'val1', displayName: 'Value One', description: 'First value' }, + { value: 'val2', displayName: 'Value Two', description: 'Second value' } + ] + }); + expect(component.formatDefaultValue()).toBe('Value One'); + }); + + it('should return the raw default value when it does not match any allowable value', async () => { + const { component } = await setup({ + defaultValue: 'unknown', + allowableValues: [{ value: 'val1', displayName: 'Value One', description: 'First value' }] + }); + expect(component.formatDefaultValue()).toBe('unknown'); + }); + }); + + describe('Property type formatting', () => { + it('should return STRING when property type is undefined', async () => { + const { component } = await setup({ propertyType: undefined }); + expect(component.formatPropertyType()).toBe('STRING'); + }); + + it('should return the property type when defined', async () => { + const { component } = await setup({ propertyType: 'INTEGER' }); + expect(component.formatPropertyType()).toBe('INTEGER'); + }); + }); + + describe('Dependency sorting', () => { + it('should sort dependencies alphabetically by property name', async () => { + const { component } = await setup(); + const dependencies = [ + { propertyName: 'zeta', dependentValues: ['a'] }, + { propertyName: 'alpha', dependentValues: ['b'] }, + { propertyName: 'gamma', dependentValues: ['c'] } + ]; + + const sorted = component.sortDependencies(dependencies); + + expect(sorted[0].propertyName).toBe('alpha'); + expect(sorted[1].propertyName).toBe('gamma'); + expect(sorted[2].propertyName).toBe('zeta'); + }); + + it('should not mutate the original array', async () => { + const { component } = await setup(); + const dependencies = [ + { propertyName: 'zeta', dependentValues: ['a'] }, + { propertyName: 'alpha', dependentValues: ['b'] } + ]; + + component.sortDependencies(dependencies); + + expect(dependencies[0].propertyName).toBe('zeta'); + }); + }); + + describe('Dependent value formatting', () => { + it('should return empty string when dependentValues is undefined', async () => { + const { component } = await setup(); + const dependency = { propertyName: 'prop1' }; + + expect(component.formatDependentValue(dependency)).toBe(''); + }); + + it('should join dependent values with comma', async () => { + const { component } = await setup(); + const dependency = { propertyName: 'prop1', dependentValues: ['val1', 'val2', 'val3'] }; + + expect(component.formatDependentValue(dependency)).toBe('val1, val2, val3'); + }); + + it('should use display names from lookupProperty when available', async () => { + const { component } = await setup(); + const properties: ConnectorPropertyDescriptor[] = [ + { + name: 'dep-prop', + description: 'Dependency property', + required: false, + allowableValuesFetchable: false, + allowableValues: [ + { value: 'v1', displayName: 'Display Value 1', description: '' }, + { value: 'v2', displayName: 'Display Value 2', description: '' } + ] + } + ]; + component.lookupProperty = (name: string) => properties.find((p) => p.name === name); + + const dependency = { propertyName: 'dep-prop', dependentValues: ['v1', 'v2'] }; + + expect(component.formatDependentValue(dependency)).toBe('Display Value 1, Display Value 2'); + }); + + it('should fall back to raw values when lookupProperty does not find the property', async () => { + const { component } = await setup(); + component.lookupProperty = () => undefined; + + const dependency = { propertyName: 'unknown-prop', dependentValues: ['raw1', 'raw2'] }; + + expect(component.formatDependentValue(dependency)).toBe('raw1, raw2'); + }); + }); +}); diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.ts new file mode 100644 index 000000000000..fae2f0094511 --- /dev/null +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/common/connector-property-definition/connector-property-definition.component.ts @@ -0,0 +1,88 @@ +/* + * 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. + */ + +import { Component, Input, inject } from '@angular/core'; +import { NiFiCommon, NifiTooltipDirective, TextTip } from '@nifi/shared'; +import { ConnectorPropertyDescriptor, ConnectorPropertyDependency } from '../../../state/connector-definition'; +import { PropertyAllowableValue } from '../../../state'; + +@Component({ + selector: 'connector-property-definition', + imports: [NifiTooltipDirective], + templateUrl: './connector-property-definition.component.html', + styleUrl: './connector-property-definition.component.scss' +}) +export class ConnectorPropertyDefinitionComponent { + private nifiCommon = inject(NiFiCommon); + + @Input() propertyDescriptor!: ConnectorPropertyDescriptor; + @Input() lookupProperty: ((name: string) => ConnectorPropertyDescriptor | undefined) | undefined; + + protected readonly TextTip = TextTip; + + formatDefaultValue(): string | undefined { + if (!this.propertyDescriptor.defaultValue) { + return undefined; + } + + if (this.propertyDescriptor.allowableValues) { + const defaultAllowableValue: PropertyAllowableValue | undefined = + this.propertyDescriptor.allowableValues.find( + (allowableValue) => allowableValue.value === this.propertyDescriptor.defaultValue + ); + + if (defaultAllowableValue) { + return defaultAllowableValue.displayName; + } + } + + return this.propertyDescriptor.defaultValue; + } + + formatPropertyType(): string { + return this.propertyDescriptor.propertyType || 'STRING'; + } + + sortDependencies(dependencies: ConnectorPropertyDependency[]): ConnectorPropertyDependency[] { + return dependencies.slice().sort((a, b) => { + return this.nifiCommon.compareString(a.propertyName, b.propertyName); + }); + } + + formatDependentValue(dependency: ConnectorPropertyDependency): string { + if (dependency.dependentValues) { + if (this.lookupProperty) { + const propertyDependency = this.lookupProperty(dependency.propertyName); + + if (propertyDependency?.allowableValues) { + return dependency.dependentValues + .map((dependentValue) => { + const dependentAllowableValue = propertyDependency.allowableValues?.find( + (allowableValue) => dependentValue === allowableValue.value + ); + return dependentAllowableValue ? dependentAllowableValue.displayName : dependentValue; + }) + .join(', '); + } + } + + return dependency.dependentValues.join(', '); + } + + return ''; + } +} diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.html b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.html new file mode 100644 index 000000000000..849ce1db7310 --- /dev/null +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.html @@ -0,0 +1,209 @@ + + +@if (connectorDefinitionState) { + @if (isInitialLoading(connectorDefinitionState)) { + + } @else { + @if (connectorDefinitionState.connectorDefinition; as connectorDefinition) { +
+

+ {{ formatExtensionName(connectorDefinition.type) }} {{ connectorDefinition.version }} +

+
+ @if (connectorDefinition.deprecated) { +
+ @if (connectorDefinition.deprecationReason) { +
{{ connectorDefinition.deprecationReason }}
+ } @else { +
+ Please be aware this connector is deprecated and may be removed in the near future. +
+ } + @if (connectorDefinition.deprecationAlternatives) { +
+
See Alternatives
+ +
+ } +
+ } +
+
Bundle
+
+ {{ connectorDefinition.group }} | {{ connectorDefinition.artifact }} +
+
+ @if (connectorDefinition.typeDescription) { +
+
Description
+
{{ connectorDefinition.typeDescription }}
+
+ } + @if (connectorDefinition.tags) { +
+
Tags
+
+ {{ connectorDefinition.tags.join(', ') }} +
+
+ } + @if (hasConfigurationSteps(connectorDefinition.configurationSteps)) { +
+
+

Configuration Steps

+
+ + +
+
+ + @for ( + step of connectorDefinition.configurationSteps; + track step.name; + let first = $first + ) { + + + {{ step.name }} + +
+ @if (step.description) { +
{{ step.description }}
+ } + @if (step.documented) { + + + Step Documentation + +
+ @if (isStepDocumentationLoading(step.name)) { + + } @else if (getStepDocumentationError(step.name); as error) { +
{{ error }}
+ } @else if (getStepDocumentation(step.name); as documentation) { + + } +
+
+ } + @if (hasPropertyGroups(step.propertyGroups)) { + @for (group of step.propertyGroups; track group.name) { +
+
+

{{ group.name }}

+
+ @if (group.description) { +
+ {{ group.description }} +
+ } + @if (hasProperties(group.properties)) { +
+
+ In the list below required properties are shown with + an asterisk (*). Other properties are considered + optional. +
+
+ + +
+
+ + @for ( + property of group.properties; + track property.name + ) { + + + + {{ formatPropertyTitle(property) }} + + + {{ property.description }} + + + + + } + + } +
+ } + } +
+
+ } +
+
+ } + @if (connectorDefinition.seeAlso) { +
+

See Also

+ +
+ } +
+
+ } @else if (connectorDefinitionState.error) { +
+ {{ connectorDefinitionState.error }} +
+ } + } +} diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.scss b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.scss new file mode 100644 index 000000000000..eb0ceabcbddf --- /dev/null +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.scss @@ -0,0 +1,20 @@ +/* + * 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. + */ + +:host { + display: block; +} diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.spec.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.spec.ts new file mode 100644 index 000000000000..41ac8cac5f86 --- /dev/null +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.spec.ts @@ -0,0 +1,415 @@ +/* + * 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. + */ + +import { TestBed } from '@angular/core/testing'; +import { MockStore, provideMockStore } from '@ngrx/store/testing'; + +import { ConnectorDefinitionComponent } from './connector-definition.component'; +import { + ConfigurationStep, + connectorDefinitionFeatureKey, + ConnectorDefinitionState, + ConnectorPropertyDescriptor, + ConnectorPropertyGroup +} from '../../state/connector-definition'; +import { initialConnectorDefinitionState } from '../../state/connector-definition/connector-definition.reducer'; +import { documentationFeatureKey } from '../../state'; +import { initialState as initialErrorState } from '../../../../state/error/error.reducer'; +import { errorFeatureKey } from '../../../../state/error'; +import { initialState as initialCurrentUserState } from '../../../../state/current-user/current-user.reducer'; +import { currentUserFeatureKey } from '../../../../state/current-user'; +import { ConnectorDefinition } from '../../state/connector-definition'; +import { loadStepDocumentation } from '../../state/connector-definition/connector-definition.actions'; + +describe('ConnectorDefinitionComponent', () => { + function createMockProperty(name: string, required: boolean, description: string): ConnectorPropertyDescriptor { + return { + name, + description, + required, + allowableValuesFetchable: false + }; + } + + function createMockPropertyGroup(name: string, properties: ConnectorPropertyDescriptor[]): ConnectorPropertyGroup { + return { + name, + description: `Description for ${name}`, + properties + }; + } + + function createMockConfigurationStep(name: string, propertyGroups: ConnectorPropertyGroup[]): ConfigurationStep { + return { + name, + description: `Description for ${name}`, + documented: false, + propertyGroups + }; + } + + async function setup(stateOverrides?: Partial) { + const initialState = { + [errorFeatureKey]: initialErrorState, + [currentUserFeatureKey]: initialCurrentUserState, + [documentationFeatureKey]: { + [connectorDefinitionFeatureKey]: { + ...initialConnectorDefinitionState, + ...stateOverrides + } + } + }; + + await TestBed.configureTestingModule({ + imports: [ConnectorDefinitionComponent], + providers: [provideMockStore({ initialState })] + }).compileComponents(); + + const fixture = TestBed.createComponent(ConnectorDefinitionComponent); + const component = fixture.componentInstance; + const store = TestBed.inject(MockStore); + + fixture.detectChanges(); + + return { component, fixture, store }; + } + + describe('Component initialization', () => { + it('should create', async () => { + const { component } = await setup(); + expect(component).toBeTruthy(); + }); + + it('should have null connectorDefinitionState initially', async () => { + const { component } = await setup(); + expect(component.connectorDefinitionState).toBeDefined(); + }); + }); + + describe('Loading state logic', () => { + it('should return true for isInitialLoading when connectorDefinition and error are null', async () => { + const { component } = await setup(); + const state: ConnectorDefinitionState = { + connectorDefinition: null, + error: null, + status: 'pending', + stepDocumentation: {} + }; + expect(component.isInitialLoading(state)).toBe(true); + }); + + it('should return false for isInitialLoading when connectorDefinition is present', async () => { + const { component } = await setup(); + const state: ConnectorDefinitionState = { + connectorDefinition: { + group: 'org.apache.nifi', + artifact: 'nifi-test-nar', + version: '1.0.0', + type: 'org.apache.nifi.TestConnector', + typeDescription: 'Test Connector', + buildInfo: { + revision: 'abc123', + version: '1.0.0' + }, + additionalDetails: false + }, + error: null, + status: 'success', + stepDocumentation: {} + }; + expect(component.isInitialLoading(state)).toBe(false); + }); + + it('should return false for isInitialLoading when error is present', async () => { + const { component } = await setup(); + const state: ConnectorDefinitionState = { + connectorDefinition: null, + error: 'Failed to load', + status: 'error', + stepDocumentation: {} + }; + expect(component.isInitialLoading(state)).toBe(false); + }); + }); + + describe('Configuration step helpers', () => { + it('should return true for hasConfigurationSteps when steps array has items', async () => { + const { component } = await setup(); + const steps = [createMockConfigurationStep('Step 1', [])]; + expect(component.hasConfigurationSteps(steps)).toBe(true); + }); + + it('should return false for hasConfigurationSteps when steps array is empty', async () => { + const { component } = await setup(); + expect(component.hasConfigurationSteps([])).toBe(false); + }); + + it('should return false for hasConfigurationSteps when steps is undefined', async () => { + const { component } = await setup(); + expect(component.hasConfigurationSteps(undefined)).toBe(false); + }); + }); + + describe('Property group helpers', () => { + it('should return true for hasPropertyGroups when groups array has items', async () => { + const { component } = await setup(); + const groups = [createMockPropertyGroup('Group 1', [])]; + expect(component.hasPropertyGroups(groups)).toBe(true); + }); + + it('should return false for hasPropertyGroups when groups array is empty', async () => { + const { component } = await setup(); + expect(component.hasPropertyGroups([])).toBe(false); + }); + + it('should return false for hasPropertyGroups when groups is undefined', async () => { + const { component } = await setup(); + expect(component.hasPropertyGroups(undefined)).toBe(false); + }); + }); + + describe('Property helpers', () => { + it('should return true for hasProperties when properties array has items', async () => { + const { component } = await setup(); + const properties = [createMockProperty('prop1', true, 'Description')]; + expect(component.hasProperties(properties)).toBe(true); + }); + + it('should return false for hasProperties when properties array is empty', async () => { + const { component } = await setup(); + expect(component.hasProperties([])).toBe(false); + }); + + it('should return false for hasProperties when properties is undefined', async () => { + const { component } = await setup(); + expect(component.hasProperties(undefined)).toBe(false); + }); + }); + + describe('Property title formatting', () => { + it('should append asterisk for required properties', async () => { + const { component } = await setup(); + const property = createMockProperty('Required Property', true, 'Description'); + expect(component.formatPropertyTitle(property)).toBe('Required Property*'); + }); + + it('should not append asterisk for optional properties', async () => { + const { component } = await setup(); + const property = createMockProperty('Optional Property', false, 'Description'); + expect(component.formatPropertyTitle(property)).toBe('Optional Property'); + }); + }); + + describe('Property lookup', () => { + it('should return a function that finds properties by name', async () => { + const { component } = await setup(); + const properties = [ + createMockProperty('prop1', true, 'First property'), + createMockProperty('prop2', false, 'Second property') + ]; + + const lookupFn = component.lookupProperty(properties); + + expect(lookupFn('prop1')).toEqual(properties[0]); + expect(lookupFn('prop2')).toEqual(properties[1]); + expect(lookupFn('nonexistent')).toBeUndefined(); + }); + }); + + describe('Step documentation', () => { + it('should return false for isStepDocumentationLoading when step is not in state', async () => { + const { component } = await setup(); + expect(component.isStepDocumentationLoading('Some Step')).toBe(false); + }); + + it('should return true for isStepDocumentationLoading when step status is loading', async () => { + const { component } = await setup({ + stepDocumentation: { + 'Some Step': { documentation: null, error: null, status: 'loading' } + } + }); + expect(component.isStepDocumentationLoading('Some Step')).toBe(true); + }); + + it('should return undefined for getStepDocumentation when step is not in state', async () => { + const { component } = await setup(); + expect(component.getStepDocumentation('Some Step')).toBeUndefined(); + }); + + it('should return documentation when step has loaded documentation', async () => { + const { component } = await setup({ + stepDocumentation: { + 'Some Step': { documentation: '# Step Documentation', error: null, status: 'success' } + } + }); + expect(component.getStepDocumentation('Some Step')).toBe('# Step Documentation'); + }); + + it('should return undefined for getStepDocumentationError when step is not in state', async () => { + const { component } = await setup(); + expect(component.getStepDocumentationError('Some Step')).toBeUndefined(); + }); + + it('should return error when step has an error', async () => { + const { component } = await setup({ + stepDocumentation: { + 'Some Step': { documentation: null, error: 'Failed to load', status: 'error' } + } + }); + expect(component.getStepDocumentationError('Some Step')).toBe('Failed to load'); + }); + }); + + describe('onStepExpanded', () => { + function createMockConnectorDefinition(): ConnectorDefinition { + return { + group: 'org.apache.nifi', + artifact: 'nifi-test-nar', + version: '1.0.0', + type: 'org.apache.nifi.TestConnector', + typeDescription: 'Test Connector', + buildInfo: { revision: 'abc123', version: '1.0.0' }, + additionalDetails: false + }; + } + + it('should call loadStepDocumentation when step is documented', async () => { + const { component, store } = await setup(); + const dispatchSpy = jest.spyOn(store, 'dispatch'); + const connectorDefinition = createMockConnectorDefinition(); + const step = createMockConfigurationStep('Documented Step', []); + step.documented = true; + + component.onStepExpanded(connectorDefinition, step); + + expect(dispatchSpy).toHaveBeenCalledWith( + loadStepDocumentation({ + coordinates: { + group: connectorDefinition.group, + artifact: connectorDefinition.artifact, + version: connectorDefinition.version, + type: connectorDefinition.type + }, + stepName: 'Documented Step' + }) + ); + }); + + it('should not call loadStepDocumentation when step is not documented', async () => { + const { component, store } = await setup(); + const dispatchSpy = jest.spyOn(store, 'dispatch'); + dispatchSpy.mockClear(); + const connectorDefinition = createMockConnectorDefinition(); + const step = createMockConfigurationStep('Undocumented Step', []); + step.documented = false; + + component.onStepExpanded(connectorDefinition, step); + + expect(dispatchSpy).not.toHaveBeenCalled(); + }); + }); + + describe('loadStepDocumentation deduplication', () => { + function createMockConnectorDefinition(): ConnectorDefinition { + return { + group: 'org.apache.nifi', + artifact: 'nifi-test-nar', + version: '1.0.0', + type: 'org.apache.nifi.TestConnector', + typeDescription: 'Test Connector', + buildInfo: { revision: 'abc123', version: '1.0.0' }, + additionalDetails: false + }; + } + + it('should not dispatch when step is already loading', async () => { + const { component, store } = await setup({ + stepDocumentation: { + 'Loading Step': { documentation: null, error: null, status: 'loading' } + } + }); + const dispatchSpy = jest.spyOn(store, 'dispatch'); + dispatchSpy.mockClear(); + const connectorDefinition = createMockConnectorDefinition(); + + component.loadStepDocumentation(connectorDefinition, 'Loading Step'); + + expect(dispatchSpy).not.toHaveBeenCalled(); + }); + + it('should not dispatch when step already has documentation', async () => { + const { component, store } = await setup({ + stepDocumentation: { + 'Loaded Step': { documentation: '# Docs', error: null, status: 'success' } + } + }); + const dispatchSpy = jest.spyOn(store, 'dispatch'); + dispatchSpy.mockClear(); + const connectorDefinition = createMockConnectorDefinition(); + + component.loadStepDocumentation(connectorDefinition, 'Loaded Step'); + + expect(dispatchSpy).not.toHaveBeenCalled(); + }); + + it('should dispatch when step is not in state', async () => { + const { component, store } = await setup(); + const dispatchSpy = jest.spyOn(store, 'dispatch'); + dispatchSpy.mockClear(); + const connectorDefinition = createMockConnectorDefinition(); + + component.loadStepDocumentation(connectorDefinition, 'New Step'); + + expect(dispatchSpy).toHaveBeenCalledWith( + loadStepDocumentation({ + coordinates: { + group: connectorDefinition.group, + artifact: connectorDefinition.artifact, + version: connectorDefinition.version, + type: connectorDefinition.type + }, + stepName: 'New Step' + }) + ); + }); + + it('should dispatch when step had an error', async () => { + const { component, store } = await setup({ + stepDocumentation: { + 'Error Step': { documentation: null, error: 'Failed', status: 'error' } + } + }); + const dispatchSpy = jest.spyOn(store, 'dispatch'); + dispatchSpy.mockClear(); + const connectorDefinition = createMockConnectorDefinition(); + + component.loadStepDocumentation(connectorDefinition, 'Error Step'); + + expect(dispatchSpy).toHaveBeenCalledWith( + loadStepDocumentation({ + coordinates: { + group: connectorDefinition.group, + artifact: connectorDefinition.artifact, + version: connectorDefinition.version, + type: connectorDefinition.type + }, + stepName: 'Error Step' + }) + ); + }); + }); +}); diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.ts new file mode 100644 index 000000000000..7e57bb0dea89 --- /dev/null +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/documentation/ui/connector-definition/connector-definition.component.ts @@ -0,0 +1,204 @@ +/* + * 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. + */ + +import { Component, OnDestroy, ViewChild, inject } from '@angular/core'; +import { Store } from '@ngrx/store'; +import { NiFiState } from '../../../../state'; +import { NgxSkeletonLoaderModule } from 'ngx-skeleton-loader'; +import { ComponentType, isDefinedAndNotNull, NiFiCommon } from '@nifi/shared'; +import { takeUntilDestroyed } from '@angular/core/rxjs-interop'; +import { selectDefinitionCoordinatesFromRouteForComponentType } from '../../state/documentation/documentation.selectors'; +import { distinctUntilChanged } from 'rxjs'; +import { + ConnectorDefinitionState, + ConfigurationStep, + ConnectorPropertyGroup, + ConnectorPropertyDescriptor, + ConnectorDefinition, + StepDocumentationState +} from '../../state/connector-definition'; +import { + loadConnectorDefinition, + loadStepDocumentation, + resetConnectorDefinitionState +} from '../../state/connector-definition/connector-definition.actions'; +import { selectConnectorDefinitionState } from '../../state/connector-definition/connector-definition.selectors'; +import { MatAccordion, MatExpansionModule } from '@angular/material/expansion'; +import { MatButtonModule } from '@angular/material/button'; +import { SeeAlsoComponent } from '../common/see-also/see-also.component'; +import { MarkdownComponent } from 'ngx-markdown'; +import { ConnectorPropertyDefinitionComponent } from '../common/connector-property-definition/connector-property-definition.component'; + +@Component({ + selector: 'connector-definition', + imports: [ + NgxSkeletonLoaderModule, + MatExpansionModule, + MatButtonModule, + SeeAlsoComponent, + MarkdownComponent, + ConnectorPropertyDefinitionComponent + ], + templateUrl: './connector-definition.component.html', + styleUrl: './connector-definition.component.scss' +}) +export class ConnectorDefinitionComponent implements OnDestroy { + private store = inject>(Store); + private nifiCommon = inject(NiFiCommon); + + @ViewChild('stepsAccordion') stepsAccordion!: MatAccordion; + + connectorDefinitionState: ConnectorDefinitionState | null = null; + + constructor() { + this.store + .select(selectDefinitionCoordinatesFromRouteForComponentType(ComponentType.Connector)) + .pipe( + isDefinedAndNotNull(), + distinctUntilChanged( + (a, b) => + a.group === b.group && a.artifact === b.artifact && a.version === b.version && a.type === b.type + ), + takeUntilDestroyed() + ) + .subscribe((coordinates) => { + this.store.dispatch( + loadConnectorDefinition({ + coordinates + }) + ); + }); + + this.store + .select(selectConnectorDefinitionState) + .pipe(takeUntilDestroyed()) + .subscribe((connectorDefinitionState) => { + const previousState = this.connectorDefinitionState; + this.connectorDefinitionState = connectorDefinitionState; + + if (connectorDefinitionState.status === 'loading') { + window.scrollTo({ top: 0, left: 0 }); + } + + if ( + previousState?.status !== 'success' && + connectorDefinitionState.status === 'success' && + connectorDefinitionState.connectorDefinition + ) { + const firstStep = connectorDefinitionState.connectorDefinition.configurationSteps?.[0]; + if (firstStep?.documented) { + this.loadStepDocumentation(connectorDefinitionState.connectorDefinition, firstStep.name); + } + } + }); + } + + isInitialLoading(state: ConnectorDefinitionState): boolean { + return state.connectorDefinition === null && state.error === null; + } + + formatExtensionName(type: string): string { + return this.nifiCommon.getComponentTypeLabel(type); + } + + hasConfigurationSteps(steps: ConfigurationStep[] | undefined): boolean { + return steps !== undefined && steps.length > 0; + } + + hasPropertyGroups(groups: ConnectorPropertyGroup[] | undefined): boolean { + return groups !== undefined && groups.length > 0; + } + + hasProperties(properties: ConnectorPropertyDescriptor[] | undefined): boolean { + return properties !== undefined && properties.length > 0; + } + + formatPropertyTitle(descriptor: ConnectorPropertyDescriptor): string { + if (descriptor.required) { + return `${descriptor.name}*`; + } + return descriptor.name; + } + + lookupProperty( + properties: ConnectorPropertyDescriptor[] + ): (name: string) => ConnectorPropertyDescriptor | undefined { + return (name: string) => properties.find((prop) => prop.name === name); + } + + expandAllSteps(): void { + this.stepsAccordion.openAll(); + } + + collapseAllSteps(): void { + this.stepsAccordion.closeAll(); + } + + expandAllProperties(accordion: MatAccordion): void { + accordion.openAll(); + } + + collapseAllProperties(accordion: MatAccordion): void { + accordion.closeAll(); + } + + onStepExpanded(connectorDefinition: ConnectorDefinition, step: ConfigurationStep): void { + if (step.documented) { + this.loadStepDocumentation(connectorDefinition, step.name); + } + } + + loadStepDocumentation(connectorDefinition: ConnectorDefinition, stepName: string): void { + const stepState = this.getStepDocumentationState(stepName); + if (stepState && (stepState.status === 'loading' || stepState.status === 'success')) { + return; + } + + this.store.dispatch( + loadStepDocumentation({ + coordinates: { + group: connectorDefinition.group, + artifact: connectorDefinition.artifact, + version: connectorDefinition.version, + type: connectorDefinition.type + }, + stepName + }) + ); + } + + getStepDocumentationState(stepName: string): StepDocumentationState | undefined { + return this.connectorDefinitionState?.stepDocumentation[stepName]; + } + + isStepDocumentationLoading(stepName: string): boolean { + const state = this.getStepDocumentationState(stepName); + return state?.status === 'loading'; + } + + getStepDocumentation(stepName: string): string | undefined { + return this.getStepDocumentationState(stepName)?.documentation ?? undefined; + } + + getStepDocumentationError(stepName: string): string | undefined { + return this.getStepDocumentationState(stepName)?.error ?? undefined; + } + + ngOnDestroy(): void { + this.store.dispatch(resetConnectorDefinitionState()); + } +} diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/feature/flow-designer.module.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/feature/flow-designer.module.ts index d68b3c9f7520..bb3e63033a5d 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/feature/flow-designer.module.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/feature/flow-designer.module.ts @@ -30,6 +30,7 @@ import { ParameterEffects } from '../state/parameter/parameter.effects'; import { QueueEffects } from '../state/queue/queue.effects'; import { BannerText } from '../../../ui/common/banner-text/banner-text.component'; import { FlowAnalysisEffects } from '../state/flow-analysis/flow-analysis.effects'; +import { ComponentTypeNamePipe } from '@nifi/shared'; @NgModule({ declarations: [FlowDesigner], @@ -49,6 +50,7 @@ import { FlowAnalysisEffects } from '../state/flow-analysis/flow-analysis.effect NgOptimizedImage, MatDialogModule, BannerText - ] + ], + providers: [ComponentTypeNamePipe] }) export class FlowDesignerModule {} diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/service/canvas-context-menu.service.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/service/canvas-context-menu.service.ts index ab7cdbe04b1d..2e8baea059e5 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/service/canvas-context-menu.service.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/service/canvas-context-menu.service.ts @@ -887,7 +887,8 @@ export class CanvasContextMenu implements ContextMenuDefinitionProvider { const selectionData = selection.datum(); this.store.dispatch( navigateToProvenanceForComponent({ - id: selectionData.id + id: selectionData.id, + componentType: selectionData.type }) ); } diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/state/flow/flow.actions.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/state/flow/flow.actions.ts index a7fdc481555b..380a437f7eaf 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/state/flow/flow.actions.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/state/flow/flow.actions.ts @@ -121,6 +121,7 @@ import { } from '../../../../state/shared'; import { ErrorContext } from '../../../../state/error'; import { CopyResponseContext, CopyResponseEntity } from '../../../../state/copy'; +import { ComponentType } from '@nifi/shared'; const CANVAS_PREFIX = '[Canvas]'; @@ -614,7 +615,7 @@ export const renderConnectionsForComponent = createAction( export const navigateToProvenanceForComponent = createAction( `${CANVAS_PREFIX} Navigate To Provenance For Component`, - props<{ id: string }>() + props<{ id: string; componentType: ComponentType }>() ); export const replayLastProvenanceEvent = createAction( diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/state/flow/flow.effects.spec.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/state/flow/flow.effects.spec.ts index ba287885d743..70cf7bcc7104 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/state/flow/flow.effects.spec.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/state/flow/flow.effects.spec.ts @@ -58,9 +58,10 @@ import { CopyPasteService } from '../../service/copy-paste.service'; import { CanvasView } from '../../service/canvas-view.service'; import { BirdseyeView } from '../../service/birdseye-view.service'; import { selectDisconnectionAcknowledged } from '../../../../state/cluster-summary/cluster-summary.selectors'; -import { ComponentType } from '@nifi/shared'; +import { ComponentType, ComponentTypeNamePipe } from '@nifi/shared'; import { ParameterContextService } from '../../../parameter-contexts/service/parameter-contexts.service'; import { HttpErrorResponse } from '@angular/common/http'; +import { provideRouter, Router } from '@angular/router'; import { ErrorHelper } from '../../../../service/error-helper.service'; import { selectConnectedStateChanged } from '../../../../state/cluster-summary/cluster-summary.selectors'; @@ -775,6 +776,8 @@ describe('FlowEffects', () => { imports: [], providers: [ FlowEffects, + ComponentTypeNamePipe, + provideRouter([]), provideMockActions(() => action$), provideMockStore({ selectors: [ @@ -1182,4 +1185,103 @@ describe('FlowEffects', () => { ); }); }); + + describe('navigateToProvenanceForComponent$', () => { + let router: Router; + + beforeEach(() => { + router = TestBed.inject(Router); + jest.spyOn(router, 'navigate').mockImplementation(() => Promise.resolve(true)); + store.overrideSelector(selectCurrentProcessGroupId, 'pg-123'); + store.refreshState(); + }); + + it('should navigate to provenance with Processor type in back navigation route and context', () => { + effects.navigateToProvenanceForComponent$.subscribe(); + + action$.next( + FlowActions.navigateToProvenanceForComponent({ + id: 'comp-1', + componentType: ComponentType.Processor + }) + ); + + expect(router.navigate).toHaveBeenCalledWith(['/provenance'], { + queryParams: { componentId: 'comp-1' }, + state: { + backNavigation: { + route: ['/process-groups', 'pg-123', ComponentType.Processor, 'comp-1'], + routeBoundary: ['/provenance'], + context: 'Processor' + } + } + }); + }); + + it('should navigate to provenance with Funnel type in back navigation route and context', () => { + effects.navigateToProvenanceForComponent$.subscribe(); + + action$.next( + FlowActions.navigateToProvenanceForComponent({ + id: 'funnel-1', + componentType: ComponentType.Funnel + }) + ); + + expect(router.navigate).toHaveBeenCalledWith(['/provenance'], { + queryParams: { componentId: 'funnel-1' }, + state: { + backNavigation: { + route: ['/process-groups', 'pg-123', ComponentType.Funnel, 'funnel-1'], + routeBoundary: ['/provenance'], + context: 'Funnel' + } + } + }); + }); + + it('should navigate to provenance with InputPort type in back navigation route and context', () => { + effects.navigateToProvenanceForComponent$.subscribe(); + + action$.next( + FlowActions.navigateToProvenanceForComponent({ + id: 'port-1', + componentType: ComponentType.InputPort + }) + ); + + expect(router.navigate).toHaveBeenCalledWith(['/provenance'], { + queryParams: { componentId: 'port-1' }, + state: { + backNavigation: { + route: ['/process-groups', 'pg-123', ComponentType.InputPort, 'port-1'], + routeBoundary: ['/provenance'], + context: 'Input Port' + } + } + }); + }); + + it('should navigate to provenance with OutputPort type in back navigation route and context', () => { + effects.navigateToProvenanceForComponent$.subscribe(); + + action$.next( + FlowActions.navigateToProvenanceForComponent({ + id: 'port-2', + componentType: ComponentType.OutputPort + }) + ); + + expect(router.navigate).toHaveBeenCalledWith(['/provenance'], { + queryParams: { componentId: 'port-2' }, + state: { + backNavigation: { + route: ['/process-groups', 'pg-123', ComponentType.OutputPort, 'port-2'], + routeBoundary: ['/provenance'], + context: 'Output Port' + } + } + }); + }); + }); }); diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/state/flow/flow.effects.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/state/flow/flow.effects.ts index 658b630af5d3..79fba048ad0a 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/state/flow/flow.effects.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/flow-designer/state/flow/flow.effects.ts @@ -128,6 +128,7 @@ import { EditProcessGroup } from '../../ui/canvas/items/process-group/edit-proce import { ControllerServiceService } from '../../service/controller-service.service'; import { ComponentType, + ComponentTypeNamePipe, isDefinedAndNotNull, LARGE_DIALOG, MEDIUM_DIALOG, @@ -200,6 +201,7 @@ export class FlowEffects { private parameterContextService = inject(ParameterContextService); private extensionTypesService = inject(ExtensionTypesService); private errorHelper = inject(ErrorHelper); + private componentTypeNamePipe = inject(ComponentTypeNamePipe); private copyPasteService = inject(CopyPasteService); private createProcessGroupDialogRef: MatDialogRef | undefined; @@ -2990,16 +2992,16 @@ export class FlowEffects { () => this.actions$.pipe( ofType(FlowActions.navigateToProvenanceForComponent), - map((action) => action.id), + map((action) => ({ id: action.id, componentType: action.componentType })), concatLatestFrom(() => this.store.select(selectCurrentProcessGroupId)), - tap(([componentId, processGroupId]) => { + tap(([{ id: componentId, componentType }, processGroupId]) => { this.router.navigate(['/provenance'], { queryParams: { componentId }, state: { backNavigation: { - route: ['/process-groups', processGroupId, ComponentType.Processor, componentId], + route: ['/process-groups', processGroupId, componentType, componentId], routeBoundary: ['/provenance'], - context: 'Processor' + context: this.componentTypeNamePipe.transform(componentType) } as BackNavigation } }); diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/provenance/state/provenance-event-listing/index.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/provenance/state/provenance-event-listing/index.ts index 8cb87708e5c6..f8d95b4fba2a 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/provenance/state/provenance-event-listing/index.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/provenance/state/provenance-event-listing/index.ts @@ -41,6 +41,7 @@ export interface ProvenanceEventRequest { export interface GoToProvenanceEventSourceRequest { eventId?: number; componentId?: string; + componentType?: string; groupId?: string; clusterNodeId?: string; } diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/provenance/state/provenance-event-listing/provenance-event-listing.effects.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/provenance/state/provenance-event-listing/provenance-event-listing.effects.ts index 46cb2e791526..9ff84a4d93af 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/provenance/state/provenance-event-listing/provenance-event-listing.effects.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/provenance/state/provenance-event-listing/provenance-event-listing.effects.ts @@ -444,7 +444,9 @@ export class ProvenanceEventListingEffects { this.provenanceService.getProvenanceEvent(request.eventId, request.clusterNodeId).subscribe({ next: (response) => { const event: any = response.provenanceEvent; - this.router.navigate(this.getEventComponentLink(event.groupId, event.componentId)); + this.router.navigate( + this.getEventComponentLink(event.groupId, event.componentId, event.componentType) + ); }, error: (errorResponse: HttpErrorResponse) => { this.store.dispatch( @@ -455,7 +457,9 @@ export class ProvenanceEventListingEffects { } }); } else if (request.groupId && request.componentId) { - this.router.navigate(this.getEventComponentLink(request.groupId, request.componentId)); + this.router.navigate( + this.getEventComponentLink(request.groupId, request.componentId, request.componentType) + ); } }) ), @@ -495,15 +499,19 @@ export class ProvenanceEventListingEffects { { dispatch: false } ); - private getEventComponentLink(groupId: string, componentId: string): string[] { + private getEventComponentLink(groupId: string, componentId: string, componentType?: string): string[] { let link: string[]; if (groupId == componentId) { link = ['/process-groups', componentId]; - } else if (componentId === 'Connection' || componentId === 'Load Balanced Connection') { + } else if (componentType === 'Connection' || componentType === 'Load Balanced Connection') { link = ['/process-groups', groupId, 'Connection', componentId]; - } else if (componentId === 'Output Port') { + } else if (componentType === 'Output Port') { link = ['/process-groups', groupId, 'OutputPort', componentId]; + } else if (componentType === 'Input Port') { + link = ['/process-groups', groupId, 'InputPort', componentId]; + } else if (componentType === 'Funnel') { + link = ['/process-groups', groupId, 'Funnel', componentId]; } else { link = ['/process-groups', groupId, 'Processor', componentId]; } diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/provenance/ui/provenance-event-listing/provenance-event-table/provenance-event-table.component.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/provenance/ui/provenance-event-listing/provenance-event-table/provenance-event-table.component.ts index e2df47bb3584..adb62661308e 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/provenance/ui/provenance-event-listing/provenance-event-table/provenance-event-table.component.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/provenance/ui/provenance-event-listing/provenance-event-table/provenance-event-table.component.ts @@ -338,12 +338,13 @@ export class ProvenanceEventTable implements AfterViewInit { return false; } - return !(event.componentId === 'Remote Output Port' || event.componentId === 'Remote Input Port'); + return !(event.componentType === 'Remote Output Port' || event.componentType === 'Remote Input Port'); } goToClicked(event: ProvenanceEventSummary): void { this.goToEventSource({ componentId: event.componentId, + componentType: event.componentType, groupId: event.groupId }); } diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/flow-analysis-rules/flow-analysis-rule-table/flow-analysis-rule-table.component.spec.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/flow-analysis-rules/flow-analysis-rule-table/flow-analysis-rule-table.component.spec.ts index dca5347d2eb5..953caea16845 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/flow-analysis-rules/flow-analysis-rule-table/flow-analysis-rule-table.component.spec.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/flow-analysis-rules/flow-analysis-rule-table/flow-analysis-rule-table.component.spec.ts @@ -134,6 +134,10 @@ describe('FlowAnalysisRuleTable', () => { canRead: true, canWrite: true }, + connectorsPermissions: { + canRead: true, + canWrite: true + }, componentRestrictionPermissions: [], canVersionFlows: true, logoutSupported: true diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/management-controller-services/management-controller-services.component.spec.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/management-controller-services/management-controller-services.component.spec.ts index 55a5cf4db5ed..82f7012812fa 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/management-controller-services/management-controller-services.component.spec.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/management-controller-services/management-controller-services.component.spec.ts @@ -127,6 +127,10 @@ describe('ManagementControllerServices', () => { canRead: true, canWrite: true }, + connectorsPermissions: { + canRead: true, + canWrite: true + }, restrictedComponentsPermissions: { canRead: true, canWrite: true diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/parameter-providers/parameter-providers-table/parameter-providers-table.component.spec.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/parameter-providers/parameter-providers-table/parameter-providers-table.component.spec.ts index 0133931981f8..76d51030a015 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/parameter-providers/parameter-providers-table/parameter-providers-table.component.spec.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/parameter-providers/parameter-providers-table/parameter-providers-table.component.spec.ts @@ -122,6 +122,10 @@ describe('ParameterProvidersTable', () => { canRead: true, canWrite: true }, + connectorsPermissions: { + canRead: true, + canWrite: true + }, restrictedComponentsPermissions: { canRead: true, canWrite: true diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/reporting-tasks/reporting-task-table/reporting-task-table.component.spec.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/reporting-tasks/reporting-task-table/reporting-task-table.component.spec.ts index 9008c53a15c1..cc79f9c2283d 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/reporting-tasks/reporting-task-table/reporting-task-table.component.spec.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/settings/ui/reporting-tasks/reporting-task-table/reporting-task-table.component.spec.ts @@ -138,6 +138,10 @@ describe('ReportingTaskTable', () => { controllerPermissions: { canRead: true, canWrite: true + }, + connectorsPermissions: { + canRead: true, + canWrite: true } }; } diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/users/ui/user-listing/user-table/user-table.component.spec.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/users/ui/user-listing/user-table/user-table.component.spec.ts index c4258373bbf4..e69bf072f9de 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/users/ui/user-listing/user-table/user-table.component.spec.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/pages/users/ui/user-listing/user-table/user-table.component.spec.ts @@ -61,6 +61,10 @@ describe('UserTable', () => { canRead: true, canWrite: true }, + connectorsPermissions: { + canRead: false, + canWrite: false + }, restrictedComponentsPermissions: { canRead: false, canWrite: true diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/service/extension-types.service.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/service/extension-types.service.ts index 477b1b6510cf..f76675842516 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/service/extension-types.service.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/service/extension-types.service.ts @@ -99,4 +99,8 @@ export class ExtensionTypesService { getParameterProviderTypes(): Observable { return this.httpClient.get(`${ExtensionTypesService.API}/flow/parameter-provider-types`); } + + getConnectorTypes(): Observable { + return this.httpClient.get(`${ExtensionTypesService.API}/flow/connector-types`); + } } diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/current-user/current-user.reducer.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/current-user/current-user.reducer.ts index a1b57ea94fd3..3825549351f3 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/current-user/current-user.reducer.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/current-user/current-user.reducer.ts @@ -34,6 +34,7 @@ export const initialState: CurrentUserState = { controllerPermissions: NO_PERMISSIONS, countersPermissions: NO_PERMISSIONS, parameterContextPermissions: NO_PERMISSIONS, + connectorsPermissions: NO_PERMISSIONS, policiesPermissions: NO_PERMISSIONS, provenancePermissions: NO_PERMISSIONS, restrictedComponentsPermissions: NO_PERMISSIONS, diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/current-user/index.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/current-user/index.ts index 4dfd972c66d6..2f05556c7c38 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/current-user/index.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/current-user/index.ts @@ -41,6 +41,7 @@ export interface CurrentUser { policiesPermissions: Permissions; systemPermissions: Permissions; parameterContextPermissions: Permissions; + connectorsPermissions: Permissions; restrictedComponentsPermissions: Permissions; componentRestrictionPermissions: ComponentRestrictionPermission[]; } diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/extension-types.effects.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/extension-types.effects.ts index 256d15e192ea..98d411850acb 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/extension-types.effects.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/extension-types.effects.ts @@ -136,7 +136,8 @@ export class ExtensionTypesEffects { this.extensionTypesService.getReportingTaskTypes(), this.extensionTypesService.getRegistryClientTypes(), this.extensionTypesService.getParameterProviderTypes(), - this.extensionTypesService.getFlowAnalysisRuleTypes() + this.extensionTypesService.getFlowAnalysisRuleTypes(), + this.extensionTypesService.getConnectorTypes() ]).pipe( map( ([ @@ -145,7 +146,8 @@ export class ExtensionTypesEffects { reportingTaskTypes, registryClientTypes, parameterProviderTypes, - flowAnalysisRuleTypes + flowAnalysisRuleTypes, + connectorTypes ]) => ExtensionTypesActions.loadExtensionTypesForDocumentationSuccess({ response: { @@ -154,7 +156,8 @@ export class ExtensionTypesEffects { reportingTaskTypes: reportingTaskTypes.reportingTaskTypes, registryClientTypes: registryClientTypes.flowRegistryClientTypes, parameterProviderTypes: parameterProviderTypes.parameterProviderTypes, - flowAnalysisRuleTypes: flowAnalysisRuleTypes.flowAnalysisRuleTypes + flowAnalysisRuleTypes: flowAnalysisRuleTypes.flowAnalysisRuleTypes, + connectorTypes: connectorTypes.connectorTypes } }) ), diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/extension-types.reducer.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/extension-types.reducer.ts index a06944ca42b3..3ac1f8c91824 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/extension-types.reducer.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/extension-types.reducer.ts @@ -37,6 +37,7 @@ export const initialExtensionsTypesState: ExtensionTypesState = { registryClientTypes: [], flowAnalysisRuleTypes: [], parameterProviderTypes: [], + connectorTypes: [], status: 'pending' }; @@ -56,6 +57,7 @@ export const extensionTypesReducer = createReducer( registryClientTypes: [], parameterProviderTypes: [], flowAnalysisRuleTypes: [], + connectorTypes: [], status: 'loading' as const }) ), @@ -92,6 +94,7 @@ export const extensionTypesReducer = createReducer( registryClientTypes: response.registryClientTypes, parameterProviderTypes: response.parameterProviderTypes, flowAnalysisRuleTypes: response.flowAnalysisRuleTypes, + connectorTypes: response.connectorTypes, status: 'success' as const })), on(extensionTypesApiError, (state) => ({ diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/extension-types.selectors.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/extension-types.selectors.ts index 65cc3e4f57c9..059a4da62e68 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/extension-types.selectors.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/extension-types.selectors.ts @@ -61,6 +61,11 @@ export const selectParameterProviderTypes = createSelector( (state: ExtensionTypesState) => state.parameterProviderTypes ); +export const selectConnectorTypes = createSelector( + selectExtensionTypesState, + (state: ExtensionTypesState) => state.connectorTypes +); + export const selectTypesToIdentifyComponentRestrictions = createSelector( selectExtensionTypesState, (state: ExtensionTypesState) => { @@ -139,6 +144,7 @@ export const selectExtensionFromTypes = (extensionTypes: string[]) => reportingTaskTypes: state.reportingTaskTypes.filter(typeFilter), registryClientTypes: state.registryClientTypes.filter(typeFilter), parameterProviderTypes: state.parameterProviderTypes.filter(typeFilter), - flowAnalysisRuleTypes: state.flowAnalysisRuleTypes.filter(typeFilter) + flowAnalysisRuleTypes: state.flowAnalysisRuleTypes.filter(typeFilter), + connectorTypes: state.connectorTypes.filter(typeFilter) } as LoadExtensionTypesForDocumentationResponse; }); diff --git a/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/index.ts b/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/index.ts index 05d2d351d090..249772cb9422 100644 --- a/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/index.ts +++ b/nifi-frontend/src/main/frontend/apps/nifi/src/app/state/extension-types/index.ts @@ -50,6 +50,7 @@ export interface LoadExtensionTypesForDocumentationResponse { registryClientTypes: DocumentedType[]; flowAnalysisRuleTypes: DocumentedType[]; parameterProviderTypes: DocumentedType[]; + connectorTypes: DocumentedType[]; } export interface ExtensionTypesState { @@ -60,5 +61,6 @@ export interface ExtensionTypesState { registryClientTypes: DocumentedType[]; flowAnalysisRuleTypes: DocumentedType[]; parameterProviderTypes: DocumentedType[]; + connectorTypes: DocumentedType[]; status: ExtensionTypesLoadingStatus; } diff --git a/nifi-frontend/src/main/frontend/libs/shared/src/services/nifi-common.service.ts b/nifi-frontend/src/main/frontend/libs/shared/src/services/nifi-common.service.ts index bb6e285c85fd..75d5973b5cb9 100644 --- a/nifi-frontend/src/main/frontend/libs/shared/src/services/nifi-common.service.ts +++ b/nifi-frontend/src/main/frontend/libs/shared/src/services/nifi-common.service.ts @@ -63,6 +63,11 @@ export class NiFiCommon { value: 'parameter-contexts', description: 'Allows users to view/modify Parameter Contexts' }, + { + text: 'access connectors', + value: 'connectors', + description: 'Allows users to view/modify Connectors' + }, { text: 'query provenance', value: 'provenance', diff --git a/nifi-frontend/src/main/frontend/libs/shared/src/types/index.ts b/nifi-frontend/src/main/frontend/libs/shared/src/types/index.ts index 5f1e21ab133e..25e922995dcc 100644 --- a/nifi-frontend/src/main/frontend/libs/shared/src/types/index.ts +++ b/nifi-frontend/src/main/frontend/libs/shared/src/types/index.ts @@ -57,6 +57,7 @@ export enum ComponentType { FlowAnalysisRule = 'FlowAnalysisRule', ParameterProvider = 'ParameterProvider', FlowRegistryClient = 'FlowRegistryClient', + Connector = 'Connector', Flow = 'Flow' } diff --git a/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConfigurationStep.java b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConfigurationStep.java new file mode 100644 index 000000000000..457e6d8fe8b1 --- /dev/null +++ b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConfigurationStep.java @@ -0,0 +1,80 @@ +/* + * 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.extension.manifest; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlAccessType; +import jakarta.xml.bind.annotation.XmlAccessorType; +import jakarta.xml.bind.annotation.XmlElement; +import jakarta.xml.bind.annotation.XmlElementWrapper; + +import java.util.List; + +/** + * Represents a configuration step for a Connector. + */ +@XmlAccessorType(XmlAccessType.FIELD) +public class ConfigurationStep { + + private String name; + private String description; + + @XmlElementWrapper + @XmlElement(name = "stepDependency") + private List stepDependencies; + + @XmlElementWrapper + @XmlElement(name = "propertyGroup") + private List propertyGroups; + + @Schema(description = "The name of the configuration step") + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + @Schema(description = "The description of the configuration step") + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + @Schema(description = "The dependencies that this step has on other steps") + public List getStepDependencies() { + return stepDependencies; + } + + public void setStepDependencies(List stepDependencies) { + this.stepDependencies = stepDependencies; + } + + @Schema(description = "The property groups in this configuration step") + public List getPropertyGroups() { + return propertyGroups; + } + + public void setPropertyGroups(List propertyGroups) { + this.propertyGroups = propertyGroups; + } +} + diff --git a/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConfigurationStepDependency.java b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConfigurationStepDependency.java new file mode 100644 index 000000000000..ac4381b3e075 --- /dev/null +++ b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConfigurationStepDependency.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.extension.manifest; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlAccessType; +import jakarta.xml.bind.annotation.XmlAccessorType; +import jakarta.xml.bind.annotation.XmlElement; +import jakarta.xml.bind.annotation.XmlElementWrapper; + +import java.util.List; + +/** + * Represents a dependency that a configuration step has on another step's property. + */ +@XmlAccessorType(XmlAccessType.FIELD) +public class ConfigurationStepDependency { + + private String stepName; + private String propertyName; + + @XmlElementWrapper + @XmlElement(name = "dependentValue") + private List dependentValues; + + @Schema(description = "The name of the step that this step depends on") + public String getStepName() { + return stepName; + } + + public void setStepName(String stepName) { + this.stepName = stepName; + } + + @Schema(description = "The name of the property within the step that this step depends on") + public String getPropertyName() { + return propertyName; + } + + public void setPropertyName(String propertyName) { + this.propertyName = propertyName; + } + + @Schema(description = "The values of the dependent property that enable this step") + public List getDependentValues() { + return dependentValues; + } + + public void setDependentValues(List dependentValues) { + this.dependentValues = dependentValues; + } +} + diff --git a/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorProperty.java b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorProperty.java new file mode 100644 index 000000000000..812a45a88090 --- /dev/null +++ b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorProperty.java @@ -0,0 +1,120 @@ +/* + * 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.extension.manifest; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlAccessType; +import jakarta.xml.bind.annotation.XmlAccessorType; +import jakarta.xml.bind.annotation.XmlElement; +import jakarta.xml.bind.annotation.XmlElementWrapper; + +import java.util.List; + +/** + * Represents a property descriptor for a Connector. + */ +@XmlAccessorType(XmlAccessType.FIELD) +public class ConnectorProperty { + + private String name; + private String description; + private String defaultValue; + private boolean required; + private ConnectorPropertyType propertyType; + private boolean allowableValuesFetchable; + + @XmlElementWrapper + @XmlElement(name = "allowableValue") + private List allowableValues; + + @XmlElementWrapper + @XmlElement(name = "dependency") + private List dependencies; + + @Schema(description = "The name of the property") + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + @Schema(description = "The description of the property") + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + @Schema(description = "The default value of the property") + public String getDefaultValue() { + return defaultValue; + } + + public void setDefaultValue(String defaultValue) { + this.defaultValue = defaultValue; + } + + @Schema(description = "Whether or not the property is required") + public boolean isRequired() { + return required; + } + + public void setRequired(boolean required) { + this.required = required; + } + + @Schema(description = "The type of the property") + public ConnectorPropertyType getPropertyType() { + return propertyType; + } + + public void setPropertyType(ConnectorPropertyType propertyType) { + this.propertyType = propertyType; + } + + @Schema(description = "Whether or not the allowable values can be fetched dynamically") + public boolean isAllowableValuesFetchable() { + return allowableValuesFetchable; + } + + public void setAllowableValuesFetchable(boolean allowableValuesFetchable) { + this.allowableValuesFetchable = allowableValuesFetchable; + } + + @Schema(description = "The allowable values for this property") + public List getAllowableValues() { + return allowableValues; + } + + public void setAllowableValues(List allowableValues) { + this.allowableValues = allowableValues; + } + + @Schema(description = "The properties that this property depends on") + public List getDependencies() { + return dependencies; + } + + public void setDependencies(List dependencies) { + this.dependencies = dependencies; + } +} + diff --git a/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorPropertyDependency.java b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorPropertyDependency.java new file mode 100644 index 000000000000..dc1fb67a6b84 --- /dev/null +++ b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorPropertyDependency.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.extension.manifest; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlAccessType; +import jakarta.xml.bind.annotation.XmlAccessorType; +import jakarta.xml.bind.annotation.XmlElement; +import jakarta.xml.bind.annotation.XmlElementWrapper; + +import java.util.List; + +/** + * Represents a dependency that a connector property has on another property. + */ +@XmlAccessorType(XmlAccessType.FIELD) +public class ConnectorPropertyDependency { + + private String propertyName; + + @XmlElementWrapper + @XmlElement(name = "dependentValue") + private List dependentValues; + + @Schema(description = "The name of the property that this property depends on") + public String getPropertyName() { + return propertyName; + } + + public void setPropertyName(String propertyName) { + this.propertyName = propertyName; + } + + @Schema(description = "The values of the dependent property that enable this property") + public List getDependentValues() { + return dependentValues; + } + + public void setDependentValues(List dependentValues) { + this.dependentValues = dependentValues; + } +} + diff --git a/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorPropertyGroup.java b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorPropertyGroup.java new file mode 100644 index 000000000000..e6d411517c32 --- /dev/null +++ b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorPropertyGroup.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.extension.manifest; + +import io.swagger.v3.oas.annotations.media.Schema; +import jakarta.xml.bind.annotation.XmlAccessType; +import jakarta.xml.bind.annotation.XmlAccessorType; +import jakarta.xml.bind.annotation.XmlElement; +import jakarta.xml.bind.annotation.XmlElementWrapper; + +import java.util.List; + +/** + * Represents a group of properties within a configuration step. + */ +@XmlAccessorType(XmlAccessType.FIELD) +public class ConnectorPropertyGroup { + + private String name; + private String description; + + @XmlElementWrapper + @XmlElement(name = "property") + private List properties; + + @Schema(description = "The name of the property group") + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + @Schema(description = "The description of the property group") + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + @Schema(description = "The properties in this group") + public List getProperties() { + return properties; + } + + public void setProperties(List properties) { + this.properties = properties; + } +} + diff --git a/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorPropertyType.java b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorPropertyType.java new file mode 100644 index 000000000000..07ced332c753 --- /dev/null +++ b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ConnectorPropertyType.java @@ -0,0 +1,43 @@ +/* + * 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.extension.manifest; + +/** + * Possible types for connector properties. + */ +public enum ConnectorPropertyType { + + STRING, + + INTEGER, + + BOOLEAN, + + FLOAT, + + DOUBLE, + + STRING_LIST, + + SECRET, + + ASSET, + + ASSET_LIST; + +} + diff --git a/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/Extension.java b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/Extension.java index 53e739cbadf9..56ba1d4928e4 100644 --- a/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/Extension.java +++ b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/Extension.java @@ -107,6 +107,11 @@ public class Extension { @XmlElement(name = "multiProcessorUseCase") private List multiProcessorUseCases; + // Connector-specific fields + @XmlElementWrapper + @XmlElement(name = "configurationStep") + private List configurationSteps; + @Schema(description = "The name of the extension") public String getName() { return name; @@ -358,6 +363,15 @@ public void setMultiProcessorUseCases(final List multiPro this.multiProcessorUseCases = multiProcessorUseCases; } + @Schema(description = "The configuration steps for a Connector extension") + public List getConfigurationSteps() { + return configurationSteps; + } + + public void setConfigurationSteps(final List configurationSteps) { + this.configurationSteps = configurationSteps; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ExtensionType.java b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ExtensionType.java index 0413e260ed12..a72511dabd09 100644 --- a/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ExtensionType.java +++ b/nifi-manifest/nifi-extension-manifest-model/src/main/java/org/apache/nifi/extension/manifest/ExtensionType.java @@ -31,6 +31,8 @@ public enum ExtensionType { PARAMETER_PROVIDER, - FLOW_REGISTRY_CLIENT; + FLOW_REGISTRY_CLIENT, + + CONNECTOR; } diff --git a/nifi-manifest/nifi-runtime-manifest-core/src/main/java/org/apache/nifi/runtime/manifest/ComponentManifestBuilder.java b/nifi-manifest/nifi-runtime-manifest-core/src/main/java/org/apache/nifi/runtime/manifest/ComponentManifestBuilder.java index 9eb3f697ffb0..4ced8551411e 100644 --- a/nifi-manifest/nifi-runtime-manifest-core/src/main/java/org/apache/nifi/runtime/manifest/ComponentManifestBuilder.java +++ b/nifi-manifest/nifi-runtime-manifest-core/src/main/java/org/apache/nifi/runtime/manifest/ComponentManifestBuilder.java @@ -17,6 +17,7 @@ package org.apache.nifi.runtime.manifest; import org.apache.nifi.c2.protocol.component.api.ComponentManifest; +import org.apache.nifi.c2.protocol.component.api.ConnectorDefinition; import org.apache.nifi.c2.protocol.component.api.ControllerServiceDefinition; import org.apache.nifi.c2.protocol.component.api.FlowAnalysisRuleDefinition; import org.apache.nifi.c2.protocol.component.api.FlowRegistryClientDefinition; @@ -65,6 +66,12 @@ public interface ComponentManifestBuilder { */ ComponentManifestBuilder addFlowRegistryClient(FlowRegistryClientDefinition flowRegistryClientDefinition); + /** + * @param connectorDefinition a connector definition to add + * @return the builder + */ + ComponentManifestBuilder addConnector(ConnectorDefinition connectorDefinition); + /** * @return a component manifest containing all the added definitions */ diff --git a/nifi-manifest/nifi-runtime-manifest-core/src/main/java/org/apache/nifi/runtime/manifest/impl/StandardComponentManifestBuilder.java b/nifi-manifest/nifi-runtime-manifest-core/src/main/java/org/apache/nifi/runtime/manifest/impl/StandardComponentManifestBuilder.java index b6b67181bc6d..712aab218768 100644 --- a/nifi-manifest/nifi-runtime-manifest-core/src/main/java/org/apache/nifi/runtime/manifest/impl/StandardComponentManifestBuilder.java +++ b/nifi-manifest/nifi-runtime-manifest-core/src/main/java/org/apache/nifi/runtime/manifest/impl/StandardComponentManifestBuilder.java @@ -17,6 +17,7 @@ package org.apache.nifi.runtime.manifest.impl; import org.apache.nifi.c2.protocol.component.api.ComponentManifest; +import org.apache.nifi.c2.protocol.component.api.ConnectorDefinition; import org.apache.nifi.c2.protocol.component.api.ControllerServiceDefinition; import org.apache.nifi.c2.protocol.component.api.FlowAnalysisRuleDefinition; import org.apache.nifi.c2.protocol.component.api.FlowRegistryClientDefinition; @@ -39,6 +40,7 @@ public class StandardComponentManifestBuilder implements ComponentManifestBuilde private final List parameterProviders = new ArrayList<>(); private final List flowAnalysisRules = new ArrayList<>(); private final List flowRegistryClients = new ArrayList<>(); + private final List connectors = new ArrayList<>(); @Override public ComponentManifestBuilder addProcessor(final ProcessorDefinition processorDefinition) { @@ -94,6 +96,15 @@ public ComponentManifestBuilder addFlowRegistryClient(final FlowRegistryClientDe return this; } + @Override + public ComponentManifestBuilder addConnector(final ConnectorDefinition connectorDefinition) { + if (connectorDefinition == null) { + throw new IllegalArgumentException("Connector definition cannot be null"); + } + connectors.add(connectorDefinition); + return this; + } + @Override public ComponentManifest build() { final ComponentManifest componentManifest = new ComponentManifest(); @@ -103,6 +114,7 @@ public ComponentManifest build() { componentManifest.setParameterProviders(new ArrayList<>(parameterProviders)); componentManifest.setFlowAnalysisRules(new ArrayList<>(flowAnalysisRules)); componentManifest.setFlowRegistryClients(new ArrayList<>(flowRegistryClients)); + componentManifest.setConnectors(new ArrayList<>(connectors)); return componentManifest; } diff --git a/nifi-manifest/nifi-runtime-manifest-core/src/main/java/org/apache/nifi/runtime/manifest/impl/StandardRuntimeManifestBuilder.java b/nifi-manifest/nifi-runtime-manifest-core/src/main/java/org/apache/nifi/runtime/manifest/impl/StandardRuntimeManifestBuilder.java index a080378e5eae..91318bac00a6 100644 --- a/nifi-manifest/nifi-runtime-manifest-core/src/main/java/org/apache/nifi/runtime/manifest/impl/StandardRuntimeManifestBuilder.java +++ b/nifi-manifest/nifi-runtime-manifest-core/src/main/java/org/apache/nifi/runtime/manifest/impl/StandardRuntimeManifestBuilder.java @@ -20,6 +20,13 @@ import org.apache.nifi.c2.protocol.component.api.BuildInfo; import org.apache.nifi.c2.protocol.component.api.Bundle; import org.apache.nifi.c2.protocol.component.api.ConfigurableComponentDefinition; +import org.apache.nifi.c2.protocol.component.api.ConfigurationStep; +import org.apache.nifi.c2.protocol.component.api.ConfigurationStepDependency; +import org.apache.nifi.c2.protocol.component.api.ConnectorDefinition; +import org.apache.nifi.c2.protocol.component.api.ConnectorPropertyDependency; +import org.apache.nifi.c2.protocol.component.api.ConnectorPropertyDescriptor; +import org.apache.nifi.c2.protocol.component.api.ConnectorPropertyGroup; +import org.apache.nifi.c2.protocol.component.api.ConnectorPropertyType; import org.apache.nifi.c2.protocol.component.api.ControllerServiceDefinition; import org.apache.nifi.c2.protocol.component.api.DefinedType; import org.apache.nifi.c2.protocol.component.api.ExtensionComponent; @@ -46,6 +53,7 @@ import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.extension.manifest.AllowableValue; import org.apache.nifi.extension.manifest.Attribute; +import org.apache.nifi.extension.manifest.ConnectorProperty; import org.apache.nifi.extension.manifest.DefaultSchedule; import org.apache.nifi.extension.manifest.DefaultSettings; import org.apache.nifi.extension.manifest.Dependency; @@ -226,6 +234,9 @@ private void addExtension(final ExtensionManifest extensionManifest, final Exten case FLOW_REGISTRY_CLIENT: addFlowRegistryClientDefinition(extensionManifest, extension, additionalDetails, componentManifestBuilder); break; + case CONNECTOR: + addConnectorDefinition(extensionManifest, extension, additionalDetails, componentManifestBuilder); + break; } } @@ -410,6 +421,121 @@ private void addFlowRegistryClientDefinition(final ExtensionManifest extensionMa componentManifestBuilder.addFlowRegistryClient(flowRegistryClientDefinition); } + private void addConnectorDefinition(final ExtensionManifest extensionManifest, final Extension extension, final String additionalDetails, + final ComponentManifestBuilder componentManifestBuilder) { + final ConnectorDefinition connectorDefinition = new ConnectorDefinition(); + populateDefinedType(extensionManifest, extension, connectorDefinition); + populateExtensionComponent(extensionManifest, extension, additionalDetails, connectorDefinition); + + // Populate configuration steps + final List manifestSteps = extension.getConfigurationSteps(); + if (isNotEmpty(manifestSteps)) { + connectorDefinition.setConfigurationSteps( + manifestSteps.stream() + .map(this::getConfigurationStep) + .collect(Collectors.toList()) + ); + } + + componentManifestBuilder.addConnector(connectorDefinition); + } + + private ConfigurationStep getConfigurationStep(final org.apache.nifi.extension.manifest.ConfigurationStep manifestStep) { + final ConfigurationStep step = new ConfigurationStep(); + step.setName(manifestStep.getName()); + step.setDescription(manifestStep.getDescription()); + + // Convert step dependencies + final List manifestDeps = manifestStep.getStepDependencies(); + if (isNotEmpty(manifestDeps)) { + step.setStepDependencies( + manifestDeps.stream() + .map(this::getConfigurationStepDependency) + .collect(Collectors.toList()) + ); + } + + // Convert property groups + final List manifestGroups = manifestStep.getPropertyGroups(); + if (isNotEmpty(manifestGroups)) { + step.setPropertyGroups( + manifestGroups.stream() + .map(this::getConnectorPropertyGroup) + .collect(Collectors.toList()) + ); + } + + return step; + } + + private ConfigurationStepDependency getConfigurationStepDependency( + final org.apache.nifi.extension.manifest.ConfigurationStepDependency manifestDep) { + final ConfigurationStepDependency dep = new ConfigurationStepDependency(); + dep.setStepName(manifestDep.getStepName()); + dep.setPropertyName(manifestDep.getPropertyName()); + dep.setDependentValues(manifestDep.getDependentValues()); + return dep; + } + + private ConnectorPropertyGroup getConnectorPropertyGroup( + final org.apache.nifi.extension.manifest.ConnectorPropertyGroup manifestGroup) { + final ConnectorPropertyGroup group = new ConnectorPropertyGroup(); + group.setName(manifestGroup.getName()); + group.setDescription(manifestGroup.getDescription()); + + final List manifestProps = manifestGroup.getProperties(); + if (isNotEmpty(manifestProps)) { + group.setProperties( + manifestProps.stream() + .map(this::getConnectorPropertyDescriptor) + .collect(Collectors.toList()) + ); + } + + return group; + } + + private ConnectorPropertyDescriptor getConnectorPropertyDescriptor(final ConnectorProperty manifestProp) { + final ConnectorPropertyDescriptor prop = new ConnectorPropertyDescriptor(); + prop.setName(manifestProp.getName()); + prop.setDescription(manifestProp.getDescription()); + prop.setDefaultValue(manifestProp.getDefaultValue()); + prop.setRequired(manifestProp.isRequired()); + prop.setAllowableValuesFetchable(manifestProp.isAllowableValuesFetchable()); + + // Convert property type + final org.apache.nifi.extension.manifest.ConnectorPropertyType manifestType = manifestProp.getPropertyType(); + if (manifestType != null) { + prop.setPropertyType(ConnectorPropertyType.valueOf(manifestType.name())); + } + + // Convert allowable values + final List manifestValues = manifestProp.getAllowableValues(); + if (isNotEmpty(manifestValues)) { + prop.setAllowableValues(getPropertyAllowableValues(manifestValues)); + } + + // Convert property dependencies + final List manifestDeps = manifestProp.getDependencies(); + if (isNotEmpty(manifestDeps)) { + prop.setDependencies( + manifestDeps.stream() + .map(this::getConnectorPropertyDependency) + .collect(Collectors.toList()) + ); + } + + return prop; + } + + private ConnectorPropertyDependency getConnectorPropertyDependency( + final org.apache.nifi.extension.manifest.ConnectorPropertyDependency manifestDep) { + final ConnectorPropertyDependency dep = new ConnectorPropertyDependency(); + dep.setPropertyName(manifestDep.getPropertyName()); + dep.setDependentValues(manifestDep.getDependentValues()); + return dep; + } + private void addFlowAnalysisRuleDefinition(final ExtensionManifest extensionManifest, final Extension extension, final String additionalDetails, final ComponentManifestBuilder componentManifestBuilder) { final FlowAnalysisRuleDefinition flowAnalysisRuleDefinition = new FlowAnalysisRuleDefinition(); diff --git a/nifi-manifest/nifi-runtime-manifest-core/src/test/java/org/apache/nifi/runtime/manifest/impl/StandardRuntimeManifestBuilderConnectorTest.java b/nifi-manifest/nifi-runtime-manifest-core/src/test/java/org/apache/nifi/runtime/manifest/impl/StandardRuntimeManifestBuilderConnectorTest.java new file mode 100644 index 000000000000..8585dc479574 --- /dev/null +++ b/nifi-manifest/nifi-runtime-manifest-core/src/test/java/org/apache/nifi/runtime/manifest/impl/StandardRuntimeManifestBuilderConnectorTest.java @@ -0,0 +1,349 @@ +/* + * 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.runtime.manifest.impl; + +import org.apache.nifi.c2.protocol.component.api.ConfigurationStep; +import org.apache.nifi.c2.protocol.component.api.ConfigurationStepDependency; +import org.apache.nifi.c2.protocol.component.api.ConnectorDefinition; +import org.apache.nifi.c2.protocol.component.api.ConnectorPropertyDescriptor; +import org.apache.nifi.c2.protocol.component.api.ConnectorPropertyGroup; +import org.apache.nifi.c2.protocol.component.api.ConnectorPropertyType; +import org.apache.nifi.c2.protocol.component.api.RuntimeManifest; +import org.apache.nifi.extension.manifest.AllowableValue; +import org.apache.nifi.extension.manifest.ConnectorProperty; +import org.apache.nifi.extension.manifest.ConnectorPropertyDependency; +import org.apache.nifi.extension.manifest.Extension; +import org.apache.nifi.extension.manifest.ExtensionManifest; +import org.apache.nifi.extension.manifest.ExtensionType; +import org.apache.nifi.extension.manifest.ParentNar; +import org.apache.nifi.runtime.manifest.ExtensionManifestContainer; +import org.junit.jupiter.api.Test; + +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class StandardRuntimeManifestBuilderConnectorTest { + + private static final String TEST_CONNECTOR_TYPE = "org.apache.nifi.connectors.TestConnector"; + private static final String TEST_GROUP = "org.apache.nifi"; + private static final String TEST_ARTIFACT = "nifi-test-nar"; + private static final String TEST_VERSION = "2.0.0"; + private static final String STEP_NAME = "Connection Settings"; + private static final String STEP_DESCRIPTION = "Configure connection parameters"; + private static final String GROUP_NAME = "Authentication"; + private static final String GROUP_DESCRIPTION = "Authentication settings"; + private static final String PROPERTY_NAME = "username"; + private static final String PROPERTY_DESCRIPTION = "The username for authentication"; + private static final String PROPERTY_DEFAULT = "admin"; + + @Test + void testConnectorDefinitionIsBuiltFromExtensionManifest() { + final ExtensionManifest extensionManifest = createExtensionManifest(); + final Extension connectorExtension = createConnectorExtension(); + extensionManifest.setExtensions(List.of(connectorExtension)); + + final ExtensionManifestContainer container = new ExtensionManifestContainer(extensionManifest, null); + final RuntimeManifest runtimeManifest = new StandardRuntimeManifestBuilder() + .addBundle(container) + .build(); + + assertNotNull(runtimeManifest); + assertNotNull(runtimeManifest.getBundles()); + assertEquals(1, runtimeManifest.getBundles().size()); + + final List connectors = runtimeManifest.getBundles().get(0).getComponentManifest().getConnectors(); + assertNotNull(connectors); + assertEquals(1, connectors.size()); + + final ConnectorDefinition connector = connectors.get(0); + assertEquals(TEST_CONNECTOR_TYPE, connector.getType()); + } + + @Test + void testConfigurationStepsAreConverted() { + final ExtensionManifest extensionManifest = createExtensionManifest(); + final Extension connectorExtension = createConnectorExtensionWithConfigurationSteps(); + extensionManifest.setExtensions(List.of(connectorExtension)); + + final ExtensionManifestContainer container = new ExtensionManifestContainer(extensionManifest, null); + final RuntimeManifest runtimeManifest = new StandardRuntimeManifestBuilder() + .addBundle(container) + .build(); + + final ConnectorDefinition connector = runtimeManifest.getBundles().get(0).getComponentManifest().getConnectors().get(0); + assertNotNull(connector.getConfigurationSteps()); + assertEquals(1, connector.getConfigurationSteps().size()); + + final ConfigurationStep step = connector.getConfigurationSteps().get(0); + assertEquals(STEP_NAME, step.getName()); + assertEquals(STEP_DESCRIPTION, step.getDescription()); + } + + @Test + void testConfigurationStepDependenciesAreConverted() { + final ExtensionManifest extensionManifest = createExtensionManifest(); + final Extension connectorExtension = createConnectorExtensionWithStepDependencies(); + extensionManifest.setExtensions(List.of(connectorExtension)); + + final ExtensionManifestContainer container = new ExtensionManifestContainer(extensionManifest, null); + final RuntimeManifest runtimeManifest = new StandardRuntimeManifestBuilder() + .addBundle(container) + .build(); + + final ConfigurationStep step = runtimeManifest.getBundles().get(0).getComponentManifest().getConnectors().get(0).getConfigurationSteps().get(0); + assertNotNull(step.getStepDependencies()); + assertEquals(1, step.getStepDependencies().size()); + + final ConfigurationStepDependency dependency = step.getStepDependencies().get(0); + assertEquals("previousStep", dependency.getStepName()); + assertEquals("connectionType", dependency.getPropertyName()); + assertEquals(List.of("advanced"), dependency.getDependentValues()); + } + + @Test + void testPropertyGroupsAreConverted() { + final ExtensionManifest extensionManifest = createExtensionManifest(); + final Extension connectorExtension = createConnectorExtensionWithPropertyGroups(); + extensionManifest.setExtensions(List.of(connectorExtension)); + + final ExtensionManifestContainer container = new ExtensionManifestContainer(extensionManifest, null); + final RuntimeManifest runtimeManifest = new StandardRuntimeManifestBuilder() + .addBundle(container) + .build(); + + final ConfigurationStep step = runtimeManifest.getBundles().get(0).getComponentManifest().getConnectors().get(0).getConfigurationSteps().get(0); + assertNotNull(step.getPropertyGroups()); + assertEquals(1, step.getPropertyGroups().size()); + + final ConnectorPropertyGroup group = step.getPropertyGroups().get(0); + assertEquals(GROUP_NAME, group.getName()); + assertEquals(GROUP_DESCRIPTION, group.getDescription()); + } + + @Test + void testConnectorPropertiesAreConverted() { + final ExtensionManifest extensionManifest = createExtensionManifest(); + final Extension connectorExtension = createConnectorExtensionWithProperties(); + extensionManifest.setExtensions(List.of(connectorExtension)); + + final ExtensionManifestContainer container = new ExtensionManifestContainer(extensionManifest, null); + final RuntimeManifest runtimeManifest = new StandardRuntimeManifestBuilder() + .addBundle(container) + .build(); + + final ConnectorPropertyGroup group = runtimeManifest.getBundles().get(0).getComponentManifest().getConnectors().get(0) + .getConfigurationSteps().get(0).getPropertyGroups().get(0); + assertNotNull(group.getProperties()); + assertEquals(1, group.getProperties().size()); + + final ConnectorPropertyDescriptor property = group.getProperties().get(0); + assertEquals(PROPERTY_NAME, property.getName()); + assertEquals(PROPERTY_DESCRIPTION, property.getDescription()); + assertEquals(PROPERTY_DEFAULT, property.getDefaultValue()); + assertTrue(property.isRequired()); + assertFalse(property.isAllowableValuesFetchable()); + assertEquals(ConnectorPropertyType.STRING, property.getPropertyType()); + } + + @Test + void testConnectorPropertyAllowableValuesAreConverted() { + final ExtensionManifest extensionManifest = createExtensionManifest(); + final Extension connectorExtension = createConnectorExtensionWithAllowableValues(); + extensionManifest.setExtensions(List.of(connectorExtension)); + + final ExtensionManifestContainer container = new ExtensionManifestContainer(extensionManifest, null); + final RuntimeManifest runtimeManifest = new StandardRuntimeManifestBuilder() + .addBundle(container) + .build(); + + final ConnectorPropertyDescriptor property = runtimeManifest.getBundles().get(0).getComponentManifest().getConnectors().get(0) + .getConfigurationSteps().get(0).getPropertyGroups().get(0).getProperties().get(0); + assertNotNull(property.getAllowableValues()); + assertEquals(2, property.getAllowableValues().size()); + assertEquals("option1", property.getAllowableValues().get(0).getValue()); + assertEquals("Option 1", property.getAllowableValues().get(0).getDisplayName()); + } + + @Test + void testConnectorPropertyDependenciesAreConverted() { + final ExtensionManifest extensionManifest = createExtensionManifest(); + final Extension connectorExtension = createConnectorExtensionWithPropertyDependencies(); + extensionManifest.setExtensions(List.of(connectorExtension)); + + final ExtensionManifestContainer container = new ExtensionManifestContainer(extensionManifest, null); + final RuntimeManifest runtimeManifest = new StandardRuntimeManifestBuilder() + .addBundle(container) + .build(); + + final ConnectorPropertyDescriptor property = runtimeManifest.getBundles().get(0).getComponentManifest().getConnectors().get(0) + .getConfigurationSteps().get(0).getPropertyGroups().get(0).getProperties().get(0); + assertNotNull(property.getDependencies()); + assertEquals(1, property.getDependencies().size()); + assertEquals("authType", property.getDependencies().get(0).getPropertyName()); + assertEquals(List.of("basic", "oauth"), property.getDependencies().get(0).getDependentValues()); + } + + private ExtensionManifest createExtensionManifest() { + final ExtensionManifest manifest = new ExtensionManifest(); + manifest.setGroupId(TEST_GROUP); + manifest.setArtifactId(TEST_ARTIFACT); + manifest.setVersion(TEST_VERSION); + + final ParentNar parentNar = new ParentNar(); + parentNar.setGroupId("org.apache.nifi"); + parentNar.setArtifactId("nifi-standard-services-api-nar"); + parentNar.setVersion(TEST_VERSION); + manifest.setParentNar(parentNar); + + return manifest; + } + + private Extension createConnectorExtension() { + final Extension extension = new Extension(); + extension.setName(TEST_CONNECTOR_TYPE); + extension.setType(ExtensionType.CONNECTOR); + return extension; + } + + private Extension createConnectorExtensionWithConfigurationSteps() { + final Extension extension = createConnectorExtension(); + + final org.apache.nifi.extension.manifest.ConfigurationStep step = new org.apache.nifi.extension.manifest.ConfigurationStep(); + step.setName(STEP_NAME); + step.setDescription(STEP_DESCRIPTION); + extension.setConfigurationSteps(List.of(step)); + + return extension; + } + + private Extension createConnectorExtensionWithStepDependencies() { + final Extension extension = createConnectorExtension(); + + final org.apache.nifi.extension.manifest.ConfigurationStepDependency dependency = new org.apache.nifi.extension.manifest.ConfigurationStepDependency(); + dependency.setStepName("previousStep"); + dependency.setPropertyName("connectionType"); + dependency.setDependentValues(List.of("advanced")); + + final org.apache.nifi.extension.manifest.ConfigurationStep step = new org.apache.nifi.extension.manifest.ConfigurationStep(); + step.setName(STEP_NAME); + step.setDescription(STEP_DESCRIPTION); + step.setStepDependencies(List.of(dependency)); + extension.setConfigurationSteps(List.of(step)); + + return extension; + } + + private Extension createConnectorExtensionWithPropertyGroups() { + final Extension extension = createConnectorExtension(); + + final org.apache.nifi.extension.manifest.ConnectorPropertyGroup group = new org.apache.nifi.extension.manifest.ConnectorPropertyGroup(); + group.setName(GROUP_NAME); + group.setDescription(GROUP_DESCRIPTION); + + final org.apache.nifi.extension.manifest.ConfigurationStep step = new org.apache.nifi.extension.manifest.ConfigurationStep(); + step.setName(STEP_NAME); + step.setDescription(STEP_DESCRIPTION); + step.setPropertyGroups(List.of(group)); + extension.setConfigurationSteps(List.of(step)); + + return extension; + } + + private Extension createConnectorExtensionWithProperties() { + final Extension extension = createConnectorExtension(); + + final ConnectorProperty property = new ConnectorProperty(); + property.setName(PROPERTY_NAME); + property.setDescription(PROPERTY_DESCRIPTION); + property.setDefaultValue(PROPERTY_DEFAULT); + property.setRequired(true); + property.setAllowableValuesFetchable(false); + property.setPropertyType(org.apache.nifi.extension.manifest.ConnectorPropertyType.STRING); + + final org.apache.nifi.extension.manifest.ConnectorPropertyGroup group = new org.apache.nifi.extension.manifest.ConnectorPropertyGroup(); + group.setName(GROUP_NAME); + group.setDescription(GROUP_DESCRIPTION); + group.setProperties(List.of(property)); + + final org.apache.nifi.extension.manifest.ConfigurationStep step = new org.apache.nifi.extension.manifest.ConfigurationStep(); + step.setName(STEP_NAME); + step.setDescription(STEP_DESCRIPTION); + step.setPropertyGroups(List.of(group)); + extension.setConfigurationSteps(List.of(step)); + + return extension; + } + + private Extension createConnectorExtensionWithAllowableValues() { + final Extension extension = createConnectorExtension(); + + final AllowableValue value1 = new AllowableValue(); + value1.setValue("option1"); + value1.setDisplayName("Option 1"); + value1.setDescription("First option"); + + final AllowableValue value2 = new AllowableValue(); + value2.setValue("option2"); + value2.setDisplayName("Option 2"); + value2.setDescription("Second option"); + + final ConnectorProperty property = new ConnectorProperty(); + property.setName(PROPERTY_NAME); + property.setDescription(PROPERTY_DESCRIPTION); + property.setAllowableValues(List.of(value1, value2)); + + final org.apache.nifi.extension.manifest.ConnectorPropertyGroup group = new org.apache.nifi.extension.manifest.ConnectorPropertyGroup(); + group.setName(GROUP_NAME); + group.setProperties(List.of(property)); + + final org.apache.nifi.extension.manifest.ConfigurationStep step = new org.apache.nifi.extension.manifest.ConfigurationStep(); + step.setName(STEP_NAME); + step.setPropertyGroups(List.of(group)); + extension.setConfigurationSteps(List.of(step)); + + return extension; + } + + private Extension createConnectorExtensionWithPropertyDependencies() { + final Extension extension = createConnectorExtension(); + + final ConnectorPropertyDependency dependency = new ConnectorPropertyDependency(); + dependency.setPropertyName("authType"); + dependency.setDependentValues(List.of("basic", "oauth")); + + final ConnectorProperty property = new ConnectorProperty(); + property.setName(PROPERTY_NAME); + property.setDescription(PROPERTY_DESCRIPTION); + property.setDependencies(List.of(dependency)); + + final org.apache.nifi.extension.manifest.ConnectorPropertyGroup group = new org.apache.nifi.extension.manifest.ConnectorPropertyGroup(); + group.setName(GROUP_NAME); + group.setProperties(List.of(property)); + + final org.apache.nifi.extension.manifest.ConfigurationStep step = new org.apache.nifi.extension.manifest.ConfigurationStep(); + step.setName(STEP_NAME); + step.setPropertyGroups(List.of(group)); + extension.setConfigurationSteps(List.of(step)); + + return extension; + } +} + diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java index 826cea5dc3cc..6253c9e08de9 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockValidationContext.java @@ -31,6 +31,7 @@ import org.apache.nifi.parameter.ParameterLookup; import org.apache.nifi.parameter.ParameterParser; import org.apache.nifi.parameter.ParameterReference; +import org.apache.nifi.parameter.ParameterTokenList; import java.util.Collection; import java.util.Collections; @@ -219,6 +220,13 @@ public boolean isParameterSet(final String parameterName) { return contextParameters.containsKey(parameterName) && contextParameters.get(parameterName) != null; } + @Override + public String evaluateParameters(final String value) { + final ParameterParser parameterParser = new ExpressionLanguageAgnosticParameterParser(); + final ParameterTokenList parameterTokenList = parameterParser.parseTokens(value); + return parameterTokenList.substitute(parameterLookup); + } + private String getEffectiveValue(final PropertyDescriptor descriptor) { final String configuredValue = context.getProperties().get(descriptor); return getEffectiveValue(descriptor, configuredValue); diff --git a/nifi-server-api/src/main/java/org/apache/nifi/nar/ExtensionMapping.java b/nifi-server-api/src/main/java/org/apache/nifi/nar/ExtensionMapping.java index 851fefecb744..1d4b9cd8759a 100644 --- a/nifi-server-api/src/main/java/org/apache/nifi/nar/ExtensionMapping.java +++ b/nifi-server-api/src/main/java/org/apache/nifi/nar/ExtensionMapping.java @@ -34,6 +34,7 @@ public class ExtensionMapping { private final Map> flowAnalysisRuleNames = new HashMap<>(); private final Map> parameterProviderNames = new HashMap<>(); private final Map> flowRegistryClientNames = new HashMap<>(); + private final Map> connectorNames = new HashMap<>(); private final BiFunction, Set, Set> merger = (oldValue, newValue) -> { final Set merged = new HashSet<>(); @@ -102,6 +103,16 @@ void addAllFlowRegistryClients(final BundleCoordinate coordinate, final Collecti }); } + void addConnector(final BundleCoordinate coordinate, final String connectorName) { + connectorNames.computeIfAbsent(connectorName, name -> new HashSet<>()).add(coordinate); + } + + void addAllConnectors(final BundleCoordinate coordinate, final Collection connectorNames) { + connectorNames.forEach(name -> { + addConnector(coordinate, name); + }); + } + void merge(final ExtensionMapping other) { other.getProcessorNames().forEach((name, otherCoordinates) -> { processorNames.merge(name, otherCoordinates, merger); @@ -121,6 +132,9 @@ void merge(final ExtensionMapping other) { other.getFlowRegistryClientNames().forEach((name, otherCoordinates) -> { flowRegistryClientNames.merge(name, otherCoordinates, merger); }); + other.getConnectorNames().forEach((name, otherCoordinates) -> { + connectorNames.merge(name, otherCoordinates, merger); + }); } public Map> getProcessorNames() { @@ -147,6 +161,10 @@ public Map> getFlowRegistryClientNames() { return Collections.unmodifiableMap(flowRegistryClientNames); } + public Map> getConnectorNames() { + return Collections.unmodifiableMap(connectorNames); + } + public Map> getAllExtensionNames() { final Map> extensionNames = new HashMap<>(); extensionNames.putAll(processorNames); @@ -155,6 +173,7 @@ public Map> getAllExtensionNames() { extensionNames.putAll(flowAnalysisRuleNames); extensionNames.putAll(parameterProviderNames); extensionNames.putAll(flowRegistryClientNames); + extensionNames.putAll(connectorNames); return extensionNames; } @@ -179,6 +198,9 @@ public int size() { for (final Set coordinates : flowRegistryClientNames.values()) { size += coordinates.size(); } + for (final Set coordinates : connectorNames.values()) { + size += coordinates.size(); + } return size; } @@ -189,6 +211,7 @@ public boolean isEmpty() { && reportingTaskNames.isEmpty() && flowAnalysisRuleNames.isEmpty() && parameterProviderNames.isEmpty() - && flowRegistryClientNames.isEmpty(); + && flowRegistryClientNames.isEmpty() + && connectorNames.isEmpty(); } } diff --git a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/StatelessDataflow.java b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/StatelessDataflow.java index 452b985345ac..7cc2a3eea020 100644 --- a/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/StatelessDataflow.java +++ b/nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/StatelessDataflow.java @@ -109,4 +109,13 @@ default void shutdown() { OptionalLong getCounter(String componentId, String counterName); Map getCounters(Pattern counterNamePattern); + + /** + * Returns the latest time at which any component in the dataflow had activity, or an empty OptionalLong if there has been no activity. + * + * @return the latest activity time in milliseconds since epoch, or empty if no activity has occurred + */ + default OptionalLong getLatestActivityTime() { + return OptionalLong.empty(); + } } diff --git a/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/AllowListClassLoader.java b/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/AllowListClassLoader.java index 26ec99b86cea..3a90031d7019 100644 --- a/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/AllowListClassLoader.java +++ b/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/AllowListClassLoader.java @@ -99,6 +99,11 @@ private boolean isClassAllowed(final String name, final Class clazz) { return true; } + // TODO: Delete this + if (name.startsWith("org.mockito") || name.startsWith("net.bytebuddy") || name.startsWith("org.testcontainers")) { + return true; + } + // If the class has a module whose name is allowed, allow it. // The module is obtained by calling Class.getModule(). However, that method is only available in Java 9. // Since this codebase must be Java 8 compatible we can't make that method call. So we use Reflection to determine diff --git a/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/StatelessBootstrap.java b/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/StatelessBootstrap.java index ff2064bc895e..a0bd035eb1c9 100644 --- a/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/StatelessBootstrap.java +++ b/nifi-stateless/nifi-stateless-bootstrap/src/main/java/org/apache/nifi/stateless/bootstrap/StatelessBootstrap.java @@ -149,7 +149,7 @@ public static StatelessBootstrap bootstrap(final StatelessEngineConfiguration en * @param parent the parent class loader that the given BlockListClassLoader should delegate to for classes that it does not block * @return an AllowListClassLoader that allows only the appropriate classes to be loaded from the given parent */ - protected static AllowListClassLoader createExtensionRootClassLoader(final File narDirectory, final ClassLoader parent) throws IOException { + public static AllowListClassLoader createExtensionRootClassLoader(final File narDirectory, final ClassLoader parent) throws IOException { final File[] narDirectoryFiles = narDirectory.listFiles(); if (narDirectoryFiles == null) { throw new IOException("Could not get a listing of the NAR directory"); diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/reporting/StatelessReportingTaskNode.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/reporting/StatelessReportingTaskNode.java index 8ec54520b5bc..a5601ca98a66 100644 --- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/reporting/StatelessReportingTaskNode.java +++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/reporting/StatelessReportingTaskNode.java @@ -19,6 +19,7 @@ import org.apache.nifi.authorization.Resource; import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.validation.ValidationTrigger; import org.apache.nifi.controller.LoggableComponent; @@ -50,7 +51,7 @@ public StatelessReportingTaskNode(final LoggableComponent reporti } @Override - protected List validateConfig() { + protected List validateConfig(final ValidationContext validationContext) { return Collections.emptyList(); } diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/scheduling/StatelessProcessScheduler.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/scheduling/StatelessProcessScheduler.java index 2f6c38ac8faa..802bbcb1ed80 100644 --- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/scheduling/StatelessProcessScheduler.java +++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/controller/scheduling/StatelessProcessScheduler.java @@ -20,6 +20,7 @@ import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.annotation.notification.PrimaryNodeState; +import org.apache.nifi.components.connector.ConnectorNode; import org.apache.nifi.components.validation.ValidationStatus; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.Funnel; @@ -128,7 +129,7 @@ public void initialize(final StatelessProcessSchedulerInitializationContext cont } @Override - public Future startProcessor(final ProcessorNode procNode, final boolean failIfStopping) { + public CompletableFuture startProcessor(final ProcessorNode procNode, final boolean failIfStopping) { final CompletableFuture future = new CompletableFuture<>(); final SchedulingAgentCallback callback = new SchedulingAgentCallback() { @Override @@ -195,7 +196,7 @@ public void onProcessorRemoved(final ProcessorNode procNode) { } @Override - public Future startStatelessGroup(final StatelessGroupNode groupNode) { + public CompletableFuture startStatelessGroup(final StatelessGroupNode groupNode) { throw new UnsupportedOperationException(); } @@ -328,12 +329,32 @@ public CompletableFuture enableControllerService(final ControllerServiceNo return service.enable(componentLifeCycleThreadPool, ADMINISTRATIVE_YIELD_MILLIS, true); } + @Override + public CompletableFuture enableControllerService(final ControllerServiceNode service, final ConfigurationContext configurationContext) { + logger.info("Enabling {}", service); + return service.enable(componentLifeCycleThreadPool, ADMINISTRATIVE_YIELD_MILLIS, true, configurationContext); + } + @Override public CompletableFuture disableControllerService(final ControllerServiceNode service) { logger.info("Disabling {}", service); return service.disable(componentLifeCycleThreadPool); } + @Override + public CompletableFuture startConnector(final ConnectorNode connectorNode) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture stopConnector(final ConnectorNode connectorNode) { + throw new UnsupportedOperationException(); + } + + @Override + public void onConnectorRemoved(final ConnectorNode connectorNode) { + } + @Override public CompletableFuture disableControllerServices(final List services) { if (services == null || services.isEmpty()) { diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessFlowManager.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessFlowManager.java index 09010dd7dff4..a637745bb61e 100644 --- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessFlowManager.java +++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessFlowManager.java @@ -21,6 +21,7 @@ import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored; import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.bundle.BundleCoordinate; +import org.apache.nifi.components.connector.ConnectorNode; import org.apache.nifi.components.state.StateManager; import org.apache.nifi.connectable.Connectable; import org.apache.nifi.connectable.ConnectableType; @@ -225,7 +226,7 @@ public Port createLocalOutputPort(final String id, final String name) { } @Override - public ProcessGroup createProcessGroup(final String id) { + public ProcessGroup createProcessGroup(final String id, final String connectorId) { final ProcessGroup created = new StandardProcessGroup(id, statelessEngine.getControllerServiceProvider(), statelessEngine.getProcessScheduler(), statelessEngine.getPropertyEncryptor(), @@ -236,7 +237,8 @@ public ProcessGroup createProcessGroup(final String id) { new StatelessNodeTypeProvider(), null, group -> null, - statelessEngine.getAssetManager()); + statelessEngine.getAssetManager(), + connectorId); onProcessGroupAdded(created); return created; @@ -442,6 +444,21 @@ public ControllerServiceNode getRootControllerService(final String serviceIdenti public void removeRootControllerService(final ControllerServiceNode service) { } + @Override + public ConnectorNode createConnector(final String type, final String id, final BundleCoordinate coordinate, final boolean firstTimeAdded, final boolean registerLogObserver) { + throw new UnsupportedOperationException(); + } + + @Override + public List getAllConnectors() { + return List.of(); + } + + @Override + public ConnectorNode getConnector(final String id) { + return null; + } + @Override protected Authorizable getParameterContextParent() { return null; diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java index f50913288871..5c317d7bf752 100644 --- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java +++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java @@ -962,6 +962,11 @@ public Map getCounters(final Pattern counterNamePattern) { .collect(Collectors.toMap(Counter::getName, Counter::getValue)); } + @Override + public OptionalLong getLatestActivityTime() { + return rootGroup.getFlowFileActivity().getLatestActivityTime(); + } + private String findInstanceId(final String componentId) { return rootGroup.findAllProcessors().stream() .filter(processor -> Objects.equals(processor.getIdentifier(), componentId) || Objects.equals(processor.getVersionedComponentId().orElse(""), componentId)) diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/queue/StatelessFlowFileQueue.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/queue/StatelessFlowFileQueue.java index caaa607b104b..2c30d9168392 100644 --- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/queue/StatelessFlowFileQueue.java +++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/queue/StatelessFlowFileQueue.java @@ -17,6 +17,7 @@ package org.apache.nifi.stateless.queue; +import org.apache.nifi.components.connector.DropFlowFileSummary; import org.apache.nifi.controller.queue.DropFlowFileStatus; import org.apache.nifi.controller.queue.ListFlowFileStatus; import org.apache.nifi.controller.queue.LoadBalanceCompression; @@ -32,6 +33,7 @@ import org.apache.nifi.processor.FlowFileFilter; import org.apache.nifi.util.FormatUtils; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -45,6 +47,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Predicate; public class StatelessFlowFileQueue implements DrainableFlowFileQueue { private final String identifier; @@ -317,6 +320,11 @@ public DropFlowFileStatus cancelDropFlowFileRequest(final String requestIdentifi throw new UnsupportedOperationException("Cannot drop FlowFiles from a queue in Stateless NiFi"); } + @Override + public DropFlowFileSummary dropFlowFiles(final Predicate predicate) throws IOException { + throw new UnsupportedOperationException("Cannot drop FlowFiles from a queue in Stateless NiFi"); + } + @Override public ListFlowFileStatus listFlowFiles(final String requestIdentifier, final int maxResults) { throw new UnsupportedOperationException("Cannot list FlowFiles in a queue in Stateless NiFi"); diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/pom.xml b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/pom.xml index f877db14d043..fd23562cd9e9 100644 --- a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/pom.xml +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/pom.xml @@ -43,6 +43,11 @@ com.fasterxml.jackson.core jackson-databind + + org.apache.nifi + nifi-connector-utils + 2.9.0-SNAPSHOT + org.apache.nifi nifi-system-test-extensions-services-api diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/AssetConnector.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/AssetConnector.java new file mode 100644 index 000000000000..2e927bb038a7 --- /dev/null +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/AssetConnector.java @@ -0,0 +1,93 @@ +/* + * 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.connectors.tests.system; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.ConnectorPropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorPropertyGroup; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.PropertyType; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.flow.VersionedExternalFlow; + +import java.util.List; +import java.util.Map; + +/** + * Example Connector used by system tests for exercising Connector Asset APIs. + *

+ * The connector defines a single configuration step with one property whose type is ASSET. + * The connector itself does not manipulate the flow; it is intended only to drive API-level tests. + */ +public class AssetConnector extends AbstractConnector { + + static final ConnectorPropertyDescriptor ASSET_PROPERTY = new ConnectorPropertyDescriptor.Builder() + .name("Test Asset") + .description("Asset used for validating Connector Asset APIs in system tests") + .type(PropertyType.ASSET) + .required(true) + .build(); + + private static final ConnectorPropertyGroup ASSET_GROUP = new ConnectorPropertyGroup.Builder() + .name("Asset Configuration") + .description("Configuration properties related to Assets") + .addProperty(ASSET_PROPERTY) + .build(); + + private static final ConfigurationStep ASSET_CONFIGURATION_STEP = new ConfigurationStep.Builder() + .name("Asset Configuration") + .description("Configuration Step that exposes an ASSET-type property") + .propertyGroups(List.of(ASSET_GROUP)) + .build(); + + @Override + public VersionedExternalFlow getInitialFlow() { + // This Connector is intended only for exercising the REST API and does not manage a flow. + return null; + } + + @Override + public void prepareForUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + // No-op: this connector does not manipulate the flow. + } + + @Override + public List getConfigurationSteps() { + return List.of(ASSET_CONFIGURATION_STEP); + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + // No-op: this connector does not manipulate the flow. + } + + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) throws FlowUpdateException { + // No-op: there is no additional behavior when the step is configured. + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map overrides, final FlowContext flowContext) { + // No additional verification is required for this simple test connector. + return List.of(); + } +} + + diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/BundleResolutionConnector.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/BundleResolutionConnector.java new file mode 100644 index 000000000000..f15d668ef51f --- /dev/null +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/BundleResolutionConnector.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.connectors.tests.system; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.ConfigVerificationResult.Outcome; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.BundleCompatibility; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.ConnectorPropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorPropertyGroup; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.PropertyType; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.Position; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedParameter; +import org.apache.nifi.flow.VersionedParameterContext; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +/** + * A test connector that exercises bundle resolution capabilities. + * It creates a flow with processors using different bundle specifications to test + * how the framework handles unavailable bundles based on the configured BundleCompatability strategy. + */ +public class BundleResolutionConnector extends AbstractConnector { + + private static final String BUNDLE_COMPATABILITY_STEP = "Bundle Resolution"; + + private static final ConnectorPropertyDescriptor BUNDLE_COMPATABILITY_PROPERTY = new ConnectorPropertyDescriptor.Builder() + .name("Bundle Compatability") + .description("Specifies how bundle resolution should be handled when the specified bundle is not available.") + .required(true) + .type(PropertyType.STRING) + .allowableValues( + BundleCompatibility.REQUIRE_EXACT_BUNDLE.name(), + BundleCompatibility.RESOLVE_BUNDLE.name(), + BundleCompatibility.RESOLVE_NEWEST_BUNDLE.name() + ) + .defaultValue(BundleCompatibility.REQUIRE_EXACT_BUNDLE.name()) + .build(); + + private static final ConnectorPropertyGroup BUNDLE_PROPERTY_GROUP = new ConnectorPropertyGroup.Builder() + .name("Bundle Settings") + .description("Settings for bundle resolution behavior.") + .properties(List.of(BUNDLE_COMPATABILITY_PROPERTY)) + .build(); + + private static final ConfigurationStep BUNDLE_STEP = new ConfigurationStep.Builder() + .name(BUNDLE_COMPATABILITY_STEP) + .propertyGroups(List.of(BUNDLE_PROPERTY_GROUP)) + .build(); + + private final List configurationSteps = List.of(BUNDLE_STEP); + + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) throws FlowUpdateException { + if (BUNDLE_COMPATABILITY_STEP.equals(stepName)) { + final String compatabilityValue = workingContext.getConfigurationContext() + .getProperty(BUNDLE_STEP, BUNDLE_COMPATABILITY_PROPERTY) + .getValue(); + final BundleCompatibility bundleCompatability = BundleCompatibility.valueOf(compatabilityValue); + + final VersionedExternalFlow flow = createFlowWithBundleScenarios(); + getInitializationContext().updateFlow(workingContext, flow, bundleCompatability); + } + } + + @Override + public VersionedExternalFlow getInitialFlow() { + final VersionedProcessGroup group = VersionedFlowUtils.createProcessGroup(UUID.randomUUID().toString(), "Bundle Resolution Flow"); + + final VersionedParameter compatabilityParam = new VersionedParameter(); + compatabilityParam.setName("BUNDLE_COMPATABILITY"); + compatabilityParam.setValue(BundleCompatibility.REQUIRE_EXACT_BUNDLE.name()); + compatabilityParam.setSensitive(false); + compatabilityParam.setProvided(false); + compatabilityParam.setReferencedAssets(List.of()); + + final VersionedParameterContext parameterContext = new VersionedParameterContext(); + parameterContext.setName("Bundle Resolution Parameter Context"); + parameterContext.setParameters(Set.of(compatabilityParam)); + + final VersionedExternalFlow flow = new VersionedExternalFlow(); + flow.setParameterContexts(Map.of(parameterContext.getName(), parameterContext)); + flow.setFlowContents(group); + return flow; + } + + private VersionedExternalFlow createFlowWithBundleScenarios() { + final VersionedProcessGroup group = VersionedFlowUtils.createProcessGroup(UUID.randomUUID().toString(), "Bundle Resolution Flow"); + + // Add a processor with an unavailable bundle (fake version) that should be resolved based on BundleCompatability + // Uses the system test GenerateFlowFile processor which is available in the system test extensions bundle + final Bundle nonexistentBundle = new Bundle("org.apache.nifi", "nifi-system-test-extensions-nar", "0.0.0-NONEXISTENT"); + + final VersionedProcessor testProcessor = VersionedFlowUtils.addProcessor(group, + "org.apache.nifi.processors.tests.system.GenerateFlowFile", nonexistentBundle, + "GenerateFlowFile for Bundle Resolution Test", new Position(100, 100)); + testProcessor.setSchedulingPeriod("1 sec"); + testProcessor.setAutoTerminatedRelationships(Set.of("success")); + + final VersionedParameterContext parameterContext = new VersionedParameterContext(); + parameterContext.setName("Bundle Resolution Parameter Context"); + parameterContext.setParameters(Set.of()); + + final VersionedExternalFlow flow = new VersionedExternalFlow(); + flow.setParameterContexts(Map.of(parameterContext.getName(), parameterContext)); + flow.setFlowContents(group); + return flow; + } + + @Override + public List verifyConfigurationStep(final String stepName, + final Map propertyValueOverrides, + final FlowContext flowContext) { + return List.of(new ConfigVerificationResult.Builder() + .outcome(Outcome.SUCCESSFUL) + .subject(stepName) + .verificationStepName("Bundle Resolution Verification") + .explanation("Bundle resolution configuration verified successfully.") + .build()); + } + + @Override + public List getConfigurationSteps() { + return configurationSteps; + } + + @Override + public void applyUpdate(final FlowContext workingFlowContext, final FlowContext activeFlowContext) throws FlowUpdateException { + final String compatabilityValue = workingFlowContext.getConfigurationContext() + .getProperty(BUNDLE_STEP, BUNDLE_COMPATABILITY_PROPERTY) + .getValue(); + final BundleCompatibility bundleCompatability = BundleCompatibility.valueOf(compatabilityValue); + + final VersionedExternalFlow flow = createFlowWithBundleScenarios(); + getInitializationContext().updateFlow(activeFlowContext, flow, bundleCompatability); + } +} diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/CalculateConnector.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/CalculateConnector.java new file mode 100644 index 000000000000..bb767219154d --- /dev/null +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/CalculateConnector.java @@ -0,0 +1,226 @@ +/* + * 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.connectors.tests.system; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.ConnectorPropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorPropertyGroup; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.InvocationFailedException; +import org.apache.nifi.components.connector.PropertyType; +import org.apache.nifi.components.connector.StepConfigurationContext; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.components.ProcessorFacade; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.Position; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** + * A test connector that invokes a ConnectorMethod on a CalculateProcessor using its own POJO types. + * This connector is used to test JSON marshalling of complex objects across ClassLoader boundaries. + */ +public class CalculateConnector extends AbstractConnector { + + /** + * A POJO representing a calculation request. This is intentionally a different class than + * the Processor's record type to test cross-ClassLoader JSON marshalling. + */ + public static class Calculation { + private int operand1; + private int operand2; + private String operation; + + public Calculation() { + } + + public Calculation(final int operand1, final int operand2, final String operation) { + this.operand1 = operand1; + this.operand2 = operand2; + this.operation = operation; + } + + public int getOperand1() { + return operand1; + } + + public void setOperand1(final int operand1) { + this.operand1 = operand1; + } + + public int getOperand2() { + return operand2; + } + + public void setOperand2(final int operand2) { + this.operand2 = operand2; + } + + public String getOperation() { + return operation; + } + + public void setOperation(final String operation) { + this.operation = operation; + } + } + + /** + * A POJO representing the result of a calculation. This is intentionally a different class than + * the Processor's record type to test cross-ClassLoader JSON marshalling. + */ + public static class CalculatedResult { + private Calculation calculation; + private int result; + + public Calculation getCalculation() { + return calculation; + } + + public void setCalculation(final Calculation calculation) { + this.calculation = calculation; + } + + public int getResult() { + return result; + } + + public void setResult(final int result) { + this.result = result; + } + } + + private static final ConnectorPropertyDescriptor OPERAND_1 = new ConnectorPropertyDescriptor.Builder() + .name("Operand 1") + .description("The first operand for the calculation") + .type(PropertyType.STRING) + .required(true) + .addValidator(StandardValidators.INTEGER_VALIDATOR) + .build(); + + private static final ConnectorPropertyDescriptor OPERAND_2 = new ConnectorPropertyDescriptor.Builder() + .name("Operand 2") + .description("The second operand for the calculation") + .type(PropertyType.STRING) + .required(true) + .addValidator(StandardValidators.INTEGER_VALIDATOR) + .build(); + + private static final ConnectorPropertyDescriptor OPERATION = new ConnectorPropertyDescriptor.Builder() + .name("Operation") + .description("The operation to perform (ADD, SUBTRACT, MULTIPLY, DIVIDE)") + .type(PropertyType.STRING) + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + private static final ConnectorPropertyDescriptor OUTPUT_FILE = new ConnectorPropertyDescriptor.Builder() + .name("Output File") + .description("The file to write the calculation result to") + .type(PropertyType.STRING) + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + private static final ConnectorPropertyGroup CALCULATION_GROUP = new ConnectorPropertyGroup.Builder() + .name("Calculation Configuration") + .description("Configuration properties for the calculation") + .properties(List.of(OPERAND_1, OPERAND_2, OPERATION, OUTPUT_FILE)) + .build(); + + private static final ConfigurationStep CALCULATION_STEP = new ConfigurationStep.Builder() + .name("Calculation") + .description("Configure the calculation parameters") + .propertyGroups(List.of(CALCULATION_GROUP)) + .build(); + + @Override + public VersionedExternalFlow getInitialFlow() { + final Bundle bundle = new Bundle("org.apache.nifi", "nifi-system-test-extensions-nar", "2.8.0-SNAPSHOT"); + final VersionedProcessGroup group = VersionedFlowUtils.createProcessGroup("calculate-flow-id", "Calculate Flow"); + + VersionedFlowUtils.addProcessor(group, "org.apache.nifi.processors.tests.system.Calculate", bundle, "Calculate Processor", new Position(0, 0)); + + final VersionedExternalFlow flow = new VersionedExternalFlow(); + flow.setFlowContents(group); + flow.setParameterContexts(Map.of()); + return flow; + } + + @Override + public List getConfigurationSteps() { + return List.of(CALCULATION_STEP); + } + + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) { + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + final StepConfigurationContext stepContext = workingContext.getConfigurationContext().scopedToStep(CALCULATION_STEP); + final int operand1 = stepContext.getProperty(OPERAND_1).asInteger(); + final int operand2 = stepContext.getProperty(OPERAND_2).asInteger(); + final String operation = stepContext.getProperty(OPERATION).getValue(); + final String outputFile = stepContext.getProperty(OUTPUT_FILE).getValue(); + + final ProcessorFacade processorFacade = workingContext.getRootGroup().getProcessors().stream() + .filter(p -> p.getDefinition().getType().endsWith("Calculate")) + .findFirst() + .orElseThrow(() -> new FlowUpdateException("CalculateProcessor not found in flow")); + + final Calculation calculation = new Calculation(operand1, operand2, operation); + final CalculatedResult result; + try { + result = processorFacade.invokeConnectorMethod("calculate", Map.of("calculation", calculation), CalculatedResult.class); + } catch (final InvocationFailedException e) { + throw new FlowUpdateException("Failed to invoke calculate method", e); + } + + final File file = new File(outputFile); + try (final FileWriter writer = new FileWriter(file)) { + writer.write(String.valueOf(result.getCalculation().getOperand1())); + writer.write("\n"); + writer.write(String.valueOf(result.getCalculation().getOperand2())); + writer.write("\n"); + writer.write(result.getCalculation().getOperation()); + writer.write("\n"); + writer.write(String.valueOf(result.getResult())); + } catch (final IOException e) { + throw new FlowUpdateException("Failed to write result to file", e); + } + + getLogger().info("Calculation result: {} {} {} = {}", operand1, operation, operand2, result.getResult()); + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map propertyValueOverrides, final FlowContext flowContext) { + return List.of(); + } +} + diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/ComponentLifecycleConnector.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/ComponentLifecycleConnector.java new file mode 100644 index 000000000000..81c6af171b13 --- /dev/null +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/ComponentLifecycleConnector.java @@ -0,0 +1,177 @@ +/* + * 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.connectors.tests.system; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.ControllerServiceAPI; +import org.apache.nifi.flow.ExecutionEngine; +import org.apache.nifi.flow.Position; +import org.apache.nifi.flow.ScheduledState; +import org.apache.nifi.flow.VersionedControllerService; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedPort; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; + +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +/** + * Test Connector designed to verify the complete component lifecycle. + * Creates a flow with: + * - A processor at the root level + * - A child process group with input and output ports + * - A processor within the child group + * - A stateless group with a processor + * + * This allows testing that start/stop operations properly handle all component types recursively. + */ +public class ComponentLifecycleConnector extends AbstractConnector { + + private static final Bundle SYSTEM_TEST_EXTENSIONS_BUNDLE = new Bundle("org.apache.nifi", "nifi-system-test-extensions-nar", "2.8.0-SNAPSHOT"); + + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) { + } + + @Override + public VersionedExternalFlow getInitialFlow() { + final VersionedProcessGroup rootGroup = createRootGroup(); + final VersionedExternalFlow flow = new VersionedExternalFlow(); + flow.setFlowContents(rootGroup); + return flow; + } + + private VersionedProcessGroup createRootGroup() { + final VersionedProcessGroup rootGroup = VersionedFlowUtils.createProcessGroup(UUID.randomUUID().toString(), "Component Lifecycle Root"); + rootGroup.setPosition(new Position(0, 0)); + rootGroup.setRemoteProcessGroups(new HashSet<>()); + rootGroup.setScheduledState(ScheduledState.ENABLED); + rootGroup.setExecutionEngine(ExecutionEngine.STANDARD); + + final VersionedControllerService rootControllerService = VersionedFlowUtils.addControllerService(rootGroup, + "org.apache.nifi.cs.tests.system.StandardCountService", SYSTEM_TEST_EXTENSIONS_BUNDLE, "Root Count Service"); + rootControllerService.setScheduledState(ScheduledState.ENABLED); + final ControllerServiceAPI rootServiceApi = new ControllerServiceAPI(); + rootServiceApi.setType("org.apache.nifi.cs.tests.system.CountService"); + rootServiceApi.setBundle(SYSTEM_TEST_EXTENSIONS_BUNDLE); + rootControllerService.setControllerServiceApis(Collections.singletonList(rootServiceApi)); + + final VersionedProcessor rootProcessor = VersionedFlowUtils.addProcessor(rootGroup, + "org.apache.nifi.processors.tests.system.GenerateFlowFile", SYSTEM_TEST_EXTENSIONS_BUNDLE, "Root GenerateFlowFile", new Position(100, 100)); + rootProcessor.setSchedulingPeriod("10 sec"); + + final VersionedProcessor rootTerminateProcessor = VersionedFlowUtils.addProcessor(rootGroup, + "org.apache.nifi.processors.tests.system.TerminateFlowFile", SYSTEM_TEST_EXTENSIONS_BUNDLE, "Root TerminateFlowFile", new Position(300, 100)); + + final VersionedProcessGroup childGroup = createChildGroup(rootGroup.getIdentifier()); + rootGroup.getProcessGroups().add(childGroup); + + final VersionedPort childInputPort = childGroup.getInputPorts().iterator().next(); + final VersionedPort childOutputPort = childGroup.getOutputPorts().iterator().next(); + + VersionedFlowUtils.addConnection(rootGroup, VersionedFlowUtils.createConnectableComponent(rootProcessor), + VersionedFlowUtils.createConnectableComponent(childInputPort), Set.of("success")); + VersionedFlowUtils.addConnection(rootGroup, VersionedFlowUtils.createConnectableComponent(childOutputPort), + VersionedFlowUtils.createConnectableComponent(rootTerminateProcessor), Set.of("")); + + return rootGroup; + } + + private VersionedProcessGroup createChildGroup(final String parentGroupId) { + final VersionedProcessGroup childGroup = VersionedFlowUtils.createProcessGroup("child-group-id", "Child Group"); + childGroup.setPosition(new Position(100, 300)); + childGroup.setRemoteProcessGroups(new HashSet<>()); + childGroup.setScheduledState(ScheduledState.ENABLED); + childGroup.setExecutionEngine(ExecutionEngine.STANDARD); + childGroup.setGroupIdentifier(parentGroupId); + + final VersionedControllerService childControllerService = VersionedFlowUtils.addControllerService(childGroup, + "org.apache.nifi.cs.tests.system.StandardCountService", SYSTEM_TEST_EXTENSIONS_BUNDLE, "Child Count Service"); + childControllerService.setScheduledState(ScheduledState.ENABLED); + final ControllerServiceAPI childServiceApi = new ControllerServiceAPI(); + childServiceApi.setType("org.apache.nifi.cs.tests.system.CountService"); + childServiceApi.setBundle(SYSTEM_TEST_EXTENSIONS_BUNDLE); + childControllerService.setControllerServiceApis(Collections.singletonList(childServiceApi)); + + final VersionedPort inputPort = VersionedFlowUtils.addInputPort(childGroup, "Child Input", new Position(0, 0)); + final VersionedPort outputPort = VersionedFlowUtils.addOutputPort(childGroup, "Child Output", new Position(200, 0)); + + final VersionedProcessor childProcessor = VersionedFlowUtils.addProcessor(childGroup, + "org.apache.nifi.processors.tests.system.PassThrough", SYSTEM_TEST_EXTENSIONS_BUNDLE, "Child Terminate", new Position(100, 100)); + + final VersionedProcessGroup statelessGroup = createStatelessGroup(childGroup.getIdentifier()); + childGroup.getProcessGroups().add(statelessGroup); + + final VersionedPort statelessInputPort = statelessGroup.getInputPorts().iterator().next(); + + VersionedFlowUtils.addConnection(childGroup, VersionedFlowUtils.createConnectableComponent(inputPort), + VersionedFlowUtils.createConnectableComponent(childProcessor), Set.of("")); + VersionedFlowUtils.addConnection(childGroup, VersionedFlowUtils.createConnectableComponent(inputPort), + VersionedFlowUtils.createConnectableComponent(statelessInputPort), Set.of("")); + VersionedFlowUtils.addConnection(childGroup, VersionedFlowUtils.createConnectableComponent(childProcessor), + VersionedFlowUtils.createConnectableComponent(outputPort), Set.of("success")); + + return childGroup; + } + + private VersionedProcessGroup createStatelessGroup(final String parentGroupId) { + final VersionedProcessGroup statelessGroup = VersionedFlowUtils.createProcessGroup("stateless-group-id", "Stateless Group"); + statelessGroup.setPosition(new Position(400, 100)); + statelessGroup.setRemoteProcessGroups(new HashSet<>()); + statelessGroup.setScheduledState(ScheduledState.ENABLED); + statelessGroup.setExecutionEngine(ExecutionEngine.STATELESS); + statelessGroup.setStatelessFlowTimeout("1 min"); + statelessGroup.setGroupIdentifier(parentGroupId); + + final VersionedPort statelessInput = VersionedFlowUtils.addInputPort(statelessGroup, "Stateless Input", new Position(0, 0)); + + final VersionedProcessor statelessProcessor = VersionedFlowUtils.addProcessor(statelessGroup, + "org.apache.nifi.processors.tests.system.TerminateFlowFile", SYSTEM_TEST_EXTENSIONS_BUNDLE, "Stateless Terminate", new Position(100, 100)); + + VersionedFlowUtils.addConnection(statelessGroup, VersionedFlowUtils.createConnectableComponent(statelessInput), + VersionedFlowUtils.createConnectableComponent(statelessProcessor), Set.of("")); + + return statelessGroup; + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map propertyValueOverrides, final FlowContext flowContext) { + return List.of(); + } + + @Override + public List getConfigurationSteps() { + return List.of(); + } + + @Override + public void applyUpdate(final FlowContext workingFlowContext, final FlowContext activeFlowContext) throws FlowUpdateException { + getInitializationContext().updateFlow(activeFlowContext, getInitialFlow()); + } +} diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/DataQueuingConnector.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/DataQueuingConnector.java new file mode 100644 index 000000000000..1c50941c8371 --- /dev/null +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/DataQueuingConnector.java @@ -0,0 +1,79 @@ +/* + * 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.connectors.tests.system; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.Position; +import org.apache.nifi.flow.ScheduledState; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class DataQueuingConnector extends AbstractConnector { + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) { + + } + + @Override + public VersionedExternalFlow getInitialFlow() { + final Bundle bundle = new Bundle("org.apache.nifi", "nifi-system-test-extensions-nar", "2.8.0-SNAPSHOT"); + final VersionedProcessGroup rootGroup = VersionedFlowUtils.createProcessGroup("1234", "Data Queuing Connector"); + + final VersionedProcessor generate = VersionedFlowUtils.addProcessor(rootGroup, + "org.apache.nifi.processors.tests.system.GenerateFlowFile", bundle, "GenerateFlowFile", new Position(0, 0)); + generate.getProperties().put("File Size", "1 KB"); + generate.setSchedulingPeriod("100 millis"); + + final VersionedProcessor terminate = VersionedFlowUtils.addProcessor(rootGroup, + "org.apache.nifi.processors.tests.system.TerminateFlowFile", bundle, "TerminateFlowFile", new Position(0, 0)); + terminate.setScheduledState(ScheduledState.DISABLED); + + VersionedFlowUtils.addConnection(rootGroup, VersionedFlowUtils.createConnectableComponent(generate), + VersionedFlowUtils.createConnectableComponent(terminate), Set.of("success")); + + final VersionedExternalFlow flow = new VersionedExternalFlow(); + flow.setFlowContents(rootGroup); + flow.setParameterContexts(Collections.emptyMap()); + return flow; + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map propertyValueOverrides, final FlowContext flowContext) { + return List.of(); + } + + @Override + public List getConfigurationSteps() { + return List.of(); + } + + @Override + public void applyUpdate(final FlowContext workingFlowContext, final FlowContext activeFlowContext) { + } +} diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/GatedDataQueuingConnector.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/GatedDataQueuingConnector.java new file mode 100644 index 000000000000..8c7f680d86ba --- /dev/null +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/GatedDataQueuingConnector.java @@ -0,0 +1,122 @@ +/* + * 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.connectors.tests.system; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.BundleCompatibility; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.ConnectorPropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorPropertyGroup; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.PropertyType; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.Position; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; +import org.apache.nifi.processor.util.StandardValidators; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * A connector that generates FlowFiles and allows controlling when the TerminateFlowFile processor + * will process them via a gate file. If the gate file does not exist, FlowFiles will queue up. + * When the gate file exists, the TerminateFlowFile processor will process the queued FlowFiles. + *

+ * The connector uses a configuration step to specify the gate file path. The test must configure + * the connector with a gate file path and apply the update before starting the connector. + */ +public class GatedDataQueuingConnector extends AbstractConnector { + + static final ConnectorPropertyDescriptor GATE_FILE_PATH = new ConnectorPropertyDescriptor.Builder() + .name("Gate File Path") + .description("The path to the gate file. When this file exists, the TerminateFlowFile processor " + + "will process FlowFiles. When it does not exist, FlowFiles will queue up.") + .required(true) + .type(PropertyType.STRING) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + private static final ConnectorPropertyGroup PROPERTY_GROUP = new ConnectorPropertyGroup.Builder() + .name("Gate Configuration") + .addProperty(GATE_FILE_PATH) + .build(); + + private static final ConfigurationStep CONFIG_STEP = new ConfigurationStep.Builder() + .name("Gate Configuration") + .propertyGroups(List.of(PROPERTY_GROUP)) + .build(); + + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) { + } + + @Override + public VersionedExternalFlow getInitialFlow() { + final Bundle bundle = new Bundle("org.apache.nifi", "nifi-system-test-extensions-nar", "2.8.0"); + final VersionedProcessGroup rootGroup = VersionedFlowUtils.createProcessGroup("1234", "Gated Data Queuing Connector"); + + final VersionedProcessor generate = VersionedFlowUtils.addProcessor(rootGroup, + "org.apache.nifi.processors.tests.system.GenerateFlowFile", bundle, "GenerateFlowFile", new Position(0, 0)); + generate.getProperties().put("File Size", "1 KB"); + generate.setSchedulingPeriod("100 millis"); + + final VersionedProcessor terminate = VersionedFlowUtils.addProcessor(rootGroup, + "org.apache.nifi.processors.tests.system.TerminateFlowFile", bundle, "TerminateFlowFile", new Position(0, 0)); + + VersionedFlowUtils.addConnection(rootGroup, VersionedFlowUtils.createConnectableComponent(generate), + VersionedFlowUtils.createConnectableComponent(terminate), Set.of("success")); + + final VersionedExternalFlow flow = new VersionedExternalFlow(); + flow.setFlowContents(rootGroup); + flow.setParameterContexts(Collections.emptyMap()); + return flow; + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map propertyValueOverrides, final FlowContext flowContext) { + return List.of(); + } + + @Override + public List getConfigurationSteps() { + return List.of(CONFIG_STEP); + } + + @Override + public void applyUpdate(final FlowContext workingFlowContext, final FlowContext activeFlowContext) throws FlowUpdateException { + final String gateFilePath = workingFlowContext.getConfigurationContext().getProperty(CONFIG_STEP, GATE_FILE_PATH).getValue(); + if (gateFilePath == null) { + return; + } + + final VersionedExternalFlow flow = getInitialFlow(); + final VersionedProcessGroup rootGroup = flow.getFlowContents(); + + VersionedFlowUtils.findProcessor(rootGroup, p -> p.getType().endsWith("TerminateFlowFile")) + .ifPresent(processor -> processor.getProperties().put("Gate File", gateFilePath)); + + getInitializationContext().updateFlow(activeFlowContext, flow, BundleCompatibility.RESOLVE_BUNDLE); + } +} diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/NestedProcessGroupConnector.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/NestedProcessGroupConnector.java new file mode 100644 index 000000000000..e3903a38f1de --- /dev/null +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/NestedProcessGroupConnector.java @@ -0,0 +1,75 @@ +/* + * 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.connectors.tests.system; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedProcessGroup; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Test Connector that creates a flow with a nested (child) process group. + * Used to verify that child process groups within a connector's managed flow + * can be properly accessed via the connector API. + */ +public class NestedProcessGroupConnector extends AbstractConnector { + + public static final String CHILD_PROCESS_GROUP_ID = "child-process-group-id"; + + private final List configurationSteps = List.of(); + + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) { + } + + @Override + public VersionedExternalFlow getInitialFlow() { + final VersionedProcessGroup childGroup = new VersionedProcessGroup(); + childGroup.setIdentifier(CHILD_PROCESS_GROUP_ID); + childGroup.setName("Child Process Group"); + + final VersionedProcessGroup rootGroup = new VersionedProcessGroup(); + rootGroup.setName("Nested Flow"); + rootGroup.setProcessGroups(Set.of(childGroup)); + + final VersionedExternalFlow flow = new VersionedExternalFlow(); + flow.setFlowContents(rootGroup); + return flow; + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map propertyValueOverrides, final FlowContext flowContext) { + return List.of(); + } + + @Override + public List getConfigurationSteps() { + return configurationSteps; + } + + @Override + public void applyUpdate(final FlowContext workingFlowContext, final FlowContext activeFlowContext) { + } +} + diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/NopConnector.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/NopConnector.java new file mode 100644 index 000000000000..3d28240ca256 --- /dev/null +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/NopConnector.java @@ -0,0 +1,133 @@ +/* + * 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.connectors.tests.system; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.ConfigVerificationResult.Outcome; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.Validator; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.ConnectorPropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorPropertyGroup; +import org.apache.nifi.components.connector.PropertyType; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedParameter; +import org.apache.nifi.flow.VersionedParameterContext; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.processor.util.StandardValidators; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class NopConnector extends AbstractConnector { + + private static final ConnectorPropertyDescriptor IGNORED_PROPERTY = new ConnectorPropertyDescriptor.Builder() + .name("Ignored Property") + .description("This property is ignored by the NopConnector.") + .required(false) + .type(PropertyType.STRING) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + private static final ConnectorPropertyGroup IGNORED_PROPERTY_GROUP = new ConnectorPropertyGroup.Builder() + .name("Ignored Property Group") + .description("This property group is ignored by the NopConnector.") + .properties(List.of(IGNORED_PROPERTY)) + .build(); + + private static final ConnectorPropertyDescriptor SECRET_PROPERTY = new ConnectorPropertyDescriptor.Builder() + .name("Secret Property") + .description("This is a secret property that is ignored by the NopConnector.") + .required(false) + .type(PropertyType.SECRET) + .addValidator(new SuperSecretValidator()) + .build(); + private static final ConnectorPropertyGroup SECRET_PROPERTY_GROUP = new ConnectorPropertyGroup.Builder() + .name("Secret Property Group") + .description("This property group is ignored by the NopConnector.") + .properties(List.of(SECRET_PROPERTY)) + .build(); + + private static final ConfigurationStep IGNORED_STEP = new ConfigurationStep.Builder() + .name("Ignored Step") + .propertyGroups(List.of(IGNORED_PROPERTY_GROUP, SECRET_PROPERTY_GROUP)) + .build(); + + private final List configurationSteps = List.of(IGNORED_STEP); + + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) { + } + + @Override + public VersionedExternalFlow getInitialFlow() { + final VersionedProcessGroup group = new VersionedProcessGroup(); + group.setName("Nop Flow"); + + final VersionedParameter someText = new VersionedParameter(); + someText.setName("SOME_TEXT"); + someText.setValue("Lorem ipsum"); + someText.setSensitive(false); + someText.setProvided(false); + someText.setReferencedAssets(List.of()); + + final VersionedParameterContext parameterContext = new VersionedParameterContext(); + parameterContext.setName("Nop Parameter Context"); + parameterContext.setParameters(Set.of(someText)); + + final VersionedExternalFlow flow = new VersionedExternalFlow(); + flow.setParameterContexts(Map.of(parameterContext.getName(), parameterContext)); + flow.setFlowContents(group); + return flow; + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map propertyValueOverrides, final FlowContext flowContext) { + return List.of(new ConfigVerificationResult.Builder() + .outcome(Outcome.SUCCESSFUL) + .subject(stepName) + .verificationStepName("Nop Verification") + .explanation("Successful verification with properties: " + propertyValueOverrides) + .build()); + } + + @Override + public List getConfigurationSteps() { + return configurationSteps; + } + + @Override + public void applyUpdate(final FlowContext workingFlowContext, final FlowContext activeFlowContext) { + } + + private static class SuperSecretValidator implements Validator { + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + return new ValidationResult.Builder() + .subject(subject) + .input(input) + .explanation("Value must be 'supersecret'") + .valid(input.equals("supersecret")) + .build(); + } + } +} diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/ParameterContextConnector.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/ParameterContextConnector.java new file mode 100644 index 000000000000..9d160a11f1c2 --- /dev/null +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/ParameterContextConnector.java @@ -0,0 +1,286 @@ +/* + * 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.connectors.tests.system; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.BundleCompatibility; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.ConnectorPropertyDescriptor; +import org.apache.nifi.components.connector.ConnectorPropertyGroup; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.PropertyType; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.Position; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedParameter; +import org.apache.nifi.flow.VersionedParameterContext; +import org.apache.nifi.flow.VersionedPort; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; +import org.apache.nifi.processor.util.StandardValidators; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * A test connector that creates a flow with nested Process Groups and Parameter Context inheritance. + * Used to verify that sensitive parameters and asset-referencing parameters work correctly + * when inherited from child Parameter Contexts. + * + * The flow structure: + * - Root Group (bound to Parent Parameter Context) + * - GenerateFlowFile (generates 1 FlowFile) + * - Process Group A (bound to Parent Parameter Context) + * - Input Port + * - UpdateContent (Sensitive Content = #{sensitive_param}) + * - WriteToFile (target/sensitive.txt) + * - Process Group B (bound to Parent Parameter Context) + * - Input Port + * - ReplaceWithFile (Filename = #{asset_param}) + * - WriteToFile (target/asset.txt) + * + * Parameter Context hierarchy: + * - Parent Parameter Context (inherits from Child Context A and Child Context B) + * - Child Context A: sensitive_param (sensitive) + * - Child Context B: asset_param (asset reference) + */ +public class ParameterContextConnector extends AbstractConnector { + + private static final String CONFIGURATION_STEP_NAME = "Parameter Context Configuration"; + + private static final String ROOT_GROUP_ID = "root-group"; + private static final String GROUP_A_ID = "process-group-a"; + private static final String GROUP_B_ID = "process-group-b"; + + private static final String PARENT_CONTEXT_NAME = "Parent Parameter Context"; + private static final String CHILD_CONTEXT_A_NAME = "Child Context A"; + private static final String CHILD_CONTEXT_B_NAME = "Child Context B"; + + private static final String SENSITIVE_PARAM_NAME = "sensitive_param"; + private static final String ASSET_PARAM_NAME = "asset_param"; + + static final ConnectorPropertyDescriptor SENSITIVE_VALUE = new ConnectorPropertyDescriptor.Builder() + .name("Sensitive Value") + .description("The sensitive value to be stored in the sensitive parameter") + .required(true) + .type(PropertyType.SECRET) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + static final ConnectorPropertyDescriptor ASSET_FILE = new ConnectorPropertyDescriptor.Builder() + .name("Asset File") + .description("The asset file whose contents will be used via the asset parameter") + .required(true) + .type(PropertyType.ASSET) + .build(); + + static final ConnectorPropertyDescriptor SENSITIVE_OUTPUT_FILE = new ConnectorPropertyDescriptor.Builder() + .name("Sensitive Output File") + .description("The file path where the sensitive value output will be written") + .required(true) + .type(PropertyType.STRING) + .defaultValue("target/sensitive.txt") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + static final ConnectorPropertyDescriptor ASSET_OUTPUT_FILE = new ConnectorPropertyDescriptor.Builder() + .name("Asset Output File") + .description("The file path where the asset contents output will be written") + .required(true) + .type(PropertyType.STRING) + .defaultValue("target/asset.txt") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + private static final ConnectorPropertyGroup PROPERTY_GROUP = new ConnectorPropertyGroup.Builder() + .name("Parameter Context Configuration") + .description("Configuration properties for parameter context testing") + .properties(List.of(SENSITIVE_VALUE, ASSET_FILE, SENSITIVE_OUTPUT_FILE, ASSET_OUTPUT_FILE)) + .build(); + + private static final ConfigurationStep CONFIG_STEP = new ConfigurationStep.Builder() + .name(CONFIGURATION_STEP_NAME) + .description("Configure the sensitive value and asset file for parameter context testing") + .propertyGroups(List.of(PROPERTY_GROUP)) + .build(); + + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) throws FlowUpdateException { + } + + @Override + public VersionedExternalFlow getInitialFlow() { + return createEmptyFlow(); + } + + private VersionedExternalFlow createEmptyFlow() { + final VersionedProcessGroup rootGroup = VersionedFlowUtils.createProcessGroup(ROOT_GROUP_ID, "Parameter Context Test Flow"); + + final VersionedExternalFlow flow = new VersionedExternalFlow(); + flow.setFlowContents(rootGroup); + flow.setParameterContexts(Collections.emptyMap()); + return flow; + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map propertyValueOverrides, final FlowContext flowContext) { + return List.of(); + } + + @Override + public List getConfigurationSteps() { + return List.of(CONFIG_STEP); + } + + @Override + public void applyUpdate(final FlowContext workingContext, final FlowContext activeContext) throws FlowUpdateException { + final String sensitiveValue = workingContext.getConfigurationContext().getProperty(CONFIG_STEP, SENSITIVE_VALUE).getValue(); + final String assetFilePath = workingContext.getConfigurationContext().getProperty(CONFIG_STEP, ASSET_FILE).getValue(); + final String sensitiveOutputFile = workingContext.getConfigurationContext().getProperty(CONFIG_STEP, SENSITIVE_OUTPUT_FILE).getValue(); + final String assetOutputFile = workingContext.getConfigurationContext().getProperty(CONFIG_STEP, ASSET_OUTPUT_FILE).getValue(); + + if (sensitiveValue == null || assetFilePath == null) { + return; + } + + final VersionedExternalFlow flow = createFlow(sensitiveValue, assetFilePath, sensitiveOutputFile, assetOutputFile); + getInitializationContext().updateFlow(activeContext, flow, BundleCompatibility.RESOLVE_BUNDLE); + } + + private VersionedExternalFlow createFlow(final String sensitiveValue, final String assetFilePath, + final String sensitiveOutputFile, final String assetOutputFile) { + final Bundle bundle = new Bundle("org.apache.nifi", "nifi-system-test-extensions-nar", "2.8.0-SNAPSHOT"); + final Map parameterContexts = createParameterContexts(sensitiveValue, assetFilePath); + + final VersionedProcessGroup rootGroup = VersionedFlowUtils.createProcessGroup(ROOT_GROUP_ID, "Parameter Context Test Flow"); + rootGroup.setParameterContextName(PARENT_CONTEXT_NAME); + + final VersionedProcessor generateProcessor = VersionedFlowUtils.addProcessor(rootGroup, + "org.apache.nifi.processors.tests.system.GenerateFlowFile", bundle, "GenerateFlowFile", new Position(0, 0)); + generateProcessor.getProperties().putAll(Map.of("Max FlowFiles", "1", "File Size", "0 B")); + generateProcessor.setSchedulingPeriod("60 sec"); + + final VersionedProcessGroup groupA = createProcessGroupA(bundle, sensitiveOutputFile); + rootGroup.getProcessGroups().add(groupA); + + final VersionedProcessGroup groupB = createProcessGroupB(bundle, assetOutputFile); + rootGroup.getProcessGroups().add(groupB); + + final VersionedPort inputPortA = groupA.getInputPorts().iterator().next(); + final VersionedPort inputPortB = groupB.getInputPorts().iterator().next(); + + VersionedFlowUtils.addConnection(rootGroup, VersionedFlowUtils.createConnectableComponent(generateProcessor), + VersionedFlowUtils.createConnectableComponent(inputPortA), Set.of("success")); + VersionedFlowUtils.addConnection(rootGroup, VersionedFlowUtils.createConnectableComponent(generateProcessor), + VersionedFlowUtils.createConnectableComponent(inputPortB), Set.of("success")); + + final VersionedExternalFlow flow = new VersionedExternalFlow(); + flow.setFlowContents(rootGroup); + flow.setParameterContexts(parameterContexts); + return flow; + } + + private Map createParameterContexts(final String sensitiveValue, final String assetFilePath) { + final VersionedParameter sensitiveParam = new VersionedParameter(); + sensitiveParam.setName(SENSITIVE_PARAM_NAME); + sensitiveParam.setSensitive(true); + sensitiveParam.setValue(sensitiveValue); + sensitiveParam.setProvided(false); + sensitiveParam.setReferencedAssets(List.of()); + + final VersionedParameterContext childContextA = new VersionedParameterContext(); + childContextA.setName(CHILD_CONTEXT_A_NAME); + childContextA.setParameters(Set.of(sensitiveParam)); + + final VersionedParameter assetParam = new VersionedParameter(); + assetParam.setName(ASSET_PARAM_NAME); + assetParam.setSensitive(false); + assetParam.setValue(assetFilePath); + assetParam.setProvided(false); + assetParam.setReferencedAssets(List.of()); + + final VersionedParameterContext childContextB = new VersionedParameterContext(); + childContextB.setName(CHILD_CONTEXT_B_NAME); + childContextB.setParameters(Set.of(assetParam)); + + final VersionedParameterContext parentContext = new VersionedParameterContext(); + parentContext.setName(PARENT_CONTEXT_NAME); + parentContext.setParameters(Set.of()); + parentContext.setInheritedParameterContexts(List.of(CHILD_CONTEXT_A_NAME, CHILD_CONTEXT_B_NAME)); + + final Map contexts = new HashMap<>(); + contexts.put(CHILD_CONTEXT_A_NAME, childContextA); + contexts.put(CHILD_CONTEXT_B_NAME, childContextB); + contexts.put(PARENT_CONTEXT_NAME, parentContext); + return contexts; + } + + private VersionedProcessGroup createProcessGroupA(final Bundle bundle, final String outputFile) { + final VersionedProcessGroup groupA = VersionedFlowUtils.createProcessGroup(GROUP_A_ID, "Process Group A - Sensitive Value"); + groupA.setGroupIdentifier(ROOT_GROUP_ID); + groupA.setParameterContextName(PARENT_CONTEXT_NAME); + + final VersionedPort inputPortA = VersionedFlowUtils.addInputPort(groupA, "Input Port A", new Position(0, 0)); + + final VersionedProcessor updateContent = VersionedFlowUtils.addProcessor(groupA, + "org.apache.nifi.processors.tests.system.UpdateContent", bundle, "UpdateContent", new Position(0, 0)); + updateContent.getProperties().putAll(Map.of("Sensitive Content", "#{" + SENSITIVE_PARAM_NAME + "}", "Update Strategy", "Replace")); + + final VersionedProcessor writeToFile = VersionedFlowUtils.addProcessor(groupA, + "org.apache.nifi.processors.tests.system.WriteToFile", bundle, "WriteToFile", new Position(0, 0)); + writeToFile.getProperties().put("Filename", outputFile); + writeToFile.setAutoTerminatedRelationships(Set.of("success", "failure")); + + VersionedFlowUtils.addConnection(groupA, VersionedFlowUtils.createConnectableComponent(inputPortA), + VersionedFlowUtils.createConnectableComponent(updateContent), Set.of()); + VersionedFlowUtils.addConnection(groupA, VersionedFlowUtils.createConnectableComponent(updateContent), + VersionedFlowUtils.createConnectableComponent(writeToFile), Set.of("success")); + + return groupA; + } + + private VersionedProcessGroup createProcessGroupB(final Bundle bundle, final String outputFile) { + final VersionedProcessGroup groupB = VersionedFlowUtils.createProcessGroup(GROUP_B_ID, "Process Group B - Asset Value"); + groupB.setGroupIdentifier(ROOT_GROUP_ID); + groupB.setParameterContextName(PARENT_CONTEXT_NAME); + + final VersionedPort inputPortB = VersionedFlowUtils.addInputPort(groupB, "Input Port B", new Position(0, 0)); + + final VersionedProcessor replaceWithFile = VersionedFlowUtils.addProcessor(groupB, + "org.apache.nifi.processors.tests.system.ReplaceWithFile", bundle, "ReplaceWithFile", new Position(0, 0)); + replaceWithFile.getProperties().put("Filename", "#{" + ASSET_PARAM_NAME + "}"); + + final VersionedProcessor writeToFile = VersionedFlowUtils.addProcessor(groupB, + "org.apache.nifi.processors.tests.system.WriteToFile", bundle, "WriteToFile", new Position(0, 0)); + writeToFile.getProperties().put("Filename", outputFile); + writeToFile.setAutoTerminatedRelationships(Set.of("success", "failure")); + + VersionedFlowUtils.addConnection(groupB, VersionedFlowUtils.createConnectableComponent(inputPortB), + VersionedFlowUtils.createConnectableComponent(replaceWithFile), Set.of()); + VersionedFlowUtils.addConnection(groupB, VersionedFlowUtils.createConnectableComponent(replaceWithFile), + VersionedFlowUtils.createConnectableComponent(writeToFile), Set.of("success")); + + return groupB; + } +} diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/SelectiveDropConnector.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/SelectiveDropConnector.java new file mode 100644 index 000000000000..b1b852b18afa --- /dev/null +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/connectors/tests/system/SelectiveDropConnector.java @@ -0,0 +1,150 @@ +/* + * 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.connectors.tests.system; + +import org.apache.nifi.components.ConfigVerificationResult; +import org.apache.nifi.components.connector.AbstractConnector; +import org.apache.nifi.components.connector.ConfigurationStep; +import org.apache.nifi.components.connector.DropFlowFileSummary; +import org.apache.nifi.components.connector.FlowUpdateException; +import org.apache.nifi.components.connector.components.ConnectionFacade; +import org.apache.nifi.components.connector.components.FlowContext; +import org.apache.nifi.components.connector.components.ProcessGroupFacade; +import org.apache.nifi.components.connector.util.VersionedFlowUtils; +import org.apache.nifi.flow.Bundle; +import org.apache.nifi.flow.Position; +import org.apache.nifi.flow.ScheduledState; +import org.apache.nifi.flow.VersionedConnection; +import org.apache.nifi.flow.VersionedExternalFlow; +import org.apache.nifi.flow.VersionedProcessGroup; +import org.apache.nifi.flow.VersionedProcessor; +import org.apache.nifi.flowfile.FlowFile; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * A Connector for testing selective dropping of FlowFiles. + * When started, it generates FlowFiles with a 'flowFileIndex' attribute. + * When stopped, it drops all FlowFiles where the flowFileIndex attribute has an even value. + */ +public class SelectiveDropConnector extends AbstractConnector { + + @Override + protected void onStepConfigured(final String stepName, final FlowContext workingContext) { + } + + @Override + public VersionedExternalFlow getInitialFlow() { + final Bundle bundle = new Bundle("org.apache.nifi", "nifi-system-test-extensions-nar", "2.7.0-SNAPSHOT"); + final VersionedProcessGroup rootGroup = VersionedFlowUtils.createProcessGroup("1234", "Selective Drop Connector"); + + final VersionedProcessor generate = VersionedFlowUtils.addProcessor(rootGroup, + "org.apache.nifi.processors.tests.system.GenerateFlowFile", bundle, "GenerateFlowFile", new Position(0, 0)); + generate.getProperties().putAll(Map.of( + "File Size", "1 B", + "Batch Size", "20000", + "Max FlowFiles", "20000", + "flowFileIndex", "${nextInt()}" + )); + generate.setSchedulingPeriod("10 sec"); + + final VersionedProcessor terminate = VersionedFlowUtils.addProcessor(rootGroup, + "org.apache.nifi.processors.tests.system.TerminateFlowFile", bundle, "TerminateFlowFile", new Position(0, 0)); + terminate.setScheduledState(ScheduledState.DISABLED); + + final VersionedConnection connection = VersionedFlowUtils.addConnection(rootGroup, + VersionedFlowUtils.createConnectableComponent(generate), VersionedFlowUtils.createConnectableComponent(terminate), Set.of("success")); + connection.setBackPressureDataSizeThreshold("100 GB"); + connection.setBackPressureObjectThreshold(100_000L); + + final VersionedExternalFlow flow = new VersionedExternalFlow(); + flow.setFlowContents(rootGroup); + flow.setParameterContexts(Collections.emptyMap()); + return flow; + } + + @Override + public List verifyConfigurationStep(final String stepName, final Map propertyValueOverrides, final FlowContext flowContext) { + return List.of(); + } + + @Override + public List getConfigurationSteps() { + return List.of(); + } + + @Override + public void applyUpdate(final FlowContext workingFlowContext, final FlowContext activeFlowContext) throws FlowUpdateException { + final VersionedExternalFlow flow = getInitialFlow(); + getInitializationContext().updateFlow(activeFlowContext, flow); + } + + @Override + public void stop(final FlowContext context) throws FlowUpdateException { + super.stop(context); + + final ProcessGroupFacade rootGroup = context.getRootGroup(); + final ConnectionFacade connection = findFirstConnection(rootGroup); + + if (connection == null) { + getLogger().warn("Could not find connection to perform selective drop"); + return; + } + + try { + final AtomicInteger dropCount = new AtomicInteger(); + final AtomicInteger seenCount = new AtomicInteger(); + final DropFlowFileSummary summary = connection.dropFlowFiles(ff -> { + final boolean shouldDrop = hasEvenFlowFileIndex(ff); + seenCount.incrementAndGet(); + if (shouldDrop) { + dropCount.incrementAndGet(); + } + return shouldDrop; + }); + getLogger().info("Selectively dropped {} out of {} FlowFiles ({} bytes) with even flowFileIndex values", + dropCount.get(), seenCount.get(), summary.getDroppedBytes()); + } catch (final IOException e) { + throw new FlowUpdateException("Failed to selectively drop FlowFiles", e); + } + } + + private boolean hasEvenFlowFileIndex(final FlowFile flowFile) { + return Integer.parseInt(flowFile.getAttribute("flowFileIndex")) % 2 == 0; + } + + private ConnectionFacade findFirstConnection(final ProcessGroupFacade group) { + for (final ConnectionFacade connection : group.getConnections()) { + return connection; + } + + for (final ProcessGroupFacade childGroup : group.getProcessGroups()) { + final ConnectionFacade found = findFirstConnection(childGroup); + if (found != null) { + return found; + } + } + + return null; + } +} diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/Calculate.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/Calculate.java new file mode 100644 index 000000000000..d8bc65b8a55f --- /dev/null +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/Calculate.java @@ -0,0 +1,71 @@ +/* + * 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.processors.tests.system; + +import org.apache.nifi.components.connector.components.ComponentState; +import org.apache.nifi.components.connector.components.ConnectorMethod; +import org.apache.nifi.components.connector.components.MethodArgument; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; + +/** + * A test processor that exposes a ConnectorMethod for performing calculations. + * This processor is used to test JSON marshalling of complex objects across ClassLoader boundaries. + */ +public class Calculate extends AbstractProcessor { + + @ConnectorMethod( + name = "calculate", + description = "Performs a calculation based on the provided Calculation object", + allowedStates = ComponentState.STOPPED, + arguments = { + @MethodArgument(name = "calculation", type = Calculation.class, description = "The calculation to perform", required = true) + } + ) + public Object calculate(final Calculation calculation) { + final int result = switch (calculation.operation()) { + case "ADD" -> calculation.operand1() + calculation.operand2(); + case "SUBTRACT" -> calculation.operand1() - calculation.operand2(); + case "MULTIPLY" -> calculation.operand1() * calculation.operand2(); + case "DIVIDE" -> calculation.operand1() / calculation.operand2(); + default -> throw new IllegalArgumentException("Unknown operation: " + calculation.operation()); + }; + + return new CalculatedResult(calculation, result); + } + + @Override + public void onTrigger(final ProcessContext processContext, final ProcessSession processSession) throws ProcessException { + } + + /** + * A record representing a calculation request with two operands and an operation. + */ + public record Calculation(int operand1, int operand2, String operation) { + } + + /** + * A record representing the result of a calculation, including the original calculation and the result. + */ + public record CalculatedResult(Calculation calculation, int result) { + } + +} + diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/TerminateFlowFile.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/TerminateFlowFile.java index 734766fa37eb..9ab6fd71b18c 100644 --- a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/TerminateFlowFile.java +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/TerminateFlowFile.java @@ -16,15 +16,43 @@ */ package org.apache.nifi.processors.tests.system; +import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.File; +import java.util.List; public class TerminateFlowFile extends AbstractProcessor { + + public static final PropertyDescriptor GATE_FILE = new PropertyDescriptor.Builder() + .name("Gate File") + .description("An optional file path. If specified, the processor will only process FlowFiles when this file exists. " + + "If the file does not exist, the processor will yield and return without processing any data.") + .required(false) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .build(); + + @Override + protected List getSupportedPropertyDescriptors() { + return List.of(GATE_FILE); + } + @Override public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + final String gateFilePath = context.getProperty(GATE_FILE).getValue(); + if (gateFilePath != null) { + final File gateFile = new File(gateFilePath); + if (!gateFile.exists()) { + context.yield(); + return; + } + } + FlowFile flowFile = session.get(); if (flowFile == null) { return; diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/UpdateContent.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/UpdateContent.java index 0f79a426668e..652d779fd2df 100644 --- a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/UpdateContent.java +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/UpdateContent.java @@ -19,6 +19,8 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor.Builder; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.Validator; import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.flowfile.FlowFile; @@ -29,7 +31,9 @@ import org.apache.nifi.processor.exception.ProcessException; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Set; @@ -40,9 +44,17 @@ public class UpdateContent extends AbstractProcessor { .name("Content") .displayName("Content") .description("Content to set") - .required(true) + .required(false) + .addValidator(Validator.VALID) + .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) + .build(); + static final PropertyDescriptor SENSITIVE_CONTENT = new Builder() + .name("Sensitive Content") + .displayName("Sensitive Content") + .description("Sensitive content to set (use for sensitive parameter references)") + .required(false) + .sensitive(true) .addValidator(Validator.VALID) - .defaultValue("Default Content") .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) .build(); static final PropertyDescriptor UPDATE_STRATEGY = new Builder() @@ -60,7 +72,23 @@ public class UpdateContent extends AbstractProcessor { @Override protected List getSupportedPropertyDescriptors() { - return Arrays.asList(CONTENT, UPDATE_STRATEGY); + return Arrays.asList(CONTENT, SENSITIVE_CONTENT, UPDATE_STRATEGY); + } + + @Override + protected Collection customValidate(final ValidationContext context) { + final List results = new ArrayList<>(); + final boolean hasContent = context.getProperty(CONTENT).isSet(); + final boolean hasSensitiveContent = context.getProperty(SENSITIVE_CONTENT).isSet(); + + if (hasContent == hasSensitiveContent) { + results.add(new ValidationResult.Builder() + .subject("Content") + .valid(false) + .explanation("Exactly one of 'Content' or 'Sensitive Content' must be set") + .build()); + } + return results; } @Override @@ -75,7 +103,12 @@ public void onTrigger(final ProcessContext context, final ProcessSession session return; } - final String content = context.getProperty(CONTENT).evaluateAttributeExpressions(flowFile).getValue(); + final String content; + if (context.getProperty(CONTENT).isSet()) { + content = context.getProperty(CONTENT).evaluateAttributeExpressions(flowFile).getValue(); + } else { + content = context.getProperty(SENSITIVE_CONTENT).evaluateAttributeExpressions(flowFile).getValue(); + } final byte[] contentBytes = content.getBytes(StandardCharsets.UTF_8); final String strategy = context.getProperty(UPDATE_STRATEGY).getValue(); diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.components.connector.Connector b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.components.connector.Connector new file mode 100644 index 000000000000..a2d46427433a --- /dev/null +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.components.connector.Connector @@ -0,0 +1,25 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.nifi.connectors.tests.system.AssetConnector +org.apache.nifi.connectors.tests.system.BundleResolutionConnector +org.apache.nifi.connectors.tests.system.CalculateConnector +org.apache.nifi.connectors.tests.system.ComponentLifecycleConnector +org.apache.nifi.connectors.tests.system.DataQueuingConnector +org.apache.nifi.connectors.tests.system.GatedDataQueuingConnector +org.apache.nifi.connectors.tests.system.NestedProcessGroupConnector +org.apache.nifi.connectors.tests.system.NopConnector +org.apache.nifi.connectors.tests.system.ParameterContextConnector +org.apache.nifi.connectors.tests.system.SelectiveDropConnector diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor index a12f954cb84a..dab323f15001 100644 --- a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -63,3 +63,4 @@ org.apache.nifi.processors.tests.system.WriteFlowFileCountToFile org.apache.nifi.processors.tests.system.WriteLifecycleEvents org.apache.nifi.processors.tests.system.WriteToFile org.apache.nifi.processors.tests.system.YieldSource +org.apache.nifi.processors.tests.system.Calculate \ No newline at end of file diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java index 2c3181a96db6..053615d2121a 100644 --- a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiClientUtil.java @@ -17,6 +17,7 @@ package org.apache.nifi.tests.system; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; +import org.apache.nifi.components.connector.ConnectorState; import org.apache.nifi.controller.AbstractPort; import org.apache.nifi.controller.queue.LoadBalanceCompression; import org.apache.nifi.controller.queue.LoadBalanceStrategy; @@ -27,6 +28,7 @@ import org.apache.nifi.scheduling.ExecutionNode; import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.toolkit.client.ConnectionClient; +import org.apache.nifi.toolkit.client.ConnectorClient; import org.apache.nifi.toolkit.client.NiFiClient; import org.apache.nifi.toolkit.client.NiFiClientException; import org.apache.nifi.toolkit.client.ProcessorClient; @@ -34,8 +36,11 @@ import org.apache.nifi.web.api.dto.AssetReferenceDTO; import org.apache.nifi.web.api.dto.BundleDTO; import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO; +import org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO; import org.apache.nifi.web.api.dto.ConnectableDTO; import org.apache.nifi.web.api.dto.ConnectionDTO; +import org.apache.nifi.web.api.dto.ConnectorDTO; +import org.apache.nifi.web.api.dto.ConnectorValueReferenceDTO; import org.apache.nifi.web.api.dto.ControllerServiceDTO; import org.apache.nifi.web.api.dto.CounterDTO; import org.apache.nifi.web.api.dto.CountersSnapshotDTO; @@ -54,10 +59,12 @@ import org.apache.nifi.web.api.dto.ProcessGroupDTO; import org.apache.nifi.web.api.dto.ProcessorConfigDTO; import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.api.dto.PropertyGroupConfigurationDTO; import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; import org.apache.nifi.web.api.dto.ReportingTaskDTO; import org.apache.nifi.web.api.dto.RevisionDTO; import org.apache.nifi.web.api.dto.VerifyConfigRequestDTO; +import org.apache.nifi.web.api.dto.VerifyConnectorConfigStepRequestDTO; import org.apache.nifi.web.api.dto.VersionControlInformationDTO; import org.apache.nifi.web.api.dto.VersionedFlowDTO; import org.apache.nifi.web.api.dto.flow.FlowDTO; @@ -69,8 +76,11 @@ import org.apache.nifi.web.api.dto.status.ProcessGroupStatusSnapshotDTO; import org.apache.nifi.web.api.dto.status.ProcessorStatusSnapshotDTO; import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepEntity; import org.apache.nifi.web.api.entity.ConnectionEntity; import org.apache.nifi.web.api.entity.ConnectionStatusEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ConnectorsEntity; import org.apache.nifi.web.api.entity.ControllerServiceEntity; import org.apache.nifi.web.api.entity.ControllerServiceRunStatusEntity; import org.apache.nifi.web.api.entity.ControllerServicesEntity; @@ -113,6 +123,7 @@ import org.apache.nifi.web.api.entity.ScheduleComponentsEntity; import org.apache.nifi.web.api.entity.StartVersionControlRequestEntity; import org.apache.nifi.web.api.entity.VerifyConfigRequestEntity; +import org.apache.nifi.web.api.entity.VerifyConnectorConfigStepRequestEntity; import org.apache.nifi.web.api.entity.VersionControlInformationEntity; import org.apache.nifi.web.api.entity.VersionedFlowUpdateRequestEntity; import org.junit.jupiter.api.Assertions; @@ -163,6 +174,12 @@ private ConnectionClient getConnectionClient() { return client; } + private ConnectorClient getConnectorClient() { + final ConnectorClient client = nifiClient.getConnectorClient(); + client.acknowledgeDisconnectedNode(); + return client; + } + public ProcessorEntity startProcessor(final ProcessorEntity currentEntity) throws NiFiClientException, IOException, InterruptedException { waitForValidationCompleted(currentEntity); @@ -225,6 +242,225 @@ private String simpleName(final String type) { return type.substring(lastIndex + 1); } + public ConnectorEntity createConnector(final String simpleTypeName) throws NiFiClientException, IOException { + return createConnector(NiFiSystemIT.TEST_CONNECTORS_PACKAGE + "." + simpleTypeName, NiFiSystemIT.NIFI_GROUP_ID, NiFiSystemIT.TEST_EXTENSIONS_ARTIFACT_ID, nifiVersion); + } + + public ConnectorEntity createConnector(final String type, final String bundleGroupId, final String artifactId, final String version) throws NiFiClientException, IOException { + final ConnectorDTO dto = new ConnectorDTO(); + dto.setType(type); + + final BundleDTO bundle = new BundleDTO(); + bundle.setGroup(bundleGroupId); + bundle.setArtifact(artifactId); + bundle.setVersion(version); + dto.setBundle(bundle); + + final ConnectorEntity entity = new ConnectorEntity(); + entity.setComponent(dto); + entity.setRevision(createNewRevision()); + entity.setDisconnectedNodeAcknowledged(true); + + final ConnectorEntity connector = getConnectorClient().createConnector(entity); + logger.info("Created Connector [type={}, id={}, name={}] for Test [{}]", simpleName(type), connector.getId(), connector.getComponent().getName(), testName); + return connector; + } + + public ConfigurationStepEntity configureConnector(final String connectorId, final String configurationStepName, + final Map properties) throws NiFiClientException, IOException { + final ConnectorEntity connectorEntity = getConnectorClient().getConnector(connectorId); + return configureConnector(connectorEntity, configurationStepName, properties); + } + + public ConfigurationStepEntity configureConnector(final ConnectorEntity connectorEntity, final String configurationStepName, + final Map properties) throws NiFiClientException, IOException { + final Map propertyValues = properties.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, entry -> createStringLiteralValueReference(entry.getValue()))); + + final PropertyGroupConfigurationDTO propertyGroupConfig = new PropertyGroupConfigurationDTO(); + propertyGroupConfig.setPropertyValues(propertyValues); + + final ConfigurationStepConfigurationDTO stepConfig = new ConfigurationStepConfigurationDTO(); + stepConfig.setConfigurationStepName(configurationStepName); + stepConfig.setPropertyGroupConfigurations(Collections.singletonList(propertyGroupConfig)); + + final ConfigurationStepEntity stepEntity = new ConfigurationStepEntity(); + stepEntity.setParentConnectorId(connectorEntity.getId()); + stepEntity.setParentConnectorRevision(connectorEntity.getRevision()); + stepEntity.setConfigurationStep(stepConfig); + stepEntity.setDisconnectedNodeAcknowledged(true); + + final ConfigurationStepEntity result = getConnectorClient().updateConfigurationStep(stepEntity); + logger.info("Configured Connector [id={}, step={}] for Test [{}]", connectorEntity.getId(), configurationStepName, testName); + return result; + } + + private ConnectorValueReferenceDTO createStringLiteralValueReference(final String value) { + final ConnectorValueReferenceDTO valueRef = new ConnectorValueReferenceDTO(); + valueRef.setValueType("STRING_LITERAL"); + valueRef.setValue(value); + return valueRef; + } + + public ConnectorValueReferenceDTO createSecretValueReference(final String secretProviderId, final String secretName, final String fullyQualifiedSecretName) { + final ConnectorValueReferenceDTO valueRef = new ConnectorValueReferenceDTO(); + valueRef.setValueType("SECRET_REFERENCE"); + valueRef.setSecretProviderId(secretProviderId); + valueRef.setSecretName(secretName); + valueRef.setFullyQualifiedSecretName(fullyQualifiedSecretName); + return valueRef; + } + + public ConfigurationStepEntity configureConnectorWithReferences(final String connectorId, final String configurationStepName, + final Map propertyValues) throws NiFiClientException, IOException { + final ConnectorEntity connectorEntity = getConnectorClient().getConnector(connectorId); + return configureConnectorWithReferences(connectorEntity, configurationStepName, propertyValues); + } + + public ConfigurationStepEntity configureConnectorWithReferences(final ConnectorEntity connectorEntity, final String configurationStepName, + final Map propertyValues) throws NiFiClientException, IOException { + final PropertyGroupConfigurationDTO propertyGroupConfig = new PropertyGroupConfigurationDTO(); + propertyGroupConfig.setPropertyValues(propertyValues); + + final ConfigurationStepConfigurationDTO stepConfig = new ConfigurationStepConfigurationDTO(); + stepConfig.setConfigurationStepName(configurationStepName); + stepConfig.setPropertyGroupConfigurations(Collections.singletonList(propertyGroupConfig)); + + final ConfigurationStepEntity stepEntity = new ConfigurationStepEntity(); + stepEntity.setParentConnectorId(connectorEntity.getId()); + stepEntity.setParentConnectorRevision(connectorEntity.getRevision()); + stepEntity.setConfigurationStep(stepConfig); + stepEntity.setDisconnectedNodeAcknowledged(true); + + final ConfigurationStepEntity result = getConnectorClient().updateConfigurationStep(stepEntity); + logger.info("Configured Connector [id={}, step={}] for Test [{}]", connectorEntity.getId(), configurationStepName, testName); + return result; + } + + public void applyConnectorUpdate(final ConnectorEntity connectorEntity) throws NiFiClientException, IOException, InterruptedException { + final ConnectorClient client = nifiClient.getConnectorClient(); + final ConnectorEntity initialEntity = client.getConnector(connectorEntity.getId()); + final ConnectorState initialState = ConnectorState.valueOf(initialEntity.getComponent().getState()); + client.applyUpdate(connectorEntity); + + while (true) { + final ConnectorEntity currentEntity = client.getConnector(connectorEntity.getId()); + final String state = currentEntity.getComponent().getState(); + final ConnectorState currentState = ConnectorState.valueOf(state); + + switch (currentState) { + case UPDATE_FAILED: + throw new IllegalStateException("Connector failed to update"); + case UPDATED: + case UPDATING: + case PREPARING_FOR_UPDATE: + logger.debug("Waiting for Connector [id={}] to finish updating (current state={})...", connectorEntity.getId(), state); + break; + default: + if (initialState == currentState) { + logger.info("Connector [id={}] has successfully updated and now has state of {}", connectorEntity.getId(), state); + return; + } + logger.debug("Waiting for Connector [id={}] to return to initial state of {} (current state={})...", connectorEntity.getId(), initialState, state); + break; + } + + // Wait 250 milliseconds before polling again + Thread.sleep(250L); + } + } + + public void waitForValidConnector(final String connectorId) throws NiFiClientException, IOException, InterruptedException { + waitForConnectorValidationStatus(connectorId, "VALID"); + } + + public void waitForInvalidConnector(final String connectorId) throws NiFiClientException, IOException, InterruptedException { + waitForConnectorValidationStatus(connectorId, "INVALID"); + } + + public void waitForConnectorValidationStatus(final String connectorId, final String expectedStatus) throws NiFiClientException, IOException, InterruptedException { + int iteration = 0; + while (true) { + final ConnectorEntity entity = getConnectorClient().getConnector(connectorId); + final String validationStatus = entity.getComponent().getValidationStatus(); + if (expectedStatus.equals(validationStatus)) { + return; + } + + if ("VALIDATING".equals(validationStatus)) { + logger.debug("Waiting for Connector {} to finish validating...", connectorId); + } else if (iteration++ % 30 == 0) { // Every 3 seconds log status + logger.info("Connector with ID {} has validation status {} but expected {}. Validation errors: {}", + connectorId, validationStatus, expectedStatus, entity.getComponent().getValidationErrors()); + } + + Thread.sleep(100L); + } + } + + public void stopConnectors() throws NiFiClientException, IOException, InterruptedException { + final ConnectorsEntity connectorsEntity = nifiClient.getFlowClient().getConnectors(); + for (final ConnectorEntity connector : connectorsEntity.getConnectors()) { + getConnectorClient().stopConnector(connector); + waitForConnectorStopped(connector.getId()); + } + } + + public void startConnector(final ConnectorEntity connectorEntity) throws NiFiClientException, IOException, InterruptedException { + startConnector(connectorEntity.getId()); + } + + public void startConnector(final String connectorId) throws NiFiClientException, IOException, InterruptedException { + final ConnectorEntity entity = getConnectorClient().getConnector(connectorId); + getConnectorClient().startConnector(entity); + waitForConnectorState(connectorId, ConnectorState.RUNNING); + } + + public void stopConnector(final ConnectorEntity connectorEntity) throws NiFiClientException, IOException, InterruptedException { + stopConnector(connectorEntity.getId()); + } + + public void stopConnector(final String connectorId) throws NiFiClientException, IOException, InterruptedException { + final ConnectorEntity entity = getConnectorClient().getConnector(connectorId); + getConnectorClient().stopConnector(entity); + waitForConnectorStopped(connectorId); + } + + public void waitForConnectorStopped(final String connectorId) throws NiFiClientException, IOException, InterruptedException { + waitForConnectorState(connectorId, ConnectorState.STOPPED); + } + + public void waitForConnectorState(final String connectorId, final ConnectorState desiredState) throws InterruptedException, NiFiClientException, IOException { + int iteration = 0; + while (true) { + final ConnectorEntity entity = getConnectorClient().getConnector(connectorId); + final String state = entity.getComponent().getState(); + if (desiredState.name().equals(state)) { + return; + } + + if (iteration++ % 30 == 0) { // Every 3 seconds log status + logger.info("Connector with ID {} has state {} but waiting for state {}.", connectorId, state, desiredState); + } + + Thread.sleep(100L); + } + } + + public ConnectorEntity drainConnector(final String connectorId) throws NiFiClientException, IOException { + final ConnectorEntity entity = getConnectorClient().getConnector(connectorId); + return getConnectorClient().drainConnector(entity); + } + + public ConnectorEntity cancelDrain(final String connectorId) throws NiFiClientException, IOException { + final ConnectorEntity entity = getConnectorClient().getConnector(connectorId); + return getConnectorClient().cancelDrain(entity); + } + + public void waitForConnectorDraining(final String connectorId) throws NiFiClientException, IOException, InterruptedException { + waitForConnectorState(connectorId, ConnectorState.DRAINING); + } + public ParameterProviderEntity createParameterProvider(final String simpleTypeName) throws NiFiClientException, IOException { return createParameterProvider(NiFiSystemIT.TEST_PARAM_PROVIDERS_PACKAGE + "." + simpleTypeName, NiFiSystemIT.NIFI_GROUP_ID, NiFiSystemIT.TEST_EXTENSIONS_ARTIFACT_ID, nifiVersion); } @@ -1232,6 +1468,42 @@ public void deleteControllerLevelServices() throws NiFiClientException, IOExcept } } + public void deleteConnectors() throws NiFiClientException, IOException { + final ConnectorsEntity connectors = nifiClient.getFlowClient().getConnectors(); + for (final ConnectorEntity connector : connectors.getConnectors()) { + purgeConnectorFlowFiles(connector.getId()); + connector.setDisconnectedNodeAcknowledged(true); + nifiClient.getConnectorClient().deleteConnector(connector); + } + } + + public DropRequestEntity purgeConnectorFlowFiles(final String connectorId) throws NiFiClientException, IOException { + final ConnectorClient connectorClient = getConnectorClient(); + final long maxTimestamp = System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(1L); + + DropRequestEntity requestEntity = connectorClient.createPurgeRequest(connectorId); + try { + while (requestEntity.getDropRequest().getPercentCompleted() < 100) { + if (System.currentTimeMillis() > maxTimestamp) { + throw new IOException("Timed out waiting for Connector " + connectorId + " to purge FlowFiles"); + } + + try { + Thread.sleep(50L); + } catch (final InterruptedException ie) { + Thread.currentThread().interrupt(); + return null; + } + + requestEntity = connectorClient.getPurgeRequest(connectorId, requestEntity.getDropRequest().getId()); + } + } finally { + requestEntity = connectorClient.deletePurgeRequest(connectorId, requestEntity.getDropRequest().getId()); + } + + return requestEntity; + } + public void waitForControllerServiceRunStatus(final String id, final String requestedRunStatus) throws NiFiClientException, IOException { final long maxTimestamp = System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(2L); logger.info("Waiting for Controller Service {} to have a Run Status of {}", id, requestedRunStatus); @@ -1930,6 +2202,49 @@ public List verifyFlowAnalysisRuleConfig(final Stri return results.getRequest().getResults(); } + public List verifyConnectorStepConfig(final String connectorId, final String configurationStepName, + final Map properties) throws NiFiClientException, IOException, InterruptedException { + + final Map propertyValues = properties.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, entry -> createStringLiteralValueReference(entry.getValue()))); + + return verifyConnectorStepConfigWithReferences(connectorId, configurationStepName, propertyValues); + } + + public List verifyConnectorStepConfigWithReferences(final String connectorId, final String configurationStepName, + final Map propertyValues) throws NiFiClientException, IOException, InterruptedException { + + final PropertyGroupConfigurationDTO propertyGroupConfig = new PropertyGroupConfigurationDTO(); + propertyGroupConfig.setPropertyValues(propertyValues); + + final ConfigurationStepConfigurationDTO stepConfig = new ConfigurationStepConfigurationDTO(); + stepConfig.setConfigurationStepName(configurationStepName); + stepConfig.setPropertyGroupConfigurations(Collections.singletonList(propertyGroupConfig)); + + final VerifyConnectorConfigStepRequestDTO requestDto = new VerifyConnectorConfigStepRequestDTO(); + requestDto.setConnectorId(connectorId); + requestDto.setConfigurationStepName(configurationStepName); + requestDto.setConfigurationStep(stepConfig); + + final VerifyConnectorConfigStepRequestEntity verificationRequest = new VerifyConnectorConfigStepRequestEntity(); + verificationRequest.setRequest(requestDto); + + VerifyConnectorConfigStepRequestEntity results = getConnectorClient().submitConfigStepVerificationRequest(verificationRequest); + while (!results.getRequest().isComplete()) { + Thread.sleep(50L); + results = getConnectorClient().getConfigStepVerificationRequest(connectorId, configurationStepName, results.getRequest().getRequestId()); + } + + final String failureReason = results.getRequest().getFailureReason(); + if (failureReason != null) { + throw new IllegalStateException("Configuration step verification failed: " + failureReason); + } + + getConnectorClient().deleteConfigStepVerificationRequest(connectorId, configurationStepName, results.getRequest().getRequestId()); + + return results.getRequest().getResults(); + } + public ReportingTaskEntity createReportingTask(final String type, final String bundleGroupId, final String artifactId, final String version) throws NiFiClientException, IOException { final ReportingTaskDTO dto = new ReportingTaskDTO(); diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiSystemIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiSystemIT.java index d9d5e522d700..3562174747b8 100644 --- a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiSystemIT.java +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/NiFiSystemIT.java @@ -89,6 +89,7 @@ public abstract class NiFiSystemIT implements NiFiInstanceProvider { public static final String TEST_PYTHON_EXTENSIONS_ARTIFACT_ID = "python-extensions"; public static final String TEST_PARAM_PROVIDERS_PACKAGE = "org.apache.nifi.parameter.tests.system"; public static final String TEST_PROCESSORS_PACKAGE = "org.apache.nifi.processors.tests.system"; + public static final String TEST_CONNECTORS_PACKAGE = "org.apache.nifi.connectors.tests.system"; public static final String TEST_CS_PACKAGE = "org.apache.nifi.cs.tests.system"; public static final String TEST_REPORTING_TASK_PACKAGE = "org.apache.nifi.reporting"; public static final String TEST_FLOW_ANALYSIS_RULE_PACKAGE = "org.apache.nifi.flowanalysis"; @@ -260,12 +261,14 @@ protected void destroyFlow() throws NiFiClientException, IOException, Interrupte getClientUtil().disableControllerLevelServices(); getClientUtil().disableFlowAnalysisRules(); getClientUtil().stopTransmitting("root"); + getClientUtil().stopConnectors(); getClientUtil().deleteAll("root"); getClientUtil().deleteControllerLevelServices(); getClientUtil().deleteReportingTasks(); getClientUtil().deleteFlowAnalysisRules(); getClientUtil().deleteParameterContexts(); getClientUtil().deleteParameterProviders(); + getClientUtil().deleteConnectors(); logger.info("Finished destroyFlow"); } @@ -466,6 +469,29 @@ protected void waitForQueueCount(final String connectionId, final int queueSize) logger.info("Queue Count for Connection {} is now {}", connectionId, queueSize); } + protected void waitForConnectorMinQueueCount(final String connectorId, final int minQueueSize) throws InterruptedException { + logger.info("Waiting for Queue Count of at least {} in Connector {}", minQueueSize, connectorId); + + waitFor(() -> { + try { + final ProcessGroupStatusEntity statusEntity = getNifiClient().getConnectorClient().getStatus(connectorId, true); + final int currentSize = statusEntity.getProcessGroupStatus().getAggregateSnapshot().getFlowFilesQueued(); + logEverySecond("Current Queue Size for Connector {} = {}, Waiting for at least {}", connectorId, currentSize, minQueueSize); + return currentSize >= minQueueSize; + } catch (final Exception e) { + logger.error("Failed to get connector queue count", e); + return false; + } + }); + + logger.info("Queue Count for Connector {} is now at least {}", connectorId, minQueueSize); + } + + protected int getConnectorQueuedFlowFileCount(final String connectorId) throws NiFiClientException, IOException { + final ProcessGroupStatusEntity statusEntity = getNifiClient().getConnectorClient().getStatus(connectorId, true); + return statusEntity.getProcessGroupStatus().getAggregateSnapshot().getFlowFilesQueued(); + } + private void waitForQueueCountToMatch(final String connectionId, final Predicate test, final String queueSizeDescription) throws InterruptedException { waitFor(() -> { final ConnectionStatusEntity statusEntity = getConnectionStatus(connectionId); diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/SpawnedStandaloneNiFiInstanceFactory.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/SpawnedStandaloneNiFiInstanceFactory.java index 5474a54a46e0..588ddd63d168 100644 --- a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/SpawnedStandaloneNiFiInstanceFactory.java +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/SpawnedStandaloneNiFiInstanceFactory.java @@ -18,9 +18,11 @@ import org.apache.nifi.bootstrap.command.process.ManagementServerAddressProvider; import org.apache.nifi.bootstrap.command.process.ProcessBuilderProvider; +import org.apache.nifi.bootstrap.command.process.ProcessHandleManagementServerAddressProvider; import org.apache.nifi.bootstrap.command.process.StandardManagementServerAddressProvider; import org.apache.nifi.bootstrap.command.process.StandardProcessBuilderProvider; import org.apache.nifi.bootstrap.configuration.ConfigurationProvider; +import org.apache.nifi.bootstrap.configuration.ManagementServerPath; import org.apache.nifi.bootstrap.configuration.StandardConfigurationProvider; import org.apache.nifi.toolkit.client.NiFiClient; import org.apache.nifi.toolkit.client.NiFiClientConfig; @@ -38,7 +40,13 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.PrintWriter; +import java.net.URI; +import java.net.http.HttpClient; +import java.net.http.HttpRequest; +import java.net.http.HttpResponse; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; +import java.time.Duration; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -448,6 +456,8 @@ public void quarantineTroubleshootingInfo(final File destinationDir, final Throw if (process == null) { logger.warn("NiFi instance is not running so will not capture diagnostics for {}", getInstanceDirectory()); + } else { + captureDiagnostics(destinationDir); } final File causeFile = new File(destinationDir, "test-failure-stack-trace.txt"); @@ -456,6 +466,43 @@ public void quarantineTroubleshootingInfo(final File destinationDir, final Throw } } + private void captureDiagnostics(final File destinationDir) { + final ProcessHandle processHandle = process.toHandle(); + final ManagementServerAddressProvider addressProvider = new ProcessHandleManagementServerAddressProvider(processHandle); + final Optional managementServerAddress = addressProvider.getAddress(); + + if (managementServerAddress.isEmpty()) { + logger.warn("Could not determine management server address for NiFi instance {}, will not capture diagnostics", getInstanceDirectory()); + return; + } + + final String diagnosticsUri = "http://%s%s?verbose=true".formatted(managementServerAddress.get(), ManagementServerPath.HEALTH_DIAGNOSTICS.getPath()); + logger.info("Capturing diagnostics from {} for failed test", diagnosticsUri); + + try (final HttpClient httpClient = HttpClient.newBuilder().connectTimeout(Duration.ofSeconds(5)).build()) { + final HttpRequest request = HttpRequest.newBuilder() + .uri(URI.create(diagnosticsUri)) + .timeout(Duration.ofSeconds(30)) + .GET() + .build(); + + final HttpResponse response = httpClient.send(request, HttpResponse.BodyHandlers.ofString(StandardCharsets.UTF_8)); + final int statusCode = response.statusCode(); + + if (statusCode == 200) { + final File diagnosticsFile = new File(destinationDir, "diagnostics.txt"); + try (final FileWriter writer = new FileWriter(diagnosticsFile, StandardCharsets.UTF_8)) { + writer.write(response.body()); + } + logger.info("Diagnostics captured successfully to {}", diagnosticsFile.getAbsolutePath()); + } else { + logger.warn("Failed to capture diagnostics from {}. HTTP status: {}", diagnosticsUri, statusCode); + } + } catch (final Exception e) { + logger.warn("Failed to capture diagnostics from {}", diagnosticsUri, e); + } + } + public NiFiClient createClient() throws IOException { final Properties nifiProperties = getProperties(); final String httpsPort = nifiProperties.getProperty("nifi.web.https.port"); diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/TroubleshootingTestWatcher.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/TroubleshootingTestWatcher.java index b2c1ab078f46..f800b8b5b9c3 100644 --- a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/TroubleshootingTestWatcher.java +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/TroubleshootingTestWatcher.java @@ -16,6 +16,7 @@ */ package org.apache.nifi.tests.system; +import org.junit.jupiter.api.extension.AfterTestExecutionCallback; import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.api.extension.TestWatcher; import org.slf4j.Logger; @@ -26,39 +27,103 @@ import java.util.Optional; /** - * Test Watcher responsible for writing troubleshooting logs + * Test Watcher responsible for writing troubleshooting logs. + *

+ * This extension implements {@link AfterTestExecutionCallback} to capture diagnostics from the running + * NiFi instance BEFORE the {@code @AfterEach} method tears it down. The JUnit 5 execution order is: + *

    + *
  1. {@code @BeforeEach}
  2. + *
  3. Test method (may fail)
  4. + *
  5. {@code AfterTestExecutionCallback.afterTestExecution()} - Diagnostics captured here while NiFi is still running
  6. + *
  7. {@code @AfterEach} - NiFi instance is torn down here
  8. + *
  9. {@code TestWatcher.testFailed()} - Too late for live diagnostics
  10. + *
*/ -public class TroubleshootingTestWatcher implements TestWatcher { +public class TroubleshootingTestWatcher implements TestWatcher, AfterTestExecutionCallback { private static final Logger logger = LoggerFactory.getLogger(TroubleshootingTestWatcher.class); + private static final ExtensionContext.Namespace NAMESPACE = ExtensionContext.Namespace.create(TroubleshootingTestWatcher.class); + private static final String QUARANTINE_DIR_KEY = "quarantineDir"; @Override - public void testFailed(final ExtensionContext context, final Throwable cause) { + public void afterTestExecution(final ExtensionContext context) { + // Check if the test failed by looking for an execution exception + final Optional executionException = context.getExecutionException(); + if (executionException.isEmpty()) { + return; // Test passed, no need to capture diagnostics + } + final Optional optionalTestInstance = context.getTestInstance(); - if (optionalTestInstance.isPresent()) { - final Object testInstance = optionalTestInstance.get(); - if (testInstance instanceof NiFiInstanceProvider) { - final NiFiInstanceProvider provider = (NiFiInstanceProvider) testInstance; - final String displayName = context.getDisplayName(); - try { - final String testClassName = context.getTestClass().map(Class::getSimpleName).orElse("TestClassUnknown"); - final File dir = quarantineTroubleshootingInfo(provider, testClassName, displayName, cause); - logger.info("Test Failed [{}]: Troubleshooting information stored [{}]", displayName, dir.getAbsolutePath()); - } catch (final Exception e) { - logger.error("Test Failed [{}]: Troubleshooting information not stored", displayName, e); + if (optionalTestInstance.isEmpty()) { + return; + } + + final Object testInstance = optionalTestInstance.get(); + if (!(testInstance instanceof NiFiInstanceProvider provider)) { + return; + } + + final NiFiInstance instance = provider.getNiFiInstance(); + if (instance == null) { + logger.warn("Test failed but NiFi Instance is not available for capturing diagnostics"); + return; + } + + final String displayName = context.getDisplayName(); + final String testClassName = context.getTestClass().map(Class::getSimpleName).orElse("TestClassUnknown"); + final Throwable cause = executionException.get(); + + try { + final File quarantineDir = quarantineTroubleshootingInfo(instance, testClassName, displayName, cause); + // Store the directory so testFailed() can log it + context.getStore(NAMESPACE).put(QUARANTINE_DIR_KEY, quarantineDir); + logger.info("Test Failed [{}]: Troubleshooting information (including live diagnostics) stored [{}]", displayName, quarantineDir.getAbsolutePath()); + } catch (final Exception e) { + logger.error("Test Failed [{}]: Failed to capture troubleshooting information", displayName, e); + } + } + + @Override + public void testFailed(final ExtensionContext context, final Throwable cause) { + // Diagnostics have already been captured in afterTestExecution() while NiFi was still running. + // This method is called after @AfterEach, so we just check if we already captured the info. + final File quarantineDir = context.getStore(NAMESPACE).get(QUARANTINE_DIR_KEY, File.class); + if (quarantineDir == null) { + // Diagnostics weren't captured in afterTestExecution - try to capture what we can now + final Optional optionalTestInstance = context.getTestInstance(); + if (optionalTestInstance.isPresent()) { + final Object testInstance = optionalTestInstance.get(); + if (testInstance instanceof NiFiInstanceProvider provider) { + final String displayName = context.getDisplayName(); + try { + final String testClassName = context.getTestClass().map(Class::getSimpleName).orElse("TestClassUnknown"); + final File dir = quarantineTroubleshootingInfoFallback(provider, testClassName, displayName, cause); + logger.info("Test Failed [{}]: Troubleshooting information stored (without live diagnostics) [{}]", displayName, dir.getAbsolutePath()); + } catch (final Exception e) { + logger.error("Test Failed [{}]: Troubleshooting information not stored", displayName, e); + } } } } } - private File quarantineTroubleshootingInfo(final NiFiInstanceProvider provider, final String testClassName, final String methodName, final Throwable failureCause) throws IOException { + private File quarantineTroubleshootingInfo(final NiFiInstance instance, final String testClassName, + final String methodName, final Throwable failureCause) throws IOException { + final File troubleshooting = new File("target/troubleshooting"); + final String quarantineDirName = testClassName + "-" + methodName.replace("()", ""); + final File quarantineDir = new File(troubleshooting, quarantineDirName); + quarantineDir.mkdirs(); + + instance.quarantineTroubleshootingInfo(quarantineDir, failureCause); + return quarantineDir; + } + + private File quarantineTroubleshootingInfoFallback(final NiFiInstanceProvider provider, final String testClassName, + final String methodName, final Throwable failureCause) throws IOException { NiFiInstance instance = provider.getNiFiInstance(); - // The teardown method may or may not have already run at this point. If it has, the instance will be null. - // In that case, just create a new instance and use it - it will map to the same directories. if (instance == null) { - logger.warn("While capturing troubleshooting info for {}, the NiFi Instance is not available. Will create a new one for Diagnostics purposes, but some of the diagnostics may be less " + - "accurate, since it's not the same instance that ran the test", methodName); - + logger.warn("While capturing troubleshooting info for {}, the NiFi Instance is not available. Will create a new one for Diagnostics purposes, but live diagnostics will not be available " + + "since it's not the same instance that ran the test", methodName); instance = provider.getInstanceFactory().createInstance(); } diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/FlowSynchronizationIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/FlowSynchronizationIT.java index acf3b1c9dadb..1365dcb3e37f 100644 --- a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/FlowSynchronizationIT.java +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/clustering/FlowSynchronizationIT.java @@ -688,6 +688,9 @@ public void testAddControllerServiceReferencingExistingService() throws NiFiClie @Test public void testUnnecessaryProcessorsAndConnectionsRemoved() throws NiFiClientException, IOException, InterruptedException { + // We make assertions about the number of components in the flow so make sure that we start with an empty flow. + destroyFlow(); + final ProcessorEntity generate = getClientUtil().createProcessor("GenerateFlowFile"); getClientUtil().updateProcessorSchedulingPeriod(generate, "10 mins"); diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ClusteredConnectorAssetsIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ClusteredConnectorAssetsIT.java new file mode 100644 index 000000000000..f628b5a4dd1c --- /dev/null +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ClusteredConnectorAssetsIT.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.tests.system.connectors; + +import org.apache.nifi.components.connector.ConnectorState; +import org.apache.nifi.tests.system.NiFiInstanceFactory; +import org.apache.nifi.toolkit.client.ConnectorClient; +import org.apache.nifi.toolkit.client.NiFiClientException; +import org.apache.nifi.util.file.FileUtils; +import org.apache.nifi.web.api.dto.AssetReferenceDTO; +import org.apache.nifi.web.api.dto.ConnectorValueReferenceDTO; +import org.apache.nifi.web.api.entity.AssetEntity; +import org.apache.nifi.web.api.entity.AssetsEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Clustered variants of the connector system tests that verify connector assets + * are synchronized across all cluster nodes. + * + * The structure of these tests is based on the clustered parameter context + * asset synchronization tests in {@code ClusteredParameterContextIT}. + */ +public class ClusteredConnectorAssetsIT extends ConnectorAssetsIT { + + @Override + public NiFiInstanceFactory getInstanceFactory() { + return createTwoNodeInstanceFactory(); + } + + @Test + public void testSynchronizeConnectorAssets() throws NiFiClientException, IOException, InterruptedException { + waitForAllNodesConnected(); + + // Create a Connector instance using the AssetConnector test extension + final ConnectorEntity connector = getClientUtil().createConnector("AssetConnector"); + assertNotNull(connector); + assertNotNull(connector.getId()); + + final String connectorId = connector.getId(); + + // Upload an Asset to the Connector + final File assetFile = new File("src/test/resources/sample-assets/helloworld.txt"); + final ConnectorClient connectorClient = getNifiClient().getConnectorClient(); + final AssetEntity assetEntity = connectorClient.createAsset(connectorId, assetFile.getName(), assetFile); + + assertNotNull(assetEntity); + assertNotNull(assetEntity.getAsset()); + assertNotNull(assetEntity.getAsset().getId()); + assertEquals(assetFile.getName(), assetEntity.getAsset().getName()); + + final String uploadedAssetId = assetEntity.getAsset().getId(); + + // List the Connector's Assets and verify that the uploaded Asset is present + final AssetsEntity assetsEntity = connectorClient.getAssets(connectorId); + assertNotNull(assetsEntity); + assertNotNull(assetsEntity.getAssets()); + assertFalse(assetsEntity.getAssets().isEmpty()); + + final boolean assetFound = assetsEntity.getAssets().stream() + .filter(a -> a.getAsset() != null) + .anyMatch(a -> uploadedAssetId.equals(a.getAsset().getId())); + + assertTrue(assetFound); + + // Check that the asset exists in the connector_assets directory for each node + final File node1Dir = getNiFiInstance().getNodeInstance(1).getInstanceDirectory(); + final File node1AssetsDir = new File(node1Dir, "connector_assets"); + final File node1ConnectorDir = new File(node1AssetsDir, connectorId); + assertTrue(node1ConnectorDir.exists()); + + final File node2Dir = getNiFiInstance().getNodeInstance(2).getInstanceDirectory(); + final File node2AssetsDir = new File(node2Dir, "connector_assets"); + final File node2ConnectorDir = new File(node2AssetsDir, connectorId); + assertTrue(node2ConnectorDir.exists()); + + final File[] node1AssetIdDirs = node1ConnectorDir.listFiles(); + assertNotNull(node1AssetIdDirs); + assertEquals(1, node1AssetIdDirs.length); + + final File[] node2AssetIdDirs = node2ConnectorDir.listFiles(); + assertNotNull(node2AssetIdDirs); + assertEquals(1, node2AssetIdDirs.length); + + // Configure the connector's "Test Asset" property to reference the uploaded asset + final ConnectorValueReferenceDTO assetValueReference = new ConnectorValueReferenceDTO(); + assetValueReference.setValueType("ASSET_REFERENCE"); + assetValueReference.setAssetReferences(List.of(new AssetReferenceDTO(uploadedAssetId))); + getClientUtil().configureConnectorWithReferences(connectorId, "Asset Configuration", Map.of("Test Asset", assetValueReference)); + + // Apply the updates to the connector + getClientUtil().applyConnectorUpdate(connector); + + // Start the connector before disconnecting node 2 + getClientUtil().startConnector(connectorId); + + // Stop node 2 and delete its connector_assets directory + disconnectNode(2); + getNiFiInstance().getNodeInstance(2).stop(); + + FileUtils.deleteFilesInDir(node2AssetsDir, (dir, name) -> true, null, true, true); + assertTrue(node2AssetsDir.delete()); + assertFalse(node2AssetsDir.exists()); + + // Start node 2 again and wait for it to rejoin the cluster + getNiFiInstance().getNodeInstance(2).start(true); + waitForAllNodesConnected(); + + // Verify that the connector state is RUNNING after node 2 rejoins + getClientUtil().waitForConnectorState(connectorId, ConnectorState.RUNNING); + getClientUtil().waitForValidConnector(connectorId); + + // Verify node 2 connector assets directory is recreated and contains the expected asset + assertTrue(node2AssetsDir.exists()); + assertTrue(node2ConnectorDir.exists()); + + final File[] node2AssetIdDirsAfterRestart = node2ConnectorDir.listFiles(); + assertNotNull(node2AssetIdDirsAfterRestart); + assertEquals(1, node2AssetIdDirsAfterRestart.length); + assertEquals(uploadedAssetId, node2AssetIdDirsAfterRestart[0].getName()); + + // Verify that the Asset is still returned from the Connector's Assets list + final AssetsEntity assetsAfterRestart = connectorClient.getAssets(connectorId); + assertNotNull(assetsAfterRestart); + assertNotNull(assetsAfterRestart.getAssets()); + + final boolean assetStillPresent = assetsAfterRestart.getAssets().stream() + .filter(a -> a.getAsset() != null) + .anyMatch(a -> uploadedAssetId.equals(a.getAsset().getId())); + + assertTrue(assetStillPresent); + } +} diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ClusteredConnectorDrainIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ClusteredConnectorDrainIT.java new file mode 100644 index 000000000000..9ee3fbd09b2a --- /dev/null +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ClusteredConnectorDrainIT.java @@ -0,0 +1,228 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.tests.system.connectors; + +import org.apache.nifi.components.connector.ConnectorState; +import org.apache.nifi.tests.system.NiFiInstanceFactory; +import org.apache.nifi.tests.system.NiFiSystemIT; +import org.apache.nifi.toolkit.client.NiFiClientException; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Cluster-specific tests for connector draining functionality. + * These tests verify that draining works correctly when nodes complete draining at different times. + */ +public class ClusteredConnectorDrainIT extends NiFiSystemIT { + + private static final Logger logger = LoggerFactory.getLogger(ClusteredConnectorDrainIT.class); + + @Override + public NiFiInstanceFactory getInstanceFactory() { + return createTwoNodeInstanceFactory(); + } + + /** + * Tests that when draining in a cluster: + * 1. Create gate file on Node 1 only - Node 1 finishes draining (STOPPED), Node 2 still DRAINING + * 2. Aggregate state should be DRAINING (since at least one node is still draining) + * 3. Create gate file on Node 2 - both nodes finish draining + * 4. Aggregate state should be STOPPED + */ + @Test + public void testDrainWithNodeCompletingAtDifferentTimes() throws NiFiClientException, IOException, InterruptedException { + final File node1InstanceDir = getNiFiInstance().getNodeInstance(1).getInstanceDirectory(); + final File node2InstanceDir = getNiFiInstance().getNodeInstance(2).getInstanceDirectory(); + final File node1GateFile = new File(node1InstanceDir, "gate-file.txt"); + final File node2GateFile = new File(node2InstanceDir, "gate-file.txt"); + node1GateFile.deleteOnExit(); + node2GateFile.deleteOnExit(); + deleteIfExists(node1GateFile); + deleteIfExists(node2GateFile); + + logger.info("Creating GatedDataQueuingConnector"); + final ConnectorEntity connector = getClientUtil().createConnector("GatedDataQueuingConnector"); + assertNotNull(connector); + final String connectorId = connector.getId(); + + final String gateFilePath = "./gate-file.txt"; + logger.info("Configuring connector {} with gate file path: {}", connectorId, gateFilePath); + getClientUtil().configureConnector(connector, "Gate Configuration", Map.of("Gate File Path", gateFilePath)); + getClientUtil().applyConnectorUpdate(connector); + + logger.info("Starting connector {}", connectorId); + getClientUtil().startConnector(connectorId); + + logger.info("Waiting for at least 5 FlowFiles to queue"); + waitForConnectorMinQueueCount(connectorId, 5); + + logger.info("Stopping connector {}", connectorId); + getClientUtil().stopConnector(connectorId); + getClientUtil().waitForConnectorStopped(connectorId); + + final int queuedCountBeforeDrain = getConnectorQueuedFlowFileCount(connectorId); + logger.info("Queued FlowFile count before drain: {}", queuedCountBeforeDrain); + assertTrue(queuedCountBeforeDrain > 0); + + logger.info("Initiating drain for connector {}", connectorId); + getClientUtil().drainConnector(connectorId); + + logger.info("Waiting for aggregate connector state to be DRAINING"); + getClientUtil().waitForConnectorDraining(connectorId); + + logger.info("Creating gate file for Node 1 only: {}", node1GateFile.getAbsolutePath()); + assertTrue(node1GateFile.createNewFile()); + + logger.info("Waiting for Node 1 to finish draining and become STOPPED"); + waitForNodeConnectorState(1, connectorId, ConnectorState.STOPPED); + waitForNodeConnectorState(2, connectorId, ConnectorState.DRAINING); + + logger.info("Verifying aggregate state is still DRAINING (Node 2 still draining)"); + final ConnectorEntity aggregateConnector = getNifiClient().getConnectorClient().getConnector(connectorId); + assertEquals(ConnectorState.DRAINING.name(), aggregateConnector.getComponent().getState()); + + logger.info("Creating gate file for Node 2: {}", node2GateFile.getAbsolutePath()); + assertTrue(node2GateFile.createNewFile()); + + logger.info("Waiting for aggregate state to become STOPPED"); + waitFor(() -> { + try { + final ConnectorEntity entity = getNifiClient().getConnectorClient().getConnector(connectorId); + return ConnectorState.STOPPED.name().equals(entity.getComponent().getState()); + } catch (final Exception e) { + return false; + } + }); + + final ConnectorEntity finalConnector = getNifiClient().getConnectorClient().getConnector(connectorId); + logger.info("Final aggregate connector state: {}", finalConnector.getComponent().getState()); + assertEquals(ConnectorState.STOPPED.name(), finalConnector.getComponent().getState()); + + final int finalQueuedCount = getConnectorQueuedFlowFileCount(connectorId); + logger.info("Final queued FlowFile count: {}", finalQueuedCount); + assertEquals(0, finalQueuedCount); + + logger.info("testDrainWithNodeCompletingAtDifferentTimes completed successfully"); + } + + /** + * Tests that when canceling drain in a cluster where one node has already finished: + * 1. Create gate file on Node 1 only - Node 1 finishes draining (STOPPED), Node 2 still DRAINING + * 2. Aggregate state should be DRAINING + * 3. Cancel drain - Node 2 should stop draining + * 4. Aggregate state should be STOPPED + * 5. Data should still be queued (from Node 2) + */ + @Test + public void testCancelDrainWithOneNodeAlreadyComplete() throws NiFiClientException, IOException, InterruptedException { + final File node1InstanceDir = getNiFiInstance().getNodeInstance(1).getInstanceDirectory(); + final File node2InstanceDir = getNiFiInstance().getNodeInstance(2).getInstanceDirectory(); + final File node1GateFile = new File(node1InstanceDir, "gate-file-cancel.txt"); + final File node2GateFile = new File(node2InstanceDir, "gate-file-cancel.txt"); + node1GateFile.deleteOnExit(); + node2GateFile.deleteOnExit(); + deleteIfExists(node1GateFile); + deleteIfExists(node2GateFile); + + logger.info("Creating GatedDataQueuingConnector"); + final ConnectorEntity connector = getClientUtil().createConnector("GatedDataQueuingConnector"); + assertNotNull(connector); + final String connectorId = connector.getId(); + + final String gateFilePath = "./gate-file-cancel.txt"; + logger.info("Configuring connector {} with gate file path: {}", connectorId, gateFilePath); + getClientUtil().configureConnector(connector, "Gate Configuration", Map.of("Gate File Path", gateFilePath)); + getClientUtil().applyConnectorUpdate(connector); + + logger.info("Starting connector {}", connectorId); + getClientUtil().startConnector(connectorId); + + logger.info("Waiting for at least 5 FlowFiles to queue"); + waitForConnectorMinQueueCount(connectorId, 5); + + logger.info("Stopping connector {}", connectorId); + getClientUtil().stopConnector(connectorId); + getClientUtil().waitForConnectorStopped(connectorId); + + final int queuedCountBeforeDrain = getConnectorQueuedFlowFileCount(connectorId); + logger.info("Queued FlowFile count before drain: {}", queuedCountBeforeDrain); + assertTrue(queuedCountBeforeDrain > 0); + + logger.info("Initiating drain for connector {}", connectorId); + getClientUtil().drainConnector(connectorId); + + logger.info("Waiting for aggregate connector state to be DRAINING"); + getClientUtil().waitForConnectorDraining(connectorId); + + logger.info("Creating gate file for Node 1 only: {}", node1GateFile.getAbsolutePath()); + assertTrue(node1GateFile.createNewFile()); + + logger.info("Waiting for Node 1 to finish draining and become STOPPED"); + waitForNodeConnectorState(1, connectorId, ConnectorState.STOPPED); + + logger.info("Verifying aggregate state is still DRAINING (Node 2 still draining)"); + final ConnectorEntity aggregateBeforeCancel = getNifiClient().getConnectorClient().getConnector(connectorId); + assertEquals(ConnectorState.DRAINING.name(), aggregateBeforeCancel.getComponent().getState()); + + logger.info("Canceling drain for connector {}", connectorId); + getClientUtil().cancelDrain(connectorId); + + logger.info("Waiting for aggregate state to become STOPPED after cancel"); + getClientUtil().waitForConnectorStopped(connectorId); + + final ConnectorEntity finalConnector = getNifiClient().getConnectorClient().getConnector(connectorId); + logger.info("Final aggregate connector state: {}", finalConnector.getComponent().getState()); + assertEquals(ConnectorState.STOPPED.name(), finalConnector.getComponent().getState()); + + final int finalQueuedCount = getConnectorQueuedFlowFileCount(connectorId); + logger.info("Final queued FlowFile count after cancel (should still have data from Node 2): {}", finalQueuedCount); + assertTrue(finalQueuedCount > 0); + + logger.info("testCancelDrainWithOneNodeAlreadyComplete completed successfully"); + } + + private void waitForNodeConnectorState(final int nodeIndex, final String connectorId, final ConnectorState expectedState) throws InterruptedException { + logger.info("Waiting for Node {} connector {} to reach state {}", nodeIndex, connectorId, expectedState); + waitFor(() -> { + try { + switchClientToNode(nodeIndex); + final ConnectorEntity entity = getNifiClient().getConnectorClient(DO_NOT_REPLICATE).getConnector(connectorId); + return expectedState.name().equals(entity.getComponent().getState()); + } catch (final Exception e) { + return false; + } + }); + logger.info("Node {} connector {} reached state {}", nodeIndex, connectorId, expectedState); + } + + private void deleteIfExists(final File file) { + if (file.exists()) { + assertTrue(file.delete()); + } + } +} diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ClusteredConnectorIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ClusteredConnectorIT.java new file mode 100644 index 000000000000..b07646dd179b --- /dev/null +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ClusteredConnectorIT.java @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.tests.system.connectors; + +import jakarta.ws.rs.NotFoundException; +import org.apache.nifi.tests.system.NiFiInstanceFactory; +import org.apache.nifi.toolkit.client.ConnectorClient; +import org.apache.nifi.toolkit.client.NiFiClientException; +import org.apache.nifi.web.api.dto.ConnectorConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorValueReferenceDTO; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.NodeEntity; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.fail; + +public class ClusteredConnectorIT extends ConnectorCrudIT { + private static final Logger logger = LoggerFactory.getLogger(ClusteredConnectorIT.class); + + @Override + public NiFiInstanceFactory getInstanceFactory() { + return createTwoNodeInstanceFactory(); + } + + @Test + public void testModifiedWhenDisconnected() throws NiFiClientException, IOException, InterruptedException { + // Create a new connector + final ConnectorEntity connector = getClientUtil().createConnector("NopConnector"); + assertNotNull(connector); + logger.info("Created Connector with ID {}", connector.getId()); + + // Configure the connector with an initial value and apply the update + getClientUtil().configureConnector(connector, "Ignored Step", Map.of("Ignored Property", "Initial Value")); + getClientUtil().applyConnectorUpdate(connector); + logger.info("Configured Connector with initial value and applied the update"); + + // Make requests to each node to ensure that the config has been applied + switchClientToNode(1); + ConnectorEntity node1Connector = getNifiClient().getConnectorClient(DO_NOT_REPLICATE).getConnector(connector.getId()); + assertActiveConfigurationValue(node1Connector, "Ignored Property", "Initial Value"); + assertWorkingConfigurationValue(node1Connector, "Ignored Property", "Initial Value"); + + switchClientToNode(2); + ConnectorEntity node2Connector = getNifiClient().getConnectorClient(DO_NOT_REPLICATE).getConnector(connector.getId()); + assertActiveConfigurationValue(node2Connector, "Ignored Property", "Initial Value"); + assertWorkingConfigurationValue(node2Connector, "Ignored Property", "Initial Value"); + logger.info("Validated initial configuration on both nodes"); + + // Disconnect node 2 + switchClientToNode(1); + disconnectNode(2); + logger.info("Disconnected Node 2"); + + // Make changes on node 1 and apply the change + getClientUtil().configureConnector(connector.getId(), "Ignored Step", Map.of("Ignored Property", "Applied While Disconnected")); + getNifiClient().getConnectorClient().applyUpdate(connector); + logger.info("Applied configuration change on Node 1 while Node 2 is disconnected"); + + // Configure the connector to yet another value on node 1 but do NOT apply the change + getClientUtil().configureConnector(connector.getId(), "Ignored Step", Map.of("Ignored Property", "Working Only Value")); + logger.info("Configured working configuration on Node 1 without applying the change"); + + // Verify node 1 has the expected active and working configurations + node1Connector = getNifiClient().getConnectorClient(DO_NOT_REPLICATE).getConnector(connector.getId()); + assertActiveConfigurationValue(node1Connector, "Ignored Property", "Applied While Disconnected"); + assertWorkingConfigurationValue(node1Connector, "Ignored Property", "Working Only Value"); + logger.info("Validated active and working configuration on Node 1"); + + // Reconnect Node 2 + reconnectNode(2); + waitForAllNodesConnected(); + logger.info("Reconnected Node 2"); + + // Make requests to node 2 to ensure that it properly inherited both the working and active context configuration + switchClientToNode(2); + waitFor(() -> { + try { + final ConnectorEntity latestNode2Connector = getNifiClient().getConnectorClient(DO_NOT_REPLICATE).getConnector(connector.getId()); + final String activeValue = getConfigurationValue(latestNode2Connector.getComponent().getActiveConfiguration(), "Ignored Property"); + final String workingValue = getConfigurationValue(latestNode2Connector.getComponent().getWorkingConfiguration(), "Ignored Property"); + return "Applied While Disconnected".equals(activeValue) && "Working Only Value".equals(workingValue); + } catch (final Exception e) { + return false; + } + }); + logger.info("Validated that Node 2 has received updated configuration after reconnection"); + + final ConnectorEntity finalNode2Connector = getNifiClient().getConnectorClient(DO_NOT_REPLICATE).getConnector(connector.getId()); + assertActiveConfigurationValue(finalNode2Connector, "Ignored Property", "Applied While Disconnected"); + assertWorkingConfigurationValue(finalNode2Connector, "Ignored Property", "Working Only Value"); + logger.info("Validated active and working configuration on Node 2"); + } + + @Test + public void testDeleteConnectorOnConnect() throws NiFiClientException, IOException, InterruptedException { + // Create Connector + final ConnectorEntity connector = getClientUtil().createConnector("DataQueuingConnector"); + assertNotNull(connector); + + // Disconnect node 2 + disconnectNode(2); + + // Should not be able to delete connector + final ConnectorClient connectorClient = getNifiClient().getConnectorClient(); + assertThrows(NiFiClientException.class, () -> connectorClient.deleteConnector(connector)); + + final NodeEntity node2Entity = getNodeEntity(2); + getNifiClient().getControllerClient().deleteNode(node2Entity.getNode().getNodeId()); + + // Should now be able to delete connector + connectorClient.deleteConnector(connector); + + // Should now be able to add node 2 back + getNiFiInstance().getNodeInstance(2).stop(); + getNiFiInstance().getNodeInstance(2).start(true); + waitForAllNodesConnected(); + + switchClientToNode(2); + + // We should get a 404 + try { + getNifiClient().getConnectorClient().getConnector(connector.getId()); + fail("Expected NiFiClientException but it was not thrown"); + } catch (final NiFiClientException e) { + assertInstanceOf(NotFoundException.class, e.getCause()); + } + } + + + @Test + public void testDeleteConnectorOnConnectWithDataQueued() throws NiFiClientException, IOException, InterruptedException { + // Create Connector + final ConnectorEntity connector = getClientUtil().createConnector("DataQueuingConnector"); + assertNotNull(connector); + + getNifiClient().getConnectorClient().startConnector(connector); + + Thread.sleep(1000L); // Wait 1 second to allow some data to queue + + // Disconnect node 2 + disconnectNode(2); + getNiFiInstance().getNodeInstance(2).stop(); + + // Should not be able to delete connector + final ConnectorClient connectorClient = getNifiClient().getConnectorClient(); + assertThrows(NiFiClientException.class, () -> connectorClient.deleteConnector(connector)); + + // Remove node 2 from cluster. + final NodeEntity node2Entity = getNodeEntity(2); + getNifiClient().getControllerClient().deleteNode(node2Entity.getNode().getNodeId()); + + // We cannot delete the connector directly because it has data queued. Stop Node 1, delete the flow.json.gz file, and restart Node 1. + getNiFiInstance().getNodeInstance(1).stop(); + final File node1InstanceDir = getNiFiInstance().getNodeInstance(1).getInstanceDirectory(); + final File node1ConfDir = new File(node1InstanceDir, "conf"); + final File flowJson = new File(node1ConfDir, "flow.json.gz"); + Files.delete(flowJson.toPath()); + + getNiFiInstance().getNodeInstance(1).start(true); + waitForCoordinatorElected(); + + // Should now be able to add node 2 back + getNiFiInstance().getNodeInstance(2).start(true); + waitForAllNodesConnected(); + + switchClientToNode(2); + + // We should get a 404 + try { + getNifiClient().getConnectorClient().getConnector(connector.getId()); + fail("Expected NiFiClientException but it was not thrown"); + } catch (final NiFiClientException e) { + assertInstanceOf(NotFoundException.class, e.getCause()); + } + } + + + private void assertActiveConfigurationValue(final ConnectorEntity connector, final String propertyName, final String expectedValue) { + final String actualValue = getConfigurationValue(connector.getComponent().getActiveConfiguration(), propertyName); + assertEquals(expectedValue, actualValue, "Active configuration property '" + propertyName + "' did not match expected value"); + } + + private void assertWorkingConfigurationValue(final ConnectorEntity connector, final String propertyName, final String expectedValue) { + final String actualValue = getConfigurationValue(connector.getComponent().getWorkingConfiguration(), propertyName); + assertEquals(expectedValue, actualValue, "Working configuration property '" + propertyName + "' did not match expected value"); + } + + private String getConfigurationValue(final ConnectorConfigurationDTO configuration, final String propertyName) { + final Map propertyValues = configuration.getConfigurationStepConfigurations().getFirst() + .getPropertyGroupConfigurations().getFirst().getPropertyValues(); + final ConnectorValueReferenceDTO valueRef = propertyValues.get(propertyName); + return valueRef == null ? null : valueRef.getValue(); + } +} diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorAssetsIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorAssetsIT.java new file mode 100644 index 000000000000..40a2bce775df --- /dev/null +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorAssetsIT.java @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.tests.system.connectors; + +import org.apache.nifi.components.connector.ConnectorState; +import org.apache.nifi.tests.system.NiFiSystemIT; +import org.apache.nifi.toolkit.client.ConnectorClient; +import org.apache.nifi.toolkit.client.NiFiClientException; +import org.apache.nifi.web.api.dto.AssetReferenceDTO; +import org.apache.nifi.web.api.dto.ConfigurationStepConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorValueReferenceDTO; +import org.apache.nifi.web.api.dto.PropertyGroupConfigurationDTO; +import org.apache.nifi.web.api.entity.AssetEntity; +import org.apache.nifi.web.api.entity.AssetsEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class ConnectorAssetsIT extends NiFiSystemIT { + + @Test + public void testCreateConnectorAndUploadAsset() throws NiFiClientException, IOException, InterruptedException { + // Create a Connector instance using the AssetConnector test extension + final ConnectorEntity connector = getClientUtil().createConnector("AssetConnector"); + assertNotNull(connector); + assertNotNull(connector.getId()); + + final String connectorId = connector.getId(); + + // Upload an Asset to the Connector + final File assetFile = new File("src/test/resources/sample-assets/helloworld.txt"); + final ConnectorClient connectorClient = getNifiClient().getConnectorClient(); + final AssetEntity assetEntity = connectorClient.createAsset(connectorId, assetFile.getName(), assetFile); + + assertNotNull(assetEntity); + assertNotNull(assetEntity.getAsset()); + assertNotNull(assetEntity.getAsset().getId()); + assertEquals(assetFile.getName(), assetEntity.getAsset().getName()); + + // List the Connector's Assets and verify that the uploaded Asset is present + final AssetsEntity assetsEntity = connectorClient.getAssets(connectorId); + assertNotNull(assetsEntity); + assertNotNull(assetsEntity.getAssets()); + assertFalse(assetsEntity.getAssets().isEmpty()); + + final String uploadedAssetId = assetEntity.getAsset().getId(); + final boolean assetFound = assetsEntity.getAssets().stream() + .filter(a -> a.getAsset() != null) + .anyMatch(a -> uploadedAssetId.equals(a.getAsset().getId())); + + assertTrue(assetFound); + + // Update the configuration step to reference the uploaded Asset in the Test Asset property + final String configurationStepName = "Asset Configuration"; + final String propertyGroupName = "Asset Configuration"; + final String propertyName = "Test Asset"; + + final ConfigurationStepEntity configurationStepEntity = connectorClient.getConfigurationStep(connectorId, configurationStepName); + assertNotNull(configurationStepEntity); + assertNotNull(configurationStepEntity.getConfigurationStep()); + + final ConfigurationStepConfigurationDTO configurationStepConfiguration = configurationStepEntity.getConfigurationStep(); + assertNotNull(configurationStepConfiguration.getPropertyGroupConfigurations()); + assertFalse(configurationStepConfiguration.getPropertyGroupConfigurations().isEmpty()); + + PropertyGroupConfigurationDTO assetConfigurationGroup = null; + for (final PropertyGroupConfigurationDTO group : configurationStepConfiguration.getPropertyGroupConfigurations()) { + if (propertyGroupName.equals(group.getPropertyGroupName())) { + assetConfigurationGroup = group; + break; + } + } + assertNotNull(assetConfigurationGroup); + + Map propertyValues = assetConfigurationGroup.getPropertyValues(); + if (propertyValues == null) { + propertyValues = new HashMap<>(); + assetConfigurationGroup.setPropertyValues(propertyValues); + } + + final ConnectorValueReferenceDTO assetValueReference = new ConnectorValueReferenceDTO(); + assetValueReference.setValueType("ASSET_REFERENCE"); + + final AssetReferenceDTO assetReferenceDTO = new AssetReferenceDTO(uploadedAssetId); + assetValueReference.setAssetReferences(List.of(assetReferenceDTO)); + + propertyValues.put(propertyName, assetValueReference); + + final ConfigurationStepEntity updatedConfigurationStepEntity = connectorClient.updateConfigurationStep(configurationStepEntity); + assertNotNull(updatedConfigurationStepEntity); + + // Retrieve the Connector configuration and verify that the Test Asset property has the Asset reference set + final ConnectorEntity connectorWithConfiguration = connectorClient.getConnector(connectorId); + assertNotNull(connectorWithConfiguration); + assertNotNull(connectorWithConfiguration.getComponent()); + + final ConnectorConfigurationDTO workingConfiguration = connectorWithConfiguration.getComponent().getWorkingConfiguration(); + assertNotNull(workingConfiguration); + assertNotNull(workingConfiguration.getConfigurationStepConfigurations()); + assertFalse(workingConfiguration.getConfigurationStepConfigurations().isEmpty()); + + ConfigurationStepConfigurationDTO configuredStep = null; + for (final ConfigurationStepConfigurationDTO stepConfiguration : workingConfiguration.getConfigurationStepConfigurations()) { + if (configurationStepName.equals(stepConfiguration.getConfigurationStepName())) { + configuredStep = stepConfiguration; + break; + } + } + assertNotNull(configuredStep); + assertNotNull(configuredStep.getPropertyGroupConfigurations()); + assertFalse(configuredStep.getPropertyGroupConfigurations().isEmpty()); + + PropertyGroupConfigurationDTO configuredAssetGroup = null; + for (final PropertyGroupConfigurationDTO group : configuredStep.getPropertyGroupConfigurations()) { + if (propertyGroupName.equals(group.getPropertyGroupName())) { + configuredAssetGroup = group; + break; + } + } + assertNotNull(configuredAssetGroup); + assertNotNull(configuredAssetGroup.getPropertyValues()); + + final ConnectorValueReferenceDTO configuredAssetValue = configuredAssetGroup.getPropertyValues().get(propertyName); + assertNotNull(configuredAssetValue); + assertEquals("ASSET_REFERENCE", configuredAssetValue.getValueType()); + assertNotNull(configuredAssetValue.getAssetReferences()); + assertFalse(configuredAssetValue.getAssetReferences().isEmpty()); + assertEquals(uploadedAssetId, configuredAssetValue.getAssetReferences().getFirst().getId()); + + // Update the configuration step again to remove the Asset reference + final ConfigurationStepEntity configurationStepEntityWithoutAsset = connectorClient.getConfigurationStep(connectorId, configurationStepName); + assertNotNull(configurationStepEntityWithoutAsset); + assertNotNull(configurationStepEntityWithoutAsset.getConfigurationStep()); + + final ConfigurationStepConfigurationDTO configurationStepConfigurationWithoutAsset = configurationStepEntityWithoutAsset.getConfigurationStep(); + assertNotNull(configurationStepConfigurationWithoutAsset.getPropertyGroupConfigurations()); + + PropertyGroupConfigurationDTO groupWithoutAsset = null; + for (final PropertyGroupConfigurationDTO group : configurationStepConfigurationWithoutAsset.getPropertyGroupConfigurations()) { + if (propertyGroupName.equals(group.getPropertyGroupName())) { + groupWithoutAsset = group; + break; + } + } + assertNotNull(groupWithoutAsset); + + configuredAssetValue.setAssetReferences(null); + final Map propertyValuesWithoutAsset = groupWithoutAsset.getPropertyValues(); + propertyValuesWithoutAsset.put(propertyName, configuredAssetValue); + + final ConfigurationStepEntity configurationStepEntityAfterRemoval = connectorClient.updateConfigurationStep(configurationStepEntityWithoutAsset); + assertNotNull(configurationStepEntityAfterRemoval); + + // Apply the connector update so that unreferenced assets are cleaned up based on the active configuration + final ConnectorEntity connectorBeforeApply = connectorClient.getConnector(connectorId); + assertNotNull(connectorBeforeApply); + connectorBeforeApply.setDisconnectedNodeAcknowledged(true); + + final ConnectorEntity connectorAfterApply = connectorClient.applyUpdate(connectorBeforeApply); + assertNotNull(connectorAfterApply); + + getClientUtil().waitForConnectorState(connectorId, ConnectorState.STOPPED); + + // Verify that the Asset has been removed from the Connector's Assets list + final AssetsEntity assetsAfterRemoval = connectorClient.getAssets(connectorId); + assertNotNull(assetsAfterRemoval); + + final boolean assetStillPresent = assetsAfterRemoval.getAssets() != null && assetsAfterRemoval.getAssets().stream() + .filter(a -> a.getAsset() != null) + .anyMatch(a -> uploadedAssetId.equals(a.getAsset().getId())); + + assertFalse(assetStillPresent); + + // Wait for Connector to stop before attempting to delete it. + getClientUtil().waitForConnectorStopped(connectorAfterApply.getId()); + connectorClient.deleteConnector(connectorAfterApply); + } +} + + diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorCrudIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorCrudIT.java new file mode 100644 index 000000000000..3dec256e24f5 --- /dev/null +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorCrudIT.java @@ -0,0 +1,280 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.tests.system.connectors; + +import jakarta.ws.rs.NotFoundException; +import org.apache.nifi.components.ConfigVerificationResult.Outcome; +import org.apache.nifi.components.connector.BundleCompatibility; +import org.apache.nifi.components.connector.ConnectorState; +import org.apache.nifi.tests.system.NiFiSystemIT; +import org.apache.nifi.toolkit.client.NiFiClientException; +import org.apache.nifi.web.api.dto.BundleDTO; +import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO; +import org.apache.nifi.web.api.dto.ConnectorConfigurationDTO; +import org.apache.nifi.web.api.dto.ConnectorValueReferenceDTO; +import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.api.dto.flow.FlowDTO; +import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ParameterContextsEntity; +import org.apache.nifi.web.api.entity.ParameterProviderEntity; +import org.apache.nifi.web.api.entity.ProcessGroupEntity; +import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity; +import org.apache.nifi.web.api.entity.ProcessorEntity; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +public class ConnectorCrudIT extends NiFiSystemIT { + + @Test + public void testCreateConfigureRestart() throws NiFiClientException, IOException, InterruptedException { + // Create Connector + final ConnectorEntity connector = getClientUtil().createConnector("NopConnector"); + assertNotNull(connector); + + // Ensure that Parameter Context is not created for the Connector + final ParameterContextsEntity contextsEntity = getNifiClient().getParamContextClient().getParamContexts(); + assertEquals(0, contextsEntity.getParameterContexts().size()); + + // Configure the connector and apply the configuration + getClientUtil().configureConnector(connector, "Ignored Step", Map.of("Ignored Property", "Hello, World!")); + getClientUtil().applyConnectorUpdate(connector); + + // Configure with a different value + getClientUtil().configureConnector(connector, "Ignored Step", Map.of("Ignored Property", "Hola, Mundo!")); + + // Restart NiFi and ensure that we have the expected values for both the active and working configurations + getNiFiInstance().stop(); + getNiFiInstance().start(); + + // If running in cluster, wait for all nodes to be connected + if (getNumberOfNodes() > 1) { + waitForAllNodesConnected(); + } + + final ConnectorEntity connectorAfterRestart = getNifiClient().getConnectorClient().getConnector(connector.getId()); + assertNotNull(connectorAfterRestart); + + final String connectorState = connectorAfterRestart.getComponent().getState(); + assertEquals(ConnectorState.STOPPED.name(), connectorState); + + final ConnectorConfigurationDTO activeConfig = connectorAfterRestart.getComponent().getActiveConfiguration(); + final Map activeProperties = activeConfig.getConfigurationStepConfigurations().getFirst().getPropertyGroupConfigurations().getFirst().getPropertyValues(); + final String activeIgnoredProperty = activeProperties.get("Ignored Property").getValue(); + assertEquals("Hello, World!", activeIgnoredProperty); + + final ConnectorConfigurationDTO workingConfig = connectorAfterRestart.getComponent().getWorkingConfiguration(); + final Map workingProperties = workingConfig.getConfigurationStepConfigurations().getFirst().getPropertyGroupConfigurations().getFirst().getPropertyValues(); + final String workingIgnoredProperty = workingProperties.get("Ignored Property").getValue(); + assertEquals("Hola, Mundo!", workingIgnoredProperty); + } + + @Test + public void testConfigVerification() throws NiFiClientException, IOException, InterruptedException { + final ConnectorEntity connector = getClientUtil().createConnector("NopConnector"); + + final List resultDtos = getClientUtil().verifyConnectorStepConfig(connector.getId(), "Ignored Step", + Map.of("Ignored Property", "Test Value")); + assertNotNull(resultDtos); + assertEquals(2, resultDtos.size()); + + assertTrue(resultDtos.stream().allMatch(result -> Outcome.SUCCESSFUL.name().equals(result.getOutcome()))); + assertEquals("Property Validation", resultDtos.getFirst().getVerificationStepName()); + + final ConfigVerificationResultDTO resultDto = resultDtos.get(1); + assertEquals("Nop Verification", resultDto.getVerificationStepName()); + assertTrue(resultDto.getExplanation().contains("Test Value")); + } + + @Test + public void testSecretReferences() throws NiFiClientException, IOException, InterruptedException { + // Create and configure a Parameter Provider with two secrets + final ParameterProviderEntity paramProvider = getClientUtil().createParameterProvider("PropertiesParameterProvider"); + getClientUtil().updateParameterProviderProperties(paramProvider, Map.of("parameters", "supersecret=supersecret\nother=other")); + + // Create the Nop Connector + final ConnectorEntity connector = getClientUtil().createConnector("NopConnector"); + assertNotNull(connector); + + // Verify that using a String Literal for a SECRET property should fail validation + final Map stringLiteralProperties = Map.of("Secret Property", "supersecret"); + List verificationResults = getClientUtil().verifyConnectorStepConfig(connector.getId(), "Ignored Step", stringLiteralProperties); + assertTrue(verificationResults.stream().anyMatch(result -> Outcome.FAILED.name().equals(result.getOutcome()))); + + // Configure and apply the String Literal value, then wait for invalid + getClientUtil().configureConnector(connector, "Ignored Step", stringLiteralProperties); + getClientUtil().applyConnectorUpdate(connector); + getClientUtil().waitForInvalidConnector(connector.getId()); + + // Verify that using a Secret Reference to 'other' (value "other") should fail validation + final ConnectorValueReferenceDTO otherSecretRef = getClientUtil().createSecretValueReference(paramProvider.getId(), "other", "PropertiesParameterProvider.Parameters.other"); + final Map otherSecretProperties = Map.of("Secret Property", otherSecretRef); + verificationResults = getClientUtil().verifyConnectorStepConfigWithReferences(connector.getId(), "Ignored Step", otherSecretProperties); + assertTrue(verificationResults.stream().anyMatch(result -> Outcome.FAILED.name().equals(result.getOutcome()))); + + // Configure and apply the 'other' secret reference, then wait for invalid + getClientUtil().configureConnectorWithReferences(connector.getId(), "Ignored Step", otherSecretProperties); + getClientUtil().applyConnectorUpdate(connector); + getClientUtil().waitForInvalidConnector(connector.getId()); + + // Verify that using an invalid reference should fail validation + final ConnectorValueReferenceDTO invalidRef = getClientUtil().createSecretValueReference(paramProvider.getId(), "nonexistent", "PropertiesParameterProvider.Parameters.nonexistent"); + final Map invalidSecretProperties = Map.of("Secret Property", invalidRef); + verificationResults = getClientUtil().verifyConnectorStepConfigWithReferences(connector.getId(), "Ignored Step", invalidSecretProperties); + assertTrue(verificationResults.stream().anyMatch(result -> Outcome.FAILED.name().equals(result.getOutcome()))); + + // Configure and apply the invalid secret reference, then wait for invalid + getClientUtil().configureConnectorWithReferences(connector.getId(), "Ignored Step", invalidSecretProperties); + getClientUtil().applyConnectorUpdate(connector); + getClientUtil().waitForInvalidConnector(connector.getId()); + + // Verify that using a Secret Reference to 'supersecret' (value "supersecret") should pass validation + final ConnectorValueReferenceDTO supersecretRef = getClientUtil().createSecretValueReference(paramProvider.getId(), "supersecret", "PropertiesParameterProvider.Parameters.supersecret"); + final Map supersecretProperties = Map.of("Secret Property", supersecretRef); + verificationResults = getClientUtil().verifyConnectorStepConfigWithReferences(connector.getId(), "Ignored Step", supersecretProperties); + assertTrue(verificationResults.stream().allMatch(result -> Outcome.SUCCESSFUL.name().equals(result.getOutcome()))); + + // Configure and apply the 'supersecret' secret reference, then wait for valid + getClientUtil().configureConnectorWithReferences(connector.getId(), "Ignored Step", supersecretProperties); + getClientUtil().applyConnectorUpdate(connector); + getClientUtil().waitForValidConnector(connector.getId()); + } + + @Test + public void testDeleteConnectorNoDataQueued() throws NiFiClientException, IOException { + // Create Connector + final ConnectorEntity connector = getClientUtil().createConnector("DataQueuingConnector"); + assertNotNull(connector); + + // Delete + getNifiClient().getConnectorClient().deleteConnector(connector); + } + + @Test + public void testProcessGroupAccessibility() throws NiFiClientException, IOException { + final ConnectorEntity connector = getClientUtil().createConnector("NestedProcessGroupConnector"); + assertNotNull(connector); + + final ConnectorEntity updatedConnector = getNifiClient().getConnectorClient().getConnector(connector.getId()); + final String managedProcessGroupId = updatedConnector.getComponent().getManagedProcessGroupId(); + assertNotNull(managedProcessGroupId); + + final ProcessGroupFlowEntity connectorFlowEntity = getNifiClient().getConnectorClient().getFlow(connector.getId(), managedProcessGroupId); + assertNotNull(connectorFlowEntity); + + final ProcessGroupFlowDTO connectorFlow = connectorFlowEntity.getProcessGroupFlow(); + assertNotNull(connectorFlow); + assertEquals(managedProcessGroupId, connectorFlow.getId()); + + try { + getNifiClient().getFlowClient().getProcessGroup(managedProcessGroupId); + fail("Was able to retrieve connector-managed process group via FlowClient"); + } catch (final NiFiClientException e) { + assertInstanceOf(NotFoundException.class, e.getCause()); + } + + final Set childGroups = connectorFlow.getFlow().getProcessGroups(); + assertEquals(1, childGroups.size(), "Expected exactly one child process group"); + + final ProcessGroupEntity childGroup = childGroups.iterator().next(); + final String childGroupId = childGroup.getId(); + assertNotNull(childGroupId); + + final ProcessGroupFlowEntity childFlowEntity = getNifiClient().getConnectorClient().getFlow(connector.getId(), childGroupId); + assertNotNull(childFlowEntity); + assertEquals(childGroupId, childFlowEntity.getProcessGroupFlow().getId()); + + try { + getNifiClient().getFlowClient().getProcessGroup(childGroupId); + fail("Was able to retrieve child process group of connector-managed flow via FlowClient"); + } catch (final NiFiClientException e) { + assertInstanceOf(NotFoundException.class, e.getCause()); + } + } + + @Test + public void testBundleResolutionRequireExactBundle() throws NiFiClientException, IOException, InterruptedException { + final ConnectorAndProcessor connectorAndProcessor = createBundleResolutionConnector(BundleCompatibility.REQUIRE_EXACT_BUNDLE); + final ProcessorDTO processor = connectorAndProcessor.processor(); + assertTrue(processor.getExtensionMissing()); + + final BundleDTO bundle = processor.getBundle(); + assertEquals("0.0.0-NONEXISTENT", bundle.getVersion()); + assertEquals(Boolean.TRUE, processor.getExtensionMissing()); + + final ConnectorEntity connector = connectorAndProcessor.connector(); + assertEquals("INVALID", connector.getComponent().getValidationStatus()); + final Collection validationErrors = connector.getComponent().getValidationErrors(); + assertNotNull(validationErrors); + assertEquals(1, validationErrors.size()); + final String validationError = validationErrors.iterator().next(); + assertTrue(validationError.contains("missing")); + } + + @Test + public void testBundleResolutionResolveBundle() throws NiFiClientException, IOException, InterruptedException { + final ProcessorDTO processor = createBundleResolutionConnector(BundleCompatibility.RESOLVE_BUNDLE).processor(); + assertFalse(processor.getExtensionMissing()); + + final BundleDTO bundle = processor.getBundle(); + assertNotEquals("0.0.0-NONEXISTENT", bundle.getVersion()); + } + + @Test + public void testBundleResolutionResolveNewestBundle() throws NiFiClientException, IOException, InterruptedException { + final ProcessorDTO processor = createBundleResolutionConnector(BundleCompatibility.RESOLVE_NEWEST_BUNDLE).processor(); + assertFalse(processor.getExtensionMissing()); + + final BundleDTO bundle = processor.getBundle(); + assertNotEquals("0.0.0-NONEXISTENT", bundle.getVersion()); + } + + private ConnectorAndProcessor createBundleResolutionConnector(final BundleCompatibility bundleCompatability) throws NiFiClientException, IOException, InterruptedException { + final ConnectorEntity connector = getClientUtil().createConnector("BundleResolutionConnector"); + assertNotNull(connector); + + getClientUtil().configureConnector(connector, "Bundle Resolution", Map.of("Bundle Compatability", bundleCompatability.name())); + getClientUtil().applyConnectorUpdate(connector); + + final ConnectorEntity updatedConnector = getNifiClient().getConnectorClient().getConnector(connector.getId()); + final ProcessGroupFlowEntity flowEntity = getNifiClient().getConnectorClient().getFlow(connector.getId()); + final FlowDTO flowDto = flowEntity.getProcessGroupFlow().getFlow(); + final Set processors = flowDto.getProcessors(); + + assertEquals(1, processors.size()); + + final ProcessorDTO processor = processors.iterator().next().getComponent(); + return new ConnectorAndProcessor(updatedConnector, processor); + } + + private record ConnectorAndProcessor(ConnectorEntity connector, ProcessorDTO processor) { + } +} diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorDrainIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorDrainIT.java new file mode 100644 index 000000000000..9f0321e87352 --- /dev/null +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorDrainIT.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.tests.system.connectors; + +import org.apache.nifi.components.connector.ConnectorState; +import org.apache.nifi.tests.system.NiFiSystemIT; +import org.apache.nifi.toolkit.client.NiFiClientException; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class ConnectorDrainIT extends NiFiSystemIT { + + private static final Logger logger = LoggerFactory.getLogger(ConnectorDrainIT.class); + + @Test + public void testDrainFlowFiles() throws NiFiClientException, IOException, InterruptedException { + final File gateFile = new File(getNiFiInstance().getInstanceDirectory(), "gate-file.txt"); + gateFile.deleteOnExit(); + + if (gateFile.exists()) { + assertTrue(gateFile.delete()); + } + + logger.info("Creating GatedDataQueuingConnector"); + final ConnectorEntity connector = getClientUtil().createConnector("GatedDataQueuingConnector"); + assertNotNull(connector); + final String connectorId = connector.getId(); + + logger.info("Configuring connector {} with gate file path: {}", connectorId, gateFile.getAbsolutePath()); + getClientUtil().configureConnector(connector, "Gate Configuration", Map.of("Gate File Path", gateFile.getAbsolutePath())); + getClientUtil().applyConnectorUpdate(connector); + + logger.info("Starting connector {}", connectorId); + getClientUtil().startConnector(connectorId); + + logger.info("Waiting for at least 5 FlowFiles to queue"); + waitForConnectorMinQueueCount(connectorId, 5); + + logger.info("Stopping connector {}", connectorId); + getClientUtil().stopConnector(connectorId); + getClientUtil().waitForConnectorStopped(connectorId); + + final int queuedCountBeforeDrain = getConnectorQueuedFlowFileCount(connectorId); + logger.info("Queued FlowFile count before drain: {}", queuedCountBeforeDrain); + assertTrue(queuedCountBeforeDrain > 0); + + logger.info("Initiating drain for connector {}", connectorId); + getClientUtil().drainConnector(connectorId); + + logger.info("Waiting for connector to enter DRAINING state"); + getClientUtil().waitForConnectorDraining(connectorId); + + logger.info("Sleeping for 2 seconds to verify connector remains in DRAINING state"); + Thread.sleep(2000L); + + ConnectorEntity drainingConnector = getNifiClient().getConnectorClient().getConnector(connectorId); + logger.info("Connector state after 2 seconds: {}", drainingConnector.getComponent().getState()); + assertEquals(ConnectorState.DRAINING.name(), drainingConnector.getComponent().getState()); + + final int queuedWhileDraining = getConnectorQueuedFlowFileCount(connectorId); + logger.info("Queued FlowFile count while draining (gate file absent): {}", queuedWhileDraining); + assertTrue(queuedWhileDraining > 0); + + logger.info("Creating gate file to allow draining to complete: {}", gateFile.getAbsolutePath()); + assertTrue(gateFile.createNewFile()); + + logger.info("Waiting for connector to complete draining and return to STOPPED state"); + waitFor(() -> { + try { + final ConnectorEntity entity = getNifiClient().getConnectorClient().getConnector(connectorId); + return ConnectorState.STOPPED.name().equals(entity.getComponent().getState()); + } catch (final Exception e) { + return false; + } + }); + + final ConnectorEntity finalConnector = getNifiClient().getConnectorClient().getConnector(connectorId); + logger.info("Final connector state: {}", finalConnector.getComponent().getState()); + assertEquals(ConnectorState.STOPPED.name(), finalConnector.getComponent().getState()); + + final int finalQueuedCount = getConnectorQueuedFlowFileCount(connectorId); + logger.info("Final queued FlowFile count: {}", finalQueuedCount); + assertEquals(0, finalQueuedCount); + + logger.info("testDrainFlowFiles completed successfully"); + } + + @Test + public void testCancelDrainFlowFiles() throws NiFiClientException, IOException, InterruptedException { + final File gateFile = new File(getNiFiInstance().getInstanceDirectory(), "gate-file-cancel.txt"); + gateFile.deleteOnExit(); + + if (gateFile.exists()) { + assertTrue(gateFile.delete()); + } + + logger.info("Creating GatedDataQueuingConnector"); + final ConnectorEntity connector = getClientUtil().createConnector("GatedDataQueuingConnector"); + assertNotNull(connector); + final String connectorId = connector.getId(); + + logger.info("Configuring connector {} with gate file path: {}", connectorId, gateFile.getAbsolutePath()); + getClientUtil().configureConnector(connector, "Gate Configuration", Map.of("Gate File Path", gateFile.getAbsolutePath())); + getClientUtil().applyConnectorUpdate(connector); + + logger.info("Starting connector {}", connectorId); + getClientUtil().startConnector(connectorId); + + logger.info("Waiting for at least 5 FlowFiles to queue"); + waitForConnectorMinQueueCount(connectorId, 5); + + logger.info("Stopping connector {}", connectorId); + getClientUtil().stopConnector(connectorId); + getClientUtil().waitForConnectorStopped(connectorId); + + final int queuedCountBeforeDrain = getConnectorQueuedFlowFileCount(connectorId); + logger.info("Queued FlowFile count before drain: {}", queuedCountBeforeDrain); + assertTrue(queuedCountBeforeDrain > 0); + + logger.info("Initiating drain for connector {}", connectorId); + getClientUtil().drainConnector(connectorId); + + logger.info("Waiting for connector to enter DRAINING state"); + getClientUtil().waitForConnectorDraining(connectorId); + + logger.info("Sleeping for 2 seconds to verify connector remains in DRAINING state"); + Thread.sleep(2000L); + + final ConnectorEntity drainingConnector = getNifiClient().getConnectorClient().getConnector(connectorId); + logger.info("Connector state after 2 seconds: {}", drainingConnector.getComponent().getState()); + assertEquals(ConnectorState.DRAINING.name(), drainingConnector.getComponent().getState()); + + final int queuedCountWhileDraining = getConnectorQueuedFlowFileCount(connectorId); + logger.info("Queued FlowFile count while draining: {}", queuedCountWhileDraining); + assertTrue(queuedCountWhileDraining > 0); + + logger.info("Canceling drain for connector {}", connectorId); + getClientUtil().cancelDrain(connectorId); + + logger.info("Waiting for connector to return to STOPPED state after cancel"); + getClientUtil().waitForConnectorStopped(connectorId); + + final ConnectorEntity finalConnector = getNifiClient().getConnectorClient().getConnector(connectorId); + logger.info("Final connector state: {}", finalConnector.getComponent().getState()); + assertEquals(ConnectorState.STOPPED.name(), finalConnector.getComponent().getState()); + + final int queuedCountAfterCancel = getConnectorQueuedFlowFileCount(connectorId); + logger.info("Queued FlowFile count after cancel (should still have data): {}", queuedCountAfterCancel); + assertTrue(queuedCountAfterCancel > 0); + + logger.info("testCancelDrainFlowFiles completed successfully"); + } +} diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorLifecycleIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorLifecycleIT.java new file mode 100644 index 000000000000..dc1a50de1af3 --- /dev/null +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorLifecycleIT.java @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.tests.system.connectors; + +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.groups.StatelessGroupScheduledState; +import org.apache.nifi.tests.system.NiFiSystemIT; +import org.apache.nifi.toolkit.client.NiFiClientException; +import org.apache.nifi.web.api.dto.PortDTO; +import org.apache.nifi.web.api.dto.ProcessGroupDTO; +import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.api.dto.flow.FlowDTO; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ParameterContextEntity; +import org.apache.nifi.web.api.entity.ParameterContextsEntity; +import org.apache.nifi.web.api.entity.PortEntity; +import org.apache.nifi.web.api.entity.ProcessGroupEntity; +import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity; +import org.apache.nifi.web.api.entity.ProcessorEntity; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.Set; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * System test that verifies the connector start/stop lifecycle correctly starts and stops + * all component types: processors, ports, and stateless groups. + */ +public class ConnectorLifecycleIT extends NiFiSystemIT { + + private static final Logger logger = LoggerFactory.getLogger(ConnectorLifecycleIT.class); + + + @Test + public void testStartStopStartsAndStopsAllComponents() throws NiFiClientException, IOException, InterruptedException { + logger.info("Creating ComponentLifecycleConnector"); + final ConnectorEntity connector = getClientUtil().createConnector("ComponentLifecycleConnector"); + assertNotNull(connector); + final String connectorId = connector.getId(); + + logger.info("Applying connector configuration"); + getClientUtil().applyConnectorUpdate(connector); + + logger.info("Waiting for connector to be valid"); + getClientUtil().waitForValidConnector(connectorId); + + logger.info("Starting connector {}", connectorId); + getClientUtil().startConnector(connectorId); + + // Ensure that there are no Parameter Contexts defined. When we start a flow that has a Stateless Group, + // we synchronize the Process Group with the Versioned External Flow, and we want to ensure that this does + // not register a Parameter Context + final ParameterContextsEntity contextsEntity = getNifiClient().getParamContextClient().getParamContexts(); + final Set parameterContexts = contextsEntity.getParameterContexts(); + assertNotNull(parameterContexts); + assertEquals(Collections.emptySet(), parameterContexts); + + logger.info("Verifying flow has components after start"); + verifyFlowHasComponents(connectorId); + + logger.info("Verifying all processors are running"); + waitForAllProcessorsRunning(connectorId); + + logger.info("Verifying all ports are running"); + waitForAllPortsRunning(connectorId); + + logger.info("Verifying stateless group is running"); + waitForStatelessGroupRunning(connectorId); + + logger.info("Stopping connector {}", connectorId); + getClientUtil().stopConnector(connectorId); + getClientUtil().waitForConnectorStopped(connectorId); + + logger.info("Verifying all processors are stopped"); + waitForAllProcessorsStopped(connectorId); + + logger.info("Verifying all ports are stopped"); + waitForAllPortsStopped(connectorId); + + logger.info("Verifying stateless group is stopped"); + waitForStatelessGroupStopped(connectorId); + + logger.info("testStartStopStartsAndStopsAllComponents completed successfully"); + } + + private void verifyFlowHasComponents(final String connectorId) throws NiFiClientException, IOException { + final ProcessGroupFlowEntity flowEntity = getNifiClient().getConnectorClient().getFlow(connectorId); + final FlowDTO flowDto = flowEntity.getProcessGroupFlow().getFlow(); + + boolean hasProcessors = !flowDto.getProcessors().isEmpty(); + boolean hasChildGroups = !flowDto.getProcessGroups().isEmpty(); + + logger.info("Root group has {} processors and {} child groups", + flowDto.getProcessors().size(), flowDto.getProcessGroups().size()); + + assertTrue(hasProcessors || hasChildGroups, "Flow should have processors or child groups"); + } + + private void waitForAllProcessorsRunning(final String connectorId) throws InterruptedException { + waitFor(() -> allProcessorsInState(connectorId, null, ScheduledState.RUNNING.name())); + } + + private void waitForAllProcessorsStopped(final String connectorId) throws InterruptedException { + waitFor(() -> allProcessorsInState(connectorId, null, ScheduledState.STOPPED.name())); + } + + private boolean allProcessorsInState(final String connectorId, final String groupId, final String expectedState) throws NiFiClientException, IOException { + final FlowDTO flowDto = getFlow(connectorId, groupId); + + for (final ProcessorEntity processorEntity : flowDto.getProcessors()) { + final ProcessorDTO processorDto = processorEntity.getComponent(); + final String state = processorDto.getState(); + if (!expectedState.equals(state) && !ScheduledState.DISABLED.name().equals(state)) { + logger.debug("Processor {} ({}) in group {} is in state {} (expected {})", + processorDto.getName(), processorEntity.getId(), groupId, state, expectedState); + return false; + } + } + + for (final ProcessGroupEntity childGroupEntity : flowDto.getProcessGroups()) { + final ProcessGroupDTO childGroupDto = childGroupEntity.getComponent(); + if (!"STATELESS".equals(childGroupDto.getExecutionEngine())) { + if (!allProcessorsInState(connectorId, childGroupEntity.getId(), expectedState)) { + return false; + } + } + } + + return true; + } + + private void waitForAllPortsRunning(final String connectorId) throws InterruptedException { + waitFor(() -> allPortsInState(connectorId, null, ScheduledState.RUNNING.name())); + } + + private void waitForAllPortsStopped(final String connectorId) throws InterruptedException { + waitFor(() -> allPortsInState(connectorId, null, ScheduledState.STOPPED.name())); + } + + private boolean allPortsInState(final String connectorId, final String groupId, final String expectedState) throws NiFiClientException, IOException { + final FlowDTO flowDto = getFlow(connectorId, groupId); + + for (final PortEntity portEntity : flowDto.getInputPorts()) { + final PortDTO portDto = portEntity.getComponent(); + final String state = portDto.getState(); + if (!expectedState.equals(state) && !ScheduledState.DISABLED.name().equals(state)) { + logger.debug("Input port {} is in state {} (expected {})", portDto.getName(), state, expectedState); + return false; + } + } + + for (final PortEntity portEntity : flowDto.getOutputPorts()) { + final PortDTO portDto = portEntity.getComponent(); + final String state = portDto.getState(); + if (!expectedState.equals(state) && !ScheduledState.DISABLED.name().equals(state)) { + logger.debug("Output port {} is in state {} (expected {})", portDto.getName(), state, expectedState); + return false; + } + } + + for (final ProcessGroupEntity childGroupEntity : flowDto.getProcessGroups()) { + final ProcessGroupDTO childGroupDto = childGroupEntity.getComponent(); + if (!"STATELESS".equals(childGroupDto.getExecutionEngine())) { + if (!allPortsInState(connectorId, childGroupEntity.getId(), expectedState)) { + return false; + } + } + } + + return true; + } + + private void waitForStatelessGroupRunning(final String connectorId) throws InterruptedException { + waitFor(() -> isStatelessGroupInState(connectorId, StatelessGroupScheduledState.RUNNING.name())); + } + + private void waitForStatelessGroupStopped(final String connectorId) throws InterruptedException { + waitFor(() -> isStatelessGroupInState(connectorId, StatelessGroupScheduledState.STOPPED.name())); + } + + private boolean isStatelessGroupInState(final String connectorId, final String expectedState) throws NiFiClientException, IOException { + return findStatelessGroupInState(connectorId, null, expectedState); + } + + private boolean findStatelessGroupInState(final String connectorId, final String groupId, final String expectedState) throws NiFiClientException, IOException { + final FlowDTO flowDto = getFlow(connectorId, groupId); + + for (final ProcessGroupEntity childGroupEntity : flowDto.getProcessGroups()) { + final ProcessGroupDTO childGroupDto = childGroupEntity.getComponent(); + if ("STATELESS".equals(childGroupDto.getExecutionEngine())) { + final String actualState = childGroupDto.getStatelessGroupScheduledState(); + logger.debug("Stateless group {} is in state {} (expected {})", childGroupDto.getName(), actualState, expectedState); + if (expectedState.equals(actualState)) { + return true; + } + } else { + if (findStatelessGroupInState(connectorId, childGroupEntity.getId(), expectedState)) { + return true; + } + } + } + + return false; + } + + private FlowDTO getFlow(final String connectorId, final String groupId) throws NiFiClientException, IOException { + final ProcessGroupFlowEntity flowEntity = (groupId == null) + ? getNifiClient().getConnectorClient().getFlow(connectorId) + : getNifiClient().getConnectorClient().getFlow(connectorId, groupId); + + return flowEntity.getProcessGroupFlow().getFlow(); + } +} diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorMethodMarshallingIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorMethodMarshallingIT.java new file mode 100644 index 000000000000..e57bf852d28d --- /dev/null +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorMethodMarshallingIT.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.tests.system.connectors; + +import org.apache.nifi.tests.system.NiFiSystemIT; +import org.apache.nifi.toolkit.client.NiFiClientException; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * System tests that verify JSON marshalling of complex objects across ClassLoader boundaries + * when invoking ConnectorMethods on Processors. + */ +public class ConnectorMethodMarshallingIT extends NiFiSystemIT { + + @Test + public void testComplexObjectMarshalling() throws NiFiClientException, IOException, InterruptedException { + final File outputFile = new File("target/calculate-result.txt"); + if (outputFile.exists()) { + assertTrue(outputFile.delete(), "Failed to delete existing output file"); + } + + final ConnectorEntity connector = getClientUtil().createConnector("CalculateConnector"); + assertNotNull(connector); + + getClientUtil().configureConnector(connector, "Calculation", Map.of( + "Operand 1", "10", + "Operand 2", "5", + "Operation", "ADD", + "Output File", outputFile.getAbsolutePath() + )); + + getClientUtil().applyConnectorUpdate(connector); + getClientUtil().waitForValidConnector(connector.getId()); + + assertTrue(outputFile.exists(), "Output file was not created"); + final String fileContents = Files.readString(outputFile.toPath(), StandardCharsets.UTF_8); + final String[] lines = fileContents.split("\n"); + assertEquals(4, lines.length, "Output file should contain 4 lines"); + assertEquals("10", lines[0], "First line should be operand1"); + assertEquals("5", lines[1], "Second line should be operand2"); + assertEquals("ADD", lines[2], "Third line should be the operation"); + assertEquals("15", lines[3], "Fourth line should be the result (10 + 5 = 15)"); + } +} + diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorParameterContextIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorParameterContextIT.java new file mode 100644 index 000000000000..40b46eeacb0d --- /dev/null +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorParameterContextIT.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.tests.system.connectors; + +import org.apache.nifi.tests.system.NiFiSystemIT; +import org.apache.nifi.toolkit.client.ConnectorClient; +import org.apache.nifi.toolkit.client.NiFiClientException; +import org.apache.nifi.web.api.dto.AssetReferenceDTO; +import org.apache.nifi.web.api.dto.ConnectorValueReferenceDTO; +import org.apache.nifi.web.api.entity.AssetEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ParameterProviderEntity; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * System test that validates Connectors can use Parameter Contexts with inheritance, + * sensitive parameters (via SECRET_REFERENCE), and asset-referencing parameters. + */ +public class ConnectorParameterContextIT extends NiFiSystemIT { + private static final Logger LOGGER = LoggerFactory.getLogger(ConnectorParameterContextIT.class); + private static final String SENSITIVE_SECRET_VALUE = "my-super-secret-value"; + private static final String ASSET_FILE_CONTENT = "Hello, World!"; + + @Test + public void testParameterContextInheritanceWithSensitiveAndAssetParameters() throws NiFiClientException, IOException, InterruptedException { + final File sensitiveOutputFile = new File("target/sensitive.txt"); + final File assetOutputFile = new File("target/asset.txt"); + + sensitiveOutputFile.delete(); + assetOutputFile.delete(); + + final ParameterProviderEntity paramProvider = getClientUtil().createParameterProvider("PropertiesParameterProvider"); + getClientUtil().updateParameterProviderProperties(paramProvider, Map.of("parameters", "secret=" + SENSITIVE_SECRET_VALUE)); + + final ConnectorEntity connector = getClientUtil().createConnector("ParameterContextConnector"); + assertNotNull(connector); + assertNotNull(connector.getId()); + + final String connectorId = connector.getId(); + final ConnectorClient connectorClient = getNifiClient().getConnectorClient(); + + final File assetFile = new File("src/test/resources/sample-assets/helloworld.txt"); + final AssetEntity assetEntity = connectorClient.createAsset(connectorId, assetFile.getName(), assetFile); + assertNotNull(assetEntity); + assertNotNull(assetEntity.getAsset()); + assertNotNull(assetEntity.getAsset().getId()); + + final String uploadedAssetId = assetEntity.getAsset().getId(); + + final ConnectorValueReferenceDTO secretRef = getClientUtil().createSecretValueReference( + paramProvider.getId(), "secret", "PropertiesParameterProvider.Parameters.secret"); + + final ConnectorValueReferenceDTO assetRef = new ConnectorValueReferenceDTO(); + assetRef.setValueType("ASSET_REFERENCE"); + assetRef.setAssetReferences(List.of(new AssetReferenceDTO(uploadedAssetId))); + + final Map propertyValues = new HashMap<>(); + propertyValues.put("Sensitive Value", secretRef); + propertyValues.put("Asset File", assetRef); + propertyValues.put("Sensitive Output File", createStringLiteralRef(sensitiveOutputFile.getAbsolutePath())); + propertyValues.put("Asset Output File", createStringLiteralRef(assetOutputFile.getAbsolutePath())); + + getClientUtil().configureConnectorWithReferences(connectorId, "Parameter Context Configuration", propertyValues); + LOGGER.info("Applying configuration to Connector..."); + getClientUtil().applyConnectorUpdate(connector); + LOGGER.info("Waiting for Connector to become valid..."); + getClientUtil().waitForValidConnector(connectorId); + + LOGGER.info("Connector is valid; starting Connector..."); + getClientUtil().startConnector(connectorId); + + LOGGER.info("Waiting for output files to be created..."); + waitFor(() -> sensitiveOutputFile.exists() && assetOutputFile.exists()); + + assertTrue(sensitiveOutputFile.exists(), "Sensitive output file should exist"); + assertTrue(assetOutputFile.exists(), "Asset output file should exist"); + + final String sensitiveContent = Files.readString(sensitiveOutputFile.toPath()).trim(); + final String assetContent = Files.readString(assetOutputFile.toPath()).trim(); + + assertEquals(SENSITIVE_SECRET_VALUE, sensitiveContent, "Sensitive output file should contain the secret value"); + assertEquals(ASSET_FILE_CONTENT, assetContent, "Asset output file should contain the asset file contents"); + } + + private ConnectorValueReferenceDTO createStringLiteralRef(final String value) { + final ConnectorValueReferenceDTO ref = new ConnectorValueReferenceDTO(); + ref.setValueType("STRING_LITERAL"); + ref.setValue(value); + return ref; + } +} diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorVersionResolutionIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorVersionResolutionIT.java new file mode 100644 index 000000000000..2c10e8093d6e --- /dev/null +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorVersionResolutionIT.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.tests.system.connectors; + +import org.apache.nifi.stream.io.GZIPOutputStream; +import org.apache.nifi.tests.system.InstanceConfiguration; +import org.apache.nifi.tests.system.NiFiInstanceFactory; +import org.apache.nifi.tests.system.NiFiSystemIT; +import org.apache.nifi.tests.system.SpawnedStandaloneNiFiInstanceFactory; +import org.apache.nifi.toolkit.client.NiFiClientException; +import org.apache.nifi.web.api.dto.BundleDTO; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UncheckedIOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +/** + * System test that verifies that when a flow.json.gz contains a Connector whose + * bundle version does not exist, but only one version of that Connector type is + * available, the Connector is created with the correct/available version. + */ +class ConnectorVersionResolutionIT extends NiFiSystemIT { + private static final String CONNECTOR_ID = "11111111-1111-1111-1111-111111111111"; + private static final String NONEXISTENT_VERSION = "1.0.0-nonexistent"; + + @Override + public NiFiInstanceFactory getInstanceFactory() { + final Path flowJsonInputPath = Paths.get("src/test/resources/flows/connector-version-mismatch/flow.json"); + final Path flowJsonOutputPath = Paths.get("target/connector-version-mismatch-flow.json.gz").toAbsolutePath(); + + try (final InputStream inputStream = Files.newInputStream(flowJsonInputPath); + final OutputStream outputStream = new GZIPOutputStream(Files.newOutputStream(flowJsonOutputPath))) { + inputStream.transferTo(outputStream); + } catch (final IOException e) { + throw new UncheckedIOException("Failed to compress Flow Configuration [%s]".formatted(flowJsonInputPath), e); + } + + return new SpawnedStandaloneNiFiInstanceFactory( + new InstanceConfiguration.Builder() + .bootstrapConfig("src/test/resources/conf/default/bootstrap.conf") + .instanceDirectory("target/standalone-instance") + .flowJson(flowJsonOutputPath.toFile()) + .overrideNifiProperties(getNifiPropertiesOverrides()) + .build() + ); + } + + @Override + protected boolean isAllowFactoryReuse() { + return false; + } + + @Override + protected boolean isDestroyEnvironmentAfterEachTest() { + return true; + } + + @Test + void testConnectorCreatedWithCorrectVersionWhenOnlyOneVersionExists() throws NiFiClientException, IOException { + final ConnectorEntity connector = getNifiClient().getConnectorClient().getConnector(CONNECTOR_ID); + assertNotNull(connector); + assertNotNull(connector.getComponent()); + + final BundleDTO bundle = connector.getComponent().getBundle(); + assertNotNull(bundle, "Connector bundle should not be null"); + assertEquals("org.apache.nifi", bundle.getGroup()); + assertEquals("nifi-system-test-extensions-nar", bundle.getArtifact()); + + assertNotEquals(NONEXISTENT_VERSION, bundle.getVersion(), "Connector should not have the nonexistent version from flow.json"); + assertEquals(getNiFiVersion(), bundle.getVersion(), "Connector should be created with the current NiFi version since only one version exists"); + } +} diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/SelectiveDropConnectorIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/SelectiveDropConnectorIT.java new file mode 100644 index 000000000000..7bdff5b6aa99 --- /dev/null +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/SelectiveDropConnectorIT.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.tests.system.connectors; + +import org.apache.nifi.tests.system.InstanceConfiguration; +import org.apache.nifi.tests.system.NiFiInstanceFactory; +import org.apache.nifi.tests.system.NiFiSystemIT; +import org.apache.nifi.tests.system.SpawnedStandaloneNiFiInstanceFactory; +import org.apache.nifi.toolkit.client.NiFiClientException; +import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO; +import org.apache.nifi.web.api.entity.ConnectionEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.Map; +import java.util.Set; +import java.util.function.Predicate; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +/** + * System test for the SelectiveDropConnector which tests the ability to selectively drop FlowFiles + * based on a predicate when a Connector is stopped. + */ +public class SelectiveDropConnectorIT extends NiFiSystemIT { + private static final Logger logger = LoggerFactory.getLogger(SelectiveDropConnectorIT.class); + + @Override + public NiFiInstanceFactory getInstanceFactory() { + // Ensure that each FlowFile is written to its own content claim by setting max appendable size to 1 byte + // and set the swap threshold to 2,000 FlowFiles to allow for multiple swap files to be created during the test + final Map nifiPropertiesOverrides = Map.of( + "nifi.content.claim.max.appendable.size", "1 B", + "nifi.queue.swap.threshold", "2000" + ); + + return new SpawnedStandaloneNiFiInstanceFactory( + new InstanceConfiguration.Builder() + .bootstrapConfig("src/test/resources/conf/default/bootstrap.conf") + .instanceDirectory("target/standalone-instance") + .overrideNifiProperties(nifiPropertiesOverrides) + .build()); + } + + // Ensure environment is destroyed after each test to reset nifi.properties overrides + @Override + protected boolean isDestroyEnvironmentAfterEachTest() { + return true; + } + + // Do not allow factory reuse as the test modifies nifi.properties + @Override + protected boolean isAllowFactoryReuse() { + return false; + } + + @Test + public void testSelectiveDropOnConnectorStop() throws NiFiClientException, IOException, InterruptedException { + final ConnectorEntity connector = getClientUtil().createConnector("SelectiveDropConnector"); + assertNotNull(connector); + + getNifiClient().getConnectorClient().startConnector(connector); + logger.info("Started SelectiveDropConnector, waiting for 20,000 FlowFiles to be queued"); + + waitForConnectorQueueSize(connector.getId(), 20000); + logger.info("Queue has reached 20,000 FlowFiles"); + + // We expect 9 swap files because it should be 2,000 FlowFiles in active queue and 18,000 FlowFiles swapped out. + // With 2,000 per swap file, that results in 9 swap files. + final File instanceDir = getNiFiInstance().getInstanceDirectory(); + waitForSwapFileCount(9); + + getClientUtil().stopConnector(connector); + logger.info("Stopped SelectiveDropConnector, which should have triggered selective drop of even-indexed FlowFiles"); + + final int queuedFlowFiles = getConnectorQueueSize(connector.getId()); + assertEquals(10000, queuedFlowFiles); + + final int swapFileCountAfterPurge = countSwapFiles(instanceDir); + assertEquals(9, swapFileCountAfterPurge); + + logger.info("Restarting NiFi instance to verify state persists across restart"); + getNiFiInstance().stop(); + getNiFiInstance().start(); + + final int queuedFlowFilesAfterRestart = getConnectorQueueSize(connector.getId()); + assertEquals(10000, queuedFlowFilesAfterRestart); + + final int contentFileCountWithArchive = countContentRepositoryFiles(instanceDir, true); + final int contentFileCountWithoutArchive = countContentRepositoryFiles(instanceDir, false); + assertEquals(10000, contentFileCountWithoutArchive); + assertEquals(20000, contentFileCountWithArchive); + + final int swapFileCountAfterRestart = countSwapFiles(instanceDir); + assertEquals(9, swapFileCountAfterRestart); + } + + private void waitForSwapFileCount(final int expectedCount) throws InterruptedException { + waitFor(() -> { + final File instanceDir = getNiFiInstance().getInstanceDirectory(); + final int swapFileCount = countSwapFiles(instanceDir); + return swapFileCount == expectedCount; + }); + } + + private void waitForConnectorQueueSize(final String connectorId, final int expectedSize) throws InterruptedException { + waitFor(() -> getConnectorQueueSize(connectorId) >= expectedSize); + } + + private int getConnectorQueueSize(final String connectorId) throws NiFiClientException, IOException { + final ConnectorEntity connector = getNifiClient().getConnectorClient().getConnector(connectorId); + final String managedProcessGroupId = connector.getComponent().getManagedProcessGroupId(); + if (managedProcessGroupId == null) { + return 0; + } + + final ProcessGroupFlowEntity flowEntity = getNifiClient().getConnectorClient().getFlow(connectorId, managedProcessGroupId); + final ProcessGroupFlowDTO flowDto = flowEntity.getProcessGroupFlow(); + if (flowDto == null || flowDto.getFlow() == null) { + return 0; + } + + final Set connections = flowDto.getFlow().getConnections(); + int totalQueueSize = 0; + for (final ConnectionEntity connection : connections) { + if (connection.getStatus() != null && connection.getStatus().getAggregateSnapshot() != null) { + totalQueueSize += connection.getStatus().getAggregateSnapshot().getFlowFilesQueued(); + } + } + return totalQueueSize; + } + + private int countContentRepositoryFiles(final File instanceDir, final boolean includeArchive) { + final File contentRepo = new File(instanceDir, "content_repository"); + final Predicate filter = includeArchive ? file -> true : file -> !file.getName().equals("archive"); + return countFilesRecursively(contentRepo, filter); + } + + private int countSwapFiles(final File instanceDir) { + final File flowFileRepo = new File(instanceDir, "flowfile_repository"); + final File swapDir = new File(flowFileRepo, "swap"); + final File[] swapFiles = swapDir.listFiles(); + return swapFiles == null ? 0 : swapFiles.length; + } + + private int countFilesRecursively(final File directory, final Predicate filter) { + if (!directory.exists() || !directory.isDirectory()) { + return 0; + } + + int count = 0; + final File[] files = directory.listFiles(); + if (files != null) { + for (final File file : files) { + if (file.isDirectory()) { + if (filter.test(file)) { + count += countFilesRecursively(file, filter); + } + } else if (filter.test(file)) { + count++; + } + } + } + + return count; + } +} diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/connector-version-mismatch/flow.json b/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/connector-version-mismatch/flow.json new file mode 100644 index 000000000000..d01e6130007e --- /dev/null +++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/connector-version-mismatch/flow.json @@ -0,0 +1,52 @@ +{ + "encodingVersion": { + "majorVersion": 2, + "minorVersion": 0 + }, + "maxTimerDrivenThreadCount": 10, + "registries": [], + "parameterContexts": [], + "parameterProviders": [], + "controllerServices": [], + "reportingTasks": [], + "flowAnalysisRules": [], + "connectors": [ + { + "instanceIdentifier": "11111111-1111-1111-1111-111111111111", + "name": "Test NopConnector", + "type": "org.apache.nifi.connectors.tests.system.NopConnector", + "bundle": { + "group": "org.apache.nifi", + "artifact": "nifi-system-test-extensions-nar", + "version": "1.0.0-nonexistent" + }, + "scheduledState": "ENABLED", + "activeFlowConfiguration": [], + "workingFlowConfiguration": [] + } + ], + "rootGroup": { + "identifier": "root-group-identifier", + "instanceIdentifier": "root-group-instance-id", + "name": "NiFi Flow", + "comments": "", + "position": { + "x": 0.0, + "y": 0.0 + }, + "processGroups": [], + "remoteProcessGroups": [], + "processors": [], + "inputPorts": [], + "outputPorts": [], + "connections": [], + "labels": [], + "funnels": [], + "controllerServices": [], + "defaultFlowFileExpiration": "0 sec", + "defaultBackPressureObjectThreshold": 10000, + "defaultBackPressureDataSizeThreshold": "1 GB", + "flowFileConcurrency": "UNBOUNDED", + "flowFileOutboundPolicy": "STREAM_WHEN_AVAILABLE" + } +} diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/controller-service-enabling-failure/flow.json b/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/controller-service-enabling-failure/flow.json index 32dd9ba211db..08d40b7a7795 100644 --- a/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/controller-service-enabling-failure/flow.json +++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/controller-service-enabling-failure/flow.json @@ -47,7 +47,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "Enable Failure Count": "1000" diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/migrate-properties/flow.json b/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/migrate-properties/flow.json index 69ba4d539965..0657fb2429f5 100644 --- a/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/migrate-properties/flow.json +++ b/nifi-system-tests/nifi-system-test-suite/src/test/resources/flows/migrate-properties/flow.json @@ -45,7 +45,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "count-service": "c1a97022-019a-1000-4c72-79bb12714423" @@ -82,7 +82,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "count-service": "c1a97022-019a-1000-4c72-79bb12714423" @@ -121,7 +121,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "count-service": "c1a97022-019a-1000-4c72-79bb12714423" @@ -158,7 +158,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "count-service": "c1a97022-019a-1000-4c72-79bb12714423" @@ -307,7 +307,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "start-value": "7" @@ -319,7 +319,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-api-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" } } ], @@ -337,7 +337,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "start-value": "9" @@ -349,7 +349,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-api-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" } } ], @@ -367,7 +367,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "start-value": "6" @@ -379,7 +379,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-api-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" } } ], @@ -397,7 +397,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "start-value": "10" @@ -409,7 +409,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-api-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" } } ], @@ -427,7 +427,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "start-value": "1" @@ -439,7 +439,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-api-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" } } ], @@ -457,7 +457,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "start-value": "0", @@ -470,7 +470,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-api-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" } } ], @@ -488,7 +488,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "start-value": "11" @@ -500,7 +500,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-api-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" } } ], @@ -518,7 +518,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "start-value": "3" @@ -530,7 +530,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-api-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" } } ], @@ -548,7 +548,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "start-value": "5" @@ -560,7 +560,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-api-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" } } ], @@ -578,7 +578,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "start-value": "4" @@ -590,7 +590,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-api-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" } } ], @@ -608,7 +608,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "start-value": "2" @@ -620,7 +620,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-api-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" } } ], @@ -638,7 +638,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" }, "properties": { "start-value": "8" @@ -650,7 +650,7 @@ "bundle": { "group": "org.apache.nifi", "artifact": "nifi-system-test-extensions-services-api-nar", - "version": "2.7.0-SNAPSHOT" + "version": "2.8.0-SNAPSHOT" } } ], diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/NiFiClientFactory.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/NiFiClientFactory.java index 3c9d439f67cc..c33ca2fa6612 100644 --- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/NiFiClientFactory.java +++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/NiFiClientFactory.java @@ -23,6 +23,7 @@ import org.apache.nifi.toolkit.cli.impl.command.CommandOption; import org.apache.nifi.toolkit.client.AccessClient; import org.apache.nifi.toolkit.client.ConnectionClient; +import org.apache.nifi.toolkit.client.ConnectorClient; import org.apache.nifi.toolkit.client.ControllerClient; import org.apache.nifi.toolkit.client.ControllerServicesClient; import org.apache.nifi.toolkit.client.CountersClient; @@ -412,6 +413,16 @@ public SystemDiagnosticsClient getSystemsDiagnosticsClient(final RequestConfig r return wrappedClient.getSystemsDiagnosticsClient(requestConfig); } + @Override + public ConnectorClient getConnectorClient() { + return wrappedClient.getConnectorClient(); + } + + @Override + public ConnectorClient getConnectorClient(final RequestConfig requestConfig) { + return wrappedClient.getConnectorClient(requestConfig); + } + @Override public void close() throws IOException { wrappedClient.close(); diff --git a/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/ConnectorClient.java b/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/ConnectorClient.java new file mode 100644 index 000000000000..05f9ed173c3d --- /dev/null +++ b/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/ConnectorClient.java @@ -0,0 +1,419 @@ +/* + * 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.toolkit.client; + +import org.apache.nifi.web.api.entity.AssetEntity; +import org.apache.nifi.web.api.entity.AssetsEntity; +import org.apache.nifi.web.api.entity.ComponentStateEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepNamesEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ConnectorPropertyAllowableValuesEntity; +import org.apache.nifi.web.api.entity.DropRequestEntity; +import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity; +import org.apache.nifi.web.api.entity.ProcessGroupStatusEntity; +import org.apache.nifi.web.api.entity.VerifyConnectorConfigStepRequestEntity; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; + +/** + * Client for interacting with the Connector REST endpoints. + */ +public interface ConnectorClient { + + /** + * Creates a new connector. + * + * @param connectorEntity the connector entity to create + * @return the created connector entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorEntity createConnector(ConnectorEntity connectorEntity) throws NiFiClientException, IOException; + + /** + * Gets a connector by ID. + * + * @param connectorId the connector ID + * @return the connector entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorEntity getConnector(String connectorId) throws NiFiClientException, IOException; + + /** + * Updates a connector. + * + * @param connectorEntity the connector entity with updates + * @return the updated connector entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorEntity updateConnector(ConnectorEntity connectorEntity) throws NiFiClientException, IOException; + + /** + * Deletes a connector. + * + * @param connectorId the connector ID + * @param clientId the client ID + * @param version the revision version + * @return the deleted connector entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorEntity deleteConnector(String connectorId, String clientId, long version) throws NiFiClientException, IOException; + + /** + * Deletes a connector using the information from the entity. + * + * @param connectorEntity the connector entity to delete + * @return the deleted connector entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorEntity deleteConnector(ConnectorEntity connectorEntity) throws NiFiClientException, IOException; + + /** + * Starts a connector. + * + * @param connectorId the connector ID + * @param clientId the client ID + * @param version the revision version + * @return the updated connector entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorEntity startConnector(String connectorId, String clientId, long version) throws NiFiClientException, IOException; + + /** + * Starts a connector using the information from the entity. + * + * @param connectorEntity the connector entity to start + * @return the updated connector entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorEntity startConnector(ConnectorEntity connectorEntity) throws NiFiClientException, IOException; + + /** + * Stops a connector. + * + * @param connectorId the connector ID + * @param clientId the client ID + * @param version the revision version + * @return the updated connector entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorEntity stopConnector(String connectorId, String clientId, long version) throws NiFiClientException, IOException; + + /** + * Stops a connector using the information from the entity. + * + * @param connectorEntity the connector entity to stop + * @return the updated connector entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorEntity stopConnector(ConnectorEntity connectorEntity) throws NiFiClientException, IOException; + + /** + * Initiates draining of FlowFiles for a connector. + * + * @param connectorId the connector ID + * @param clientId the client ID + * @param version the revision version + * @return the updated connector entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorEntity drainConnector(String connectorId, String clientId, long version) throws NiFiClientException, IOException; + + /** + * Initiates draining of FlowFiles for a connector using the information from the entity. + * + * @param connectorEntity the connector entity to drain + * @return the updated connector entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorEntity drainConnector(ConnectorEntity connectorEntity) throws NiFiClientException, IOException; + + /** + * Cancels an ongoing drain operation for a connector. + * + * @param connectorId the connector ID + * @param clientId the client ID + * @param version the revision version + * @return the updated connector entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorEntity cancelDrain(String connectorId, String clientId, long version) throws NiFiClientException, IOException; + + /** + * Cancels an ongoing drain operation for a connector using the information from the entity. + * + * @param connectorEntity the connector entity to cancel draining for + * @return the updated connector entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorEntity cancelDrain(ConnectorEntity connectorEntity) throws NiFiClientException, IOException; + + /** + * Gets the configuration step names for a connector. + * + * @param connectorId the connector ID + * @return the configuration step names entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConfigurationStepNamesEntity getConfigurationSteps(String connectorId) throws NiFiClientException, IOException; + + /** + * Gets a specific configuration step for a connector. + * + * @param connectorId the connector ID + * @param configurationStepName the configuration step name + * @return the configuration step entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConfigurationStepEntity getConfigurationStep(String connectorId, String configurationStepName) throws NiFiClientException, IOException; + + /** + * Gets the allowable values for a property in a configuration step. + * + * @param connectorId the connector ID + * @param configurationStepName the configuration step name + * @param propertyGroupName the property group name + * @param propertyName the property name + * @param filter optional filter for the allowable values + * @return the allowable values entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorPropertyAllowableValuesEntity getPropertyAllowableValues(String connectorId, String configurationStepName, + String propertyGroupName, String propertyName, String filter) throws NiFiClientException, IOException; + + /** + * Updates a configuration step for a connector. + * + * @param configurationStepEntity the configuration step entity with updates + * @return the updated configuration step entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConfigurationStepEntity updateConfigurationStep(ConfigurationStepEntity configurationStepEntity) throws NiFiClientException, IOException; + + /** + * Submits a configuration step verification request. + * + * @param requestEntity the verification request entity + * @return the verification request entity with status + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + VerifyConnectorConfigStepRequestEntity submitConfigStepVerificationRequest(VerifyConnectorConfigStepRequestEntity requestEntity) throws NiFiClientException, IOException; + + /** + * Gets a configuration step verification request. + * + * @param connectorId the connector ID + * @param configurationStepName the configuration step name + * @param requestId the verification request ID + * @return the verification request entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + VerifyConnectorConfigStepRequestEntity getConfigStepVerificationRequest(String connectorId, String configurationStepName, + String requestId) throws NiFiClientException, IOException; + + /** + * Deletes a configuration step verification request. + * + * @param connectorId the connector ID + * @param configurationStepName the configuration step name + * @param requestId the verification request ID + * @return the deleted verification request entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + VerifyConnectorConfigStepRequestEntity deleteConfigStepVerificationRequest(String connectorId, String configurationStepName, + String requestId) throws NiFiClientException, IOException; + + /** + * Applies an update to a connector. + * + * @param connectorEntity the connector entity with revision + * @return the updated connector entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ConnectorEntity applyUpdate(ConnectorEntity connectorEntity) throws NiFiClientException, IOException; + + /** + * Gets the flow for the process group managed by a connector. + * + * @param connectorId the connector ID + * @return the process group flow entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ProcessGroupFlowEntity getFlow(String connectorId) throws NiFiClientException, IOException; + + /** + * Gets the flow for a specific process group within a connector's managed flow. + * + * @param connectorId the connector ID + * @param processGroupId the process group ID within the connector's managed flow + * @return the process group flow entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ProcessGroupFlowEntity getFlow(String connectorId, String processGroupId) throws NiFiClientException, IOException; + + /** + * Gets the status for the process group managed by a connector. + * + * @param connectorId the connector ID + * @param recursive whether to include status for all descendant components + * @return the process group status entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ProcessGroupStatusEntity getStatus(String connectorId, boolean recursive) throws NiFiClientException, IOException; + + /** + * Creates an asset in the given connector. + * + * @param connectorId the connector ID + * @param assetName the asset name + * @param file the file + * @return the created asset entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + AssetEntity createAsset(String connectorId, String assetName, File file) throws NiFiClientException, IOException; + + /** + * Lists the assets for the given connector. + * + * @param connectorId the connector ID + * @return the list of asset entities + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + AssetsEntity getAssets(String connectorId) throws NiFiClientException, IOException; + + /** + * Retrieves the content of the given asset. + * + * @param connectorId the connector ID + * @param assetId the asset ID + * @param outputDirectory the directory to write the asset content + * @return the path to the asset content + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + Path getAssetContent(String connectorId, String assetId, File outputDirectory) throws NiFiClientException, IOException; + + /** + * Creates a request to purge all FlowFiles for the given connector. + * + * @param connectorId the connector ID + * @return the drop request entity containing the purge request status + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + DropRequestEntity createPurgeRequest(String connectorId) throws NiFiClientException, IOException; + + /** + * Gets the status of a purge request for the given connector. + * + * @param connectorId the connector ID + * @param purgeRequestId the purge request ID + * @return the drop request entity containing the purge request status + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + DropRequestEntity getPurgeRequest(String connectorId, String purgeRequestId) throws NiFiClientException, IOException; + + /** + * Deletes (cancels) a purge request for the given connector. + * + * @param connectorId the connector ID + * @param purgeRequestId the purge request ID + * @return the drop request entity containing the final purge request status + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + DropRequestEntity deletePurgeRequest(String connectorId, String purgeRequestId) throws NiFiClientException, IOException; + + /** + * Gets the state for a processor within a connector. + * + * @param connectorId the connector ID + * @param processorId the processor ID + * @return the component state entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ComponentStateEntity getProcessorState(String connectorId, String processorId) throws NiFiClientException, IOException; + + /** + * Clears the state for a processor within a connector. + * + * @param connectorId the connector ID + * @param processorId the processor ID + * @return the component state entity after clearing + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ComponentStateEntity clearProcessorState(String connectorId, String processorId) throws NiFiClientException, IOException; + + /** + * Gets the state for a controller service within a connector. + * + * @param connectorId the connector ID + * @param controllerServiceId the controller service ID + * @return the component state entity + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ComponentStateEntity getControllerServiceState(String connectorId, String controllerServiceId) throws NiFiClientException, IOException; + + /** + * Clears the state for a controller service within a connector. + * + * @param connectorId the connector ID + * @param controllerServiceId the controller service ID + * @return the component state entity after clearing + * @throws NiFiClientException if an error occurs during the request + * @throws IOException if an I/O error occurs + */ + ComponentStateEntity clearControllerServiceState(String connectorId, String controllerServiceId) throws NiFiClientException, IOException; + + /** + * Indicates that mutable requests should indicate that the client has + * acknowledged that the node is disconnected. + */ + void acknowledgeDisconnectedNode(); +} diff --git a/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/FlowClient.java b/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/FlowClient.java index e483f1c91a58..57fa3c74f560 100644 --- a/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/FlowClient.java +++ b/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/FlowClient.java @@ -20,6 +20,7 @@ import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity; import org.apache.nifi.web.api.entity.ClusterSummaryEntity; import org.apache.nifi.web.api.entity.ConnectionStatusEntity; +import org.apache.nifi.web.api.entity.ConnectorsEntity; import org.apache.nifi.web.api.entity.ControllerServiceTypesEntity; import org.apache.nifi.web.api.entity.ControllerServicesEntity; import org.apache.nifi.web.api.entity.CurrentUserEntity; @@ -229,4 +230,11 @@ VersionedFlowSnapshotMetadataSetEntity getVersions(String registryId, String buc * @return list of flows */ VersionedFlowsEntity getFlowRegistryFlows(String registryClientId, String branch, String bucket) throws NiFiClientException, IOException; + + /** + * Retrieves all of the Connectors. + * + * @return the list of Connectors + */ + ConnectorsEntity getConnectors() throws NiFiClientException, IOException; } diff --git a/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/NiFiClient.java b/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/NiFiClient.java index 44bc180e3aab..a70421202ac8 100644 --- a/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/NiFiClient.java +++ b/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/NiFiClient.java @@ -135,12 +135,18 @@ public interface NiFiClient extends Closeable { SnippetClient getSnippetClient(RequestConfig requestConfig); - // ----- SnippetClient ----- + // ----- SystemDiagnosticsClient ----- SystemDiagnosticsClient getSystemsDiagnosticsClient(); SystemDiagnosticsClient getSystemsDiagnosticsClient(RequestConfig requestConfig); + // ----- ConnectorClient ----- + + ConnectorClient getConnectorClient(); + + ConnectorClient getConnectorClient(RequestConfig requestConfig); + /** * The builder interface that implementations should provide for obtaining the * client. diff --git a/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/impl/JerseyConnectorClient.java b/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/impl/JerseyConnectorClient.java new file mode 100644 index 000000000000..7b648e7183e6 --- /dev/null +++ b/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/impl/JerseyConnectorClient.java @@ -0,0 +1,673 @@ +/* + * 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.toolkit.client.impl; + +import jakarta.ws.rs.client.Entity; +import jakarta.ws.rs.client.WebTarget; +import jakarta.ws.rs.core.MediaType; +import jakarta.ws.rs.core.Response; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.toolkit.client.ConnectorClient; +import org.apache.nifi.toolkit.client.NiFiClientException; +import org.apache.nifi.toolkit.client.RequestConfig; +import org.apache.nifi.web.api.dto.RevisionDTO; +import org.apache.nifi.web.api.entity.AssetEntity; +import org.apache.nifi.web.api.entity.AssetsEntity; +import org.apache.nifi.web.api.entity.ComponentStateEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepEntity; +import org.apache.nifi.web.api.entity.ConfigurationStepNamesEntity; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ConnectorPropertyAllowableValuesEntity; +import org.apache.nifi.web.api.entity.ConnectorRunStatusEntity; +import org.apache.nifi.web.api.entity.DropRequestEntity; +import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity; +import org.apache.nifi.web.api.entity.ProcessGroupStatusEntity; +import org.apache.nifi.web.api.entity.VerifyConnectorConfigStepRequestEntity; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardCopyOption; +import java.util.Objects; + +/** + * Jersey implementation of ConnectorClient. + */ +public class JerseyConnectorClient extends AbstractJerseyClient implements ConnectorClient { + + private final WebTarget connectorsTarget; + private volatile WebTarget connectorTarget; + + public JerseyConnectorClient(final WebTarget baseTarget) { + this(baseTarget, null); + } + + public JerseyConnectorClient(final WebTarget baseTarget, final RequestConfig requestConfig) { + super(requestConfig); + this.connectorsTarget = baseTarget.path("/connectors"); + this.connectorTarget = baseTarget.path("/connectors/{id}"); + } + + @Override + public void acknowledgeDisconnectedNode() { + connectorTarget = connectorTarget.queryParam("disconnectedNodeAcknowledged", true); + } + + @Override + public ConnectorEntity createConnector(final ConnectorEntity connectorEntity) throws NiFiClientException, IOException { + if (connectorEntity == null) { + throw new IllegalArgumentException("Connector entity cannot be null"); + } + + return executeAction("Error creating Connector", () -> { + return getRequestBuilder(connectorsTarget).post( + Entity.entity(connectorEntity, MediaType.APPLICATION_JSON_TYPE), + ConnectorEntity.class); + }); + } + + @Override + public ConnectorEntity getConnector(final String connectorId) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector ID cannot be null or blank"); + } + + return executeAction("Error retrieving Connector", () -> { + final WebTarget target = connectorTarget.resolveTemplate("id", connectorId); + return getRequestBuilder(target).get(ConnectorEntity.class); + }); + } + + @Override + public ConnectorEntity updateConnector(final ConnectorEntity connectorEntity) throws NiFiClientException, IOException { + if (connectorEntity == null) { + throw new IllegalArgumentException("Connector entity cannot be null"); + } + + return executeAction("Error updating Connector", () -> { + final WebTarget target = connectorTarget.resolveTemplate("id", connectorEntity.getId()); + return getRequestBuilder(target).put( + Entity.entity(connectorEntity, MediaType.APPLICATION_JSON_TYPE), + ConnectorEntity.class); + }); + } + + @Override + public ConnectorEntity deleteConnector(final String connectorId, final String clientId, final long version) throws NiFiClientException, IOException { + return deleteConnector(connectorId, clientId, version, false); + } + + private ConnectorEntity deleteConnector(final String connectorId, final String clientId, final long version, + final Boolean acknowledgeDisconnect) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector ID cannot be null or blank"); + } + + return executeAction("Error deleting Connector", () -> { + WebTarget target = connectorTarget + .queryParam("version", version) + .queryParam("clientId", clientId) + .resolveTemplate("id", connectorId); + + if (acknowledgeDisconnect == Boolean.TRUE) { + target = target.queryParam("disconnectedNodeAcknowledged", "true"); + } + + return getRequestBuilder(target).delete(ConnectorEntity.class); + }); + } + + @Override + public ConnectorEntity deleteConnector(final ConnectorEntity connectorEntity) throws NiFiClientException, IOException { + if (connectorEntity == null) { + throw new IllegalArgumentException("Connector entity cannot be null"); + } + return deleteConnector(connectorEntity.getId(), connectorEntity.getRevision().getClientId(), + connectorEntity.getRevision().getVersion(), connectorEntity.isDisconnectedNodeAcknowledged()); + } + + @Override + public ConnectorEntity startConnector(final String connectorId, final String clientId, final long version) throws NiFiClientException, IOException { + return updateConnectorRunStatus(connectorId, "RUNNING", clientId, version, false); + } + + @Override + public ConnectorEntity startConnector(final ConnectorEntity connectorEntity) throws NiFiClientException, IOException { + return updateConnectorRunStatus(connectorEntity.getId(), "RUNNING", connectorEntity.getRevision().getClientId(), + connectorEntity.getRevision().getVersion(), connectorEntity.isDisconnectedNodeAcknowledged()); + } + + @Override + public ConnectorEntity stopConnector(final String connectorId, final String clientId, final long version) throws NiFiClientException, IOException { + return updateConnectorRunStatus(connectorId, "STOPPED", clientId, version, false); + } + + @Override + public ConnectorEntity stopConnector(final ConnectorEntity connectorEntity) throws NiFiClientException, IOException { + return updateConnectorRunStatus(connectorEntity.getId(), "STOPPED", connectorEntity.getRevision().getClientId(), + connectorEntity.getRevision().getVersion(), connectorEntity.isDisconnectedNodeAcknowledged()); + } + + @Override + public ConnectorEntity drainConnector(final String connectorId, final String clientId, final long version) throws NiFiClientException, IOException { + return drainConnector(connectorId, clientId, version, false); + } + + @Override + public ConnectorEntity drainConnector(final ConnectorEntity connectorEntity) throws NiFiClientException, IOException { + return drainConnector(connectorEntity.getId(), connectorEntity.getRevision().getClientId(), + connectorEntity.getRevision().getVersion(), connectorEntity.isDisconnectedNodeAcknowledged()); + } + + private ConnectorEntity drainConnector(final String connectorId, final String clientId, final long version, + final Boolean disconnectedNodeAcknowledged) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector ID cannot be null or blank"); + } + + return executeAction("Error initiating connector drain", () -> { + final WebTarget target = connectorTarget + .path("/drain") + .resolveTemplate("id", connectorId); + + final ConnectorEntity requestEntity = new ConnectorEntity(); + requestEntity.setId(connectorId); + requestEntity.setDisconnectedNodeAcknowledged(disconnectedNodeAcknowledged); + + final RevisionDTO revisionDto = new RevisionDTO(); + revisionDto.setClientId(clientId); + revisionDto.setVersion(version); + requestEntity.setRevision(revisionDto); + + return getRequestBuilder(target).post( + Entity.entity(requestEntity, MediaType.APPLICATION_JSON_TYPE), + ConnectorEntity.class); + }); + } + + @Override + public ConnectorEntity cancelDrain(final String connectorId, final String clientId, final long version) throws NiFiClientException, IOException { + return cancelDrain(connectorId, clientId, version, false); + } + + @Override + public ConnectorEntity cancelDrain(final ConnectorEntity connectorEntity) throws NiFiClientException, IOException { + return cancelDrain(connectorEntity.getId(), connectorEntity.getRevision().getClientId(), + connectorEntity.getRevision().getVersion(), connectorEntity.isDisconnectedNodeAcknowledged()); + } + + private ConnectorEntity cancelDrain(final String connectorId, final String clientId, final long version, + final Boolean disconnectedNodeAcknowledged) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector ID cannot be null or blank"); + } + + return executeAction("Error canceling connector drain", () -> { + WebTarget target = connectorTarget + .path("/drain") + .queryParam("version", version) + .queryParam("clientId", clientId) + .resolveTemplate("id", connectorId); + + if (disconnectedNodeAcknowledged == Boolean.TRUE) { + target = target.queryParam("disconnectedNodeAcknowledged", "true"); + } + + return getRequestBuilder(target).delete(ConnectorEntity.class); + }); + } + + private ConnectorEntity updateConnectorRunStatus(final String connectorId, final String desiredState, final String clientId, + final long version, final Boolean disconnectedNodeAcknowledged) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector ID cannot be null or blank"); + } + + return executeAction("Error updating Connector run status", () -> { + final WebTarget target = connectorTarget + .path("/run-status") + .resolveTemplate("id", connectorId); + + final ConnectorRunStatusEntity runStatusEntity = new ConnectorRunStatusEntity(); + runStatusEntity.setState(desiredState); + runStatusEntity.setDisconnectedNodeAcknowledged(disconnectedNodeAcknowledged); + + final RevisionDTO revisionDto = new RevisionDTO(); + revisionDto.setClientId(clientId); + revisionDto.setVersion(version); + runStatusEntity.setRevision(revisionDto); + + return getRequestBuilder(target).put( + Entity.entity(runStatusEntity, MediaType.APPLICATION_JSON_TYPE), + ConnectorEntity.class); + }); + } + + @Override + public ConfigurationStepNamesEntity getConfigurationSteps(final String connectorId) throws NiFiClientException, IOException { + Objects.requireNonNull(connectorId, "Connector ID required"); + + return executeAction("Error retrieving configuration steps", () -> { + final WebTarget target = connectorTarget + .path("/configuration-steps") + .resolveTemplate("id", connectorId); + return getRequestBuilder(target).get(ConfigurationStepNamesEntity.class); + }); + } + + @Override + public ConfigurationStepEntity getConfigurationStep(final String connectorId, final String configurationStepName) throws NiFiClientException, IOException { + Objects.requireNonNull(connectorId, "Connector ID required"); + Objects.requireNonNull(configurationStepName, "Configuration step name required"); + + return executeAction("Error retrieving configuration step", () -> { + final WebTarget target = connectorTarget + .path("/configuration-steps/{configurationStepName}") + .resolveTemplate("id", connectorId) + .resolveTemplate("configurationStepName", configurationStepName); + return getRequestBuilder(target).get(ConfigurationStepEntity.class); + }); + } + + @Override + public ConnectorPropertyAllowableValuesEntity getPropertyAllowableValues(final String connectorId, final String configurationStepName, + final String propertyGroupName, final String propertyName, final String filter) throws NiFiClientException, IOException { + Objects.requireNonNull(connectorId, "Connector ID required"); + Objects.requireNonNull(configurationStepName, "Configuration step name required"); + Objects.requireNonNull(propertyGroupName, "Property group name required"); + Objects.requireNonNull(propertyName, "Property name required"); + + return executeAction("Error retrieving property allowable values", () -> { + WebTarget target = connectorTarget + .path("/configuration-steps/{configurationStepName}/property-groups/{propertyGroupName}/properties/{propertyName}/allowable-values") + .resolveTemplate("id", connectorId) + .resolveTemplate("configurationStepName", configurationStepName) + .resolveTemplate("propertyGroupName", propertyGroupName) + .resolveTemplate("propertyName", propertyName); + + if (filter != null) { + target = target.queryParam("filter", filter); + } + + return getRequestBuilder(target).get(ConnectorPropertyAllowableValuesEntity.class); + }); + } + + @Override + public ConfigurationStepEntity updateConfigurationStep(final ConfigurationStepEntity configurationStepEntity) throws NiFiClientException, IOException { + if (configurationStepEntity == null) { + throw new IllegalArgumentException("Configuration step entity cannot be null"); + } + if (configurationStepEntity.getParentConnectorId() == null) { + throw new IllegalArgumentException("Parent connector ID cannot be null"); + } + if (configurationStepEntity.getConfigurationStep() == null) { + throw new IllegalArgumentException("Configuration step cannot be null"); + } + if (configurationStepEntity.getConfigurationStep().getConfigurationStepName() == null) { + throw new IllegalArgumentException("Configuration step name cannot be null"); + } + + return executeAction("Error updating configuration step", () -> { + final WebTarget target = connectorTarget + .path("/configuration-steps/{configurationStepName}") + .resolveTemplate("id", configurationStepEntity.getParentConnectorId()) + .resolveTemplate("configurationStepName", configurationStepEntity.getConfigurationStep().getConfigurationStepName()); + + return getRequestBuilder(target).put( + Entity.entity(configurationStepEntity, MediaType.APPLICATION_JSON_TYPE), + ConfigurationStepEntity.class); + }); + } + + @Override + public VerifyConnectorConfigStepRequestEntity submitConfigStepVerificationRequest(final VerifyConnectorConfigStepRequestEntity requestEntity) + throws NiFiClientException, IOException { + if (requestEntity == null) { + throw new IllegalArgumentException("Verification request entity cannot be null"); + } + if (requestEntity.getRequest() == null) { + throw new IllegalArgumentException("Verification request DTO cannot be null"); + } + if (requestEntity.getRequest().getConnectorId() == null) { + throw new IllegalArgumentException("Connector ID cannot be null"); + } + if (requestEntity.getRequest().getConfigurationStepName() == null) { + throw new IllegalArgumentException("Configuration step name cannot be null"); + } + + return executeAction("Error submitting configuration step verification request", () -> { + final WebTarget target = connectorTarget + .path("/configuration-steps/{configurationStepName}/verify-config") + .resolveTemplate("id", requestEntity.getRequest().getConnectorId()) + .resolveTemplate("configurationStepName", requestEntity.getRequest().getConfigurationStepName()); + + return getRequestBuilder(target).post( + Entity.entity(requestEntity, MediaType.APPLICATION_JSON_TYPE), + VerifyConnectorConfigStepRequestEntity.class); + }); + } + + @Override + public VerifyConnectorConfigStepRequestEntity getConfigStepVerificationRequest(final String connectorId, final String configurationStepName, + final String requestId) throws NiFiClientException, IOException { + Objects.requireNonNull(connectorId, "Connector ID required"); + Objects.requireNonNull(configurationStepName, "Configuration step name required"); + Objects.requireNonNull(requestId, "Request ID required"); + + return executeAction("Error retrieving configuration step verification request", () -> { + final WebTarget target = connectorTarget + .path("/configuration-steps/{configurationStepName}/verify-config/{requestId}") + .resolveTemplate("id", connectorId) + .resolveTemplate("configurationStepName", configurationStepName) + .resolveTemplate("requestId", requestId); + + return getRequestBuilder(target).get(VerifyConnectorConfigStepRequestEntity.class); + }); + } + + @Override + public VerifyConnectorConfigStepRequestEntity deleteConfigStepVerificationRequest(final String connectorId, final String configurationStepName, + final String requestId) throws NiFiClientException, IOException { + Objects.requireNonNull(connectorId, "Connector ID required"); + Objects.requireNonNull(configurationStepName, "Configuration step name required"); + Objects.requireNonNull(requestId, "Request ID required"); + + return executeAction("Error deleting configuration step verification request", () -> { + final WebTarget target = connectorTarget + .path("/configuration-steps/{configurationStepName}/verify-config/{requestId}") + .resolveTemplate("id", connectorId) + .resolveTemplate("configurationStepName", configurationStepName) + .resolveTemplate("requestId", requestId); + + return getRequestBuilder(target).delete(VerifyConnectorConfigStepRequestEntity.class); + }); + } + + @Override + public ConnectorEntity applyUpdate(final ConnectorEntity connectorEntity) throws NiFiClientException, IOException { + if (connectorEntity == null) { + throw new IllegalArgumentException("Connector entity cannot be null"); + } + if (StringUtils.isBlank(connectorEntity.getId())) { + throw new IllegalArgumentException("Connector id cannot be null or blank"); + } + if (connectorEntity.getRevision() == null) { + throw new IllegalArgumentException("Revision cannot be null"); + } + + final String connectorId = connectorEntity.getId(); + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector ID cannot be null or blank"); + } + + return executeAction("Error applying connector update", () -> { + final WebTarget target = connectorTarget + .path("/apply-update") + .resolveTemplate("id", connectorId); + + return getRequestBuilder(target).post( + Entity.entity(connectorEntity, MediaType.APPLICATION_JSON_TYPE), + ConnectorEntity.class); + }); + } + + @Override + public ProcessGroupFlowEntity getFlow(final String connectorId) throws NiFiClientException, IOException { + final ConnectorEntity connector = getConnector(connectorId); + final String managedProcessGroupId = connector.getComponent().getManagedProcessGroupId(); + return getFlow(connectorId, managedProcessGroupId); + } + + @Override + public ProcessGroupFlowEntity getFlow(final String connectorId, final String processGroupId) throws NiFiClientException, IOException { + Objects.requireNonNull(connectorId, "Connector ID required"); + Objects.requireNonNull(processGroupId, "Process Group ID required"); + + return executeAction("Error retrieving connector flow", () -> { + WebTarget target = connectorTarget + .path("/flow/process-groups/{processGroupId}") + .resolveTemplate("id", connectorId) + .resolveTemplate("processGroupId", processGroupId); + + return getRequestBuilder(target).get(ProcessGroupFlowEntity.class); + }); + } + + @Override + public ProcessGroupStatusEntity getStatus(final String connectorId, final boolean recursive) throws NiFiClientException, IOException { + Objects.requireNonNull(connectorId, "Connector ID required"); + + return executeAction("Error retrieving connector status", () -> { + WebTarget target = connectorTarget + .path("/status") + .resolveTemplate("id", connectorId); + + if (recursive) { + target = target.queryParam("recursive", "true"); + } + + return getRequestBuilder(target).get(ProcessGroupStatusEntity.class); + }); + } + + @Override + public AssetEntity createAsset(final String connectorId, final String assetName, final File file) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector id cannot be null or blank"); + } + if (StringUtils.isBlank(assetName)) { + throw new IllegalArgumentException("Asset name cannot be null or blank"); + } + if (file == null) { + throw new IllegalArgumentException("File cannot be null"); + } + if (!file.exists()) { + throw new FileNotFoundException(file.getAbsolutePath()); + } + + try (final InputStream assetInputStream = new FileInputStream(file)) { + return executeAction("Error creating Connector Asset " + assetName + " for Connector " + connectorId, () -> { + final WebTarget target = connectorsTarget + .path("{id}/assets") + .resolveTemplate("id", connectorId); + + return getRequestBuilder(target) + .header("Filename", assetName) + .post( + Entity.entity(assetInputStream, MediaType.APPLICATION_OCTET_STREAM_TYPE), + AssetEntity.class); + }); + } + } + + @Override + public AssetsEntity getAssets(final String connectorId) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector id cannot be null or blank"); + } + + return executeAction("Error retrieving Connector assets", () -> { + final WebTarget target = connectorsTarget + .path("{id}/assets") + .resolveTemplate("id", connectorId); + return getRequestBuilder(target).get(AssetsEntity.class); + }); + } + + @Override + public Path getAssetContent(final String connectorId, final String assetId, final File outputDirectory) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector id cannot be null or blank"); + } + if (StringUtils.isBlank(assetId)) { + throw new IllegalArgumentException("Asset id cannot be null or blank"); + } + + return executeAction("Error getting Connector asset content", () -> { + final WebTarget target = connectorsTarget + .path("{id}/assets/{assetId}") + .resolveTemplate("id", connectorId) + .resolveTemplate("assetId", assetId); + + final Response response = getRequestBuilder(target) + .accept(MediaType.APPLICATION_OCTET_STREAM_TYPE) + .get(); + + final String filename = getContentDispositionFilename(response); + final File assetFile = new File(outputDirectory, filename); + + try (final InputStream responseInputStream = response.readEntity(InputStream.class)) { + Files.copy(responseInputStream, assetFile.toPath(), StandardCopyOption.REPLACE_EXISTING); + return assetFile.toPath(); + } + }); + } + + @Override + public DropRequestEntity createPurgeRequest(final String connectorId) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector id cannot be null or blank"); + } + + return executeAction("Error creating purge request for Connector " + connectorId, () -> { + final WebTarget target = connectorsTarget + .path("{id}/purge-requests") + .resolveTemplate("id", connectorId); + + return getRequestBuilder(target).post(null, DropRequestEntity.class); + }); + } + + @Override + public DropRequestEntity getPurgeRequest(final String connectorId, final String purgeRequestId) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector id cannot be null or blank"); + } + if (StringUtils.isBlank(purgeRequestId)) { + throw new IllegalArgumentException("Purge request id cannot be null or blank"); + } + + return executeAction("Error getting purge request for Connector " + connectorId, () -> { + final WebTarget target = connectorsTarget + .path("{id}/purge-requests/{purgeRequestId}") + .resolveTemplate("id", connectorId) + .resolveTemplate("purgeRequestId", purgeRequestId); + + return getRequestBuilder(target).get(DropRequestEntity.class); + }); + } + + @Override + public DropRequestEntity deletePurgeRequest(final String connectorId, final String purgeRequestId) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector id cannot be null or blank"); + } + if (StringUtils.isBlank(purgeRequestId)) { + throw new IllegalArgumentException("Purge request id cannot be null or blank"); + } + + return executeAction("Error deleting purge request for Connector " + connectorId, () -> { + final WebTarget target = connectorsTarget + .path("{id}/purge-requests/{purgeRequestId}") + .resolveTemplate("id", connectorId) + .resolveTemplate("purgeRequestId", purgeRequestId); + + return getRequestBuilder(target).delete(DropRequestEntity.class); + }); + } + + @Override + public ComponentStateEntity getProcessorState(final String connectorId, final String processorId) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector id cannot be null or blank"); + } + if (StringUtils.isBlank(processorId)) { + throw new IllegalArgumentException("Processor id cannot be null or blank"); + } + + return executeAction("Error retrieving processor state for Connector " + connectorId, () -> { + final WebTarget target = connectorTarget + .path("/processors/{processorId}/state") + .resolveTemplate("id", connectorId) + .resolveTemplate("processorId", processorId); + + return getRequestBuilder(target).get(ComponentStateEntity.class); + }); + } + + @Override + public ComponentStateEntity clearProcessorState(final String connectorId, final String processorId) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector id cannot be null or blank"); + } + if (StringUtils.isBlank(processorId)) { + throw new IllegalArgumentException("Processor id cannot be null or blank"); + } + + return executeAction("Error clearing processor state for Connector " + connectorId, () -> { + final WebTarget target = connectorTarget + .path("/processors/{processorId}/state/clear-requests") + .resolveTemplate("id", connectorId) + .resolveTemplate("processorId", processorId); + + return getRequestBuilder(target).post(null, ComponentStateEntity.class); + }); + } + + @Override + public ComponentStateEntity getControllerServiceState(final String connectorId, final String controllerServiceId) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector id cannot be null or blank"); + } + if (StringUtils.isBlank(controllerServiceId)) { + throw new IllegalArgumentException("Controller service id cannot be null or blank"); + } + + return executeAction("Error retrieving controller service state for Connector " + connectorId, () -> { + final WebTarget target = connectorTarget + .path("/controller-services/{controllerServiceId}/state") + .resolveTemplate("id", connectorId) + .resolveTemplate("controllerServiceId", controllerServiceId); + + return getRequestBuilder(target).get(ComponentStateEntity.class); + }); + } + + @Override + public ComponentStateEntity clearControllerServiceState(final String connectorId, final String controllerServiceId) throws NiFiClientException, IOException { + if (StringUtils.isBlank(connectorId)) { + throw new IllegalArgumentException("Connector id cannot be null or blank"); + } + if (StringUtils.isBlank(controllerServiceId)) { + throw new IllegalArgumentException("Controller service id cannot be null or blank"); + } + + return executeAction("Error clearing controller service state for Connector " + connectorId, () -> { + final WebTarget target = connectorTarget + .path("/controller-services/{controllerServiceId}/state/clear-requests") + .resolveTemplate("id", connectorId) + .resolveTemplate("controllerServiceId", controllerServiceId); + + return getRequestBuilder(target).post(null, ComponentStateEntity.class); + }); + } +} diff --git a/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/impl/JerseyFlowClient.java b/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/impl/JerseyFlowClient.java index 114c4e01c235..f0cb9450cacf 100644 --- a/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/impl/JerseyFlowClient.java +++ b/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/impl/JerseyFlowClient.java @@ -32,6 +32,7 @@ import org.apache.nifi.web.api.entity.ClusterSummaryEntity; import org.apache.nifi.web.api.entity.ComponentEntity; import org.apache.nifi.web.api.entity.ConnectionStatusEntity; +import org.apache.nifi.web.api.entity.ConnectorsEntity; import org.apache.nifi.web.api.entity.ControllerServiceTypesEntity; import org.apache.nifi.web.api.entity.ControllerServicesEntity; import org.apache.nifi.web.api.entity.CurrentUserEntity; @@ -400,4 +401,12 @@ public VersionedFlowsEntity getFlowRegistryFlows(String registryClientId, String return getRequestBuilder(target).get(VersionedFlowsEntity.class); }); } + + @Override + public ConnectorsEntity getConnectors() throws NiFiClientException, IOException { + return executeAction("Error retrieving Connectors", () -> { + final WebTarget target = flowTarget.path("/connectors"); + return getRequestBuilder(target).get(ConnectorsEntity.class); + }); + } } diff --git a/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/impl/JerseyNiFiClient.java b/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/impl/JerseyNiFiClient.java index af5697e262c1..c99f15d9e97e 100644 --- a/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/impl/JerseyNiFiClient.java +++ b/nifi-toolkit/nifi-toolkit-client/src/main/java/org/apache/nifi/toolkit/client/impl/JerseyNiFiClient.java @@ -26,6 +26,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.nifi.toolkit.client.AccessClient; import org.apache.nifi.toolkit.client.ConnectionClient; +import org.apache.nifi.toolkit.client.ConnectorClient; import org.apache.nifi.toolkit.client.ControllerClient; import org.apache.nifi.toolkit.client.ControllerServicesClient; import org.apache.nifi.toolkit.client.CountersClient; @@ -311,6 +312,16 @@ public SystemDiagnosticsClient getSystemsDiagnosticsClient(final RequestConfig r return new JerseySystemDiagnosticsClient(baseTarget, requestConfig); } + @Override + public ConnectorClient getConnectorClient() { + return new JerseyConnectorClient(baseTarget); + } + + @Override + public ConnectorClient getConnectorClient(final RequestConfig requestConfig) { + return new JerseyConnectorClient(baseTarget, requestConfig); + } + @Override public void close() { if (this.client != null) { @@ -351,7 +362,7 @@ private static JacksonJaxbJsonProvider jacksonJaxbJsonProvider() { JacksonJaxbJsonProvider jacksonJaxbJsonProvider = new JacksonJaxbJsonProvider(); ObjectMapper mapper = new ObjectMapper(); - mapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL)); + mapper.setDefaultPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.ALWAYS)); mapper.setAnnotationIntrospector(new JakartaXmlBindAnnotationIntrospector(mapper.getTypeFactory())); // Ignore unknown properties so that deployed client remain compatible with // future versions of NiFi that add new fields diff --git a/pom.xml b/pom.xml index e7e7934c82d3..e95220ea3e5a 100644 --- a/pom.xml +++ b/pom.xml @@ -32,6 +32,7 @@ nifi-bootstrap nifi-code-coverage nifi-mock + nifi-connector-mock-bundle nifi-extension-bundles nifi-extension-bom nifi-assembly @@ -44,6 +45,7 @@ nifi-registry nifi-toolkit nifi-manifest + nifi-connectors c2 https://nifi.apache.org From 8854332c702164fb617990a1ba17894adcc7b3ee Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Tue, 10 Mar 2026 15:54:13 -0400 Subject: [PATCH 2/4] NIFI-15258: Added license headers to cursor rules files; addressed maven build lifecycle issue --- .cursor/rules/building.mdc | 14 ++++++++++++++ .cursor/rules/code-style.mdc | 14 ++++++++++++++ .cursor/rules/ending-conditions.mdc | 14 ++++++++++++++ .cursor/rules/extension-development.mdc | 14 ++++++++++++++ .cursor/rules/extension-testing.mdc | 14 ++++++++++++++ .cursor/rules/framework-testing.mdc | 14 ++++++++++++++ .cursor/rules/persona.mdc | 14 ++++++++++++++ .cursor/rules/testing-standards.mdc | 14 ++++++++++++++ .../nifi-connector-mock-integration-tests/pom.xml | 2 +- .../nifi-connector-mock/pom.xml | 2 +- .../steps/Kafka_Connection.md | 14 ++++++++++++++ .../steps/Kafka_Topics.md | 14 ++++++++++++++ .../steps/S3_Configuration.md | 14 ++++++++++++++ .../nifi-kafka-to-s3-integration-tests/pom.xml | 2 +- 14 files changed, 157 insertions(+), 3 deletions(-) diff --git a/.cursor/rules/building.mdc b/.cursor/rules/building.mdc index af4829a42628..7ec9fabd47c4 100644 --- a/.cursor/rules/building.mdc +++ b/.cursor/rules/building.mdc @@ -2,6 +2,20 @@ description: Maven build instructions for the NiFi codebase alwaysApply: true --- + # Building diff --git a/.cursor/rules/code-style.mdc b/.cursor/rules/code-style.mdc index ca7e9cfed5a6..5432070ecf71 100644 --- a/.cursor/rules/code-style.mdc +++ b/.cursor/rules/code-style.mdc @@ -3,6 +3,20 @@ description: Java code style conventions for the NiFi codebase globs: "**/*.java" alwaysApply: false --- + # Code Style diff --git a/.cursor/rules/ending-conditions.mdc b/.cursor/rules/ending-conditions.mdc index aa2b607801a1..8242bb340057 100644 --- a/.cursor/rules/ending-conditions.mdc +++ b/.cursor/rules/ending-conditions.mdc @@ -2,6 +2,20 @@ description: Task completion checklist that must be verified before considering any task done alwaysApply: true --- + # Ending Conditions diff --git a/.cursor/rules/extension-development.mdc b/.cursor/rules/extension-development.mdc index 74e53f34d5f1..614bc79b09eb 100644 --- a/.cursor/rules/extension-development.mdc +++ b/.cursor/rules/extension-development.mdc @@ -2,6 +2,20 @@ description: Development patterns for NiFi extensions (Processors, Controller Services, Connectors). Covers Property Descriptors, Relationships, and common patterns. alwaysApply: false --- + # Extension Development diff --git a/.cursor/rules/extension-testing.mdc b/.cursor/rules/extension-testing.mdc index 97692657289b..e5143336f38c 100644 --- a/.cursor/rules/extension-testing.mdc +++ b/.cursor/rules/extension-testing.mdc @@ -2,6 +2,20 @@ description: Testing guidance for NiFi extensions (Processors, Controller Services, Connectors). Covers nifi-mock and TestRunner usage. alwaysApply: false --- + # Extension Testing diff --git a/.cursor/rules/framework-testing.mdc b/.cursor/rules/framework-testing.mdc index d25680b5021d..3343b0d0102e 100644 --- a/.cursor/rules/framework-testing.mdc +++ b/.cursor/rules/framework-testing.mdc @@ -2,6 +2,20 @@ description: Testing guidance for NiFi framework code (not extensions). Covers when to use unit, integration, and system tests for framework classes. alwaysApply: false --- + # Framework Testing diff --git a/.cursor/rules/persona.mdc b/.cursor/rules/persona.mdc index 3e4a66ed6455..d741efe5d992 100644 --- a/.cursor/rules/persona.mdc +++ b/.cursor/rules/persona.mdc @@ -2,6 +2,20 @@ description: AI persona and general approach for working on the Apache NiFi codebase alwaysApply: true --- + # AI Persona diff --git a/.cursor/rules/testing-standards.mdc b/.cursor/rules/testing-standards.mdc index 6cb98f4853c4..676b099dd70e 100644 --- a/.cursor/rules/testing-standards.mdc +++ b/.cursor/rules/testing-standards.mdc @@ -3,6 +3,20 @@ description: Shared test coding conventions for all NiFi automated tests (framew globs: "**/src/test/**/*.java" alwaysApply: false --- + # Testing Standards diff --git a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/pom.xml b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/pom.xml index 88ab60703157..d227ec9e68a8 100644 --- a/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/pom.xml +++ b/nifi-connector-mock-bundle/nifi-connector-mock-test-bundle/nifi-connector-mock-integration-tests/pom.xml @@ -112,7 +112,7 @@ copy-nar - generate-resources + package copy-dependencies diff --git a/nifi-connector-mock-bundle/nifi-connector-mock/pom.xml b/nifi-connector-mock-bundle/nifi-connector-mock/pom.xml index e32af9beacc9..ef7d8adb538e 100644 --- a/nifi-connector-mock-bundle/nifi-connector-mock/pom.xml +++ b/nifi-connector-mock-bundle/nifi-connector-mock/pom.xml @@ -130,7 +130,7 @@ copy-nar - generate-resources + package copy-dependencies diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Connection.md b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Connection.md index 1621b370616a..a84b962d1962 100644 --- a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Connection.md +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Connection.md @@ -1,3 +1,17 @@ + # Kafka Connection Configuration This step configures the connection to your Apache Kafka cluster. diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Topics.md b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Topics.md index 2e2739676c13..aa511e6c4a16 100644 --- a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Topics.md +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/Kafka_Topics.md @@ -1,3 +1,17 @@ + # Kafka Topics Configuration This step configures which Kafka topics to consume from and how to consume them. diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/S3_Configuration.md b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/S3_Configuration.md index fcd6092d4344..ea3391828c10 100644 --- a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/S3_Configuration.md +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-connector/src/main/resources/docs/org.apache.nifi.connectors.kafkas3.KafkaToS3/steps/S3_Configuration.md @@ -1,3 +1,17 @@ + # S3 Configuration This step configures the connection to Amazon S3 or an S3-compatible storage system. diff --git a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-integration-tests/pom.xml b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-integration-tests/pom.xml index 628f7b5fed63..70f12a0652c1 100644 --- a/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-integration-tests/pom.xml +++ b/nifi-connectors/nifi-kafka-to-s3-bundle/nifi-kafka-to-s3-integration-tests/pom.xml @@ -222,7 +222,7 @@ copy-nar - generate-resources + package copy-dependencies From a8a99c18e32cf2df10a83c4097b5f529ba60c3c4 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Wed, 11 Mar 2026 17:50:47 -0400 Subject: [PATCH 3/4] NIFI-15258: Ensure that components wihtin a Connector are periodically validated --- .../components/connector/ConnectorNode.java | 8 ++ .../connector/StandardConnectorNode.java | 13 +++ .../validation/TriggerValidationTask.java | 5 + .../ConnectorProcessorValidationIT.java | 108 ++++++++++++++++++ 4 files changed, 134 insertions(+) create mode 100644 nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorProcessorValidationIT.java diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorNode.java index 93e90c1adb7a..bec52c287547 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/components/connector/ConnectorNode.java @@ -25,6 +25,7 @@ import org.apache.nifi.components.VersionedComponent; import org.apache.nifi.components.validation.ValidationState; import org.apache.nifi.components.validation.ValidationStatus; +import org.apache.nifi.components.validation.ValidationTrigger; import org.apache.nifi.connectable.FlowFileTransferCounts; import org.apache.nifi.engine.FlowEngine; import org.apache.nifi.flow.Bundle; @@ -149,6 +150,13 @@ public interface ConnectorNode extends ComponentAuthorizable, VersionedComponent */ ValidationState performValidation(); + /** + * Triggers validation of all processors and controller services within the Connector's managed ProcessGroup. + * + * @param validationTrigger the ValidationTrigger to use for triggering component validation + */ + void validateComponents(ValidationTrigger validationTrigger); + /** * Returns the current validation status of the connector. * @return the current ValidationStatus diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorNode.java index d819554f3775..d2f53dcdd43a 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/connector/StandardConnectorNode.java @@ -32,6 +32,7 @@ import org.apache.nifi.components.validation.DisabledServiceValidationResult; import org.apache.nifi.components.validation.ValidationState; import org.apache.nifi.components.validation.ValidationStatus; +import org.apache.nifi.components.validation.ValidationTrigger; import org.apache.nifi.connectable.FlowFileActivity; import org.apache.nifi.connectable.FlowFileTransferCounts; import org.apache.nifi.controller.ProcessorNode; @@ -1408,6 +1409,18 @@ public ValidationState performValidation() { } } + @Override + public void validateComponents(final ValidationTrigger validationTrigger) { + final ProcessGroup managedGroup = activeFlowContext.getManagedProcessGroup(); + + for (final ProcessorNode processor : managedGroup.findAllProcessors()) { + validationTrigger.trigger(processor); + } + + for (final ControllerServiceNode service : managedGroup.findAllControllerServices()) { + validationTrigger.trigger(service); + } + } private void validateManagedFlowComponents(final List results) { final ProcessGroup managedProcessGroup = activeFlowContext.getManagedProcessGroup(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/validation/TriggerValidationTask.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/validation/TriggerValidationTask.java index 69c32bf74687..d05033a8b4d3 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/validation/TriggerValidationTask.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/components/validation/TriggerValidationTask.java @@ -17,6 +17,7 @@ package org.apache.nifi.components.validation; +import org.apache.nifi.components.connector.ConnectorNode; import org.apache.nifi.controller.ComponentNode; import org.apache.nifi.controller.flow.FlowManager; import org.slf4j.Logger; @@ -61,6 +62,10 @@ public void run() { for (final ComponentNode node : flowManager.getAllFlowRegistryClients()) { validationTrigger.trigger(node); } + + for (final ConnectorNode connector : flowManager.getAllConnectors()) { + connector.validateComponents(validationTrigger); + } } catch (final Throwable t) { logger.error("Encountered unexpected error when attempting to validate components", t); } diff --git a/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorProcessorValidationIT.java b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorProcessorValidationIT.java new file mode 100644 index 000000000000..e140d883ca3f --- /dev/null +++ b/nifi-system-tests/nifi-system-test-suite/src/test/java/org/apache/nifi/tests/system/connectors/ConnectorProcessorValidationIT.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.tests.system.connectors; + +import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.tests.system.NiFiSystemIT; +import org.apache.nifi.toolkit.client.NiFiClientException; +import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.api.dto.flow.FlowDTO; +import org.apache.nifi.web.api.entity.ConnectorEntity; +import org.apache.nifi.web.api.entity.ProcessGroupEntity; +import org.apache.nifi.web.api.entity.ProcessGroupFlowEntity; +import org.apache.nifi.web.api.entity.ProcessorEntity; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Verifies that processors within a Connector's managed ProcessGroup complete validation + * after a NiFi restart. Reproduces the issue where connector-internal processors get stuck + * in VALIDATING state because TriggerValidationTask only walks the root group hierarchy, + * and connector-managed groups are not descendants of the root group. + */ +public class ConnectorProcessorValidationIT extends NiFiSystemIT { + + private static final Logger logger = LoggerFactory.getLogger(ConnectorProcessorValidationIT.class); + + @Test + public void testStoppedConnectorProcessorsValidateAfterRestart() throws NiFiClientException, IOException, InterruptedException { + final ConnectorEntity connector = getClientUtil().createConnector("ComponentLifecycleConnector"); + final String connectorId = connector.getId(); + + getClientUtil().applyConnectorUpdate(connector); + getClientUtil().waitForValidConnector(connectorId); + waitForAllConnectorProcessorsValidated(connectorId); + logger.info("All connector processors validated before restart"); + + getNiFiInstance().stop(); + getNiFiInstance().start(); + + getClientUtil().waitForValidConnector(connectorId); + waitForAllConnectorProcessorsValidated(connectorId); + logger.info("All connector processors validated after restart"); + } + + private void waitForAllConnectorProcessorsValidated(final String connectorId) throws InterruptedException { + waitFor(() -> { + try { + final List stuckProcessors = getValidatingProcessors(connectorId, null); + if (!stuckProcessors.isEmpty()) { + logger.debug("{} connector processor(s) still in VALIDATING state: {}", + stuckProcessors.size(), stuckProcessors.stream().map(ProcessorDTO::getName).toList()); + return false; + } + + return true; + } catch (final Exception e) { + logger.debug("Failed to retrieve connector flow for validation check", e); + return false; + } + }); + } + + private List getValidatingProcessors(final String connectorId, final String groupId) throws NiFiClientException, IOException { + final ProcessGroupFlowEntity flowEntity = (groupId == null) + ? getNifiClient().getConnectorClient().getFlow(connectorId) + : getNifiClient().getConnectorClient().getFlow(connectorId, groupId); + + final FlowDTO flowDto = flowEntity.getProcessGroupFlow().getFlow(); + final List matching = new ArrayList<>(); + + for (final ProcessorEntity processorEntity : flowDto.getProcessors()) { + final ProcessorDTO dto = processorEntity.getComponent(); + if (ScheduledState.DISABLED.name().equals(dto.getState())) { + continue; + } + + if (ProcessorDTO.VALIDATING.equalsIgnoreCase(dto.getValidationStatus())) { + matching.add(dto); + } + } + + for (final ProcessGroupEntity childGroupEntity : flowDto.getProcessGroups()) { + matching.addAll(getValidatingProcessors(connectorId, childGroupEntity.getId())); + } + + return matching; + } +} From 2209d96b5b8d0dd993c08f7fcec633ea9bd0c0c3 Mon Sep 17 00:00:00 2001 From: Matt Gilman Date: Mon, 16 Mar 2026 10:32:43 -0400 Subject: [PATCH 4/4] NIFI-15714: Updating the logic to locate components within Connectors when authorizing and populating provenance events. (#11005) This closes #11005 --- .../nifi/controller/FlowController.java | 56 ++++ ...StandardProvenanceAuthorizableFactory.java | 6 +- ...dardProvenanceAuthorizableFactoryTest.java | 242 ++++++++++++++++++ .../nifi/web/controller/ControllerFacade.java | 15 +- .../web/controller/ControllerFacadeTest.java | 227 +++++++++++++++- 5 files changed, 532 insertions(+), 14 deletions(-) create mode 100644 nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/provenance/StandardProvenanceAuthorizableFactoryTest.java diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index e83b60bde9d6..7051bab362a8 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -51,10 +51,12 @@ import org.apache.nifi.components.ClassLoaderAwarePythonBridge; import org.apache.nifi.components.connector.ConnectorConfigurationProvider; import org.apache.nifi.components.connector.ConnectorConfigurationProviderInitializationContext; +import org.apache.nifi.components.connector.ConnectorNode; import org.apache.nifi.components.connector.ConnectorRepository; import org.apache.nifi.components.connector.ConnectorRepositoryInitializationContext; import org.apache.nifi.components.connector.ConnectorRequestReplicator; import org.apache.nifi.components.connector.ConnectorValidationTrigger; +import org.apache.nifi.components.connector.FrameworkFlowContext; import org.apache.nifi.components.connector.StandardConnectorConfigurationProviderInitializationContext; import org.apache.nifi.components.connector.StandardConnectorRepoInitializationContext; import org.apache.nifi.components.connector.StandardConnectorRepository; @@ -1023,6 +1025,60 @@ public ConnectorRepository getConnectorRepository() { return connectorRepository; } + /** + * Finds a Connection by ID, searching both the root process group hierarchy + * and all connector-managed process groups. + * + * @param connectionId the connection identifier + * @return the Connection, or null if not found + */ + public Connection findConnectionIncludingConnectorManaged(final String connectionId) { + final Connection connection = flowManager.getRootGroup().findConnection(connectionId); + if (connection != null) { + return connection; + } + + for (final ConnectorNode connector : connectorRepository.getConnectors()) { + final FrameworkFlowContext flowContext = connector.getActiveFlowContext(); + if (flowContext != null) { + final ProcessGroup managedGroup = flowContext.getManagedProcessGroup(); + final Connection managedConnection = managedGroup.findConnection(connectionId); + if (managedConnection != null) { + return managedConnection; + } + } + } + + return null; + } + + /** + * Finds a RemoteGroupPort by ID, searching both the root process group hierarchy + * and all connector-managed process groups. + * + * @param remoteGroupPortId the remote group port identifier + * @return the RemoteGroupPort, or null if not found + */ + public RemoteGroupPort findRemoteGroupPortIncludingConnectorManaged(final String remoteGroupPortId) { + final RemoteGroupPort remoteGroupPort = flowManager.getRootGroup().findRemoteGroupPort(remoteGroupPortId); + if (remoteGroupPort != null) { + return remoteGroupPort; + } + + for (final ConnectorNode connector : connectorRepository.getConnectors()) { + final FrameworkFlowContext flowContext = connector.getActiveFlowContext(); + if (flowContext != null) { + final ProcessGroup managedGroup = flowContext.getManagedProcessGroup(); + final RemoteGroupPort managedPort = managedGroup.findRemoteGroupPort(remoteGroupPortId); + if (managedPort != null) { + return managedPort; + } + } + } + + return null; + } + private PythonBridge createPythonBridge(final NiFiProperties nifiProperties, final ControllerServiceProvider serviceProvider) { final String pythonCommand = nifiProperties.getProperty(NiFiProperties.PYTHON_COMMAND); if (pythonCommand == null) { diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/provenance/StandardProvenanceAuthorizableFactory.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/provenance/StandardProvenanceAuthorizableFactory.java index bcc06a33934b..0a2d3387a697 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/provenance/StandardProvenanceAuthorizableFactory.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/provenance/StandardProvenanceAuthorizableFactory.java @@ -50,7 +50,7 @@ public Authorizable createLocalDataAuthorizable(final String componentId) { final Connectable connectable = flowManager.findConnectable(componentId); if (connectable == null) { // if the component id is not a connectable then consider a connection - final Connection connection = flowManager.getRootGroup().findConnection(componentId); + final Connection connection = flowController.findConnectionIncludingConnectorManaged(componentId); if (connection == null) { throw new ResourceNotFoundException("The component that generated this event is no longer part of the data flow."); @@ -70,7 +70,7 @@ public Authorizable createLocalDataAuthorizable(final String componentId) { public Authorizable createRemoteDataAuthorizable(String remoteGroupPortId) { final DataAuthorizable authorizable; - final RemoteGroupPort remoteGroupPort = flowController.getFlowManager().getRootGroup().findRemoteGroupPort(remoteGroupPortId); + final RemoteGroupPort remoteGroupPort = flowController.findRemoteGroupPortIncludingConnectorManaged(remoteGroupPortId); if (remoteGroupPort == null) { throw new ResourceNotFoundException("The component that generated this event is no longer part of the data flow."); } else { @@ -98,7 +98,7 @@ public Authorizable createProvenanceDataAuthorizable(String componentId) { final Connectable connectable = flowManager.findConnectable(componentId); if (connectable == null) { // if the component id is not a connectable then consider a connection - final Connection connection = flowManager.getRootGroup().findConnection(componentId); + final Connection connection = flowController.findConnectionIncludingConnectorManaged(componentId); if (connection == null) { throw new ResourceNotFoundException("The component that generated this event is no longer part of the data flow."); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/provenance/StandardProvenanceAuthorizableFactoryTest.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/provenance/StandardProvenanceAuthorizableFactoryTest.java new file mode 100644 index 000000000000..5692c82fd90b --- /dev/null +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/provenance/StandardProvenanceAuthorizableFactoryTest.java @@ -0,0 +1,242 @@ +/* + * 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.provenance; + +import org.apache.nifi.authorization.resource.Authorizable; +import org.apache.nifi.authorization.resource.DataAuthorizable; +import org.apache.nifi.authorization.resource.EnforcePolicyPermissionsThroughBaseResource; +import org.apache.nifi.authorization.resource.ProvenanceDataAuthorizable; +import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.connectable.Connection; +import org.apache.nifi.controller.FlowController; +import org.apache.nifi.controller.flow.FlowManager; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.groups.RemoteProcessGroup; +import org.apache.nifi.remote.RemoteGroupPort; +import org.apache.nifi.web.ResourceNotFoundException; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) +class StandardProvenanceAuthorizableFactoryTest { + + private StandardProvenanceAuthorizableFactory factory; + + @Mock + private FlowController flowController; + + @Mock + private FlowManager flowManager; + + @Mock + private ProcessGroup rootGroup; + + @Mock + private Connection rootConnection; + + @Mock + private Connection connectorConnection; + + @Mock + private Connectable connectable; + + @Mock + private Connectable connectionSource; + + @Mock + private Connectable connectorConnectionSource; + + @Mock + private RemoteProcessGroup remoteProcessGroup; + + private static final String ROOT_GROUP_ID = "root-group-id"; + private static final String CONNECTABLE_ID = "connectable-id"; + private static final String ROOT_CONNECTION_ID = "root-connection-id"; + private static final String CONNECTOR_CONNECTION_ID = "connector-connection-id"; + private static final String ROOT_REMOTE_PORT_ID = "root-remote-port-id"; + private static final String CONNECTOR_REMOTE_PORT_ID = "connector-remote-port-id"; + private static final String NON_EXISTENT_ID = "non-existent-id"; + + @BeforeEach + void setUp() { + factory = new StandardProvenanceAuthorizableFactory(flowController); + + when(flowController.getFlowManager()).thenReturn(flowManager); + when(flowManager.getRootGroup()).thenReturn(rootGroup); + when(flowManager.getRootGroupId()).thenReturn(ROOT_GROUP_ID); + + // Connectable lookups (flat map, works for all components) + when(flowManager.findConnectable(CONNECTABLE_ID)).thenReturn(connectable); + when(flowManager.findConnectable(ROOT_CONNECTION_ID)).thenReturn(null); + when(flowManager.findConnectable(CONNECTOR_CONNECTION_ID)).thenReturn(null); + when(flowManager.findConnectable(NON_EXISTENT_ID)).thenReturn(null); + + // Connection lookups via FlowController (includes connector-managed PGs) + when(rootConnection.getSource()).thenReturn(connectionSource); + when(connectorConnection.getSource()).thenReturn(connectorConnectionSource); + when(flowController.findConnectionIncludingConnectorManaged(ROOT_CONNECTION_ID)).thenReturn(rootConnection); + when(flowController.findConnectionIncludingConnectorManaged(CONNECTOR_CONNECTION_ID)).thenReturn(connectorConnection); + when(flowController.findConnectionIncludingConnectorManaged(NON_EXISTENT_ID)).thenReturn(null); + + // Remote group port lookups via FlowController + when(flowController.findRemoteGroupPortIncludingConnectorManaged(NON_EXISTENT_ID)).thenReturn(null); + } + + @Nested + class CreateLocalDataAuthorizable { + + @Test + void testRootGroupIdReturnsDataAuthorizableForRootGroup() { + final Authorizable result = factory.createLocalDataAuthorizable(ROOT_GROUP_ID); + + assertInstanceOf(DataAuthorizable.class, result); + assertEquals(rootGroup, ((EnforcePolicyPermissionsThroughBaseResource) result).getBaseAuthorizable()); + } + + @Test + void testConnectableReturnsDataAuthorizableForConnectable() { + final Authorizable result = factory.createLocalDataAuthorizable(CONNECTABLE_ID); + + assertInstanceOf(DataAuthorizable.class, result); + assertEquals(connectable, ((EnforcePolicyPermissionsThroughBaseResource) result).getBaseAuthorizable()); + } + + @Test + void testConnectionInRootGroupReturnsDataAuthorizableForSource() { + final Authorizable result = factory.createLocalDataAuthorizable(ROOT_CONNECTION_ID); + + assertInstanceOf(DataAuthorizable.class, result); + assertEquals(connectionSource, ((EnforcePolicyPermissionsThroughBaseResource) result).getBaseAuthorizable()); + } + + @Test + void testConnectionInConnectorManagedGroupReturnsDataAuthorizableForSource() { + final Authorizable result = factory.createLocalDataAuthorizable(CONNECTOR_CONNECTION_ID); + + assertInstanceOf(DataAuthorizable.class, result); + assertEquals(connectorConnectionSource, ((EnforcePolicyPermissionsThroughBaseResource) result).getBaseAuthorizable()); + } + + @Test + void testNonExistentComponentThrowsResourceNotFoundException() { + assertThrows(ResourceNotFoundException.class, () -> + factory.createLocalDataAuthorizable(NON_EXISTENT_ID) + ); + } + } + + @Nested + class CreateProvenanceDataAuthorizable { + + @Test + void testRootGroupIdReturnsProvenanceDataAuthorizableForRootGroup() { + final Authorizable result = factory.createProvenanceDataAuthorizable(ROOT_GROUP_ID); + + assertInstanceOf(ProvenanceDataAuthorizable.class, result); + assertEquals(rootGroup, ((EnforcePolicyPermissionsThroughBaseResource) result).getBaseAuthorizable()); + } + + @Test + void testConnectableReturnsProvenanceDataAuthorizableForConnectable() { + final Authorizable result = factory.createProvenanceDataAuthorizable(CONNECTABLE_ID); + + assertInstanceOf(ProvenanceDataAuthorizable.class, result); + assertEquals(connectable, ((EnforcePolicyPermissionsThroughBaseResource) result).getBaseAuthorizable()); + } + + @Test + void testConnectionInRootGroupReturnsProvenanceDataAuthorizableForSource() { + final Authorizable result = factory.createProvenanceDataAuthorizable(ROOT_CONNECTION_ID); + + assertInstanceOf(ProvenanceDataAuthorizable.class, result); + assertEquals(connectionSource, ((EnforcePolicyPermissionsThroughBaseResource) result).getBaseAuthorizable()); + } + + @Test + void testConnectionInConnectorManagedGroupReturnsProvenanceDataAuthorizableForSource() { + final Authorizable result = factory.createProvenanceDataAuthorizable(CONNECTOR_CONNECTION_ID); + + assertInstanceOf(ProvenanceDataAuthorizable.class, result); + assertEquals(connectorConnectionSource, ((EnforcePolicyPermissionsThroughBaseResource) result).getBaseAuthorizable()); + } + + @Test + void testNonExistentComponentThrowsResourceNotFoundException() { + assertThrows(ResourceNotFoundException.class, () -> + factory.createProvenanceDataAuthorizable(NON_EXISTENT_ID) + ); + } + } + + @Nested + class CreateRemoteDataAuthorizable { + + private RemoteGroupPort rootRemoteGroupPort; + private RemoteGroupPort connectorRemoteGroupPort; + + @Mock + private RemoteProcessGroup connectorRemoteProcessGroup; + + @BeforeEach + void setUpRemotePorts() { + rootRemoteGroupPort = mock(RemoteGroupPort.class); + connectorRemoteGroupPort = mock(RemoteGroupPort.class); + + when(flowController.findRemoteGroupPortIncludingConnectorManaged(ROOT_REMOTE_PORT_ID)).thenReturn(rootRemoteGroupPort); + when(rootRemoteGroupPort.getRemoteProcessGroup()).thenReturn(remoteProcessGroup); + + when(flowController.findRemoteGroupPortIncludingConnectorManaged(CONNECTOR_REMOTE_PORT_ID)).thenReturn(connectorRemoteGroupPort); + when(connectorRemoteGroupPort.getRemoteProcessGroup()).thenReturn(connectorRemoteProcessGroup); + } + + @Test + void testRemoteGroupPortInRootGroupReturnsDataAuthorizableForRemoteProcessGroup() { + final Authorizable result = factory.createRemoteDataAuthorizable(ROOT_REMOTE_PORT_ID); + + assertInstanceOf(DataAuthorizable.class, result); + assertEquals(remoteProcessGroup, ((EnforcePolicyPermissionsThroughBaseResource) result).getBaseAuthorizable()); + } + + @Test + void testRemoteGroupPortInConnectorManagedGroupReturnsDataAuthorizableForRemoteProcessGroup() { + final Authorizable result = factory.createRemoteDataAuthorizable(CONNECTOR_REMOTE_PORT_ID); + + assertInstanceOf(DataAuthorizable.class, result); + assertEquals(connectorRemoteProcessGroup, ((EnforcePolicyPermissionsThroughBaseResource) result).getBaseAuthorizable()); + } + + @Test + void testNonExistentRemoteGroupPortThrowsResourceNotFoundException() { + assertThrows(ResourceNotFoundException.class, () -> + factory.createRemoteDataAuthorizable(NON_EXISTENT_ID) + ); + } + } +} diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java index feb9a8099308..e957880e5cd2 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java @@ -1859,7 +1859,6 @@ private ProvenanceEventDTO createProvenanceEventDto(final ProvenanceEventRecord private void setComponentDetails(final ProvenanceEventDTO dto) { final NiFiUser user = NiFiUserUtils.getNiFiUser(); - final ProcessGroup root = getRootGroup(); final Connectable connectable = findLocalConnectable(dto.getComponentId()); if (connectable != null) { @@ -1879,15 +1878,11 @@ private void setComponentDetails(final ProvenanceEventDTO dto) { return; } - final RemoteGroupPort remoteGroupPort = root.findRemoteGroupPort(dto.getComponentId()); + final RemoteGroupPort remoteGroupPort = flowController.findRemoteGroupPortIncludingConnectorManaged(dto.getComponentId()); if (remoteGroupPort != null) { - final String remoteGroupPortGroupId = remoteGroupPort.getProcessGroupIdentifier(); - dto.setGroupId(remoteGroupPortGroupId); - - final ProcessGroup remotePortGroup = root.findProcessGroup(remoteGroupPortGroupId); - if (remotePortGroup != null) { - remotePortGroup.getConnectorIdentifier().ifPresent(dto::setConnectorId); - } + final ProcessGroup remotePortGroup = remoteGroupPort.getProcessGroup(); + dto.setGroupId(remotePortGroup.getIdentifier()); + remotePortGroup.getConnectorIdentifier().ifPresent(dto::setConnectorId); // if the user is approved for this component policy, provide additional details, otherwise override/redact as necessary if (Result.Approved.equals(remoteGroupPort.checkAuthorization(authorizer, RequestAction.READ, user).getResult())) { @@ -1900,7 +1895,7 @@ private void setComponentDetails(final ProvenanceEventDTO dto) { return; } - final Connection connection = root.findConnection(dto.getComponentId()); + final Connection connection = flowController.findConnectionIncludingConnectorManaged(dto.getComponentId()); if (connection != null) { final ProcessGroup connectionGroup = connection.getProcessGroup(); dto.setGroupId(connectionGroup.getIdentifier()); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerFacadeTest.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerFacadeTest.java index 86a231d92c79..ebbdeae4ed65 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerFacadeTest.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerFacadeTest.java @@ -28,6 +28,7 @@ import org.apache.nifi.c2.protocol.component.api.ConnectorDefinition; import org.apache.nifi.c2.protocol.component.api.RuntimeManifest; import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.connectable.Connection; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.flow.FlowManager; import org.apache.nifi.groups.ProcessGroup; @@ -36,6 +37,7 @@ import org.apache.nifi.provenance.ProvenanceEventRecord; import org.apache.nifi.provenance.ProvenanceEventType; import org.apache.nifi.provenance.ProvenanceRepository; +import org.apache.nifi.remote.RemoteGroupPort; import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO; import org.apache.nifi.web.api.dto.search.SearchResultsDTO; import org.apache.nifi.web.search.query.SearchQuery; @@ -235,6 +237,107 @@ public void testGetProvenanceEventConnectorIdIsNullWhenComponentNotManagedByConn assertEquals(componentId, result.getComponentId()); } + @Test + public void testGetProvenanceEventSetsDetailsForConnectionInRootGroup() throws IOException { + final String connectionId = "root-connection-id"; + final String groupId = "connection-group-id"; + final String connectorId = "connector-id"; + final String connectionName = "My Connection"; + + final ControllerFacade facade = createConnectionProvenanceFacade(connectionId, groupId, + Optional.of(connectorId), connectionName); + + final ProvenanceEventDTO result = facade.getProvenanceEvent(1L); + + assertNotNull(result); + assertEquals(groupId, result.getGroupId()); + assertEquals(connectorId, result.getConnectorId()); + assertEquals(connectionName, result.getComponentName()); + } + + @Test + public void testGetProvenanceEventSetsDetailsForConnectionInConnectorManagedGroup() throws IOException { + final String connectionId = "connector-managed-connection-id"; + final String groupId = "managed-group-id"; + final String connectorId = "managed-connector-id"; + final String connectionName = "Managed Connection"; + + final ControllerFacade facade = createConnectionProvenanceFacade(connectionId, groupId, + Optional.of(connectorId), connectionName); + + final ProvenanceEventDTO result = facade.getProvenanceEvent(1L); + + assertNotNull(result); + assertEquals(groupId, result.getGroupId()); + assertEquals(connectorId, result.getConnectorId()); + assertEquals(connectionName, result.getComponentName()); + } + + @Test + public void testGetProvenanceEventSetsDetailsForConnectionWithNoConnectorId() throws IOException { + final String connectionId = "plain-connection-id"; + final String groupId = "plain-group-id"; + + final ControllerFacade facade = createConnectionProvenanceFacade(connectionId, groupId, + Optional.empty(), "Plain Connection"); + + final ProvenanceEventDTO result = facade.getProvenanceEvent(1L); + + assertNotNull(result); + assertEquals(groupId, result.getGroupId()); + assertNull(result.getConnectorId()); + } + + @Test + public void testGetProvenanceEventSetsDetailsForRemoteGroupPort() throws IOException { + final String remotePortId = "remote-port-id"; + final String groupId = "remote-port-group-id"; + final String connectorId = "remote-connector-id"; + final String portName = "Remote Port"; + + final ControllerFacade facade = createRemoteGroupPortProvenanceFacade(remotePortId, groupId, + Optional.of(connectorId), portName); + + final ProvenanceEventDTO result = facade.getProvenanceEvent(1L); + + assertNotNull(result); + assertEquals(groupId, result.getGroupId()); + assertEquals(connectorId, result.getConnectorId()); + assertEquals(portName, result.getComponentName()); + } + + @Test + public void testGetProvenanceEventSetsDetailsForRemoteGroupPortInConnectorManagedGroup() throws IOException { + final String remotePortId = "managed-remote-port-id"; + final String groupId = "managed-remote-group-id"; + final String connectorId = "managed-remote-connector-id"; + final String portName = "Managed Remote Port"; + + final ControllerFacade facade = createRemoteGroupPortProvenanceFacade(remotePortId, groupId, + Optional.of(connectorId), portName); + + final ProvenanceEventDTO result = facade.getProvenanceEvent(1L); + + assertNotNull(result); + assertEquals(groupId, result.getGroupId()); + assertEquals(connectorId, result.getConnectorId()); + assertEquals(portName, result.getComponentName()); + } + + @Test + public void testGetProvenanceEventComponentNotFoundLeavesFieldsUnset() throws IOException { + final String componentId = "unknown-component-id"; + + final ControllerFacade facade = createNotFoundProvenanceFacade(componentId); + + final ProvenanceEventDTO result = facade.getProvenanceEvent(1L); + + assertNotNull(result); + assertNull(result.getGroupId()); + assertNull(result.getConnectorId()); + assertNull(result.getComponentName()); + } + /** * Creates a ControllerFacade wired with mocks sufficient to test provenance event creation * through the connectable branch of setComponentDetails. @@ -271,7 +374,6 @@ private ControllerFacade createProvenanceFacade(final String componentId, final // Mock FlowManager final FlowManager flowManager = mock(FlowManager.class); when(flowManager.findConnectable(componentId)).thenReturn(connectable); - when(flowManager.getRootGroup()).thenReturn(mock(ProcessGroup.class)); // Mock ProvenanceRepository final ProvenanceRepository provenanceRepository = mock(ProvenanceRepository.class); @@ -302,5 +404,128 @@ private ControllerFacade createProvenanceFacade(final String componentId, final return facade; } + + /** + * Creates a ControllerFacade wired with mocks to test provenance event creation + * through the connection branch of setComponentDetails. + */ + private ControllerFacade createConnectionProvenanceFacade(final String connectionId, final String groupId, + final Optional connectorIdentifier, + final String connectionName) throws IOException { + final ProvenanceEventRecord event = createMockProvenanceEvent(connectionId); + + final ProcessGroup connectionGroup = mock(ProcessGroup.class); + when(connectionGroup.getIdentifier()).thenReturn(groupId); + when(connectionGroup.getConnectorIdentifier()).thenReturn(connectorIdentifier); + + final Connection connection = mock(Connection.class); + when(connection.getProcessGroup()).thenReturn(connectionGroup); + when(connection.getName()).thenReturn(connectionName); + when(connection.getRelationships()).thenReturn(List.of()); + when(connection.checkAuthorization(any(), any(), any())).thenReturn(AuthorizationResult.approved()); + + final FlowManager flowManager = mock(FlowManager.class); + when(flowManager.findConnectable(connectionId)).thenReturn(null); + + final FlowController flowController = createMockFlowController(connectionId, event, flowManager); + when(flowController.findRemoteGroupPortIncludingConnectorManaged(connectionId)).thenReturn(null); + when(flowController.findConnectionIncludingConnectorManaged(connectionId)).thenReturn(connection); + + final ControllerFacade facade = new ControllerFacade(); + facade.setFlowController(flowController); + facade.setAuthorizer(mock(Authorizer.class)); + + return facade; + } + + /** + * Creates a ControllerFacade wired with mocks to test provenance event creation + * through the remote group port branch of setComponentDetails. + */ + private ControllerFacade createRemoteGroupPortProvenanceFacade(final String remotePortId, final String groupId, + final Optional connectorIdentifier, + final String portName) throws IOException { + final ProvenanceEventRecord event = createMockProvenanceEvent(remotePortId); + + final ProcessGroup portGroup = mock(ProcessGroup.class); + when(portGroup.getIdentifier()).thenReturn(groupId); + when(portGroup.getConnectorIdentifier()).thenReturn(connectorIdentifier); + + final RemoteGroupPort remoteGroupPort = mock(RemoteGroupPort.class); + when(remoteGroupPort.getProcessGroup()).thenReturn(portGroup); + when(remoteGroupPort.getName()).thenReturn(portName); + when(remoteGroupPort.checkAuthorization(any(), any(), any())).thenReturn(AuthorizationResult.approved()); + + final FlowManager flowManager = mock(FlowManager.class); + when(flowManager.findConnectable(remotePortId)).thenReturn(null); + + final FlowController flowController = createMockFlowController(remotePortId, event, flowManager); + when(flowController.findRemoteGroupPortIncludingConnectorManaged(remotePortId)).thenReturn(remoteGroupPort); + + final ControllerFacade facade = new ControllerFacade(); + facade.setFlowController(flowController); + facade.setAuthorizer(mock(Authorizer.class)); + + return facade; + } + + /** + * Creates a ControllerFacade wired with mocks where the component is not found + * through any branch of setComponentDetails. + */ + private ControllerFacade createNotFoundProvenanceFacade(final String componentId) throws IOException { + final ProvenanceEventRecord event = createMockProvenanceEvent(componentId); + + final FlowManager flowManager = mock(FlowManager.class); + when(flowManager.findConnectable(componentId)).thenReturn(null); + + final FlowController flowController = createMockFlowController(componentId, event, flowManager); + when(flowController.findRemoteGroupPortIncludingConnectorManaged(componentId)).thenReturn(null); + when(flowController.findConnectionIncludingConnectorManaged(componentId)).thenReturn(null); + + final ControllerFacade facade = new ControllerFacade(); + facade.setFlowController(flowController); + facade.setAuthorizer(mock(Authorizer.class)); + + return facade; + } + + private ProvenanceEventRecord createMockProvenanceEvent(final String componentId) { + final ProvenanceEventRecord event = mock(ProvenanceEventRecord.class); + when(event.getEventId()).thenReturn(1L); + when(event.getEventTime()).thenReturn(System.currentTimeMillis()); + when(event.getEventType()).thenReturn(ProvenanceEventType.DROP); + when(event.getFlowFileUuid()).thenReturn("test-flowfile-uuid"); + when(event.getFileSize()).thenReturn(1024L); + when(event.getComponentId()).thenReturn(componentId); + when(event.getComponentType()).thenReturn("Connection"); + when(event.getUpdatedAttributes()).thenReturn(Map.of()); + when(event.getPreviousAttributes()).thenReturn(Map.of()); + when(event.getParentUuids()).thenReturn(List.of()); + when(event.getChildUuids()).thenReturn(List.of()); + when(event.getEventDuration()).thenReturn(-1L); + when(event.getLineageStartDate()).thenReturn(0L); + return event; + } + + private FlowController createMockFlowController(final String componentId, final ProvenanceEventRecord event, + final FlowManager flowManager) throws IOException { + final ProvenanceRepository provenanceRepository = mock(ProvenanceRepository.class); + when(provenanceRepository.getEvent(eq(1L), any(NiFiUser.class))).thenReturn(event); + + final Authorizable dataAuthorizable = mock(Authorizable.class); + when(dataAuthorizable.checkAuthorization(any(), eq(RequestAction.READ), any(NiFiUser.class), any())) + .thenReturn(AuthorizationResult.denied("test")); + + final ProvenanceAuthorizableFactory provenanceAuthorizableFactory = mock(ProvenanceAuthorizableFactory.class); + when(provenanceAuthorizableFactory.createLocalDataAuthorizable(componentId)).thenReturn(dataAuthorizable); + + final FlowController flowController = mock(FlowController.class); + when(flowController.getFlowManager()).thenReturn(flowManager); + when(flowController.getProvenanceRepository()).thenReturn(provenanceRepository); + when(flowController.getProvenanceAuthorizableFactory()).thenReturn(provenanceAuthorizableFactory); + + return flowController; + } }