From 95e04702310e0ae83bb8ca0106c07719ed088591 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Thu, 19 May 2016 10:42:39 -0400 Subject: [PATCH] NIFI-1897: Refactoring to allow requests to be replicated from a node to other nodes --- .../java/org/apache/nifi/web/Revision.java | 1 + nifi-assembly/pom.xml | 29 +- .../org/apache/nifi/util/NiFiProperties.java | 154 +- .../apache/nifi/io/socket/SocketListener.java | 8 +- .../util/StandardProcessorTestRunner.java | 2 +- .../apache/nifi/web/api/dto/ComponentDTO.java | 18 + .../nifi/web/api/entity/ComponentEntity.java | 22 + .../coordination/ClusterCoordinator.java | 81 +- .../heartbeat/HeartbeatMonitor.java | 0 .../coordination/heartbeat/NodeHeartbeat.java | 6 +- .../coordination/node/ClusterRoles.java} | 23 +- .../coordination/node/DisconnectionCode.java | 10 + .../node/NodeConnectionState.java | 8 +- .../node/NodeConnectionStatus.java | 105 +- .../apache/nifi/cluster/event/NodeEvent.java} | 25 +- ...l.java => AbstractNodeProtocolSender.java} | 48 +- ...=> ClusterCoordinationProtocolSender.java} | 13 +- .../cluster/protocol/ComponentRevision.java | 104 + .../cluster/protocol/ConnectionResponse.java | 31 +- .../nifi/cluster/protocol/Heartbeat.java | 14 +- ...erCoordinationProtocolSenderListener.java} | 15 +- .../impl/ClusterServiceDiscovery.java | 180 -- .../protocol/impl/ClusterServiceLocator.java | 228 -- .../impl/ClusterServicesBroadcaster.java | 185 -- .../impl/MulticastProtocolListener.java | 203 -- .../impl/NodeProtocolSenderListener.java | 1 - .../protocol/impl/SocketProtocolListener.java | 17 +- ...ardClusterCoordinationProtocolSender.java} | 70 +- .../message/AdaptedConnectionResponse.java | 30 +- .../jaxb/message/AdaptedHeartbeat.java | 12 +- .../message/AdaptedNodeConnectionStatus.java | 30 + .../message/ConnectionResponseAdapter.java | 5 +- .../jaxb/message/DataFlowAdapter.java | 8 +- .../jaxb/message/HeartbeatAdapter.java | 4 +- .../jaxb/message/JaxbProtocolUtils.java | 1 - .../message/NodeConnectionStatusAdapter.java | 11 +- .../message/NodeStatusChangeMessage.java | 32 +- .../message/ReconnectionRequestMessage.java | 22 + .../MulticastConfigurationFactoryBean.java | 60 - .../nifi-cluster-protocol-context.xml | 48 +- .../ClusterManagerProtocolSenderImplTest.java | 133 - .../impl/ClusterServiceDiscoveryTest.java | 134 - .../impl/ClusterServiceLocatorTest.java | 121 - .../impl/ClusterServicesBroadcasterTest.java | 131 - .../impl/MulticastProtocolListenerTest.java | 168 -- .../impl/NodeProtocolSenderImplTest.java | 168 -- .../jaxb/message/TestJaxbProtocolUtils.java | 68 + .../cluster/client/MulticastTestClient.java | 151 -- .../heartbeat/AbstractHeartbeatMonitor.java | 52 +- .../ClusterProtocolHeartbeatMonitor.java | 50 +- .../heartbeat/CuratorHeartbeatMonitor.java | 376 --- .../heartbeat/StandardNodeHeartbeat.java | 16 +- .../http/StandardHttpResponseMerger.java | 70 +- .../endpoints/AbstractNodeStatusEndpoint.java | 9 +- .../endpoints/AbstractSingleDTOEndpoint.java | 115 + .../AbstractSingleEntityEndpoint.java | 56 +- .../BulletinBoardEndpointMerger.java | 2 +- .../ComponentStateEndpointMerger.java | 2 +- .../ControllerServiceEndpointMerger.java | 2 +- .../ControllerStatusEndpointMerger.java | 9 +- .../endpoints/DropRequestEndpiontMerger.java | 2 +- .../http/endpoints/FlowMerger.java | 265 ++ .../endpoints/FlowSnippetEndpointMerger.java | 14 +- .../ListFlowFilesEndpointMerger.java | 2 +- .../endpoints/ProcessorEndpointMerger.java | 17 +- .../ProvenanceEventEndpointMerger.java | 2 +- .../RemoteProcessGroupEndpointMerger.java | 2 +- .../ReportingTaskEndpointMerger.java | 2 +- .../http/replication/RequestReplicator.java | 26 +- .../ThreadPoolRequestReplicator.java | 137 +- .../node/CuratorNodeProtocolSender.java | 116 + .../node/NodeClusterCoordinator.java | 772 ++++++ ...ClusterNode.java => NodeStateManager.java} | 26 +- .../org/apache/nifi/cluster/event/Event.java | 43 +- .../nifi/cluster/event/EventManager.java | 62 - .../cluster/event/impl/EventManagerImpl.java | 141 -- .../nifi/cluster/flow/ClusterDataFlow.java | 55 - .../apache/nifi/cluster/flow/DataFlowDao.java | 60 - .../flow/DataFlowManagementService.java | 124 - .../nifi/cluster/flow/StaleFlowException.java | 40 - .../cluster/flow/impl/DataFlowDaoImpl.java | 595 ----- .../impl/DataFlowManagementServiceImpl.java | 403 --- .../nifi/cluster/manager/ClusterManager.java | 142 -- .../cluster/manager/HttpClusterManager.java | 129 - .../manager/HttpRequestReplicator.java | 84 - .../cluster/manager/HttpResponseMapper.java | 41 - .../nifi/cluster/manager/NodeResponse.java | 48 +- .../exception/BlockedByFirewallException.java | 2 +- .../IneligiblePrimaryNodeException.java | 39 - .../PrimaryRoleAssignmentException.java | 39 - .../SafeModeMutableRequestException.java | 39 - .../manager/impl/ClusteredEventAccess.java | 146 -- .../impl/ClusteredReportingContext.java | 217 -- .../impl/HttpRequestReplicatorImpl.java | 512 ---- .../manager/impl/HttpResponseMapperImpl.java | 82 - .../manager/impl/WebClusterManager.java | 2159 ----------------- .../impl/WebClusterManagerCoordinator.java | 264 -- .../org/apache/nifi/cluster/node/Node.java | 190 -- ...agerProtocolServiceLocatorFactoryBean.java | 113 - ...leBasedClusterNodeFirewallFactoryBean.java | 2 +- ...hreadPoolRequestReplicatorFactoryBean.java | 77 + .../spring/WebClusterManagerFactoryBean.java | 118 - .../reporting/ClusteredReportingTaskNode.java | 75 - .../nifi-cluster-manager-context.xml | 71 +- .../TestAbstractHeartbeatMonitor.java | 131 +- .../TestCuratorHeartbeatMonitor.java | 355 --- .../TestThreadPoolRequestReplicator.java | 115 +- .../node/TestNodeClusterCoordinator.java | 497 ++++ .../event/impl/EventManagerImplTest.java | 121 - .../DataFlowManagementServiceImplTest.java | 344 --- .../impl/HttpRequestReplicatorImplTest.java | 374 --- .../impl/HttpResponseMapperImplTest.java | 128 - .../org/apache/nifi/lifecycle/LifeCycle.java | 0 .../nifi/lifecycle/LifeCycleException.java | 0 .../lifecycle/LifeCycleStartException.java | 0 .../lifecycle/LifeCycleStopException.java | 0 .../org/apache/nifi/services/FlowService.java | 24 +- .../org/apache/nifi/web/FlowModification.java | 0 .../nifi/web/revision/DeleteRevisionTask.java | 0 .../ExpiredRevisionClaimException.java | 0 .../revision/ReadOnlyRevisionCallback.java | 0 .../nifi/web/revision/RevisionClaim.java | 0 .../nifi/web/revision/RevisionManager.java | 12 + .../nifi/web/revision/RevisionUpdate.java | 0 .../nifi/web/revision/UpdateRevisionTask.java | 0 .../nifi/controller/FlowController.java | 295 +-- .../nifi/controller/StandardFlowService.java | 255 +- .../cluster/ClusterProtocolHeartbeater.java | 3 +- .../nifi/controller/cluster/Heartbeater.java | 4 + .../CuratorLeaderElectionManager.java | 16 +- .../nifi/events/StandardEventReporter.java} | 27 +- .../nifi/groups/StandardProcessGroup.java | 1 + .../StandardXMLFlowConfigurationDAO.java | 5 + .../spring/FlowControllerFactoryBean.java | 42 +- .../StandardFlowServiceFactoryBean.java | 35 +- .../src/main/resources/nifi-context.xml | 8 + .../controller/StandardFlowServiceTest.java | 9 +- .../scheduling/TestProcessorLifecycle.java | 3 +- .../src/main/resources/conf/bootstrap.conf | 10 +- .../src/main/resources/conf/logback.xml | 1 + .../src/main/resources/conf/nifi.properties | 33 +- .../apache/nifi/web/server/JettyServer.java | 51 +- .../apache/nifi/web/NiFiServiceFacade.java | 18 + .../nifi/web/StandardNiFiContentAccess.java | 69 +- .../nifi/web/StandardNiFiServiceFacade.java | 157 +- .../StandardNiFiWebConfigurationContext.java | 101 +- .../nifi/web/StandardNiFiWebContext.java | 46 +- .../nifi/web/api/ApplicationResource.java | 292 ++- .../apache/nifi/web/api/ClusterResource.java | 136 +- .../nifi/web/api/ConnectionResource.java | 80 +- .../nifi/web/api/ControllerResource.java | 102 +- .../web/api/ControllerServiceResource.java | 110 +- .../nifi/web/api/FlowFileQueueResource.java | 129 +- .../org/apache/nifi/web/api/FlowResource.java | 316 +-- .../apache/nifi/web/api/FunnelResource.java | 71 +- .../nifi/web/api/InputPortResource.java | 67 +- .../apache/nifi/web/api/LabelResource.java | 71 +- .../org/apache/nifi/web/api/NodeResource.java | 219 ++ .../nifi/web/api/OutputPortResource.java | 71 +- .../nifi/web/api/ProcessGroupResource.java | 212 +- .../nifi/web/api/ProcessorResource.java | 132 +- .../nifi/web/api/ProvenanceResource.java | 238 +- .../web/api/RemoteProcessGroupResource.java | 83 +- .../nifi/web/api/ReportingTaskResource.java | 94 +- .../apache/nifi/web/api/ResourceResource.java | 45 +- .../nifi/web/api/SiteToSiteResource.java | 38 +- .../apache/nifi/web/api/SnippetResource.java | 60 +- .../web/api/SystemDiagnosticsResource.java | 67 +- .../apache/nifi/web/api/TemplateResource.java | 56 +- .../IneligiblePrimaryNodeExceptionMapper.java | 47 - ...r.java => InterruptedExceptionMapper.java} | 20 +- ...SafeModeMutableRequestExceptionMapper.java | 48 - .../apache/nifi/web/api/dto/DtoFactory.java | 79 +- .../ApplicationStartupContextListener.java | 154 +- .../nifi/web/controller/ControllerFacade.java | 62 +- .../ControllerServiceProviderFactoryBean.java | 19 +- .../OptimisticLockingManagerFactoryBean.java | 67 - .../ReportingTaskProviderFactoryBean.java | 19 +- .../StateManagerProviderFactoryBean.java | 26 +- .../main/resources/nifi-web-api-context.xml | 110 +- .../web/StandardOptimisticLockingManager.java | 149 -- .../web/revision/NaiveRevisionManager.java | 60 +- .../revision/TestNaiveRevisionManager.java | 111 +- 183 files changed, 5245 insertions(+), 12897 deletions(-) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-framework-cluster => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java (64%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-framework-cluster => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java (100%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-framework-cluster => nifi-framework-cluster-protocol}/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/NodeHeartbeat.java (93%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java => nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/ClusterRoles.java} (64%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java => nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/event/NodeEvent.java} (66%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/{impl/NodeProtocolSenderImpl.java => AbstractNodeProtocolSender.java} (73%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/{ClusterManagerProtocolSender.java => ClusterCoordinationProtocolSender.java} (84%) create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ComponentRevision.java rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/{ClusterManagerProtocolSenderListener.java => ClusterCoordinationProtocolSenderListener.java} (84%) delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/{ClusterManagerProtocolSenderImpl.java => StandardClusterCoordinationProtocolSender.java} (80%) delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/jaxb/message/TestJaxbProtocolUtils.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/CuratorHeartbeatMonitor.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleDTOEndpoint.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowMerger.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/CuratorNodeProtocolSender.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/{ClusterNode.java => NodeStateManager.java} (53%) delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManagerCoordinator.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/node/Node.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ThreadPoolRequestReplicatorFactoryBean.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestCuratorHeartbeatMonitor.java create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-framework-core => nifi-framework-core-api}/src/main/java/org/apache/nifi/lifecycle/LifeCycle.java (100%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-framework-core => nifi-framework-core-api}/src/main/java/org/apache/nifi/lifecycle/LifeCycleException.java (100%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-framework-core => nifi-framework-core-api}/src/main/java/org/apache/nifi/lifecycle/LifeCycleStartException.java (100%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-framework-core => nifi-framework-core-api}/src/main/java/org/apache/nifi/lifecycle/LifeCycleStopException.java (100%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-framework-core => nifi-framework-core-api}/src/main/java/org/apache/nifi/services/FlowService.java (83%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-web/nifi-web-optimistic-locking => nifi-framework-core-api}/src/main/java/org/apache/nifi/web/FlowModification.java (100%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-web/nifi-web-optimistic-locking => nifi-framework-core-api}/src/main/java/org/apache/nifi/web/revision/DeleteRevisionTask.java (100%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-web/nifi-web-optimistic-locking => nifi-framework-core-api}/src/main/java/org/apache/nifi/web/revision/ExpiredRevisionClaimException.java (100%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-web/nifi-web-optimistic-locking => nifi-framework-core-api}/src/main/java/org/apache/nifi/web/revision/ReadOnlyRevisionCallback.java (100%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-web/nifi-web-optimistic-locking => nifi-framework-core-api}/src/main/java/org/apache/nifi/web/revision/RevisionClaim.java (100%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-web/nifi-web-optimistic-locking => nifi-framework-core-api}/src/main/java/org/apache/nifi/web/revision/RevisionManager.java (96%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-web/nifi-web-optimistic-locking => nifi-framework-core-api}/src/main/java/org/apache/nifi/web/revision/RevisionUpdate.java (100%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-web/nifi-web-optimistic-locking => nifi-framework-core-api}/src/main/java/org/apache/nifi/web/revision/UpdateRevisionTask.java (100%) rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/{nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterNodes.java => nifi-framework-core/src/main/java/org/apache/nifi/events/StandardEventReporter.java} (52%) create mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/NodeResource.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/IneligiblePrimaryNodeExceptionMapper.java rename nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/{PrimaryRoleAssignmentExceptionMapper.java => InterruptedExceptionMapper.java} (62%) delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/config/SafeModeMutableRequestExceptionMapper.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/spring/OptimisticLockingManagerFactoryBean.java delete mode 100644 nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/StandardOptimisticLockingManager.java diff --git a/nifi-api/src/main/java/org/apache/nifi/web/Revision.java b/nifi-api/src/main/java/org/apache/nifi/web/Revision.java index 4c47dde79232..1308e1b75256 100644 --- a/nifi-api/src/main/java/org/apache/nifi/web/Revision.java +++ b/nifi-api/src/main/java/org/apache/nifi/web/Revision.java @@ -27,6 +27,7 @@ * @Threadsafe */ public class Revision implements Serializable { + private static final long serialVersionUID = 988658790374170022L; /** * the version number diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml index aad25619a278..140db10073cd 100644 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -442,37 +442,18 @@ language governing permissions and limitations under the License. --> 5 sec false - 30 sec - 45 sec - false - - - 500 ms - 3 - 1 sec + 5 sec + 5 sec false - 2 + 10 - - - false - - - - 10 - 30 sec - 30 sec - 10 - 5 sec - 10 - 0 sec - - 1 min + 25 + 15 secs 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 1b560fcb6fe1..e6d5bf8db60a 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 @@ -152,20 +152,14 @@ public class NiFiProperties extends Properties { public static final String CLUSTER_PROTOCOL_IS_SECURE = "nifi.cluster.protocol.is.secure"; public static final String CLUSTER_PROTOCOL_SOCKET_TIMEOUT = "nifi.cluster.protocol.socket.timeout"; public static final String CLUSTER_PROTOCOL_CONNECTION_HANDSHAKE_TIMEOUT = "nifi.cluster.protocol.connection.handshake.timeout"; - public static final String CLUSTER_PROTOCOL_USE_MULTICAST = "nifi.cluster.protocol.use.multicast"; - public static final String CLUSTER_PROTOCOL_MULTICAST_ADDRESS = "nifi.cluster.protocol.multicast.address"; - public static final String CLUSTER_PROTOCOL_MULTICAST_PORT = "nifi.cluster.protocol.multicast.port"; - public static final String CLUSTER_PROTOCOL_MULTICAST_SERVICE_BROADCAST_DELAY = "nifi.cluster.protocol.multicast.service.broadcast.delay"; - public static final String CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS = "nifi.cluster.protocol.multicast.service.locator.attempts"; - public static final String CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY = "nifi.cluster.protocol.multicast.service.locator.attempts.delay"; // cluster node properties public static final String CLUSTER_IS_NODE = "nifi.cluster.is.node"; public static final String CLUSTER_NODE_ADDRESS = "nifi.cluster.node.address"; public static final String CLUSTER_NODE_PROTOCOL_PORT = "nifi.cluster.node.protocol.port"; public static final String CLUSTER_NODE_PROTOCOL_THREADS = "nifi.cluster.node.protocol.threads"; - public static final String CLUSTER_NODE_UNICAST_MANAGER_ADDRESS = "nifi.cluster.node.unicast.manager.address"; - public static final String CLUSTER_NODE_UNICAST_MANAGER_PROTOCOL_PORT = "nifi.cluster.node.unicast.manager.protocol.port"; + public static final String REQUEST_REPLICATION_CLAIM_TIMEOUT = "nifi.cluster.request.replication.claim.timeout"; + public static final String CLUSTER_FIREWALL_FILE = "nifi.cluster.firewall.file"; // zookeeper properties public static final String ZOOKEEPER_CONNECT_STRING = "nifi.zookeeper.connect.string"; @@ -173,20 +167,6 @@ public class NiFiProperties extends Properties { public static final String ZOOKEEPER_SESSION_TIMEOUT = "nifi.zookeeper.session.timeout"; public static final String ZOOKEEPER_ROOT_NODE = "nifi.zookeeper.root.node"; - // cluster manager properties - public static final String CLUSTER_IS_MANAGER = "nifi.cluster.is.manager"; - public static final String CLUSTER_MANAGER_ADDRESS = "nifi.cluster.manager.address"; - public static final String CLUSTER_MANAGER_PROTOCOL_PORT = "nifi.cluster.manager.protocol.port"; - public static final String CLUSTER_MANAGER_NODE_FIREWALL_FILE = "nifi.cluster.manager.node.firewall.file"; - public static final String CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE = "nifi.cluster.manager.node.event.history.size"; - public static final String CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT = "nifi.cluster.manager.node.api.connection.timeout"; - public static final String CLUSTER_MANAGER_NODE_API_READ_TIMEOUT = "nifi.cluster.manager.node.api.read.timeout"; - public static final String CLUSTER_MANAGER_NODE_API_REQUEST_THREADS = "nifi.cluster.manager.node.api.request.threads"; - public static final String CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY = "nifi.cluster.manager.flow.retrieval.delay"; - public static final String CLUSTER_MANAGER_PROTOCOL_THREADS = "nifi.cluster.manager.protocol.threads"; - public static final String CLUSTER_MANAGER_SAFEMODE_DURATION = "nifi.cluster.manager.safemode.duration"; - - public static final String REQUEST_REPLICATION_CLAIM_TIMEOUT = "nifi.cluster.request.replication.claim.timeout"; // kerberos properties public static final String KERBEROS_KRB5_FILE = "nifi.kerberos.krb5.file"; @@ -241,17 +221,7 @@ public class NiFiProperties extends Properties { // cluster node defaults public static final int DEFAULT_CLUSTER_NODE_PROTOCOL_THREADS = 2; - - // cluster manager defaults - public static final int DEFAULT_CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE = 10; - public static final String DEFAULT_CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT = "30 sec"; - public static final String DEFAULT_CLUSTER_MANAGER_NODE_API_READ_TIMEOUT = "30 sec"; - public static final int DEFAULT_CLUSTER_MANAGER_NODE_API_NUM_REQUEST_THREADS = 10; - public static final String DEFAULT_CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY = "5 sec"; - public static final int DEFAULT_CLUSTER_MANAGER_PROTOCOL_THREADS = 10; - public static final String DEFAULT_CLUSTER_MANAGER_SAFEMODE_DURATION = "0 sec"; - - public static final String DEFAULT_REQUEST_REPLICATION_CLAIM_TIMEOUT = "1 min"; + public static final String DEFAULT_REQUEST_REPLICATION_CLAIM_TIMEOUT = "15 secs"; // state management defaults public static final String DEFAULT_STATE_MANAGEMENT_CONFIG_FILE = "conf/state-management.xml"; @@ -652,24 +622,6 @@ public String getClusterProtocolConnectionHandshakeTimeout() { DEFAULT_CLUSTER_PROTOCOL_CONNECTION_HANDSHAKE_TIMEOUT); } - public boolean getClusterProtocolUseMulticast() { - return Boolean.parseBoolean(getProperty(CLUSTER_PROTOCOL_USE_MULTICAST)); - } - - public InetSocketAddress getClusterProtocolMulticastAddress() { - try { - String multicastAddress = getProperty(CLUSTER_PROTOCOL_MULTICAST_ADDRESS); - int multicastPort = Integer.parseInt(getProperty(CLUSTER_PROTOCOL_MULTICAST_PORT)); - return new InetSocketAddress(multicastAddress, multicastPort); - } catch (Exception ex) { - throw new RuntimeException("Invalid multicast address/port due to: " + ex, ex); - } - } - - public String getClusterProtocolMulticastServiceBroadcastDelay() { - return getProperty(CLUSTER_PROTOCOL_MULTICAST_SERVICE_BROADCAST_DELAY); - } - public File getPersistentStateDirectory() { final String dirName = getProperty(PERSISTENT_STATE_DIRECTORY, DEFAULT_PERSISTENT_STATE_DIRECTORY); @@ -680,19 +632,6 @@ public File getPersistentStateDirectory() { return file; } - public int getClusterProtocolMulticastServiceLocatorAttempts() { - try { - return Integer - .parseInt(getProperty(CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS)); - } catch (NumberFormatException nfe) { - return DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS; - } - } - - public String getClusterProtocolMulticastServiceLocatorAttemptsDelay() { - return getProperty(CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY, - DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY); - } // getters for cluster node properties // public boolean isNode() { @@ -728,48 +667,13 @@ public int getClusterNodeProtocolThreads() { } } - public InetSocketAddress getClusterNodeUnicastManagerProtocolAddress() { - try { - String socketAddress = getProperty(CLUSTER_NODE_UNICAST_MANAGER_ADDRESS); - if (StringUtils.isBlank(socketAddress)) { - socketAddress = "localhost"; - } - int socketPort = Integer - .parseInt(getProperty(CLUSTER_NODE_UNICAST_MANAGER_PROTOCOL_PORT)); - return InetSocketAddress.createUnresolved(socketAddress, socketPort); - } catch (Exception ex) { - throw new RuntimeException("Invalid unicast manager address/port due to: " + ex, ex); - } - } - - // getters for cluster manager properties // - public boolean isClusterManager() { - return Boolean.parseBoolean(getProperty(CLUSTER_IS_MANAGER)); - } - public InetSocketAddress getClusterManagerProtocolAddress() { - try { - String socketAddress = getProperty(CLUSTER_MANAGER_ADDRESS); - if (StringUtils.isBlank(socketAddress)) { - socketAddress = "localhost"; - } - int socketPort = getClusterManagerProtocolPort(); - return InetSocketAddress.createUnresolved(socketAddress, socketPort); - } catch (Exception ex) { - throw new RuntimeException("Invalid manager protocol address/port due to: " + ex, ex); - } - } - - public Integer getClusterManagerProtocolPort() { - try { - return Integer.parseInt(getProperty(CLUSTER_MANAGER_PROTOCOL_PORT)); - } catch (NumberFormatException nfe) { - return null; - } + public boolean isClustered() { + return Boolean.parseBoolean(getProperty(CLUSTER_IS_NODE)); } - public File getClusterManagerNodeFirewallFile() { - final String firewallFile = getProperty(CLUSTER_MANAGER_NODE_FIREWALL_FILE); + public File getClusterNodeFirewallFile() { + final String firewallFile = getProperty(CLUSTER_FIREWALL_FILE); if (StringUtils.isBlank(firewallFile)) { return null; } else { @@ -777,50 +681,6 @@ public File getClusterManagerNodeFirewallFile() { } } - public int getClusterManagerNodeEventHistorySize() { - try { - return Integer.parseInt(getProperty(CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE)); - } catch (NumberFormatException nfe) { - return DEFAULT_CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE; - } - } - - public String getClusterManagerNodeApiConnectionTimeout() { - return getProperty(CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT, - DEFAULT_CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT); - } - - public String getClusterManagerNodeApiReadTimeout() { - return getProperty(CLUSTER_MANAGER_NODE_API_READ_TIMEOUT, - DEFAULT_CLUSTER_MANAGER_NODE_API_READ_TIMEOUT); - } - - public int getClusterManagerNodeApiRequestThreads() { - try { - return Integer.parseInt(getProperty(CLUSTER_MANAGER_NODE_API_REQUEST_THREADS)); - } catch (NumberFormatException nfe) { - return DEFAULT_CLUSTER_MANAGER_NODE_API_NUM_REQUEST_THREADS; - } - } - - public String getClusterManagerFlowRetrievalDelay() { - return getProperty(CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY, - DEFAULT_CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY); - } - - public int getClusterManagerProtocolThreads() { - try { - return Integer.parseInt(getProperty(CLUSTER_MANAGER_PROTOCOL_THREADS)); - } catch (NumberFormatException nfe) { - return DEFAULT_CLUSTER_MANAGER_PROTOCOL_THREADS; - } - } - - public String getClusterManagerSafeModeDuration() { - return getProperty(CLUSTER_MANAGER_SAFEMODE_DURATION, - DEFAULT_CLUSTER_MANAGER_SAFEMODE_DURATION); - } - public String getClusterProtocolManagerToNodeApiScheme() { final String isSecureProperty = getProperty(CLUSTER_PROTOCOL_IS_SECURE); if (Boolean.valueOf(isSecureProperty)) { diff --git a/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java b/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java index b509035774df..718a386676a6 100644 --- a/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java +++ b/nifi-commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java @@ -33,8 +33,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -import org.apache.nifi.logging.NiFiLog; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,7 +43,7 @@ public abstract class SocketListener { private static final int DEFAULT_SHUTDOWN_LISTENER_SECONDS = 5; - private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(SocketListener.class)); + private static final Logger logger = LoggerFactory.getLogger(SocketListener.class); private volatile ExecutorService executorService; // volatile to guarantee most current value is visible private volatile ServerSocket serverSocket; // volatile to guarantee most current value is visible private final int numThreads; @@ -96,7 +94,7 @@ public void start() throws IOException { @Override public Thread newThread(final Runnable r) { final Thread newThread = defaultThreadFactory.newThread(r); - newThread.setName("Process NCM Request-" + threadCounter.incrementAndGet()); + newThread.setName("Process Cluster Protocol Request-" + threadCounter.incrementAndGet()); return newThread; } }); @@ -152,6 +150,8 @@ public void run() { }); t.setName("Cluster Socket Listener"); t.start(); + + logger.info("Now listening for connections from nodes on port " + port); } public boolean isRunning() { diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java index 9e87d98b801b..89793538f819 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java @@ -435,7 +435,7 @@ public void enqueue(final byte[] data) { @Override public void enqueue(final String data) { - enqueue(data.getBytes(StandardCharsets.UTF_8), Collections.emptyMap()); + enqueue(data.getBytes(StandardCharsets.UTF_8), Collections. emptyMap()); } @Override diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentDTO.java index ca5e06fa1de2..0ef7fef4e451 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentDTO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ComponentDTO.java @@ -91,4 +91,22 @@ public PositionDTO getPosition() { public void setPosition(final PositionDTO position) { this.position = position; } + + @Override + public int hashCode() { + return id == null ? 187 : 187 * id.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + + if (obj.getClass() != getClass()) { + return false; + } + + return id.equals(((ComponentDTO) obj).getId()); + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentEntity.java index ab9889900c47..2f7798ee7353 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentEntity.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ComponentEntity.java @@ -112,4 +112,26 @@ public AccessPolicyDTO getAccessPolicy() { public void setAccessPolicy(AccessPolicyDTO accessPolicy) { this.accessPolicy = accessPolicy; } + + @Override + public int hashCode() { + return id.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + + if (obj == this) { + return true; + } + + if (obj.getClass() != getClass()) { + return false; + } + + return id.equals(((ComponentEntity) obj).getId()); + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java similarity index 64% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java index fa49a62ba984..7cd9f8068f3b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/ClusterCoordinator.java @@ -24,8 +24,10 @@ import org.apache.nifi.cluster.coordination.node.DisconnectionCode; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; +import org.apache.nifi.cluster.event.NodeEvent; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.reporting.Severity; +import org.apache.nifi.services.FlowService; /** *

@@ -36,11 +38,18 @@ public interface ClusterCoordinator { /** * Sends a request to the node to connect to the cluster. This will immediately - * set the NodeConnectionStatus to DISCONNECTED. + * set the NodeConnectionStatus to CONNECTING. * * @param nodeId the identifier of the node + * @param userDn the DN of the user that requested that the node connect, or null if the action is not user-driven */ - void requestNodeConnect(NodeIdentifier nodeId); + void requestNodeConnect(NodeIdentifier nodeId, String userDn); + + /** + * Notifies the Cluster Coordinator that the NiFi instance is being shutdown so that + * the coordinator is able to perform cleanup of resources + */ + void shutdown(); /** * Indicates that the node has sent a valid heartbeat and should now @@ -71,6 +80,14 @@ public interface ClusterCoordinator { */ void disconnectionRequestedByNode(NodeIdentifier nodeId, DisconnectionCode disconnectionCode, String explanation); + /** + * Removes the given disconnected node from the cluster + * + * @param nodeId the node to remove + * @param userDn the DN of the user requesting that the node be removed + */ + void removeNode(NodeIdentifier nodeId, String userDn); + /** * Returns the current status of the node with the given identifier * @@ -84,10 +101,10 @@ public interface ClusterCoordinator { /** * Returns the identifiers of all nodes that have the given connection state * - * @param state the state + * @param states the states of interest * @return the identifiers of all nodes that have the given connection state */ - Set getNodeIdentifiers(NodeConnectionState state); + Set getNodeIdentifiers(NodeConnectionState... states); /** * Returns a Map of NodeConnectionStatus to all Node Identifiers that have that status. @@ -118,11 +135,12 @@ public interface ClusterCoordinator { void reportEvent(NodeIdentifier nodeId, Severity severity, String event); /** - * Updates the node that is considered the Primary Node + * Updates the roles held by the given node * - * @param nodeId the id of the Primary Node + * @param nodeId the id of the node to update + * @param roles the new roles that the node possesses */ - void setPrimaryNode(NodeIdentifier nodeId); + void updateNodeRoles(NodeIdentifier nodeId, Set roles); /** * Returns the NodeIdentifier that exists that has the given UUID, or null if no NodeIdentifier @@ -133,4 +151,53 @@ public interface ClusterCoordinator { * exists for the given UUID */ NodeIdentifier getNodeIdentifier(String uuid); + + /** + * Returns all of the events that have occurred for the given node + * + * @param nodeId the identifier of the node + * @return all of the events that have occurred for the given node + */ + List getNodeEvents(NodeIdentifier nodeId); + + /** + * @return the identifier of the node that is elected primary, or null if either there is no + * primary or the primary is not known by this node. + */ + NodeIdentifier getPrimaryNode(); + + /** + * Updates the Flow Service to use for obtaining the current flow + * + * @param flowService the flow service to use for obtaining the current flow + */ + void setFlowService(FlowService flowService); + + /** + * Clears the current state of all nodes and replaces them with the values provided in the given map + * + * @param statusMap the new states of all nodes in the cluster + */ + void resetNodeStatuses(Map statusMap); + + /** + * Notifies the Cluster Coordinator of the Node Identifier that the coordinator is currently running on + * + * @param nodeId the ID of the current node + */ + void setLocalNodeIdentifier(NodeIdentifier nodeId); + + /** + * Notifies the Cluster Coordinator whether or not the node is connected to the cluster + * + * @param connected true if the node is connected to a cluster, false otherwise. + */ + void setConnected(boolean connected); + + /** + * Indicates whether or not the node is currently connected to the cluster + * + * @return true if connected, false otherwise + */ + boolean isConnected(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java similarity index 100% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/HeartbeatMonitor.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/NodeHeartbeat.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/NodeHeartbeat.java similarity index 93% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/NodeHeartbeat.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/NodeHeartbeat.java index bd66022d2373..2ddda790121a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/NodeHeartbeat.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/NodeHeartbeat.java @@ -17,6 +17,8 @@ package org.apache.nifi.cluster.coordination.heartbeat; +import java.util.Set; + import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; import org.apache.nifi.cluster.protocol.NodeIdentifier; @@ -38,9 +40,9 @@ public interface NodeHeartbeat { NodeConnectionStatus getConnectionStatus(); /** - * @return true if the node is the Primary Node in the cluster, false otherwise + * @return the set of Roles that the node currently possesses. */ - boolean isPrimary(); + Set getRoles(); /** * @return the number of FlowFiles that are queued up on the node diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/ClusterRoles.java similarity index 64% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/ClusterRoles.java index 166632ca92d2..611faa40f84d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/ClusterRoles.java @@ -14,23 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.cluster.flow; -/** - * Represents the various state of a flow managed by the cluster. - * - * The semantics of the values are: - *

    - *
  • CURRENT - the flow is current
  • - *
  • STALE - the flow is not current, but is eligible to be updated.
  • - *
  • UNKNOWN - the flow is not current and is not eligible to be updated. - *
  • - *
- * - */ -public enum PersistedFlowState { +package org.apache.nifi.cluster.coordination.node; + +public class ClusterRoles { + + public static final String PRIMARY_NODE = "Primary Node"; - CURRENT, - STALE, - UNKNOWN + public static final String CLUSTER_COORDINATOR = "Cluster Coordinator"; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/DisconnectionCode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/DisconnectionCode.java index bd20c3f21672..ae18699c385b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/DisconnectionCode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/DisconnectionCode.java @@ -57,6 +57,16 @@ public enum DisconnectionCode { */ MISMATCHED_FLOWS("Node's Flow did not Match Cluster Flow"), + /** + * Cannot communicate with the node + */ + UNABLE_TO_COMMUNICATE("Unable to Communicate with Node"), + + /** + * Node did not service a request that was replicated to it + */ + FAILED_TO_SERVICE_REQUEST("Failed to Service Request"), + /** * Node is being shut down */ diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionState.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionState.java index 9627b2d7aac6..8d5824f17103 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionState.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionState.java @@ -46,5 +46,11 @@ public enum NodeConnectionState { * A node that is not connected to the cluster. * A DISCONNECTED node can transition to CONNECTING. */ - DISCONNECTED + DISCONNECTED, + + /** + * A NodeConnectionState of REMOVED indicates that the node was removed from the cluster + * and is used in order to notify other nodes in the cluster. + */ + REMOVED; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionStatus.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionStatus.java index cd2a6a7d5d43..4570a241e848 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionStatus.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/coordination/node/NodeConnectionStatus.java @@ -17,8 +17,15 @@ package org.apache.nifi.cluster.coordination.node; +import java.util.Collections; +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; + import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.cluster.protocol.jaxb.message.NodeConnectionStatusAdapter; /** @@ -26,33 +33,55 @@ */ @XmlJavaTypeAdapter(NodeConnectionStatusAdapter.class) public class NodeConnectionStatus { + private static final AtomicLong idGenerator = new AtomicLong(0L); + + private final long updateId; + private final NodeIdentifier nodeId; private final NodeConnectionState state; private final DisconnectionCode disconnectCode; private final String disconnectReason; private final Long connectionRequestTime; + private final Set roles; + + public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state) { + this(nodeId, state, null, null, null, null); + } + + public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final Set roles) { + this(nodeId, state, null, null, null, roles); + } + + public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final long connectionRequestTime) { + this(nodeId, state, null, null, connectionRequestTime, null); + } - public NodeConnectionStatus(final NodeConnectionState state) { - this(state, null, null, null); + public NodeConnectionStatus(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode) { + this(nodeId, NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionCode.name(), null, null); } - public NodeConnectionStatus(final NodeConnectionState state, final long connectionRequestTime) { - this(state, null, null, connectionRequestTime); + public NodeConnectionStatus(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String disconnectionExplanation) { + this(nodeId, NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionExplanation, null, null); } - public NodeConnectionStatus(final DisconnectionCode disconnectionCode) { - this(NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionCode.name(), null); + public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectionCode, final Set roles) { + this(nodeId, state, disconnectionCode, disconnectionCode.name(), null, roles); } - public NodeConnectionStatus(final DisconnectionCode disconnectionCode, final String disconnectionExplanation) { - this(NodeConnectionState.DISCONNECTED, disconnectionCode, disconnectionExplanation, null); + public NodeConnectionStatus(final NodeConnectionStatus status, final Set roles) { + this(status.getNodeIdentifier(), status.getState(), status.getDisconnectCode(), status.getDisconnectReason(), status.getConnectionRequestTime(), roles); } - public NodeConnectionStatus(final NodeConnectionState state, final DisconnectionCode disconnectionCode) { - this(state, disconnectionCode, disconnectionCode.name(), null); + public NodeConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectCode, + final String disconnectReason, final Long connectionRequestTime, final Set roles) { + this(idGenerator.getAndIncrement(), nodeId, state, disconnectCode, disconnectReason, connectionRequestTime, roles); } - public NodeConnectionStatus(final NodeConnectionState state, final DisconnectionCode disconnectCode, final String disconnectReason, final Long connectionRequestTime) { + public NodeConnectionStatus(final long updateId, final NodeIdentifier nodeId, final NodeConnectionState state, final DisconnectionCode disconnectCode, + final String disconnectReason, final Long connectionRequestTime, final Set roles) { + this.updateId = updateId; + this.nodeId = nodeId; this.state = state; + this.roles = roles == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(roles)); if (state == NodeConnectionState.DISCONNECTED && disconnectCode == null) { this.disconnectCode = DisconnectionCode.UNKNOWN; this.disconnectReason = this.disconnectCode.toString(); @@ -61,7 +90,19 @@ public NodeConnectionStatus(final NodeConnectionState state, final Disconnection this.disconnectReason = disconnectReason; } - this.connectionRequestTime = connectionRequestTime; + this.connectionRequestTime = (connectionRequestTime == null && state == NodeConnectionState.CONNECTING) ? Long.valueOf(System.currentTimeMillis()) : connectionRequestTime; + } + + public long getUpdateIdentifier() { + return updateId; + } + + public Set getRoles() { + return roles; + } + + public NodeIdentifier getNodeIdentifier() { + return nodeId; } public NodeConnectionState getState() { @@ -88,7 +129,47 @@ public String toString() { if (state == NodeConnectionState.DISCONNECTED || state == NodeConnectionState.DISCONNECTING) { sb.append(", Disconnect Code=").append(getDisconnectCode()).append(", Disconnect Reason=").append(getDisconnectReason()); } + sb.append(", updateId=").append(getUpdateIdentifier()); sb.append("]"); return sb.toString(); } + + /** + * Updates the ID Generator so that it is at least equal to the given minimum value + * + * @param minimumValue the minimum value that the ID Generator should be set to + */ + static void updateIdGenerator(long minimumValue) { + idGenerator.updateAndGet(curValue -> Math.max(minimumValue, curValue)); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((nodeId == null) ? 0 : nodeId.hashCode()); + result = prime * result + ((roles == null) ? 0 : roles.hashCode()); + result = prime * result + ((state == null) ? 0 : state.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null) { + return false; + } + + if (!(obj instanceof NodeConnectionStatus)) { + return false; + } + + NodeConnectionStatus other = (NodeConnectionStatus) obj; + return Objects.deepEquals(getNodeIdentifier(), other.getNodeIdentifier()) + && Objects.deepEquals(getRoles(), other.getRoles()) + && Objects.deepEquals(getState(), other.getState()); + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/event/NodeEvent.java similarity index 66% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/event/NodeEvent.java index aed86c91ec77..15accb6f8b32 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/event/NodeEvent.java @@ -14,26 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.cluster.flow; -/** - * A base exception for data access exceptions. - * - */ -public class DaoException extends RuntimeException { +package org.apache.nifi.cluster.event; + +import org.apache.nifi.reporting.Severity; - public DaoException() { - } +public interface NodeEvent { + Severity getSeverity(); - public DaoException(String msg) { - super(msg); - } + String getMessage(); - public DaoException(Throwable cause) { - super(cause); - } + String getSource(); - public DaoException(String msg, Throwable cause) { - super(msg, cause); - } + long getTimestamp(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/AbstractNodeProtocolSender.java similarity index 73% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/AbstractNodeProtocolSender.java index 23b120901a06..53bd27d6cb0f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/AbstractNodeProtocolSender.java @@ -14,18 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.cluster.protocol.impl; + +package org.apache.nifi.cluster.protocol; import java.io.IOException; import java.net.InetSocketAddress; import java.net.Socket; import java.security.cert.CertificateException; -import org.apache.nifi.cluster.protocol.NodeProtocolSender; -import org.apache.nifi.cluster.protocol.ProtocolContext; -import org.apache.nifi.cluster.protocol.ProtocolException; -import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; -import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller; -import org.apache.nifi.cluster.protocol.UnknownServiceAddressException; + import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; @@ -33,26 +29,13 @@ import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType; import org.apache.nifi.io.socket.SocketConfiguration; import org.apache.nifi.io.socket.SocketUtils; -import org.apache.nifi.io.socket.multicast.DiscoverableService; import org.apache.nifi.security.util.CertificateUtils; -public class NodeProtocolSenderImpl implements NodeProtocolSender { - +public abstract class AbstractNodeProtocolSender implements NodeProtocolSender { private final SocketConfiguration socketConfiguration; - private final ClusterServiceLocator clusterManagerProtocolServiceLocator; private final ProtocolContext protocolContext; - public NodeProtocolSenderImpl(final ClusterServiceLocator clusterManagerProtocolServiceLocator, - final SocketConfiguration socketConfiguration, final ProtocolContext protocolContext) { - if (clusterManagerProtocolServiceLocator == null) { - throw new IllegalArgumentException("Protocol Service Locator may not be null."); - } else if (socketConfiguration == null) { - throw new IllegalArgumentException("Socket configuration may not be null."); - } else if (protocolContext == null) { - throw new IllegalArgumentException("Protocol Context may not be null."); - } - - this.clusterManagerProtocolServiceLocator = clusterManagerProtocolServiceLocator; + public AbstractNodeProtocolSender(final SocketConfiguration socketConfiguration, final ProtocolContext protocolContext) { this.socketConfiguration = socketConfiguration; this.protocolContext = protocolContext; } @@ -79,7 +62,8 @@ public ConnectionResponseMessage requestConnection(final ConnectionRequestMessag final ProtocolMessageUnmarshaller unmarshaller = protocolContext.createUnmarshaller(); response = unmarshaller.unmarshal(socket.getInputStream()); } catch (final IOException ioe) { - throw new ProtocolException("Failed unmarshalling '" + MessageType.CONNECTION_RESPONSE + "' protocol message due to: " + ioe, ioe); + throw new ProtocolException("Failed unmarshalling '" + MessageType.CONNECTION_RESPONSE + "' protocol message from " + + socket.getRemoteSocketAddress() + " due to: " + ioe, ioe); } if (MessageType.CONNECTION_RESPONSE == response.getType()) { @@ -118,17 +102,17 @@ private String getNCMDN(Socket socket) { } private Socket createSocket() { - // determine the cluster manager's address - final DiscoverableService service = clusterManagerProtocolServiceLocator.getService(); - if (service == null) { - throw new UnknownServiceAddressException("Cluster Manager's service is not known. Verify a cluster manager is running."); - } - + InetSocketAddress socketAddress = null; try { // create a socket - return SocketUtils.createSocket(service.getServiceAddress(), socketConfiguration); + socketAddress = getServiceAddress(); + return SocketUtils.createSocket(socketAddress, socketConfiguration); } catch (final IOException ioe) { - throw new ProtocolException("Failed to create socket due to: " + ioe, ioe); + if (socketAddress == null) { + throw new ProtocolException("Failed to create socket due to: " + ioe, ioe); + } else { + throw new ProtocolException("Failed to create socket to " + socketAddress + " due to: " + ioe, ioe); + } } } @@ -156,4 +140,6 @@ private void sendProtocolMessage(final ProtocolMessage msg, final String hostnam SocketUtils.closeQuietly(socket); } } + + protected abstract InetSocketAddress getServiceAddress() throws IOException; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterCoordinationProtocolSender.java similarity index 84% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterCoordinationProtocolSender.java index 010aed715868..a1af0f8f5cc1 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterCoordinationProtocolSender.java @@ -19,8 +19,6 @@ import java.util.Set; import org.apache.nifi.cluster.protocol.message.DisconnectMessage; -import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; -import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage; import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage; import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage; @@ -30,16 +28,7 @@ * An interface for sending protocol messages from the cluster manager to nodes. * */ -public interface ClusterManagerProtocolSender { - - /** - * Sends a "flow request" message to a node. - * - * @param msg a message - * @return the response - * @throws ProtocolException if communication failed - */ - FlowResponseMessage requestFlow(FlowRequestMessage msg) throws ProtocolException; +public interface ClusterCoordinationProtocolSender { /** * Sends a "reconnection request" message to a node. diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ComponentRevision.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ComponentRevision.java new file mode 100644 index 000000000000..5c5257a00f2e --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ComponentRevision.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.protocol; + +import org.apache.nifi.web.Revision; + +public class ComponentRevision { + private Long version; + private String clientId; + private String componentId; + + public Long getVersion() { + return version; + } + + public void setVersion(Long version) { + this.version = version; + } + + public String getClientId() { + return clientId; + } + + public void setClientId(String clientId) { + this.clientId = clientId; + } + + public String getComponentId() { + return componentId; + } + + public void setComponentId(String componentId) { + this.componentId = componentId; + } + + public Revision toRevision() { + return new Revision(getVersion(), getClientId(), getComponentId()); + } + + public static ComponentRevision fromRevision(final Revision revision) { + final ComponentRevision componentRevision = new ComponentRevision(); + componentRevision.setVersion(revision.getVersion()); + componentRevision.setClientId(revision.getClientId()); + componentRevision.setComponentId(revision.getComponentId()); + return componentRevision; + } + + @Override + public boolean equals(final Object obj) { + if (obj == null) { + return false; + } + if (obj == this) { + return true; + } + + if ((obj instanceof ComponentRevision) == false) { + return false; + } + + ComponentRevision thatRevision = (ComponentRevision) obj; + // ensure that component ID's are the same (including null) + if (thatRevision.getComponentId() == null && getComponentId() != null) { + return false; + } + if (thatRevision.getComponentId() != null && getComponentId() == null) { + return false; + } + if (thatRevision.getComponentId() != null && !thatRevision.getComponentId().equals(getComponentId())) { + return false; + } + + if (this.version != null && this.version.equals(thatRevision.version)) { + return true; + } else { + return clientId != null && !clientId.trim().isEmpty() && clientId.equals(thatRevision.getClientId()); + } + + } + + @Override + public int hashCode() { + int hash = 5; + hash = 59 * hash + (this.componentId != null ? this.componentId.hashCode() : 0); + hash = 59 * hash + (this.version != null ? this.version.hashCode() : 0); + hash = 59 * hash + (this.clientId != null ? this.clientId.hashCode() : 0); + return hash; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java index e6d8cf6f8865..f6c6f8cb2c02 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java @@ -16,8 +16,13 @@ */ package org.apache.nifi.cluster.protocol; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; import org.apache.nifi.cluster.protocol.jaxb.message.ConnectionResponseAdapter; /** @@ -34,15 +39,19 @@ public class ConnectionResponse { private final String rejectionReason; private final int tryLaterSeconds; private final NodeIdentifier nodeIdentifier; - private final StandardDataFlow dataFlow; + private final DataFlow dataFlow; private final Integer managerRemoteInputPort; private final Boolean managerRemoteCommsSecure; private final String instanceId; + private final List nodeStatuses; + private final List componentRevisions; private volatile String clusterManagerDN; - public ConnectionResponse(final NodeIdentifier nodeIdentifier, final StandardDataFlow dataFlow, - final Integer managerRemoteInputPort, final Boolean managerRemoteCommsSecure, final String instanceId) { + public ConnectionResponse(final NodeIdentifier nodeIdentifier, final DataFlow dataFlow, + final Integer managerRemoteInputPort, final Boolean managerRemoteCommsSecure, final String instanceId, + final List nodeStatuses, final List componentRevisions) { + if (nodeIdentifier == null) { throw new IllegalArgumentException("Node identifier may not be empty or null."); } else if (dataFlow == null) { @@ -55,6 +64,8 @@ public ConnectionResponse(final NodeIdentifier nodeIdentifier, final StandardDat this.managerRemoteInputPort = managerRemoteInputPort; this.managerRemoteCommsSecure = managerRemoteCommsSecure; this.instanceId = instanceId; + this.nodeStatuses = Collections.unmodifiableList(new ArrayList<>(nodeStatuses)); + this.componentRevisions = componentRevisions == null ? Collections.emptyList() : Collections.unmodifiableList(new ArrayList<>(componentRevisions)); } public ConnectionResponse(final int tryLaterSeconds) { @@ -68,6 +79,8 @@ public ConnectionResponse(final int tryLaterSeconds) { this.managerRemoteInputPort = null; this.managerRemoteCommsSecure = null; this.instanceId = null; + this.nodeStatuses = null; + this.componentRevisions = null; } private ConnectionResponse(final String rejectionReason) { @@ -78,6 +91,8 @@ private ConnectionResponse(final String rejectionReason) { this.managerRemoteInputPort = null; this.managerRemoteCommsSecure = null; this.instanceId = null; + this.nodeStatuses = null; + this.componentRevisions = null; } public static ConnectionResponse createBlockedByFirewallResponse() { @@ -104,7 +119,7 @@ public int getTryLaterSeconds() { return tryLaterSeconds; } - public StandardDataFlow getDataFlow() { + public DataFlow getDataFlow() { return dataFlow; } @@ -128,6 +143,14 @@ public void setClusterManagerDN(final String dn) { this.clusterManagerDN = dn; } + public List getNodeConnectionStatuses() { + return nodeStatuses; + } + + public List getComponentRevisions() { + return componentRevisions; + } + /** * @return the DN of the NCM, if it is available or null * otherwise diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java index f2b0fde9ecc0..2135f20dc05e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java @@ -16,7 +16,11 @@ */ package org.apache.nifi.cluster.protocol; +import java.util.Collections; import java.util.Date; +import java.util.HashSet; +import java.util.Set; + import javax.xml.bind.annotation.XmlTransient; import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; @@ -31,17 +35,17 @@ public class Heartbeat { private final NodeIdentifier nodeIdentifier; - private final boolean primary; + private final Set roles; private final NodeConnectionStatus connectionStatus; private final long createdTimestamp; private final byte[] payload; - public Heartbeat(final NodeIdentifier nodeIdentifier, final boolean primary, final NodeConnectionStatus connectionStatus, final byte[] payload) { + public Heartbeat(final NodeIdentifier nodeIdentifier, final Set roles, final NodeConnectionStatus connectionStatus, final byte[] payload) { if (nodeIdentifier == null) { throw new IllegalArgumentException("Node Identifier may not be null."); } this.nodeIdentifier = nodeIdentifier; - this.primary = primary; + this.roles = roles == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(roles)); this.connectionStatus = connectionStatus; this.payload = payload; this.createdTimestamp = new Date().getTime(); @@ -55,8 +59,8 @@ public byte[] getPayload() { return payload; } - public boolean isPrimary() { - return primary; + public Set getRoles() { + return roles; } public NodeConnectionStatus getConnectionStatus() { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterCoordinationProtocolSenderListener.java similarity index 84% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterCoordinationProtocolSenderListener.java index 2fc05b01a38e..e97712a7260d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterCoordinationProtocolSenderListener.java @@ -20,14 +20,12 @@ import java.util.Collection; import java.util.Set; -import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender; +import org.apache.nifi.cluster.protocol.ClusterCoordinationProtocolSender; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.cluster.protocol.ProtocolException; import org.apache.nifi.cluster.protocol.ProtocolHandler; import org.apache.nifi.cluster.protocol.ProtocolListener; import org.apache.nifi.cluster.protocol.message.DisconnectMessage; -import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; -import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage; import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage; import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage; @@ -38,13 +36,13 @@ * cluster manager. * */ -public class ClusterManagerProtocolSenderListener implements ClusterManagerProtocolSender, ProtocolListener { +public class ClusterCoordinationProtocolSenderListener implements ClusterCoordinationProtocolSender, ProtocolListener { - private final ClusterManagerProtocolSender sender; + private final ClusterCoordinationProtocolSender sender; private final ProtocolListener listener; - public ClusterManagerProtocolSenderListener(final ClusterManagerProtocolSender sender, final ProtocolListener listener) { + public ClusterCoordinationProtocolSenderListener(final ClusterCoordinationProtocolSender sender, final ProtocolListener listener) { if (sender == null) { throw new IllegalArgumentException("ClusterManagerProtocolSender may not be null."); } else if (listener == null) { @@ -96,11 +94,6 @@ public void setBulletinRepository(final BulletinRepository bulletinRepository) { sender.setBulletinRepository(bulletinRepository); } - @Override - public FlowResponseMessage requestFlow(final FlowRequestMessage msg) throws ProtocolException { - return sender.requestFlow(msg); - } - @Override public ReconnectionResponseMessage requestReconnection(final ReconnectionRequestMessage msg) throws ProtocolException { return sender.requestReconnection(msg); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java deleted file mode 100644 index f808c83ab731..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java +++ /dev/null @@ -1,180 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.protocol.impl; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.Collection; -import java.util.Collections; -import org.apache.nifi.io.socket.multicast.DiscoverableService; -import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; -import org.apache.nifi.io.socket.multicast.MulticastConfiguration; -import org.apache.nifi.io.socket.multicast.MulticastServiceDiscovery; -import org.apache.nifi.reporting.BulletinRepository; - -import org.apache.commons.lang3.StringUtils; -import org.apache.nifi.cluster.protocol.ProtocolContext; -import org.apache.nifi.cluster.protocol.ProtocolException; -import org.apache.nifi.cluster.protocol.ProtocolHandler; -import org.apache.nifi.cluster.protocol.ProtocolListener; -import org.apache.nifi.cluster.protocol.message.ProtocolMessage; -import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType; -import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * An implementation for discovering services by way of "service broadcast" type - * protocol messages over multicast. - * - * The client caller is responsible for starting and stopping the service - * discovery. The instance must be stopped before termination of the JVM to - * ensure proper resource clean-up. - * - */ -public class ClusterServiceDiscovery implements MulticastServiceDiscovery, ProtocolListener { - - private static final Logger logger = LoggerFactory.getLogger(ClusterServiceDiscovery.class); - private final String serviceName; - private final MulticastConfiguration multicastConfiguration; - private final MulticastProtocolListener listener; - private volatile BulletinRepository bulletinRepository; - - /* - * guarded by this - */ - private DiscoverableService service; - - public ClusterServiceDiscovery(final String serviceName, final InetSocketAddress multicastAddress, - final MulticastConfiguration multicastConfiguration, final ProtocolContext protocolContext) { - - if (StringUtils.isBlank(serviceName)) { - throw new IllegalArgumentException("Service name may not be null or empty."); - } else if (multicastAddress == null) { - throw new IllegalArgumentException("Multicast address may not be null."); - } else if (multicastAddress.getAddress().isMulticastAddress() == false) { - throw new IllegalArgumentException("Multicast group must be a Class D address."); - } else if (protocolContext == null) { - throw new IllegalArgumentException("Protocol Context may not be null."); - } else if (multicastConfiguration == null) { - throw new IllegalArgumentException("Multicast configuration may not be null."); - } - - this.serviceName = serviceName; - this.multicastConfiguration = multicastConfiguration; - this.listener = new MulticastProtocolListener(1, multicastAddress, multicastConfiguration, protocolContext); - listener.addHandler(new ClusterManagerServiceBroadcastHandler()); - } - - @Override - public void setBulletinRepository(final BulletinRepository bulletinRepository) { - this.bulletinRepository = bulletinRepository; - } - - @Override - public synchronized DiscoverableService getService() { - return service; - } - - @Override - public InetSocketAddress getMulticastAddress() { - return listener.getMulticastAddress(); - } - - @Override - public Collection getHandlers() { - return Collections.unmodifiableCollection(listener.getHandlers()); - } - - @Override - public void addHandler(ProtocolHandler handler) { - listener.addHandler(handler); - } - - @Override - public boolean removeHandler(ProtocolHandler handler) { - return listener.removeHandler(handler); - } - - @Override - public boolean isRunning() { - return listener.isRunning(); - } - - @Override - public void start() throws IOException { - if (isRunning()) { - throw new IllegalStateException("Instance is already running."); - } - listener.start(); - } - - @Override - public void stop() throws IOException { - if (isRunning() == false) { - throw new IllegalStateException("Instance is already stopped."); - } - listener.stop(); - } - - public String getServiceName() { - return serviceName; - } - - public MulticastConfiguration getMulticastConfiguration() { - return multicastConfiguration; - } - - private class ClusterManagerServiceBroadcastHandler implements ProtocolHandler { - - @Override - public boolean canHandle(final ProtocolMessage msg) { - return MessageType.SERVICE_BROADCAST == msg.getType(); - } - - @Override - public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolException { - synchronized (ClusterServiceDiscovery.this) { - if (canHandle(msg) == false) { - throw new ProtocolException("Handler cannot handle message type: " + msg.getType()); - } else { - final ServiceBroadcastMessage broadcastMsg = (ServiceBroadcastMessage) msg; - if (serviceName.equals(broadcastMsg.getServiceName())) { - final DiscoverableService oldService = service; - if (oldService == null - || broadcastMsg.getAddress().equalsIgnoreCase(oldService.getServiceAddress().getHostName()) == false - || broadcastMsg.getPort() != oldService.getServiceAddress().getPort()) { - service = new DiscoverableServiceImpl(serviceName, InetSocketAddress.createUnresolved(broadcastMsg.getAddress(), broadcastMsg.getPort())); - final InetSocketAddress oldServiceAddress = (oldService == null) ? null : oldService.getServiceAddress(); - logger.info(String.format("Updating cluster service address for '%s' from '%s' to '%s'", - serviceName, prettyPrint(oldServiceAddress), prettyPrint(service.getServiceAddress()))); - } - } - return null; - } - } - } - } - - private String prettyPrint(final InetSocketAddress address) { - if (address == null) { - return "0.0.0.0:0"; - } else { - return address.getHostName() + ":" + address.getPort(); - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java deleted file mode 100644 index a49847f89d37..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java +++ /dev/null @@ -1,228 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.protocol.impl; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.nifi.io.socket.multicast.DiscoverableService; -import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; -import org.apache.nifi.io.socket.multicast.ServiceDiscovery; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Implements the ServiceLocator interface for locating the socket address of a - * cluster service. Depending on configuration, the address may be located using - * service discovery. If using service discovery, then the service methods must - * be used for starting and stopping discovery. - * - * Service discovery may be used in conjunction with a fixed port. In this case, - * the service discovery will yield the service IP/host while the fixed port - * will be used for the port. - * - * Alternatively, the instance may be configured with exact service location, in - * which case, no service discovery occurs and the caller will always receive - * the configured service. - * - */ -public class ClusterServiceLocator implements ServiceDiscovery { - - private static final Logger logger = LoggerFactory.getLogger(ClusterServiceLocator.class); - - private final String serviceName; - - private final ClusterServiceDiscovery serviceDiscovery; - - private final DiscoverableService fixedService; - - private final int fixedServicePort; - - private final AttemptsConfig attemptsConfig = new AttemptsConfig(); - - private final AtomicBoolean running = new AtomicBoolean(false); - - public ClusterServiceLocator(final ClusterServiceDiscovery serviceDiscovery) { - if (serviceDiscovery == null) { - throw new IllegalArgumentException("ClusterServiceDiscovery may not be null."); - } - this.serviceDiscovery = serviceDiscovery; - this.fixedService = null; - this.fixedServicePort = 0; - this.serviceName = serviceDiscovery.getServiceName(); - } - - public ClusterServiceLocator(final ClusterServiceDiscovery serviceDiscovery, final int fixedServicePort) { - if (serviceDiscovery == null) { - throw new IllegalArgumentException("ClusterServiceDiscovery may not be null."); - } - this.serviceDiscovery = serviceDiscovery; - this.fixedService = null; - this.fixedServicePort = fixedServicePort; - this.serviceName = serviceDiscovery.getServiceName(); - } - - public ClusterServiceLocator(final DiscoverableService fixedService) { - if (fixedService == null) { - throw new IllegalArgumentException("Service may not be null."); - } - this.serviceDiscovery = null; - this.fixedService = fixedService; - this.fixedServicePort = 0; - this.serviceName = fixedService.getServiceName(); - } - - @Override - public DiscoverableService getService() { - - final int numAttemptsValue; - final int secondsBetweenAttempts; - synchronized (this) { - numAttemptsValue = attemptsConfig.numAttempts; - secondsBetweenAttempts = attemptsConfig.getTimeBetweenAttempts(); - } - - // try for a configured amount of attempts to retrieve the service address - for (int i = 0; i < numAttemptsValue; i++) { - - if (fixedService != null) { - return fixedService; - } else if (serviceDiscovery != null) { - - final DiscoverableService discoveredService = serviceDiscovery.getService(); - - // if we received an address - if (discoveredService != null) { - // if we were configured with a fixed port, then use the discovered host and fixed port; otherwise use the discovered address - if (fixedServicePort > 0) { - // create service using discovered service name and address with fixed service port - final InetSocketAddress addr = InetSocketAddress.createUnresolved(discoveredService.getServiceAddress().getHostName(), fixedServicePort); - final DiscoverableService result = new DiscoverableServiceImpl(discoveredService.getServiceName(), addr); - return result; - } else { - return discoveredService; - } - } - } - - // could not obtain service address, so sleep a bit - try { - logger.debug(String.format("Locating Cluster Service '%s' Attempt: %d of %d failed. Trying again in %d seconds.", - serviceName, (i + 1), numAttemptsValue, secondsBetweenAttempts)); - Thread.sleep(secondsBetweenAttempts * 1000); - } catch (final InterruptedException ie) { - break; - } - - } - - return null; - } - - public boolean isRunning() { - if (serviceDiscovery != null) { - return serviceDiscovery.isRunning(); - } else { - return running.get(); - } - } - - public void start() throws IOException { - - if (isRunning()) { - throw new IllegalStateException("Instance is already started."); - } - - if (serviceDiscovery != null) { - serviceDiscovery.start(); - } - running.set(true); - } - - public void stop() throws IOException { - - if (isRunning() == false) { - throw new IllegalStateException("Instance is already stopped."); - } - - if (serviceDiscovery != null) { - serviceDiscovery.stop(); - } - running.set(false); - } - - public synchronized void setAttemptsConfig(final AttemptsConfig config) { - if (config == null) { - throw new IllegalArgumentException("Attempts configuration may not be null."); - } - this.attemptsConfig.numAttempts = config.numAttempts; - this.attemptsConfig.timeBetweenAttempts = config.timeBetweenAttempts; - this.attemptsConfig.timeBetweenAttempsUnit = config.timeBetweenAttempsUnit; - } - - public synchronized AttemptsConfig getAttemptsConfig() { - final AttemptsConfig config = new AttemptsConfig(); - config.numAttempts = this.attemptsConfig.numAttempts; - config.timeBetweenAttempts = this.attemptsConfig.timeBetweenAttempts; - config.timeBetweenAttempsUnit = this.attemptsConfig.timeBetweenAttempsUnit; - return config; - } - - public static class AttemptsConfig { - - private int numAttempts = 1; - - private int timeBetweenAttempts = 1; - - private TimeUnit timeBetweenAttempsUnit = TimeUnit.SECONDS; - - public int getNumAttempts() { - return numAttempts; - } - - public void setNumAttempts(int numAttempts) { - if (numAttempts <= 0) { - throw new IllegalArgumentException("Number of attempts must be positive: " + numAttempts); - } - this.numAttempts = numAttempts; - } - - public TimeUnit getTimeBetweenAttemptsUnit() { - return timeBetweenAttempsUnit; - } - - public void setTimeBetweenAttempsUnit(TimeUnit timeBetweenAttempsUnit) { - if (timeBetweenAttempts <= 0) { - throw new IllegalArgumentException("Time between attempts must be positive: " + numAttempts); - } - this.timeBetweenAttempsUnit = timeBetweenAttempsUnit; - } - - public int getTimeBetweenAttempts() { - return timeBetweenAttempts; - } - - public void setTimeBetweenAttempts(int timeBetweenAttempts) { - if (timeBetweenAttempts <= 0) { - throw new IllegalArgumentException("Time between attempts must be positive: " + numAttempts); - } - this.timeBetweenAttempts = timeBetweenAttempts; - } - - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java deleted file mode 100644 index 0bb13d40afd3..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.protocol.impl; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.net.DatagramPacket; -import java.net.InetSocketAddress; -import java.net.MulticastSocket; -import java.util.Collections; -import java.util.Set; -import java.util.Timer; -import java.util.TimerTask; -import java.util.concurrent.CopyOnWriteArraySet; -import java.util.concurrent.TimeUnit; - -import org.apache.nifi.io.socket.multicast.DiscoverableService; -import org.apache.nifi.io.socket.multicast.MulticastConfiguration; -import org.apache.nifi.io.socket.multicast.MulticastServicesBroadcaster; -import org.apache.nifi.io.socket.multicast.MulticastUtils; -import org.apache.nifi.logging.NiFiLog; -import org.apache.nifi.util.FormatUtils; -import org.apache.nifi.cluster.protocol.ProtocolContext; -import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; -import org.apache.nifi.cluster.protocol.message.ProtocolMessage; -import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Broadcasts services used by the clustering software using multicast - * communication. A configurable delay occurs after broadcasting the collection - * of services. - * - * The client caller is responsible for starting and stopping the broadcasting. - * The instance must be stopped before termination of the JVM to ensure proper - * resource clean-up. - * - */ -public class ClusterServicesBroadcaster implements MulticastServicesBroadcaster { - - private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(ClusterServicesBroadcaster.class)); - - private final Set services = new CopyOnWriteArraySet<>(); - - private final InetSocketAddress multicastAddress; - - private final MulticastConfiguration multicastConfiguration; - - private final ProtocolContext protocolContext; - - private final int broadcastDelayMs; - - private Timer broadcaster; - - private MulticastSocket multicastSocket; - - public ClusterServicesBroadcaster(final InetSocketAddress multicastAddress, - final MulticastConfiguration multicastConfiguration, - final ProtocolContext protocolContext, final String broadcastDelay) { - - if (multicastAddress == null) { - throw new IllegalArgumentException("Multicast address may not be null."); - } else if (multicastAddress.getAddress().isMulticastAddress() == false) { - throw new IllegalArgumentException("Multicast group address is not a Class D IP address."); - } else if (protocolContext == null) { - throw new IllegalArgumentException("Protocol Context may not be null."); - } else if (multicastConfiguration == null) { - throw new IllegalArgumentException("Multicast configuration may not be null."); - } - - this.services.addAll(services); - this.multicastAddress = multicastAddress; - this.multicastConfiguration = multicastConfiguration; - this.protocolContext = protocolContext; - this.broadcastDelayMs = (int) FormatUtils.getTimeDuration(broadcastDelay, TimeUnit.MILLISECONDS); - } - - public void start() throws IOException { - - if (isRunning()) { - throw new IllegalStateException("Instance is already started."); - } - - // setup socket - multicastSocket = MulticastUtils.createMulticastSocket(multicastConfiguration); - - // setup broadcaster - broadcaster = new Timer("Cluster Services Broadcaster", /* is daemon */ true); - broadcaster.schedule(new TimerTask() { - @Override - public void run() { - for (final DiscoverableService service : services) { - try { - - final InetSocketAddress serviceAddress = service.getServiceAddress(); - logger.debug(String.format("Broadcasting Cluster Service '%s' at address %s:%d", - service.getServiceName(), serviceAddress.getHostName(), serviceAddress.getPort())); - - // create message - final ServiceBroadcastMessage msg = new ServiceBroadcastMessage(); - msg.setServiceName(service.getServiceName()); - msg.setAddress(serviceAddress.getHostName()); - msg.setPort(serviceAddress.getPort()); - - // marshal message to output stream - final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - marshaller.marshal(msg, baos); - final byte[] packetBytes = baos.toByteArray(); - - // send message - final DatagramPacket packet = new DatagramPacket(packetBytes, packetBytes.length, multicastAddress); - multicastSocket.send(packet); - - } catch (final Exception ex) { - logger.warn(String.format("Cluster Services Broadcaster failed broadcasting service '%s' due to: %s", service.getServiceName(), ex), ex); - } - } - } - }, 0, broadcastDelayMs); - } - - public boolean isRunning() { - return (broadcaster != null); - } - - public void stop() { - - if (isRunning() == false) { - throw new IllegalStateException("Instance is already stopped."); - } - - broadcaster.cancel(); - broadcaster = null; - - // close socket - MulticastUtils.closeQuietly(multicastSocket); - - } - - @Override - public int getBroadcastDelayMs() { - return broadcastDelayMs; - } - - @Override - public Set getServices() { - return Collections.unmodifiableSet(services); - } - - @Override - public InetSocketAddress getMulticastAddress() { - return multicastAddress; - } - - @Override - public boolean addService(final DiscoverableService service) { - return services.add(service); - } - - @Override - public boolean removeService(final String serviceName) { - for (final DiscoverableService service : services) { - if (service.getServiceName().equals(serviceName)) { - return services.remove(service); - } - } - return false; - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java deleted file mode 100644 index 8ca51419fab2..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java +++ /dev/null @@ -1,203 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.protocol.impl; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.net.DatagramPacket; -import java.net.InetSocketAddress; -import java.net.MulticastSocket; -import java.util.Collection; -import java.util.Collections; -import java.util.UUID; -import java.util.concurrent.CopyOnWriteArrayList; -import org.apache.nifi.io.socket.multicast.MulticastConfiguration; -import org.apache.nifi.io.socket.multicast.MulticastListener; -import org.apache.nifi.logging.NiFiLog; -import org.apache.nifi.reporting.Bulletin; -import org.apache.nifi.reporting.BulletinRepository; -import org.apache.nifi.cluster.protocol.ProtocolContext; -import org.apache.nifi.cluster.protocol.ProtocolException; -import org.apache.nifi.cluster.protocol.ProtocolHandler; -import org.apache.nifi.cluster.protocol.ProtocolListener; -import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; -import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller; -import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage; -import org.apache.nifi.cluster.protocol.message.ProtocolMessage; -import org.apache.nifi.events.BulletinFactory; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Implements a listener for protocol messages sent over multicast. If a message - * is of type MulticastProtocolMessage, then the underlying protocol message is - * passed to the handler. If the receiving handler produces a message response, - * then the message is wrapped with a MulticastProtocolMessage before being sent - * to the originator. - * - * The client caller is responsible for starting and stopping the listener. The - * instance must be stopped before termination of the JVM to ensure proper - * resource clean-up. - * - */ -public class MulticastProtocolListener extends MulticastListener implements ProtocolListener { - - private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(MulticastProtocolListener.class)); - - // immutable members - private final Collection handlers = new CopyOnWriteArrayList<>(); - private final String listenerId = UUID.randomUUID().toString(); - private final ProtocolContext protocolContext; - private volatile BulletinRepository bulletinRepository; - - public MulticastProtocolListener( - final int numThreads, - final InetSocketAddress multicastAddress, - final MulticastConfiguration configuration, - final ProtocolContext protocolContext) { - - super(numThreads, multicastAddress, configuration); - - if (protocolContext == null) { - throw new IllegalArgumentException("Protocol Context may not be null."); - } - this.protocolContext = protocolContext; - } - - @Override - public void setBulletinRepository(final BulletinRepository bulletinRepository) { - this.bulletinRepository = bulletinRepository; - } - - @Override - public void start() throws IOException { - - if (super.isRunning()) { - throw new IllegalStateException("Instance is already started."); - } - - super.start(); - - } - - @Override - public void stop() throws IOException { - - if (super.isRunning() == false) { - throw new IllegalStateException("Instance is already stopped."); - } - - // shutdown listener - super.stop(); - - } - - @Override - public Collection getHandlers() { - return Collections.unmodifiableCollection(handlers); - } - - @Override - public void addHandler(final ProtocolHandler handler) { - if (handler == null) { - throw new NullPointerException("Protocol handler may not be null."); - } - handlers.add(handler); - } - - @Override - public boolean removeHandler(final ProtocolHandler handler) { - return handlers.remove(handler); - } - - @Override - public void dispatchRequest(final MulticastSocket multicastSocket, final DatagramPacket packet) { - - try { - - // unmarshall message - final ProtocolMessageUnmarshaller unmarshaller = protocolContext.createUnmarshaller(); - final ProtocolMessage request = unmarshaller.unmarshal(new ByteArrayInputStream(packet.getData(), 0, packet.getLength())); - - // unwrap multicast message, if necessary - final ProtocolMessage unwrappedRequest; - if (request instanceof MulticastProtocolMessage) { - final MulticastProtocolMessage multicastRequest = (MulticastProtocolMessage) request; - // don't process a message we sent - if (listenerId.equals(multicastRequest.getId())) { - return; - } else { - unwrappedRequest = multicastRequest.getProtocolMessage(); - } - } else { - unwrappedRequest = request; - } - - // dispatch message to handler - ProtocolHandler desiredHandler = null; - for (final ProtocolHandler handler : getHandlers()) { - if (handler.canHandle(unwrappedRequest)) { - desiredHandler = handler; - break; - } - } - - // if no handler found, throw exception; otherwise handle request - if (desiredHandler == null) { - throw new ProtocolException("No handler assigned to handle message type: " + request.getType()); - } else { - final ProtocolMessage response = desiredHandler.handle(request); - if (response != null) { - try { - - // wrap with listener id - final MulticastProtocolMessage multicastResponse = new MulticastProtocolMessage(listenerId, response); - - // marshal message - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); - marshaller.marshal(multicastResponse, baos); - final byte[] responseBytes = baos.toByteArray(); - - final int maxPacketSizeBytes = getMaxPacketSizeBytes(); - if (responseBytes.length > maxPacketSizeBytes) { - logger.warn("Cluster protocol handler '" + desiredHandler.getClass() - + "' produced a multicast response with length greater than configured max packet size '" + maxPacketSizeBytes + "'"); - } - - // create and send packet - final DatagramPacket responseDatagram = new DatagramPacket(responseBytes, responseBytes.length, getMulticastAddress().getAddress(), getMulticastAddress().getPort()); - multicastSocket.send(responseDatagram); - - } catch (final IOException ioe) { - throw new ProtocolException("Failed marshalling protocol message in response to message type: " + request.getType() + " due to: " + ioe, ioe); - } - } - } - - } catch (final Throwable t) { - logger.warn("Failed processing protocol message due to " + t, t); - - if (bulletinRepository != null) { - final Bulletin bulletin = BulletinFactory.createBulletin("Clustering", "WARNING", "Failed to process Protocol Message due to " + t.toString()); - bulletinRepository.addBulletin(bulletin); - } - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java index 3d0eb8e338ed..cc403310a9f0 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java @@ -30,7 +30,6 @@ import org.apache.nifi.reporting.BulletinRepository; public class NodeProtocolSenderListener implements NodeProtocolSender, ProtocolListener { - private final NodeProtocolSender sender; private final ProtocolListener listener; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java index 1345df318a38..fd93b24405a6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java @@ -25,6 +25,7 @@ import java.util.UUID; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.TimeUnit; + import org.apache.nifi.cluster.protocol.ProtocolContext; import org.apache.nifi.cluster.protocol.ProtocolException; import org.apache.nifi.cluster.protocol.ProtocolHandler; @@ -35,7 +36,6 @@ import org.apache.nifi.events.BulletinFactory; import org.apache.nifi.io.socket.ServerSocketConfiguration; import org.apache.nifi.io.socket.SocketListener; -import org.apache.nifi.logging.NiFiLog; import org.apache.nifi.reporting.Bulletin; import org.apache.nifi.reporting.BulletinRepository; import org.apache.nifi.security.util.CertificateUtils; @@ -49,7 +49,7 @@ */ public class SocketProtocolListener extends SocketListener implements ProtocolListener { - private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(SocketProtocolListener.class)); + private static final Logger logger = LoggerFactory.getLogger(SocketProtocolListener.class); private final ProtocolContext protocolContext; private final Collection handlers = new CopyOnWriteArrayList<>(); private volatile BulletinRepository bulletinRepository; @@ -122,7 +122,7 @@ public void dispatchRequest(final Socket socket) { final StopWatch stopWatch = new StopWatch(true); hostname = socket.getInetAddress().getHostName(); final String requestId = UUID.randomUUID().toString(); - logger.info("Received request {} from {}", requestId, hostname); + logger.debug("Received request {} from {}", requestId, hostname); String requestorDn = getRequestorDN(socket); @@ -130,20 +130,23 @@ public void dispatchRequest(final Socket socket) { final ProtocolMessageUnmarshaller unmarshaller = protocolContext.createUnmarshaller(); final InputStream inStream = socket.getInputStream(); final CopyingInputStream copyingInputStream = new CopyingInputStream(inStream, maxMsgBuffer); // don't copy more than 1 MB - logger.debug("Request {} has a message length of {}", requestId, copyingInputStream.getNumberOfBytesCopied()); final ProtocolMessage request; try { request = unmarshaller.unmarshal(copyingInputStream); } finally { receivedMessage = copyingInputStream.getBytesRead(); + if (logger.isDebugEnabled()) { + logger.debug("Received message: " + new String(receivedMessage)); + } } request.setRequestorDN(requestorDn); // dispatch message to handler ProtocolHandler desiredHandler = null; - for (final ProtocolHandler handler : getHandlers()) { + final Collection handlers = getHandlers(); + for (final ProtocolHandler handler : handlers) { if (handler.canHandle(request)) { desiredHandler = handler; break; @@ -152,6 +155,7 @@ public void dispatchRequest(final Socket socket) { // if no handler found, throw exception; otherwise handle request if (desiredHandler == null) { + logger.error("Received request of type {} but none of the following Protocol Handlers were able to process the request: {}", request.getType(), handlers); throw new ProtocolException("No handler assigned to handle message type: " + request.getType()); } else { final ProtocolMessage response = desiredHandler.handle(request); @@ -169,7 +173,8 @@ public void dispatchRequest(final Socket socket) { } stopWatch.stop(); - logger.info("Finished processing request {} (type={}, length={} bytes) in {} millis", requestId, request.getType(), receivedMessage.length, stopWatch.getDuration(TimeUnit.MILLISECONDS)); + logger.info("Finished processing request {} (type={}, length={} bytes) from {} in {} millis", + requestId, request.getType(), receivedMessage.length, hostname, stopWatch.getDuration(TimeUnit.MILLISECONDS)); } catch (final IOException | ProtocolException e) { logger.warn("Failed processing protocol message from " + hostname + " due to " + e, e); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.java similarity index 80% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.java index 63ba5ba66ca8..4b58886ea615 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.java @@ -24,17 +24,17 @@ import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; -import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender; +import org.apache.nifi.cluster.protocol.ClusterCoordinationProtocolSender; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.cluster.protocol.ProtocolContext; import org.apache.nifi.cluster.protocol.ProtocolException; import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller; import org.apache.nifi.cluster.protocol.message.DisconnectMessage; -import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; -import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage; import org.apache.nifi.cluster.protocol.message.ProtocolMessage; import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType; @@ -56,13 +56,13 @@ * allowable time for communication with the node. * */ -public class ClusterManagerProtocolSenderImpl implements ClusterManagerProtocolSender { +public class StandardClusterCoordinationProtocolSender implements ClusterCoordinationProtocolSender { private final ProtocolContext protocolContext; private final SocketConfiguration socketConfiguration; private int handshakeTimeoutSeconds; - public ClusterManagerProtocolSenderImpl(final SocketConfiguration socketConfiguration, final ProtocolContext protocolContext) { + public StandardClusterCoordinationProtocolSender(final SocketConfiguration socketConfiguration, final ProtocolContext protocolContext) { if (socketConfiguration == null) { throw new IllegalArgumentException("Socket configuration may not be null."); } else if (protocolContext == null) { @@ -77,47 +77,6 @@ public ClusterManagerProtocolSenderImpl(final SocketConfiguration socketConfigur public void setBulletinRepository(final BulletinRepository bulletinRepository) { } - /** - * Requests the data flow from a node. - * - * @param msg a message - * @return the message response - * @throws ProtocolException if the message failed to be sent or the - * response was malformed - */ - @Override - public FlowResponseMessage requestFlow(final FlowRequestMessage msg) throws ProtocolException { - Socket socket = null; - try { - socket = createSocket(msg.getNodeId(), false); - - try { - // marshal message to output stream - final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); - marshaller.marshal(msg, socket.getOutputStream()); - } catch (final IOException ioe) { - throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe); - } - - final ProtocolMessage response; - try { - // unmarshall response and return - final ProtocolMessageUnmarshaller unmarshaller = protocolContext.createUnmarshaller(); - response = unmarshaller.unmarshal(socket.getInputStream()); - } catch (final IOException ioe) { - throw new ProtocolException("Failed unmarshalling '" + MessageType.FLOW_RESPONSE + "' protocol message due to: " + ioe, ioe); - } - - if (MessageType.FLOW_RESPONSE == response.getType()) { - return (FlowResponseMessage) response; - } else { - throw new ProtocolException("Expected message type '" + MessageType.FLOW_RESPONSE + "' but found '" + response.getType() + "'"); - } - - } finally { - SocketUtils.closeQuietly(socket); - } - } /** * Requests a node to reconnect to the cluster. The configured value for @@ -225,8 +184,12 @@ private Socket createSocket(final String host, final int port, final boolean app @Override public void notifyNodeStatusChange(final Set nodesToNotify, final NodeStatusChangeMessage msg) { + if (nodesToNotify.isEmpty()) { + return; + } + final NiFiProperties properties = NiFiProperties.getInstance(); - final int numThreads = Math.min(nodesToNotify.size(), properties.getClusterManagerProtocolThreads()); + final int numThreads = Math.min(nodesToNotify.size(), properties.getClusterNodeProtocolThreads()); final byte[] msgBytes; try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) { @@ -237,7 +200,18 @@ public void notifyNodeStatusChange(final Set nodesToNotify, fina throw new ProtocolException("Failed to marshal NodeStatusChangeMessage", e); } - final ExecutorService executor = Executors.newFixedThreadPool(numThreads); + final ExecutorService executor = Executors.newFixedThreadPool(numThreads, new ThreadFactory() { + private final AtomicInteger counter = new AtomicInteger(0); + + @Override + public Thread newThread(final Runnable r) { + final Thread thread = Executors.defaultThreadFactory().newThread(r); + thread.setDaemon(true); + thread.setName("Notify Cluster of Node Status Change-" + counter.incrementAndGet()); + return thread; + } + }); + for (final NodeIdentifier nodeId : nodesToNotify) { executor.submit(new Runnable() { @Override diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java index d42515e4ea5d..a4eb46e88607 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java @@ -16,32 +16,38 @@ */ package org.apache.nifi.cluster.protocol.jaxb.message; +import java.util.List; + import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; +import org.apache.nifi.cluster.protocol.ComponentRevision; +import org.apache.nifi.cluster.protocol.DataFlow; import org.apache.nifi.cluster.protocol.NodeIdentifier; -import org.apache.nifi.cluster.protocol.StandardDataFlow; /** */ public class AdaptedConnectionResponse { - private StandardDataFlow dataFlow; + private DataFlow dataFlow; private NodeIdentifier nodeIdentifier; private String rejectionReason; private int tryLaterSeconds; private Integer managerRemoteInputPort; private Boolean managerRemoteCommsSecure; private String instanceId; + private List nodeStatuses; + private List componentRevisions; public AdaptedConnectionResponse() { } @XmlJavaTypeAdapter(DataFlowAdapter.class) - public StandardDataFlow getDataFlow() { + public DataFlow getDataFlow() { return dataFlow; } - public void setDataFlow(StandardDataFlow dataFlow) { + public void setDataFlow(DataFlow dataFlow) { this.dataFlow = dataFlow; } @@ -97,4 +103,20 @@ public void setInstanceId(String instanceId) { public String getInstanceId() { return instanceId; } + + public void setNodeConnectionStatuses(List connectionStatuses) { + this.nodeStatuses = connectionStatuses; + } + + public List getNodeConnectionStatuses() { + return this.nodeStatuses; + } + + public List getComponentRevisions() { + return componentRevisions; + } + + public void setComponentRevisions(List componentRevisions) { + this.componentRevisions = componentRevisions; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java index 9501b48a198c..f1eba521c507 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java @@ -16,6 +16,8 @@ */ package org.apache.nifi.cluster.protocol.jaxb.message; +import java.util.Set; + import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; @@ -27,7 +29,7 @@ public class AdaptedHeartbeat { private NodeIdentifier nodeIdentifier; private byte[] payload; - private boolean primary; + private Set roles; private NodeConnectionStatus connectionStatus; public AdaptedHeartbeat() { @@ -42,12 +44,12 @@ public void setNodeIdentifier(NodeIdentifier nodeIdentifier) { this.nodeIdentifier = nodeIdentifier; } - public boolean isPrimary() { - return primary; + public Set getRoles() { + return roles; } - public void setPrimary(boolean primary) { - this.primary = primary; + public void setRoles(Set roles) { + this.roles = roles; } public void setConnectionStatus(NodeConnectionStatus connectionStatus) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeConnectionStatus.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeConnectionStatus.java index f9ec3b10ea2f..9cfac2cc7bc1 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeConnectionStatus.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeConnectionStatus.java @@ -17,14 +17,36 @@ package org.apache.nifi.cluster.protocol.jaxb.message; +import java.util.Set; + import org.apache.nifi.cluster.coordination.node.DisconnectionCode; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; +import org.apache.nifi.cluster.protocol.NodeIdentifier; public class AdaptedNodeConnectionStatus { + private Long updateId; + private NodeIdentifier nodeId; private NodeConnectionState state; private DisconnectionCode disconnectCode; private String disconnectReason; private Long connectionRequestTime; + private Set roles; + + public Long getUpdateId() { + return updateId; + } + + public void setUpdateId(Long updateId) { + this.updateId = updateId; + } + + public NodeIdentifier getNodeId() { + return nodeId; + } + + public void setNodeId(NodeIdentifier nodeId) { + this.nodeId = nodeId; + } public NodeConnectionState getState() { return state; @@ -57,4 +79,12 @@ public Long getConnectionRequestTime() { public void setConnectionRequestTime(Long connectionRequestTime) { this.connectionRequestTime = connectionRequestTime; } + + public Set getRoles() { + return roles; + } + + public void setRoles(Set roles) { + this.roles = roles; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java index a1bc907c04db..ca98a8685a1b 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java @@ -34,6 +34,8 @@ public AdaptedConnectionResponse marshal(final ConnectionResponse cr) { aCr.setManagerRemoteInputPort(cr.getManagerRemoteInputPort()); aCr.setManagerRemoteCommsSecure(cr.isManagerRemoteCommsSecure()); aCr.setInstanceId(cr.getInstanceId()); + aCr.setNodeConnectionStatuses(cr.getNodeConnectionStatuses()); + aCr.setComponentRevisions(cr.getComponentRevisions()); } return aCr; } @@ -46,7 +48,8 @@ public ConnectionResponse unmarshal(final AdaptedConnectionResponse aCr) { return ConnectionResponse.createRejectionResponse(aCr.getRejectionReason()); } else { return new ConnectionResponse(aCr.getNodeIdentifier(), aCr.getDataFlow(), - aCr.getManagerRemoteInputPort(), aCr.isManagerRemoteCommsSecure(), aCr.getInstanceId()); + aCr.getManagerRemoteInputPort(), aCr.isManagerRemoteCommsSecure(), + aCr.getInstanceId(), aCr.getNodeConnectionStatuses(), aCr.getComponentRevisions()); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java index bd3e69e0dd58..fc7f54f42029 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java @@ -18,15 +18,15 @@ import javax.xml.bind.annotation.adapters.XmlAdapter; +import org.apache.nifi.cluster.protocol.DataFlow; import org.apache.nifi.cluster.protocol.StandardDataFlow; /** */ -public class DataFlowAdapter extends XmlAdapter { +public class DataFlowAdapter extends XmlAdapter { @Override - public AdaptedDataFlow marshal(final StandardDataFlow df) { - + public AdaptedDataFlow marshal(final DataFlow df) { final AdaptedDataFlow aDf = new AdaptedDataFlow(); if (df != null) { @@ -39,7 +39,7 @@ public AdaptedDataFlow marshal(final StandardDataFlow df) { } @Override - public StandardDataFlow unmarshal(final AdaptedDataFlow aDf) { + public DataFlow unmarshal(final AdaptedDataFlow aDf) { final StandardDataFlow dataFlow = new StandardDataFlow(aDf.getFlow(), aDf.getSnippets()); dataFlow.setAutoStartProcessors(aDf.isAutoStartProcessors()); return dataFlow; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java index 2666abd54196..94d26ce3751c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java @@ -35,7 +35,7 @@ public AdaptedHeartbeat marshal(final Heartbeat hb) { aHb.setPayload(hb.getPayload()); // set leader flag - aHb.setPrimary(hb.isPrimary()); + aHb.setRoles(hb.getRoles()); // set connected flag aHb.setConnectionStatus(hb.getConnectionStatus()); @@ -46,7 +46,7 @@ public AdaptedHeartbeat marshal(final Heartbeat hb) { @Override public Heartbeat unmarshal(final AdaptedHeartbeat aHb) { - return new Heartbeat(aHb.getNodeIdentifier(), aHb.isPrimary(), aHb.getConnectionStatus(), aHb.getPayload()); + return new Heartbeat(aHb.getNodeIdentifier(), aHb.getRoles(), aHb.getConnectionStatus(), aHb.getPayload()); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java index cbb6747e2a2e..eb680cb24fbc 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java @@ -37,5 +37,4 @@ private static JAXBContext initializeJaxbContext() { throw new RuntimeException("Unable to create JAXBContext.", e); } } - } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeConnectionStatusAdapter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeConnectionStatusAdapter.java index e2c302dcdab6..0093c3e741a6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeConnectionStatusAdapter.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeConnectionStatusAdapter.java @@ -25,16 +25,25 @@ public class NodeConnectionStatusAdapter extends XmlAdapter nodeStatuses; + private List componentRevisions; public ReconnectionRequestMessage() { } @@ -91,4 +97,20 @@ public void setInstanceId(final String instanceId) { public String getInstanceId() { return instanceId; } + + public void setNodeConnectionStatuses(List statuses) { + this.nodeStatuses = statuses; + } + + public List getNodeConnectionStatuses() { + return nodeStatuses; + } + + public List getComponentRevisions() { + return componentRevisions; + } + + public void setComponentRevisions(List componentRevisions) { + this.componentRevisions = componentRevisions; + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java deleted file mode 100644 index 460d3bcfc4bb..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.protocol.spring; - -import java.util.concurrent.TimeUnit; - -import org.apache.nifi.io.socket.multicast.MulticastConfiguration; -import org.apache.nifi.util.FormatUtils; -import org.apache.nifi.util.NiFiProperties; -import org.springframework.beans.factory.FactoryBean; - -/** - * Factory bean for creating a singleton MulticastConfiguration instance. - */ -public class MulticastConfigurationFactoryBean implements FactoryBean { - - private MulticastConfiguration configuration; - private NiFiProperties properties; - - @Override - public Object getObject() throws Exception { - if (configuration == null) { - configuration = new MulticastConfiguration(); - - final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterProtocolSocketTimeout(), TimeUnit.MILLISECONDS); - configuration.setSocketTimeout(timeout); - configuration.setReuseAddress(true); - } - return configuration; - - } - - @Override - public Class getObjectType() { - return MulticastConfiguration.class; - } - - @Override - public boolean isSingleton() { - return true; - } - - public void setProperties(NiFiProperties properties) { - this.properties = properties; - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml index 07ea7a4de800..ac5f51e100c8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml @@ -38,13 +38,8 @@ - - - - - - + @@ -52,33 +47,21 @@ - - - - - - - - - - - - - - - + + + - - + + - - + + @@ -92,19 +75,6 @@ - - - - - - - - - - - - - + - diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java deleted file mode 100644 index 82d09596a3df..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.protocol.impl; - -import java.io.IOException; -import java.net.InetAddress; -import org.apache.nifi.cluster.protocol.NodeIdentifier; -import org.apache.nifi.cluster.protocol.ProtocolContext; -import org.apache.nifi.cluster.protocol.ProtocolException; -import org.apache.nifi.cluster.protocol.ProtocolHandler; -import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; -import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; -import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; -import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; -import org.apache.nifi.cluster.protocol.message.PingMessage; -import org.apache.nifi.cluster.protocol.message.ProtocolMessage; -import org.apache.nifi.io.socket.ServerSocketConfiguration; -import org.apache.nifi.io.socket.SocketConfiguration; -import org.junit.After; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.fail; -import org.junit.Before; -import org.junit.Test; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -public class ClusterManagerProtocolSenderImplTest { - - private InetAddress address; - - private int port; - - private SocketProtocolListener listener; - - private ClusterManagerProtocolSenderImpl sender; - - private ProtocolHandler mockHandler; - - @Before - public void setup() throws IOException { - - address = InetAddress.getLocalHost(); - ServerSocketConfiguration serverSocketConfiguration = new ServerSocketConfiguration(); - serverSocketConfiguration.setSocketTimeout(2000); - - mockHandler = mock(ProtocolHandler.class); - - ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); - - listener = new SocketProtocolListener(5, 0, serverSocketConfiguration, protocolContext); - listener.addHandler(mockHandler); - listener.start(); - - port = listener.getPort(); - - SocketConfiguration socketConfiguration = new SocketConfiguration(); - sender = new ClusterManagerProtocolSenderImpl(socketConfiguration, protocolContext); - } - - @After - public void teardown() throws IOException { - if (listener.isRunning()) { - listener.stop(); - } - } - - @Test - public void testRequestFlow() throws Exception { - - when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); - when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new FlowResponseMessage()); - FlowRequestMessage request = new FlowRequestMessage(); - request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port, "localhost", 3821, false)); - FlowResponseMessage response = sender.requestFlow(request); - assertNotNull(response); - } - - @Test - public void testRequestFlowWithBadResponseMessage() throws Exception { - - when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); - when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage()); - FlowRequestMessage request = new FlowRequestMessage(); - request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port, "localhost", 3821, false)); - try { - sender.requestFlow(request); - fail("failed to throw exception"); - } catch (ProtocolException pe) { - } - - } - - @Test - public void testRequestFlowDelayedResponse() throws Exception { - - final int time = 250; - sender.getSocketConfiguration().setSocketTimeout(time); - - when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); - when(mockHandler.handle(any(ProtocolMessage.class))).thenAnswer(new Answer() { - @Override - public FlowResponseMessage answer(InvocationOnMock invocation) throws Throwable { - Thread.sleep(time * 3); - return new FlowResponseMessage(); - } - }); - FlowRequestMessage request = new FlowRequestMessage(); - request.setNodeId(new NodeIdentifier("id", "api-address", 1, "localhost", port, "localhost", 3821, false)); - try { - sender.requestFlow(request); - fail("failed to throw exception"); - } catch (ProtocolException pe) { - } - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java deleted file mode 100644 index 90817b2b85d3..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.protocol.impl; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.net.DatagramPacket; -import java.net.InetSocketAddress; -import java.net.MulticastSocket; -import org.apache.nifi.cluster.protocol.ProtocolContext; -import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; -import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; -import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; -import org.apache.nifi.cluster.protocol.message.PingMessage; -import org.apache.nifi.cluster.protocol.message.ProtocolMessage; -import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage; -import org.apache.nifi.io.socket.multicast.MulticastConfiguration; -import org.apache.nifi.io.socket.multicast.MulticastUtils; -import org.junit.After; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; - -/** - */ -public class ClusterServiceDiscoveryTest { - - private ClusterServiceDiscovery discovery; - - private String serviceName; - - private MulticastSocket socket; - - private InetSocketAddress multicastAddress; - - private MulticastConfiguration configuration; - - private ProtocolContext protocolContext; - - @Before - public void setup() throws Exception { - - serviceName = "some-service"; - multicastAddress = new InetSocketAddress("225.1.1.1", 22222); - configuration = new MulticastConfiguration(); - - protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); - - discovery = new ClusterServiceDiscovery(serviceName, multicastAddress, configuration, protocolContext); - discovery.start(); - - socket = MulticastUtils.createMulticastSocket(multicastAddress.getPort(), configuration); - } - - @After - public void teardown() throws IOException { - try { - if (discovery.isRunning()) { - discovery.stop(); - } - } finally { - MulticastUtils.closeQuietly(socket); - } - } - - @Ignore("Test needs to be fixed. Requires an active network connection") - @Test - public void testGetAddressOnStartup() { - assertNull(discovery.getService()); - } - - @Ignore("This test has an NPE after ignoring another...perhaps has a bad inter-test dependency") - @Test - public void testGetAddressAfterBroadcast() throws Exception { - - ServiceBroadcastMessage msg = new ServiceBroadcastMessage(); - msg.setServiceName("some-service"); - msg.setAddress("3.3.3.3"); - msg.setPort(1234); - - // marshal message to output stream - ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - marshaller.marshal(msg, baos); - byte[] requestPacketBytes = baos.toByteArray(); - DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, multicastAddress); - socket.send(packet); - - Thread.sleep(250); - - InetSocketAddress updatedAddress = discovery.getService().getServiceAddress(); - assertEquals("some-service", discovery.getServiceName()); - assertEquals("3.3.3.3", updatedAddress.getHostName()); - assertEquals(1234, updatedAddress.getPort()); - - } - - @Ignore("Test needs to be fixed. Requires an active network connection") - @Test - public void testBadBroadcastMessage() throws Exception { - - ProtocolMessage msg = new PingMessage(); - - // marshal message to output stream - ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - marshaller.marshal(msg, baos); - byte[] requestPacketBytes = baos.toByteArray(); - DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, multicastAddress); - socket.send(packet); - - Thread.sleep(250); - - assertNull(discovery.getService()); - - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java deleted file mode 100644 index ea4015044d51..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.protocol.impl; - -import java.net.InetSocketAddress; -import java.util.concurrent.TimeUnit; -import org.apache.nifi.io.socket.multicast.DiscoverableService; -import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import org.junit.Before; -import org.junit.Test; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import org.mockito.stubbing.OngoingStubbing; - -public class ClusterServiceLocatorTest { - - private ClusterServiceDiscovery mockServiceDiscovery; - - private int fixedPort; - - private DiscoverableService fixedService; - - private ClusterServiceLocator serviceDiscoveryLocator; - - private ClusterServiceLocator serviceDiscoveryFixedPortLocator; - - private ClusterServiceLocator fixedServiceLocator; - - @Before - public void setup() throws Exception { - - fixedPort = 1; - mockServiceDiscovery = mock(ClusterServiceDiscovery.class); - fixedService = new DiscoverableServiceImpl("some-service", InetSocketAddress.createUnresolved("some-host", 20)); - - serviceDiscoveryLocator = new ClusterServiceLocator(mockServiceDiscovery); - serviceDiscoveryFixedPortLocator = new ClusterServiceLocator(mockServiceDiscovery, fixedPort); - fixedServiceLocator = new ClusterServiceLocator(fixedService); - - } - - @Test - public void getServiceWhenServiceDiscoveryNotStarted() { - assertNull(serviceDiscoveryLocator.getService()); - } - - @Test - public void getServiceWhenServiceDiscoveryFixedPortNotStarted() { - assertNull(serviceDiscoveryLocator.getService()); - } - - @Test - public void getServiceWhenFixedServiceNotStarted() { - assertEquals(fixedService, fixedServiceLocator.getService()); - } - - @Test - public void getServiceNotOnFirstAttempt() { - - ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig(); - config.setNumAttempts(2); - config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS); - config.setTimeBetweenAttempts(1); - - serviceDiscoveryLocator.setAttemptsConfig(config); - - OngoingStubbing stubbing = null; - for (int i = 0; i < config.getNumAttempts() - 1; i++) { - if (stubbing == null) { - stubbing = when(mockServiceDiscovery.getService()).thenReturn(null); - } else { - stubbing.thenReturn(null); - } - } - stubbing.thenReturn(fixedService); - - assertEquals(fixedService, serviceDiscoveryLocator.getService()); - - } - - @Test - public void getServiceNotOnFirstAttemptWithFixedPort() { - - ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig(); - config.setNumAttempts(2); - config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS); - config.setTimeBetweenAttempts(1); - - serviceDiscoveryFixedPortLocator.setAttemptsConfig(config); - - OngoingStubbing stubbing = null; - for (int i = 0; i < config.getNumAttempts() - 1; i++) { - if (stubbing == null) { - stubbing = when(mockServiceDiscovery.getService()).thenReturn(null); - } else { - stubbing.thenReturn(null); - } - } - stubbing.thenReturn(fixedService); - - InetSocketAddress resultAddress = InetSocketAddress.createUnresolved(fixedService.getServiceAddress().getHostName(), fixedPort); - DiscoverableService resultService = new DiscoverableServiceImpl(fixedService.getServiceName(), resultAddress); - assertEquals(resultService, serviceDiscoveryFixedPortLocator.getService()); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java deleted file mode 100644 index 24dd17dee5d7..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.protocol.impl; - -import java.net.InetSocketAddress; -import org.apache.nifi.cluster.protocol.ProtocolContext; -import org.apache.nifi.cluster.protocol.ProtocolException; -import org.apache.nifi.cluster.protocol.ProtocolHandler; -import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; -import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; -import org.apache.nifi.cluster.protocol.message.ProtocolMessage; -import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage; -import org.apache.nifi.io.socket.multicast.DiscoverableService; -import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; -import org.apache.nifi.io.socket.multicast.MulticastConfiguration; -import org.junit.After; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; - -/** - */ -public class ClusterServicesBroadcasterTest { - - private ClusterServicesBroadcaster broadcaster; - - private MulticastProtocolListener listener; - - private DummyProtocolHandler handler; - - private InetSocketAddress multicastAddress; - - private DiscoverableService broadcastedService; - - private ProtocolContext protocolContext; - - private MulticastConfiguration configuration; - - @Before - public void setup() throws Exception { - - broadcastedService = new DiscoverableServiceImpl("some-service", new InetSocketAddress("localhost", 11111)); - - multicastAddress = new InetSocketAddress("225.1.1.1", 22222); - - configuration = new MulticastConfiguration(); - - protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); - - broadcaster = new ClusterServicesBroadcaster(multicastAddress, configuration, protocolContext, "500 ms"); - broadcaster.addService(broadcastedService); - - handler = new DummyProtocolHandler(); - listener = new MulticastProtocolListener(5, multicastAddress, configuration, protocolContext); - listener.addHandler(handler); - } - - @After - public void teardown() { - - if (broadcaster.isRunning()) { - broadcaster.stop(); - } - - try { - if (listener.isRunning()) { - listener.stop(); - } - } catch (Exception ex) { - ex.printStackTrace(System.out); - } - - } - - @Ignore("fails needs to be fixed") - @Test - public void testBroadcastReceived() throws Exception { - - broadcaster.start(); - listener.start(); - - Thread.sleep(1000); - - listener.stop(); - - assertNotNull(handler.getProtocolMessage()); - assertEquals(ProtocolMessage.MessageType.SERVICE_BROADCAST, handler.getProtocolMessage().getType()); - final ServiceBroadcastMessage msg = (ServiceBroadcastMessage) handler.getProtocolMessage(); - assertEquals(broadcastedService.getServiceName(), msg.getServiceName()); - assertEquals(broadcastedService.getServiceAddress().getHostName(), msg.getAddress()); - assertEquals(broadcastedService.getServiceAddress().getPort(), msg.getPort()); - } - - private class DummyProtocolHandler implements ProtocolHandler { - - private ProtocolMessage protocolMessage; - - @Override - public boolean canHandle(ProtocolMessage msg) { - return true; - } - - @Override - public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { - this.protocolMessage = msg; - return null; - } - - public ProtocolMessage getProtocolMessage() { - return protocolMessage; - } - - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java deleted file mode 100644 index 690d416d28f4..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java +++ /dev/null @@ -1,168 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.protocol.impl; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.net.DatagramPacket; -import java.net.InetSocketAddress; -import java.net.MulticastSocket; -import java.util.ArrayList; -import java.util.List; -import org.apache.nifi.cluster.protocol.ProtocolContext; -import org.apache.nifi.cluster.protocol.ProtocolException; -import org.apache.nifi.cluster.protocol.ProtocolHandler; -import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; -import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; -import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; -import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage; -import org.apache.nifi.cluster.protocol.message.PingMessage; -import org.apache.nifi.cluster.protocol.message.ProtocolMessage; -import org.apache.nifi.io.socket.multicast.MulticastConfiguration; -import org.apache.nifi.io.socket.multicast.MulticastUtils; -import org.junit.After; -import static org.junit.Assert.assertEquals; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; - -public class MulticastProtocolListenerTest { - - private MulticastProtocolListener listener; - - private MulticastSocket socket; - - private InetSocketAddress address; - - private MulticastConfiguration configuration; - - private ProtocolContext protocolContext; - - @Before - public void setup() throws Exception { - - address = new InetSocketAddress("226.1.1.1", 60000); - configuration = new MulticastConfiguration(); - - protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); - - listener = new MulticastProtocolListener(5, address, configuration, protocolContext); - listener.start(); - - socket = MulticastUtils.createMulticastSocket(address.getPort(), configuration); - } - - @After - public void teardown() throws IOException { - try { - if (listener.isRunning()) { - listener.stop(); - } - } finally { - MulticastUtils.closeQuietly(socket); - } - } - - @Ignore("Test needs to be reworked. Fails if on a system without actiev network connection") - @Test - public void testBadRequest() throws Exception { - DelayedProtocolHandler handler = new DelayedProtocolHandler(0); - listener.addHandler(handler); - DatagramPacket packet = new DatagramPacket(new byte[]{5}, 1, address); - socket.send(packet); - Thread.sleep(250); - assertEquals(0, handler.getMessages().size()); - } - - @Ignore("this test works sometimes and fails others - needs work to be reliable") - @Test - public void testRequest() throws Exception { - - ReflexiveProtocolHandler handler = new ReflexiveProtocolHandler(); - listener.addHandler(handler); - - ProtocolMessage msg = new PingMessage(); - MulticastProtocolMessage multicastMsg = new MulticastProtocolMessage("some-id", msg); - - // marshal message to output stream - ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - marshaller.marshal(multicastMsg, baos); - byte[] requestPacketBytes = baos.toByteArray(); - DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, address); - socket.send(packet); - - Thread.sleep(250); - assertEquals(1, handler.getMessages().size()); - assertEquals(msg.getType(), handler.getMessages().get(0).getType()); - - } - - private class ReflexiveProtocolHandler implements ProtocolHandler { - - private List messages = new ArrayList<>(); - - @Override - public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { - messages.add(msg); - return msg; - } - - @Override - public boolean canHandle(ProtocolMessage msg) { - return true; - } - - public List getMessages() { - return messages; - } - - } - - private class DelayedProtocolHandler implements ProtocolHandler { - - private int delay = 0; - - private List messages = new ArrayList<>(); - - public DelayedProtocolHandler(int delay) { - this.delay = delay; - } - - @Override - public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { - try { - messages.add(msg); - Thread.sleep(delay); - return null; - } catch (final InterruptedException ie) { - throw new ProtocolException(ie); - } - - } - - @Override - public boolean canHandle(ProtocolMessage msg) { - return true; - } - - public List getMessages() { - return messages; - } - - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java deleted file mode 100644 index bd57fe49043a..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java +++ /dev/null @@ -1,168 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.protocol.impl; - -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.UUID; - -import org.apache.nifi.cluster.protocol.ConnectionRequest; -import org.apache.nifi.cluster.protocol.ConnectionResponse; -import org.apache.nifi.cluster.protocol.NodeIdentifier; -import org.apache.nifi.cluster.protocol.ProtocolContext; -import org.apache.nifi.cluster.protocol.ProtocolException; -import org.apache.nifi.cluster.protocol.ProtocolHandler; -import org.apache.nifi.cluster.protocol.StandardDataFlow; -import org.apache.nifi.cluster.protocol.UnknownServiceAddressException; -import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; -import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; -import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; -import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; -import org.apache.nifi.cluster.protocol.message.PingMessage; -import org.apache.nifi.cluster.protocol.message.ProtocolMessage; -import org.apache.nifi.io.socket.ServerSocketConfiguration; -import org.apache.nifi.io.socket.SocketConfiguration; -import org.apache.nifi.io.socket.multicast.DiscoverableService; -import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; -import org.junit.After; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; - -@Ignore("Randomly tests... probably timing-specific") -public class NodeProtocolSenderImplTest { - - private SocketProtocolListener listener; - - private NodeProtocolSenderImpl sender; - - private DiscoverableService service; - - private ServerSocketConfiguration serverSocketConfiguration; - - private ClusterServiceLocator mockServiceLocator; - - private ProtocolHandler mockHandler; - - private NodeIdentifier nodeIdentifier; - - @Before - public void setup() throws IOException { - - serverSocketConfiguration = new ServerSocketConfiguration(); - - mockServiceLocator = mock(ClusterServiceLocator.class); - mockHandler = mock(ProtocolHandler.class); - - nodeIdentifier = new NodeIdentifier("1", "localhost", 1234, "localhost", 5678, "localhost", 3821, false); - - ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); - - listener = new SocketProtocolListener(5, 0, serverSocketConfiguration, protocolContext); - listener.setShutdownListenerSeconds(3); - listener.addHandler(mockHandler); - listener.start(); - - service = new DiscoverableServiceImpl("some-service", new InetSocketAddress("localhost", listener.getPort())); - - SocketConfiguration socketConfiguration = new SocketConfiguration(); - socketConfiguration.setReuseAddress(true); - sender = new NodeProtocolSenderImpl(mockServiceLocator, socketConfiguration, protocolContext); - } - - @After - public void teardown() throws IOException { - if (listener.isRunning()) { - listener.stop(); - } - } - - @Test - public void testConnect() throws Exception { - - when(mockServiceLocator.getService()).thenReturn(service); - when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); - ConnectionResponseMessage mockMessage = new ConnectionResponseMessage(); - mockMessage.setConnectionResponse(new ConnectionResponse(nodeIdentifier, - new StandardDataFlow("flow".getBytes("UTF-8"), new byte[0]), null, null, UUID.randomUUID().toString())); - when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(mockMessage); - - ConnectionRequestMessage request = new ConnectionRequestMessage(); - request.setConnectionRequest(new ConnectionRequest(nodeIdentifier)); - ConnectionResponseMessage response = sender.requestConnection(request); - assertNotNull(response); - } - - @Test(expected = UnknownServiceAddressException.class) - public void testConnectNoClusterManagerAddress() throws Exception { - - when(mockServiceLocator.getService()).thenReturn(null); - when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); - when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new ConnectionResponseMessage()); - - ConnectionRequestMessage request = new ConnectionRequestMessage(); - request.setConnectionRequest(new ConnectionRequest(nodeIdentifier)); - - sender.requestConnection(request); - fail("failed to throw exception"); - } - - @Test(expected = ProtocolException.class) - public void testConnectBadResponse() throws Exception { - - when(mockServiceLocator.getService()).thenReturn(service); - when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); - when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage()); - - ConnectionRequestMessage request = new ConnectionRequestMessage(); - request.setConnectionRequest(new ConnectionRequest(nodeIdentifier)); - - sender.requestConnection(request); - fail("failed to throw exception"); - - } - - @Test(expected = ProtocolException.class) - public void testConnectDelayedResponse() throws Exception { - - final int time = 250; - sender.getSocketConfiguration().setSocketTimeout(time); - when(mockServiceLocator.getService()).thenReturn(service); - when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); - when(mockHandler.handle(any(ProtocolMessage.class))).thenAnswer(new Answer() { - @Override - public ConnectionResponseMessage answer(InvocationOnMock invocation) throws Throwable { - Thread.sleep(time * 3); - return new ConnectionResponseMessage(); - } - }); - ConnectionRequestMessage request = new ConnectionRequestMessage(); - request.setConnectionRequest(new ConnectionRequest(nodeIdentifier)); - - sender.requestConnection(request); - fail("failed to throw exception"); - - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/jaxb/message/TestJaxbProtocolUtils.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/jaxb/message/TestJaxbProtocolUtils.java new file mode 100644 index 000000000000..25ab73a888b6 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/jaxb/message/TestJaxbProtocolUtils.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.cluster.protocol.jaxb.message; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.util.Collections; +import java.util.List; + +import javax.xml.bind.JAXBException; + +import org.apache.nifi.cluster.coordination.node.DisconnectionCode; +import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; +import org.apache.nifi.cluster.protocol.ComponentRevision; +import org.apache.nifi.cluster.protocol.ConnectionResponse; +import org.apache.nifi.cluster.protocol.DataFlow; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.StandardDataFlow; +import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; +import org.apache.nifi.web.Revision; +import org.junit.Test; + +public class TestJaxbProtocolUtils { + + @Test + public void testRoundTripConnectionResponse() throws JAXBException { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + final ConnectionResponseMessage msg = new ConnectionResponseMessage(); + final NodeIdentifier nodeId = new NodeIdentifier("id", "localhost", 8000, "localhost", 8001, "localhost", 8002, true); + final DataFlow dataFlow = new StandardDataFlow(new byte[0], new byte[0]); + final List nodeStatuses = Collections.singletonList(new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED)); + final List componentRevisions = Collections.singletonList(ComponentRevision.fromRevision(new Revision(8L, "client-1", "component-1"))); + msg.setConnectionResponse(new ConnectionResponse(nodeId, dataFlow, 80, false, "instance-1", nodeStatuses, componentRevisions)); + + JaxbProtocolUtils.JAXB_CONTEXT.createMarshaller().marshal(msg, baos); + final Object unmarshalled = JaxbProtocolUtils.JAXB_CONTEXT.createUnmarshaller().unmarshal(new ByteArrayInputStream(baos.toByteArray())); + assertTrue(unmarshalled instanceof ConnectionResponseMessage); + final ConnectionResponseMessage unmarshalledMsg = (ConnectionResponseMessage) unmarshalled; + + final List revisions = msg.getConnectionResponse().getComponentRevisions(); + assertEquals(1, revisions.size()); + assertEquals(8L, revisions.get(0).getVersion().longValue()); + assertEquals("client-1", revisions.get(0).getClientId()); + assertEquals("component-1", revisions.get(0).getComponentId()); + + assertEquals(revisions, unmarshalledMsg.getConnectionResponse().getComponentRevisions()); + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java deleted file mode 100644 index 0b70c61c9a49..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java +++ /dev/null @@ -1,151 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.client; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.net.DatagramPacket; -import java.net.InetSocketAddress; -import java.net.MulticastSocket; -import java.text.SimpleDateFormat; -import java.util.Date; -import java.util.Locale; -import java.util.Timer; -import java.util.TimerTask; - -import org.apache.nifi.cluster.protocol.ProtocolContext; -import org.apache.nifi.cluster.protocol.ProtocolException; -import org.apache.nifi.cluster.protocol.ProtocolHandler; -import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; -import org.apache.nifi.cluster.protocol.impl.MulticastProtocolListener; -import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; -import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; -import org.apache.nifi.cluster.protocol.message.PingMessage; -import org.apache.nifi.cluster.protocol.message.ProtocolMessage; -import org.apache.nifi.io.socket.multicast.MulticastConfiguration; -import org.apache.nifi.io.socket.multicast.MulticastUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Simple multicast test client that sends ping messages to a group address. - */ -public class MulticastTestClient { - - private static final Logger logger = LoggerFactory.getLogger(MulticastTestClient.class); - - private static final int PING_DELAY_SECONDS = 3; - - public static void main(final String... args) throws IOException { - - String group = System.getProperty("group", "225.0.0.0"); - if (group == null) { - System.out.println("Host system property 'group' was not given."); - return; - } - group = group.trim(); - if (group.length() == 0) { - System.out.println("Host system property 'group' must be non-empty."); - return; - } - - final String portStr = System.getProperty("port", "2222"); - final int port; - try { - port = Integer.parseInt(portStr); - } catch (final NumberFormatException nfe) { - System.out.println("Port system property 'port' was not a valid port."); - return; - } - - logger.info(String.format("Pinging every %s seconds using multicast address: %s:%s.", PING_DELAY_SECONDS, group, port)); - logger.info("Override defaults by using system properties '-Dgroup=' and '-Dport='."); - logger.info("The test client may be stopped by entering a newline at the command line."); - - final InetSocketAddress addr = new InetSocketAddress(group, port); - final ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); - final MulticastConfiguration multicastConfig = new MulticastConfiguration(); - multicastConfig.setReuseAddress(true); - - // setup listener - final MulticastProtocolListener listener = new MulticastProtocolListener(1, addr, multicastConfig, protocolContext); - listener.addHandler(new ProtocolHandler() { - @Override - public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { - final PingMessage pingMsg = (PingMessage) msg; - final SimpleDateFormat sdf = new SimpleDateFormat("HH:mm:ss", Locale.US); - logger.info("Pinged at: " + sdf.format(pingMsg.getDate())); - return null; - } - - @Override - public boolean canHandle(ProtocolMessage msg) { - return true; - } - }); - - // setup socket - final MulticastSocket multicastSocket = MulticastUtils.createMulticastSocket(multicastConfig); - - // setup broadcaster - final Timer broadcaster = new Timer("Multicast Test Client", /** - * is daemon * - */ - true); - - try { - - // start listening - listener.start(); - - // start broadcasting - broadcaster.schedule(new TimerTask() { - - @Override - public void run() { - try { - - final PingMessage msg = new PingMessage(); - msg.setDate(new Date()); - - // marshal message to output stream - final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - marshaller.marshal(msg, baos); - final byte[] packetBytes = baos.toByteArray(); - - // send message - final DatagramPacket packet = new DatagramPacket(packetBytes, packetBytes.length, addr); - multicastSocket.send(packet); - - } catch (final Exception ex) { - logger.warn("Failed to send message due to: " + ex, ex); - } - } - }, 0, PING_DELAY_SECONDS * 1000); - - // block until any input is received - System.in.read(); - - } finally { - broadcaster.cancel(); - if (listener.isRunning()) { - listener.stop(); - } - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java index d6838cdc0d94..db66b8112eac 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java @@ -55,9 +55,19 @@ public AbstractHeartbeatMonitor(final ClusterCoordinator clusterCoordinator, fin } @Override - public final void start() { + public synchronized final void start() { + if (!stopped) { + throw new IllegalStateException("Heartbeat Monitor cannot be started because it is already started"); + } + stopped = false; - onStart(); + logger.info("Heartbeat Monitor started"); + + try { + onStart(); + } catch (final Exception e) { + logger.error("Failed to start Heartbeat Monitor", e); + } this.future = flowEngine.scheduleWithFixedDelay(new Runnable() { @Override @@ -73,8 +83,13 @@ public void run() { } @Override - public final void stop() { + public synchronized final void stop() { + if (stopped) { + return; + } + this.stopped = true; + logger.info("Heartbeat Monitor stopped"); try { if (future != null) { @@ -112,7 +127,7 @@ protected synchronized void monitorHeartbeats() { final Map latestHeartbeats = getLatestHeartbeats(); if (latestHeartbeats == null || latestHeartbeats.isEmpty()) { // failed to fetch heartbeats; don't change anything. - clusterCoordinator.reportEvent(null, Severity.WARNING, "Failed to retrieve any new heartbeat information for nodes. " + clusterCoordinator.reportEvent(null, Severity.INFO, "Failed to retrieve any new heartbeat information for nodes. " + "Will not make any decisions based on heartbeats."); return; } @@ -133,8 +148,8 @@ protected synchronized void monitorHeartbeats() { logger.info("Finished processing {} heartbeats in {}", latestHeartbeats.size(), procStopWatch.getDuration()); // Disconnect any node that hasn't sent a heartbeat in a long time (8 times the heartbeat interval) - final long maxMillis = heartbeatIntervalMillis * 1000L * 8; - final long threshold = latestHeartbeatTime - maxMillis; + final long maxMillis = heartbeatIntervalMillis * 8; + final long threshold = System.currentTimeMillis() - maxMillis; for (final NodeHeartbeat heartbeat : latestHeartbeats.values()) { if (heartbeat.getTimestamp() < threshold) { clusterCoordinator.requestNodeDisconnect(heartbeat.getNodeIdentifier(), DisconnectionCode.LACK_OF_HEARTBEAT, @@ -165,18 +180,11 @@ private void processHeartbeat(final NodeHeartbeat heartbeat) { final NodeConnectionStatus connectionStatus = clusterCoordinator.getConnectionStatus(nodeId); if (connectionStatus == null) { - final NodeConnectionState hbConnectionState = heartbeat.getConnectionStatus().getState(); - if (hbConnectionState == NodeConnectionState.DISCONNECTED || hbConnectionState == NodeConnectionState.DISCONNECTING) { - // Node is not part of the cluster. Remove heartbeat and move on. - removeHeartbeat(nodeId); - return; - } - // Unknown node. Issue reconnect request clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from unknown node. Removing heartbeat and requesting that node connect to cluster."); removeHeartbeat(nodeId); - clusterCoordinator.requestNodeConnect(nodeId); + clusterCoordinator.requestNodeConnect(nodeId, null); return; } @@ -210,7 +218,7 @@ private void processHeartbeat(final NodeHeartbeat heartbeat) { clusterCoordinator.reportEvent(nodeId, Severity.WARNING, "Received heartbeat from node that thinks it is not yet part of the cluster," + "though the Cluster Coordinator thought it was (node claimed state was " + heartbeat.getConnectionStatus().getState() + "). Marking as Disconnected and requesting that Node reconnect to cluster"); - clusterCoordinator.requestNodeConnect(nodeId); + clusterCoordinator.requestNodeConnect(nodeId, null); return; } @@ -220,14 +228,14 @@ private void processHeartbeat(final NodeHeartbeat heartbeat) { // we cannot manually reconnect it. final DisconnectionCode disconnectionCode = connectionStatus.getDisconnectCode(); - if (disconnectionCode == DisconnectionCode.LACK_OF_HEARTBEAT) { - // record event + // Determine whether or not the node should be allowed to be in the cluster still, depending on its reason for disconnection. + if (disconnectionCode == DisconnectionCode.LACK_OF_HEARTBEAT || disconnectionCode == DisconnectionCode.UNABLE_TO_COMMUNICATE) { clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from node previously " - + "disconnected due to lack of heartbeat. Issuing reconnection request."); + + "disconnected due to " + disconnectionCode + ". Issuing reconnection request."); - clusterCoordinator.requestNodeConnect(nodeId); + clusterCoordinator.requestNodeConnect(nodeId, null); } else { - // disconnected nodes should not heartbeat, so we need to issue a disconnection request + // disconnected nodes should not heartbeat, so we need to issue a disconnection request. logger.info("Ignoring received heartbeat from disconnected node " + nodeId + ". Issuing disconnection request."); clusterCoordinator.requestNodeDisconnect(nodeId, connectionStatus.getDisconnectCode(), connectionStatus.getDisconnectReason()); removeHeartbeat(nodeId); @@ -256,9 +264,7 @@ private void processHeartbeat(final NodeHeartbeat heartbeat) { clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received first heartbeat from connecting node. Node connected."); } - if (heartbeat.isPrimary()) { - clusterCoordinator.setPrimaryNode(nodeId); - } + clusterCoordinator.updateNodeRoles(nodeId, heartbeat.getRoles()); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java index f172915c4c3d..934c81bab419 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/ClusterProtocolHeartbeatMonitor.java @@ -43,6 +43,7 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.cluster.protocol.ProtocolException; import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.ProtocolListener; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; import org.apache.nifi.cluster.protocol.message.ProtocolMessage; import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType; @@ -55,7 +56,8 @@ import org.slf4j.LoggerFactory; /** - * Uses Apache Curator to monitor heartbeats from nodes + * Uses Apache ZooKeeper to advertise the address to send heartbeats to, and then relies on the NiFi Cluster + * Protocol to receive heartbeat messages from nodes in the cluster. */ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor implements HeartbeatMonitor, ProtocolHandler { protected static final Logger logger = LoggerFactory.getLogger(ClusterProtocolHeartbeatMonitor.class); @@ -64,7 +66,6 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im private final ZooKeeperClientConfig zkClientConfig; private final String clusterNodesPath; - private volatile CuratorFramework curatorClient; private volatile Map clusterNodeIds = new HashMap<>(); private final String heartbeatAddress; @@ -82,32 +83,33 @@ public class ClusterProtocolHeartbeatMonitor extends AbstractHeartbeatMonitor im } - public ClusterProtocolHeartbeatMonitor(final ClusterCoordinator clusterCoordinator, final Properties properties) { + public ClusterProtocolHeartbeatMonitor(final ClusterCoordinator clusterCoordinator, final ProtocolListener protocolListener, final Properties properties) { super(clusterCoordinator, properties); + protocolListener.addHandler(this); this.zkClientConfig = ZooKeeperClientConfig.createConfig(properties); this.clusterNodesPath = zkClientConfig.resolvePath("cluster/nodes"); - String hostname = properties.getProperty(NiFiProperties.CLUSTER_MANAGER_ADDRESS); + String hostname = properties.getProperty(NiFiProperties.CLUSTER_NODE_ADDRESS); if (hostname == null) { try { hostname = InetAddress.getLocalHost().getHostName(); } catch (UnknownHostException e) { - throw new RuntimeException("Unable to determine local hostname and the '" + NiFiProperties.CLUSTER_MANAGER_ADDRESS + "' property is not set"); + throw new RuntimeException("Unable to determine local hostname and the '" + NiFiProperties.CLUSTER_NODE_ADDRESS + "' property is not set"); } } - final String port = properties.getProperty(NiFiProperties.CLUSTER_MANAGER_PROTOCOL_PORT); + final String port = properties.getProperty(NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT); if (port == null) { - throw new RuntimeException("Unable to determine which port Cluster Manager Protocol is listening on because the '" - + NiFiProperties.CLUSTER_MANAGER_PROTOCOL_PORT + "' property is not set"); + throw new RuntimeException("Unable to determine which port Cluster Coordinator Protocol is listening on because the '" + + NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT + "' property is not set"); } try { Integer.parseInt(port); } catch (final NumberFormatException nfe) { - throw new RuntimeException("Unable to determine which port Cluster Manager Protocol is listening on because the '" - + NiFiProperties.CLUSTER_MANAGER_PROTOCOL_PORT + "' property is set to '" + port + "', which is not a valid port number."); + throw new RuntimeException("Unable to determine which port Cluster Coordinator Protocol is listening on because the '" + + NiFiProperties.CLUSTER_NODE_PROTOCOL_PORT + "' property is set to '" + port + "', which is not a valid port number."); } heartbeatAddress = hostname + ":" + port; @@ -116,10 +118,22 @@ public ClusterProtocolHeartbeatMonitor(final ClusterCoordinator clusterCoordinat @Override public void onStart() { final RetryPolicy retryPolicy = new RetryForever(5000); - curatorClient = CuratorFrameworkFactory.newClient(zkClientConfig.getConnectString(), + final CuratorFramework curatorClient = CuratorFrameworkFactory.newClient(zkClientConfig.getConnectString(), zkClientConfig.getSessionTimeoutMillis(), zkClientConfig.getConnectionTimeoutMillis(), retryPolicy); curatorClient.start(); + // We don't know what the heartbeats look like for the nodes, since we were just elected to monitoring + // them. However, the map may be filled with old heartbeats. So we clear the heartbeats and populate the + // map with new heartbeats set to the current time and using the currently known status. We do this so + // that if we go the required amount of time without receiving a heartbeat, we do know to mark the node + // as disconnected. + heartbeatMessages.clear(); + for (final NodeIdentifier nodeId : clusterCoordinator.getNodeIdentifiers()) { + final NodeHeartbeat heartbeat = new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(), + clusterCoordinator.getConnectionStatus(nodeId), Collections.emptySet(), 0, 0L, 0, System.currentTimeMillis()); + heartbeatMessages.put(nodeId, heartbeat); + } + final Thread publishAddress = new Thread(new Runnable() { @Override public void run() { @@ -128,6 +142,8 @@ public void run() { try { try { curatorClient.setData().forPath(path, heartbeatAddress.getBytes(StandardCharsets.UTF_8)); + curatorClient.close(); + logger.info("Successfully published Cluster Heartbeat Monitor Address of {} to ZooKeeper", heartbeatAddress); return; } catch (final NoNodeException nne) { // ensure that parents are created, using a wide-open ACL because the parents contain no data @@ -162,16 +178,8 @@ public void run() { publishAddress.start(); } - private CuratorFramework getClient() { - return curatorClient; - } - @Override public void onStop() { - final CuratorFramework client = getClient(); - if (client != null) { - client.close(); - } } @Override @@ -201,7 +209,7 @@ public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolExceptio final NodeIdentifier nodeId = heartbeat.getNodeIdentifier(); final NodeConnectionStatus connectionStatus = heartbeat.getConnectionStatus(); - final boolean primary = heartbeat.isPrimary(); + final Set roles = heartbeat.getRoles(); final byte[] payloadBytes = heartbeat.getPayload(); final HeartbeatPayload payload = HeartbeatPayload.unmarshal(payloadBytes); final int activeThreadCount = payload.getActiveThreadCount(); @@ -210,7 +218,7 @@ public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolExceptio final long systemStartTime = payload.getSystemStartTime(); final NodeHeartbeat nodeHeartbeat = new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(), - connectionStatus, primary, flowFileCount, flowFileBytes, activeThreadCount, systemStartTime); + connectionStatus, roles, flowFileCount, flowFileBytes, activeThreadCount, systemStartTime); heartbeatMessages.put(heartbeat.getNodeIdentifier(), nodeHeartbeat); logger.debug("Received new heartbeat from {}", nodeId); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/CuratorHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/CuratorHeartbeatMonitor.java deleted file mode 100644 index 8114813171eb..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/CuratorHeartbeatMonitor.java +++ /dev/null @@ -1,376 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.nifi.cluster.coordination.heartbeat; - -import java.io.ByteArrayInputStream; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; - -import javax.xml.bind.JAXBContext; -import javax.xml.bind.Unmarshaller; - -import org.apache.curator.RetryPolicy; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.curator.retry.RetryForever; -import org.apache.nifi.cluster.coordination.ClusterCoordinator; -import org.apache.nifi.cluster.coordination.node.DisconnectionCode; -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.cluster.protocol.message.HeartbeatMessage; -import org.apache.nifi.controller.cluster.ZooKeeperClientConfig; -import org.apache.nifi.engine.FlowEngine; -import org.apache.nifi.reporting.Severity; -import org.apache.nifi.util.FormatUtils; -import org.apache.nifi.util.NiFiProperties; -import org.apache.nifi.util.StopWatch; -import org.apache.zookeeper.KeeperException.NoNodeException; -import org.apache.zookeeper.data.Stat; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Uses Apache Curator to monitor heartbeats from nodes - */ -public class CuratorHeartbeatMonitor implements HeartbeatMonitor { - private static final Logger logger = LoggerFactory.getLogger(CuratorHeartbeatMonitor.class); - private static final Unmarshaller unmarshaller; - - private final ClusterCoordinator clusterCoordinator; - private final ZooKeeperClientConfig zkClientConfig; - private final String heartbeatPath; - private final int heartbeatIntervalMillis; - - private volatile CuratorFramework curatorClient; - private volatile ScheduledFuture future; - private volatile Map latestHeartbeatMessages; - private volatile long latestHeartbeatTime; - - private final FlowEngine flowEngine = new FlowEngine(1, "Heartbeat Monitor", true); - - static { - try { - final JAXBContext jaxbContext = JAXBContext.newInstance(HeartbeatMessage.class); - unmarshaller = jaxbContext.createUnmarshaller(); - } catch (final Exception e) { - throw new RuntimeException("Failed to create an Unmarshaller for unmarshalling Heartbeat Messages", e); - } - } - - public CuratorHeartbeatMonitor(final ClusterCoordinator clusterCoordinator, final Properties properties) { - this.clusterCoordinator = clusterCoordinator; - this.zkClientConfig = ZooKeeperClientConfig.createConfig(properties); - this.heartbeatPath = zkClientConfig.resolvePath("cluster/heartbeats"); - - final String heartbeatInterval = properties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL, - NiFiProperties.DEFAULT_CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL); - - this.heartbeatIntervalMillis = (int) FormatUtils.getTimeDuration(heartbeatInterval, TimeUnit.MILLISECONDS); - } - - @Override - public void start() { - final RetryPolicy retryPolicy = new RetryForever(5000); - curatorClient = CuratorFrameworkFactory.newClient(zkClientConfig.getConnectString(), - zkClientConfig.getSessionTimeoutMillis(), zkClientConfig.getConnectionTimeoutMillis(), retryPolicy); - curatorClient.start(); - - this.future = flowEngine.scheduleWithFixedDelay(new Runnable() { - @Override - public void run() { - try { - monitorHeartbeats(); - } catch (final Exception e) { - clusterCoordinator.reportEvent(null, Severity.ERROR, "Failed to process heartbeats from nodes due to " + e.toString()); - logger.error("Failed to process heartbeats", e); - } - } - }, heartbeatIntervalMillis, heartbeatIntervalMillis, TimeUnit.MILLISECONDS); - } - - private CuratorFramework getClient() { - return curatorClient; - } - - @Override - public void stop() { - final CuratorFramework client = getClient(); - if (client != null) { - client.close(); - } - - if (future != null) { - future.cancel(true); - } - } - - @Override - public NodeHeartbeat getLatestHeartbeat(final NodeIdentifier nodeId) { - return latestHeartbeatMessages.get(nodeId); - } - - - /** - * Fetches all of the latest heartbeats from ZooKeeper - * and updates the Cluster Coordinator as appropriate, - * based on the heartbeats received. - * - * Visible for testing. - */ - synchronized void monitorHeartbeats() { - final StopWatch fetchStopWatch = new StopWatch(true); - final Map latestHeartbeats = fetchHeartbeats(); - if (latestHeartbeats == null || latestHeartbeats.isEmpty()) { - // failed to fetch heartbeats from ZooKeeper; don't change anything. - clusterCoordinator.reportEvent(null, Severity.WARNING, "Failed to retrieve any new heartbeat information for nodes from ZooKeeper. " - + "Will not make any decisions based on heartbeats."); - return; - } - - this.latestHeartbeatMessages = new HashMap<>(latestHeartbeats); - fetchStopWatch.stop(); - - final StopWatch procStopWatch = new StopWatch(true); - for (final NodeHeartbeat heartbeat : latestHeartbeats.values()) { - try { - processHeartbeat(heartbeat); - } catch (final Exception e) { - clusterCoordinator.reportEvent(null, Severity.ERROR, - "Received heartbeat from " + heartbeat.getNodeIdentifier() + " but failed to process heartbeat due to " + e); - logger.error("Failed to process heartbeat from {} due to {}", heartbeat.getNodeIdentifier(), e.toString()); - logger.error("", e); - } - } - - procStopWatch.stop(); - logger.info("Finished processing {} heartbeats in {} (fetch took an additional {})", - latestHeartbeats.size(), procStopWatch.getDuration(), fetchStopWatch.getDuration()); - - // Disconnect any node that hasn't sent a heartbeat in a long time (8 times the heartbeat interval) - for (final NodeHeartbeat heartbeat : latestHeartbeats.values()) { - final long maxMillis = heartbeatIntervalMillis * 1000L * 8; - final long threshold = latestHeartbeatTime - maxMillis; - - if (heartbeat.getTimestamp() < threshold) { - final int differenceSeconds = (int) TimeUnit.MILLISECONDS.toSeconds(latestHeartbeatTime - heartbeat.getTimestamp()); - - clusterCoordinator.requestNodeDisconnect(heartbeat.getNodeIdentifier(), DisconnectionCode.LACK_OF_HEARTBEAT, - "Node has not sent a heartbeat to ZooKeeper in " + differenceSeconds + " seconds"); - - try { - removeHeartbeat(heartbeat.getNodeIdentifier()); - } catch (final Exception e) { - logger.warn("Failed to remove heartbeat for {} due to {}", heartbeat.getNodeIdentifier(), e.toString()); - logger.warn("", e); - } - } - } - } - - - private void processHeartbeat(final NodeHeartbeat heartbeat) { - final NodeIdentifier nodeId = heartbeat.getNodeIdentifier(); - - // Do not process heartbeat if it's blocked by firewall. - if (clusterCoordinator.isBlockedByFirewall(nodeId.getSocketAddress())) { - clusterCoordinator.reportEvent(nodeId, Severity.WARNING, "Firewall blocked received heartbeat. Issuing disconnection request."); - - // request node to disconnect - clusterCoordinator.requestNodeDisconnect(nodeId, DisconnectionCode.BLOCKED_BY_FIREWALL, "Blocked by Firewall"); - removeHeartbeat(nodeId); - return; - } - - final NodeConnectionStatus connectionStatus = clusterCoordinator.getConnectionStatus(nodeId); - if (connectionStatus == null) { - final NodeConnectionState hbConnectionState = heartbeat.getConnectionStatus().getState(); - if (hbConnectionState == NodeConnectionState.DISCONNECTED || hbConnectionState == NodeConnectionState.DISCONNECTING) { - // Node is not part of the cluster. Remove heartbeat and move on. - removeHeartbeat(nodeId); - return; - } - - // Unknown node. Issue reconnect request - clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from unknown node. Removing heartbeat and requesting that node connect to cluster."); - removeHeartbeat(nodeId); - - clusterCoordinator.requestNodeConnect(nodeId); - return; - } - - final DisconnectionCode reportedDisconnectCode = heartbeat.getConnectionStatus().getDisconnectCode(); - if (reportedDisconnectCode != null) { - // Check if the node is notifying us that it wants to disconnect from the cluster - final boolean requestingDisconnect; - switch (reportedDisconnectCode) { - case MISMATCHED_FLOWS: - case NODE_SHUTDOWN: - case STARTUP_FAILURE: - final NodeConnectionState expectedState = connectionStatus.getState(); - requestingDisconnect = expectedState == NodeConnectionState.CONNECTED || expectedState == NodeConnectionState.CONNECTING; - break; - default: - requestingDisconnect = false; - break; - } - - if (requestingDisconnect) { - clusterCoordinator.disconnectionRequestedByNode(nodeId, heartbeat.getConnectionStatus().getDisconnectCode(), - heartbeat.getConnectionStatus().getDisconnectReason()); - removeHeartbeat(nodeId); - return; - } - } - - final NodeConnectionState connectionState = connectionStatus.getState(); - if (heartbeat.getConnectionStatus().getState() != NodeConnectionState.CONNECTED && connectionState == NodeConnectionState.CONNECTED) { - // Cluster Coordinator believes that node is connected, but node does not believe so. - clusterCoordinator.reportEvent(nodeId, Severity.WARNING, "Received heartbeat from node that thinks it is not yet part of the cluster," - + "though the Cluster Coordinator thought it was (node claimed state was " + heartbeat.getConnectionStatus().getState() - + "). Marking as Disconnected and requesting that Node reconnect to cluster"); - clusterCoordinator.requestNodeConnect(nodeId); - return; - } - - if (NodeConnectionState.DISCONNECTED == connectionState) { - // ignore heartbeats from nodes disconnected by means other than lack of heartbeat, unless it is - // the only node. We allow it if it is the only node because if we have a one-node cluster, then - // we cannot manually reconnect it. - final DisconnectionCode disconnectionCode = connectionStatus.getDisconnectCode(); - - if (disconnectionCode == DisconnectionCode.LACK_OF_HEARTBEAT) { - // record event - clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat from node previously " - + "disconnected due to lack of heartbeat. Issuing reconnection request."); - - clusterCoordinator.requestNodeConnect(nodeId); - } else { - // disconnected nodes should not heartbeat, so we need to issue a disconnection request - logger.info("Ignoring received heartbeat from disconnected node " + nodeId + ". Issuing disconnection request."); - clusterCoordinator.requestNodeDisconnect(nodeId, connectionStatus.getDisconnectCode(), connectionStatus.getDisconnectReason()); - removeHeartbeat(nodeId); - } - - return; - } - - if (NodeConnectionState.DISCONNECTING == connectionStatus.getState()) { - // ignore spurious heartbeat - removeHeartbeat(nodeId); - return; - } - - // first heartbeat causes status change from connecting to connected - if (NodeConnectionState.CONNECTING == connectionState) { - final Long connectionRequestTime = connectionStatus.getConnectionRequestTime(); - if (connectionRequestTime != null && heartbeat.getTimestamp() < connectionRequestTime) { - clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received heartbeat but ignoring because it was reported before the node was last asked to reconnect."); - removeHeartbeat(nodeId); - return; - } - - // connection complete - clusterCoordinator.finishNodeConnection(nodeId); - clusterCoordinator.reportEvent(nodeId, Severity.INFO, "Received first heartbeat from connecting node. Node connected."); - } - - if (heartbeat.isPrimary()) { - clusterCoordinator.setPrimaryNode(nodeId); - } - } - - - /** - * Fetches the latest heartbeats for each node from ZooKeeper. - * Visible for testing - */ - Map fetchHeartbeats() { - logger.debug("Fetching heartbeats from ZooKeeper"); - final List nodeIds; - try { - nodeIds = curatorClient.getChildren().forPath(heartbeatPath); - } catch (final NoNodeException nne) { - logger.info("Could not find any heartbeats in ZooKeeper because the ZNode " + heartbeatPath + " does not exist"); - return null; - } catch (final Exception e) { - logger.error("Failed to obtain heartbeats from ZooKeeper due to {}", e); - logger.error("", e); - clusterCoordinator.reportEvent(null, Severity.WARNING, "Failed to obtain heartbeats from ZooKeeper due to " + e); - - return null; - } - - logger.debug("Found {} nodes that have emitted heartbeats to ZooKeeper", nodeIds.size()); - final Map latestHeartbeats = new HashMap<>(nodeIds.size()); - for (final String nodeId : nodeIds) { - final HeartbeatMessage heartbeatMsg; - - final String nodeHeartbeatPath = heartbeatPath + "/" + nodeId; - final Stat stat = new Stat(); - try { - final byte[] serializedHeartbeatMsg = getClient().getData().storingStatIn(stat).forPath(nodeHeartbeatPath); - heartbeatMsg = (HeartbeatMessage) unmarshaller.unmarshal(new ByteArrayInputStream(serializedHeartbeatMsg)); - } catch (final Exception e) { - logger.error("Failed to obtain heartbeat from ZooKeeper for Node with ID {} due to {}", nodeId, e); - logger.error("", e); - clusterCoordinator.reportEvent(null, Severity.WARNING, "Failed to obtain heartbeat for Node with ID " + nodeId + " from ZooKeeper due to " + e); - - continue; - } - - // update timestamp to be the timestamp that ZooKeeper reports - final long lastModifiedTime = stat.getMtime(); - if (lastModifiedTime > latestHeartbeatTime) { - latestHeartbeatTime = lastModifiedTime; - } - - latestHeartbeats.put(heartbeatMsg.getHeartbeat().getNodeIdentifier(), StandardNodeHeartbeat.fromHeartbeatMessage(heartbeatMsg, lastModifiedTime)); - logger.debug("Received heartbeat from Node {}", nodeId); - } - - logger.debug("Fetched {} heartbeats from ZooKeeper", latestHeartbeats.size()); - return latestHeartbeats; - } - - @Override - public synchronized void removeHeartbeat(final NodeIdentifier nodeId) { - logger.debug("Deleting heartbeat for node {}", nodeId); - final String nodeHeartbeatPath = heartbeatPath + "/" + nodeId.getId(); - - latestHeartbeatMessages.remove(nodeId); - - try { - getClient().delete().forPath(nodeHeartbeatPath); - logger.info("Removed heartbeat from ZooKeeper for Node {}", nodeId); - } catch (final NoNodeException e) { - // node did not exist. Just return. - logger.debug("Attempted to remove heartbeat for Node with ID {} but no ZNode existed at {}", nodeId, nodeHeartbeatPath); - return; - } catch (final Exception e) { - logger.warn("Failed to remove heartbeat from ZooKeeper for Node {} due to {}", nodeId, e); - logger.warn("", e); - - clusterCoordinator.reportEvent(nodeId, Severity.WARNING, "Failed to remove node's heartbeat from ZooKeeper due to " + e); - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/StandardNodeHeartbeat.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/StandardNodeHeartbeat.java index 133bab087681..e455a76db993 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/StandardNodeHeartbeat.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/StandardNodeHeartbeat.java @@ -17,6 +17,10 @@ package org.apache.nifi.cluster.coordination.heartbeat; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + import org.apache.nifi.cluster.HeartbeatPayload; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; import org.apache.nifi.cluster.protocol.Heartbeat; @@ -28,18 +32,18 @@ public class StandardNodeHeartbeat implements NodeHeartbeat { private final NodeIdentifier nodeId; private final long timestamp; private final NodeConnectionStatus connectionStatus; - private final boolean primary; + private final Set roles; private final int flowFileCount; private final long flowFileBytes; private final int activeThreadCount; private final long systemStartTime; public StandardNodeHeartbeat(final NodeIdentifier nodeId, final long timestamp, final NodeConnectionStatus connectionStatus, - final boolean primary, final int flowFileCount, final long flowFileBytes, final int activeThreadCount, final long systemStartTime) { + final Set roles, final int flowFileCount, final long flowFileBytes, final int activeThreadCount, final long systemStartTime) { this.timestamp = timestamp; this.nodeId = nodeId; this.connectionStatus = connectionStatus; - this.primary = primary; + this.roles = roles == null ? Collections.emptySet() : Collections.unmodifiableSet(new HashSet<>(roles)); this.flowFileCount = flowFileCount; this.flowFileBytes = flowFileBytes; this.activeThreadCount = activeThreadCount; @@ -62,8 +66,8 @@ public NodeConnectionStatus getConnectionStatus() { } @Override - public boolean isPrimary() { - return primary; + public Set getRoles() { + return roles; } @Override @@ -92,7 +96,7 @@ public static StandardNodeHeartbeat fromHeartbeatMessage(final HeartbeatMessage final HeartbeatPayload payload = HeartbeatPayload.unmarshal(heartbeat.getPayload()); return new StandardNodeHeartbeat(heartbeat.getNodeIdentifier(), timestamp, heartbeat.getConnectionStatus(), - heartbeat.isPrimary(), (int) payload.getTotalFlowFileCount(), payload.getTotalFlowFileBytes(), + heartbeat.getRoles(), (int) payload.getTotalFlowFileCount(), payload.getTotalFlowFileBytes(), payload.getActiveThreadCount(), payload.getSystemStartTime()); } } \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java index 7dc598ea9b01..ce2e32a1dd20 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMerger.java @@ -35,6 +35,7 @@ import org.apache.nifi.cluster.coordination.http.endpoints.ControllerStatusEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.CountersEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.DropRequestEndpiontMerger; +import org.apache.nifi.cluster.coordination.http.endpoints.FlowMerger; import org.apache.nifi.cluster.coordination.http.endpoints.FlowSnippetEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.GroupStatusEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.ListFlowFilesEndpointMerger; @@ -52,25 +53,15 @@ import org.apache.nifi.cluster.coordination.http.endpoints.ReportingTasksEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.StatusHistoryEndpointMerger; import org.apache.nifi.cluster.coordination.http.endpoints.SystemDiagnosticsEndpointMerger; +import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator; import org.apache.nifi.cluster.manager.NodeResponse; -import org.apache.nifi.cluster.manager.StatusMerger; -import org.apache.nifi.cluster.manager.impl.WebClusterManager; -import org.apache.nifi.cluster.node.Node.Status; -import org.apache.nifi.reporting.Bulletin; -import org.apache.nifi.reporting.BulletinQuery; -import org.apache.nifi.reporting.ComponentType; import org.apache.nifi.stream.io.NullOutputStream; -import org.apache.nifi.web.api.dto.status.ControllerStatusDTO; -import org.apache.nifi.web.api.entity.ControllerStatusEntity; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class StandardHttpResponseMerger implements HttpResponseMerger { private Logger logger = LoggerFactory.getLogger(StandardHttpResponseMerger.class); - private static final int NODE_CONTINUE_STATUS_CODE = 150; - private final WebClusterManager clusterManager; - private static final List endpointMergers = new ArrayList<>(); static { endpointMergers.add(new ControllerStatusEndpointMerger()); @@ -99,14 +90,10 @@ public class StandardHttpResponseMerger implements HttpResponseMerger { endpointMergers.add(new StatusHistoryEndpointMerger()); endpointMergers.add(new SystemDiagnosticsEndpointMerger()); endpointMergers.add(new CountersEndpointMerger()); + endpointMergers.add(new FlowMerger()); } public StandardHttpResponseMerger() { - this(null); - } - - public StandardHttpResponseMerger(final WebClusterManager clusterManager) { - this.clusterManager = clusterManager; } @Override @@ -140,58 +127,9 @@ public NodeResponse mergeResponses(final URI uri, final String httpMethod, final } final NodeResponse response = merger.merge(uri, httpMethod, successResponses, problematicResponses, clientResponse); - if (clusterManager != null) { - mergeNCMBulletins(response, uri, httpMethod); - } - return response; } - /** - * This method merges bulletins from the NCM. Eventually, the NCM will go away entirely, and - * at that point, we will completely remove this and the WebClusterManager as a member variable. - * However, until then, the bulletins from the NCM are important to include, since there is no other - * node that can include them. - * - * @param clientResponse the Node Response that will be returned to the client - * @param uri the URI - * @param method the HTTP Method - * - * @deprecated this method exists only until we can remove the Cluster Manager from the picture all together. It will then be removed. - */ - @Deprecated - private void mergeNCMBulletins(final NodeResponse clientResponse, final URI uri, final String method) { - // determine if we have at least one response - final boolean hasClientResponse = clientResponse != null; - final boolean hasSuccessfulClientResponse = hasClientResponse && clientResponse.is2xx(); - - if (hasSuccessfulClientResponse && clusterManager.isControllerStatusEndpoint(uri, method)) { - // for now, we need to merge the NCM's bulletins too. - final ControllerStatusEntity responseEntity = (ControllerStatusEntity) clientResponse.getUpdatedEntity(); - final ControllerStatusDTO mergedStatus = responseEntity.getControllerStatus(); - - final int totalNodeCount = clusterManager.getNodeIds().size(); - final int connectedNodeCount = clusterManager.getNodeIds(Status.CONNECTED).size(); - - final List ncmControllerBulletins = clusterManager.getBulletinRepository().findBulletinsForController(); - mergedStatus.setBulletins(clusterManager.mergeNCMBulletins(mergedStatus.getBulletins(), ncmControllerBulletins)); - - // get the controller service bulletins - final BulletinQuery controllerServiceQuery = new BulletinQuery.Builder().sourceType(ComponentType.CONTROLLER_SERVICE).build(); - final List ncmServiceBulletins = clusterManager.getBulletinRepository().findBulletins(controllerServiceQuery); - mergedStatus.setControllerServiceBulletins(clusterManager.mergeNCMBulletins(mergedStatus.getControllerServiceBulletins(), ncmServiceBulletins)); - - // get the reporting task bulletins - final BulletinQuery reportingTaskQuery = new BulletinQuery.Builder().sourceType(ComponentType.REPORTING_TASK).build(); - final List ncmReportingTaskBulletins = clusterManager.getBulletinRepository().findBulletins(reportingTaskQuery); - mergedStatus.setReportingTaskBulletins(clusterManager.mergeNCMBulletins(mergedStatus.getReportingTaskBulletins(), ncmReportingTaskBulletins)); - - mergedStatus.setConnectedNodeCount(connectedNodeCount); - mergedStatus.setTotalNodeCount(totalNodeCount); - StatusMerger.updatePrettyPrintedFields(mergedStatus); - } - } - @Override public Set getProblematicNodeResponses(final Set allResponses) { @@ -225,7 +163,7 @@ private void drainResponses(final Set responses, final NodeRespons responses.stream() .parallel() // parallelize the draining of the responses, since we have multiple streams to consume .filter(response -> response != exclude) // don't include the explicitly excluded node - .filter(response -> response.getStatus() != NODE_CONTINUE_STATUS_CODE) // don't include any 150-NodeContinue responses because they contain no content + .filter(response -> response.getStatus() != RequestReplicator.NODE_CONTINUE_STATUS_CODE) // don't include any 150-NodeContinue responses because they contain no content .forEach(response -> drainResponse(response)); // drain all node responses that didn't get filtered out } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractNodeStatusEndpoint.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractNodeStatusEndpoint.java index cc73e3423a09..f930bf1a2273 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractNodeStatusEndpoint.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractNodeStatusEndpoint.java @@ -24,11 +24,16 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.web.api.entity.Entity; -public abstract class AbstractNodeStatusEndpoint extends AbstractSingleEntityEndpoint { +public abstract class AbstractNodeStatusEndpoint extends AbstractSingleDTOEndpoint { @Override protected final void mergeResponses(DtoType clientDto, Map dtoMap, Set successfulResponses, Set problematicResponses) { - final NodeIdentifier selectedNodeId = dtoMap.entrySet().stream().filter(e -> e.getValue() == clientDto).map(e -> e.getKey()).findFirst().orElse(null); + final NodeIdentifier selectedNodeId = dtoMap.entrySet().stream() + .filter(e -> e.getValue() == clientDto) + .map(e -> e.getKey()) + .findFirst() + .orElse(null); + if (selectedNodeId == null) { throw new IllegalArgumentException("Attempted to merge Status request but could not find the appropriate Node Identifier"); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleDTOEndpoint.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleDTOEndpoint.java new file mode 100644 index 000000000000..db3cfd188b46 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleDTOEndpoint.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.cluster.coordination.http.endpoints; + +import java.net.URI; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.entity.Entity; + +public abstract class AbstractSingleDTOEndpoint implements EndpointResponseMerger { + + @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 EntityType responseEntity = clientResponse.getClientResponse().getEntity(getEntityClass()); + final DtoType dto = getDto(responseEntity); + + final Map dtoMap = new HashMap<>(); + for (final NodeResponse nodeResponse : successfulResponses) { + final EntityType nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(getEntityClass()); + final DtoType nodeDto = getDto(nodeResponseEntity); + dtoMap.put(nodeResponse.getNodeId(), nodeDto); + } + + mergeResponses(dto, dtoMap, successfulResponses, problematicResponses); + return new NodeResponse(clientResponse, responseEntity); + } + + + /** + * Normalizes the validation errors by prepending the corresponding nodes when the error does not exist across all nodes. + * + * @param validationErrorMap map + * @param totalNodes total + * @return normalized errors + */ + protected Set normalizedMergedValidationErrors(final Map> validationErrorMap, int totalNodes) { + final Set normalizedValidationErrors = new HashSet<>(); + for (final Map.Entry> validationEntry : validationErrorMap.entrySet()) { + final String msg = validationEntry.getKey(); + final Set nodeIds = validationEntry.getValue(); + + if (nodeIds.size() == totalNodes) { + normalizedValidationErrors.add(msg); + } else { + nodeIds.forEach(id -> normalizedValidationErrors.add(id.getApiAddress() + ":" + id.getApiPort() + " -- " + msg)); + } + } + return normalizedValidationErrors; + } + + /** + * Merges the validation errors into the specified map, recording the corresponding node identifier. + * + * @param validationErrorMap map + * @param nodeId id + * @param nodeValidationErrors errors + */ + protected void mergeValidationErrors(final Map> validationErrorMap, final NodeIdentifier nodeId, final Collection nodeValidationErrors) { + if (nodeValidationErrors != null) { + nodeValidationErrors.stream().forEach( + err -> validationErrorMap.computeIfAbsent(err, k -> new HashSet()) + .add(nodeId)); + } + } + + /** + * @return the class that represents the type of Entity that is expected by this response mapper + */ + protected abstract Class getEntityClass(); + + /** + * Extracts the DTO from the given entity + * + * @param entity the entity to extract the DTO from + * @return the DTO from the given entity + */ + protected abstract DtoType getDto(EntityType entity); + + /** + * Merges the responses from all nodes in the given map into the single given DTO + * + * @param clientDto the DTO to merge responses into + * @param dtoMap the responses from all nodes + * @param successfulResponses the responses from nodes that completed the request successfully + * @param problematicResponses the responses from nodes that did not complete the request successfully + */ + protected abstract void mergeResponses(DtoType clientDto, Map dtoMap, Set successfulResponses, Set problematicResponses); + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleEntityEndpoint.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleEntityEndpoint.java index 66db949f145b..026ececcf399 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleEntityEndpoint.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/AbstractSingleEntityEndpoint.java @@ -29,8 +29,7 @@ import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.web.api.entity.Entity; -public abstract class AbstractSingleEntityEndpoint implements EndpointResponseMerger { - +public abstract class AbstractSingleEntityEndpoint implements EndpointResponseMerger { @Override public final NodeResponse merge(final URI uri, final String method, final Set successfulResponses, final Set problematicResponses, final NodeResponse clientResponse) { if (!canHandle(uri, method)) { @@ -38,19 +37,31 @@ public final NodeResponse merge(final URI uri, final String method, final Set dtoMap = new HashMap<>(); + final Map entityMap = new HashMap<>(); for (final NodeResponse nodeResponse : successfulResponses) { final EntityType nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(getEntityClass()); - final DtoType nodeDto = getDto(nodeResponseEntity); - dtoMap.put(nodeResponse.getNodeId(), nodeDto); + entityMap.put(nodeResponse.getNodeId(), nodeResponseEntity); } - mergeResponses(dto, dtoMap, successfulResponses, problematicResponses); + mergeResponses(responseEntity, entityMap, successfulResponses, problematicResponses); return new NodeResponse(clientResponse, responseEntity); } + /** + * Merges the validation errors into the specified map, recording the corresponding node identifier. + * + * @param validationErrorMap map + * @param nodeId id + * @param nodeValidationErrors errors + */ + protected void mergeValidationErrors(final Map> validationErrorMap, final NodeIdentifier nodeId, final Collection nodeValidationErrors) { + if (nodeValidationErrors != null) { + nodeValidationErrors.stream().forEach( + err -> validationErrorMap.computeIfAbsent(err, k -> new HashSet()) + .add(nodeId)); + } + } /** * Normalizes the validation errors by prepending the corresponding nodes when the error does not exist across all nodes. @@ -74,42 +85,19 @@ protected Set normalizedMergedValidationErrors(final Map> validationErrorMap, final NodeIdentifier nodeId, final Collection nodeValidationErrors) { - if (nodeValidationErrors != null) { - nodeValidationErrors.stream().forEach( - err -> validationErrorMap.computeIfAbsent(err, k -> new HashSet()) - .add(nodeId)); - } - } - /** * @return the class that represents the type of Entity that is expected by this response mapper */ protected abstract Class getEntityClass(); /** - * Extracts the DTO from the given entity - * - * @param entity the entity to extract the DTO from - * @return the DTO from the given entity - */ - protected abstract DtoType getDto(EntityType entity); - - /** - * Merges the responses from all nodes in the given map into the single given DTO + * Merges the responses from all nodes in the given map into the single given entity * - * @param clientDto the DTO to merge responses into - * @param dtoMap the responses from all nodes + * @param clientEntity the Entity to merge responses into + * @param entityMap the responses from all nodes * @param successfulResponses the responses from nodes that completed the request successfully * @param problematicResponses the responses from nodes that did not complete the request successfully */ - protected abstract void mergeResponses(DtoType clientDto, Map dtoMap, Set successfulResponses, Set problematicResponses); + protected abstract void mergeResponses(EntityType clientEntity, Map entityMap, Set successfulResponses, Set problematicResponses); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/BulletinBoardEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/BulletinBoardEndpointMerger.java index 799d2795c1c0..8da6353ae517 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/BulletinBoardEndpointMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/BulletinBoardEndpointMerger.java @@ -32,7 +32,7 @@ import org.apache.nifi.web.api.dto.BulletinDTO; import org.apache.nifi.web.api.entity.BulletinBoardEntity; -public class BulletinBoardEndpointMerger extends AbstractSingleEntityEndpoint { +public class BulletinBoardEndpointMerger extends AbstractSingleDTOEndpoint { public static final Pattern BULLETIN_BOARD_URI_PATTERN = Pattern.compile("/nifi-api/flow/bulletin-board"); @Override diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java index 0598259e8cab..9976de2620b8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ComponentStateEndpointMerger.java @@ -33,7 +33,7 @@ import org.apache.nifi.web.api.dto.StateMapDTO; import org.apache.nifi.web.api.entity.ComponentStateEntity; -public class ComponentStateEndpointMerger extends AbstractSingleEntityEndpoint { +public class ComponentStateEndpointMerger extends AbstractSingleDTOEndpoint { public static final Pattern PROCESSOR_STATE_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}/state"); public static final Pattern CONTROLLER_SERVICE_STATE_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/node/[a-f0-9\\-]{36}/state"); public static final Pattern REPORTING_TASK_STATE_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/node/[a-f0-9\\-]{36}/state"); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerServiceEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerServiceEndpointMerger.java index 448b44e1eb30..10fa28196f5e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerServiceEndpointMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerServiceEndpointMerger.java @@ -31,7 +31,7 @@ import java.util.Set; import java.util.regex.Pattern; -public class ControllerServiceEndpointMerger extends AbstractSingleEntityEndpoint { +public class ControllerServiceEndpointMerger extends AbstractSingleDTOEndpoint { public static final String CONTROLLER_SERVICES_URI = "/nifi-api/controller-services/node"; public static final Pattern CONTROLLER_SERVICE_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/node/[a-f0-9\\-]{36}"); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerStatusEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerStatusEndpointMerger.java index 50514f8d25d0..30671db686f7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerStatusEndpointMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ControllerStatusEndpointMerger.java @@ -29,7 +29,7 @@ import org.apache.nifi.web.api.dto.status.ControllerStatusDTO; import org.apache.nifi.web.api.entity.ControllerStatusEntity; -public class ControllerStatusEndpointMerger extends AbstractSingleEntityEndpoint { +public class ControllerStatusEndpointMerger extends AbstractSingleDTOEndpoint { public static final Pattern CONTROLLER_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/status"); @Override @@ -71,13 +71,6 @@ protected void mergeResponses(ControllerStatusDTO clientDto, Map { +public class DropRequestEndpiontMerger extends AbstractSingleDTOEndpoint { public static final Pattern DROP_REQUESTS_URI = Pattern.compile("/nifi-api/flowfile-queues/[a-f0-9\\-]{36}/drop-requests"); public static final Pattern DROP_REQUEST_URI = Pattern.compile("/nifi-api/flowfile-queues/[a-f0-9\\-]{36}/drop-requests/[a-f0-9\\-]{36}"); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowMerger.java new file mode 100644 index 000000000000..7546a0b982be --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowMerger.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.cluster.coordination.http.endpoints; + +import java.net.URI; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +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.flow.FlowDTO; +import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO; +import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO; +import org.apache.nifi.web.api.dto.status.PortStatusDTO; +import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO; +import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO; +import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO; +import org.apache.nifi.web.api.entity.ConnectionEntity; +import org.apache.nifi.web.api.entity.FunnelEntity; +import org.apache.nifi.web.api.entity.LabelEntity; +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.apache.nifi.web.api.entity.RemoteProcessGroupEntity; + +public class FlowMerger extends AbstractSingleDTOEndpoint { + public static final Pattern FLOW_URI_PATTERN = Pattern.compile("/nifi-api/flow/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))"); + + @Override + public boolean canHandle(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && 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) { + + final FlowDTO flowDto = clientDto.getFlow(); + + final Map> connections = new HashMap<>(); + final Map> funnels = new HashMap<>(); + final Map> inputPorts = new HashMap<>(); + final Map> labels = new HashMap<>(); + final Map> outputPorts = new HashMap<>(); + final Map> processors = new HashMap<>(); + final Map> rpgs = new HashMap<>(); + final Map> processGroups = new HashMap<>(); + + // Create mapping of ComponentID -> all components with that ID (one per node) + for (final ProcessGroupFlowDTO nodeGroupFlowDto : dtoMap.values()) { + final FlowDTO nodeFlowDto = nodeGroupFlowDto.getFlow(); + + // Merge connection statuses + for (final ConnectionEntity entity : nodeFlowDto.getConnections()) { + connections.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity); + } + + for (final FunnelEntity entity : nodeFlowDto.getFunnels()) { + funnels.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity); + } + + for (final PortEntity entity : nodeFlowDto.getInputPorts()) { + inputPorts.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity); + } + + for (final PortEntity entity : nodeFlowDto.getOutputPorts()) { + outputPorts.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity); + } + + for (final LabelEntity entity : nodeFlowDto.getLabels()) { + labels.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity); + } + + for (final ProcessorEntity entity : nodeFlowDto.getProcessors()) { + processors.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity); + } + + for (final RemoteProcessGroupEntity entity : nodeFlowDto.getRemoteProcessGroups()) { + rpgs.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity); + } + + for (final ProcessGroupEntity entity : nodeFlowDto.getProcessGroups()) { + processGroups.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity); + } + } + + // + // Merge the components that are grouped together by ID + // + + // Merge connections + final Set mergedConnections = new HashSet<>(); + for (final List connectionList : connections.values()) { + mergedConnections.add(mergeConnections(connectionList)); + } + flowDto.setConnections(mergedConnections); + + // Merge funnel statuses + final Set mergedFunnels = new HashSet<>(); + for (final List funnelList : funnels.values()) { + mergedFunnels.add(mergeFunnels(funnelList)); + } + flowDto.setFunnels(mergedFunnels); + + // Merge input ports + final Set mergedInputPorts = new HashSet<>(); + for (final List portList : inputPorts.values()) { + mergedInputPorts.add(mergePorts(portList)); + } + flowDto.setInputPorts(mergedInputPorts); + + // Merge output ports + final Set mergedOutputPorts = new HashSet<>(); + for (final List portList : outputPorts.values()) { + mergedOutputPorts.add(mergePorts(portList)); + } + flowDto.setOutputPorts(mergedOutputPorts); + + // Merge labels + final Set mergedLabels = new HashSet<>(); + for (final List labelList : labels.values()) { + mergedLabels.add(mergeLabels(labelList)); + } + flowDto.setLabels(mergedLabels); + + + // Merge processors + final Set mergedProcessors = new HashSet<>(); + for (final List processorList : processors.values()) { + mergedProcessors.add(mergeProcessors(processorList)); + } + flowDto.setProcessors(mergedProcessors); + + + // Merge Remote Process Groups + final Set mergedRpgs = new HashSet<>(); + for (final List rpgList : rpgs.values()) { + mergedRpgs.add(mergeRemoteProcessGroups(rpgList)); + } + flowDto.setRemoteProcessGroups(mergedRpgs); + + + // Merge Process Groups + final Set mergedGroups = new HashSet<>(); + for (final List groupList : processGroups.values()) { + mergedGroups.add(mergeProcessGroups(groupList)); + } + flowDto.setProcessGroups(mergedGroups); + } + + private ConnectionEntity mergeConnections(final List connections) { + final ConnectionEntity merged = connections.get(0); + final ConnectionStatusDTO statusDto = merged.getStatus(); + statusDto.setNodeSnapshots(null); + + for (final ConnectionEntity entity : connections) { + if (entity != merged) { + StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot()); + } + } + + return merged; + } + + private PortEntity mergePorts(final List ports) { + final PortEntity merged = ports.get(0); + final PortStatusDTO statusDto = merged.getStatus(); + statusDto.setNodeSnapshots(null); + + for (final PortEntity entity : ports) { + if (entity != merged) { + StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot()); + } + } + + return merged; + } + + private FunnelEntity mergeFunnels(final List funnels) { + return funnels.get(0); + } + + private LabelEntity mergeLabels(final List labels) { + return labels.get(0); + } + + private ProcessorEntity mergeProcessors(final List processors) { + final ProcessorEntity merged = processors.get(0); + final ProcessorStatusDTO statusDto = merged.getStatus(); + statusDto.setNodeSnapshots(null); + + for (final ProcessorEntity entity : processors) { + if (entity != merged) { + StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot()); + } + } + + return merged; + } + + + private RemoteProcessGroupEntity mergeRemoteProcessGroups(final List rpgs) { + final RemoteProcessGroupEntity merged = rpgs.get(0); + final RemoteProcessGroupStatusDTO statusDto = merged.getStatus(); + statusDto.setNodeSnapshots(null); + + for (final RemoteProcessGroupEntity entity : rpgs) { + if (entity != merged) { + StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot()); + } + } + + return merged; + } + + private ProcessGroupEntity mergeProcessGroups(final List groups) { + final ProcessGroupEntity merged = groups.get(0); + final ProcessGroupStatusDTO statusDto = merged.getStatus(); + statusDto.setNodeSnapshots(null); + + for (final ProcessGroupEntity entity : groups) { + if (entity != merged) { + StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot()); + } + } + + // We merge only the statuses of the Remote Process Groups. The child components are not + // necessary for a FlowProcessGroupDTO, so we just ensure that they are null + merged.getComponent().setContents(null); + + return merged; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowSnippetEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowSnippetEndpointMerger.java index 2063de4d4996..78ccad6ea76e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowSnippetEndpointMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/FlowSnippetEndpointMerger.java @@ -57,22 +57,12 @@ public NodeResponse merge(final URI uri, final String method, Set final FlowDTO nodeContents = nodeResponseEntity.getFlow(); for (final ProcessorEntity nodeProcessor : nodeContents.getProcessors()) { - Map innerMap = processorMap.get(nodeProcessor.getId()); - if (innerMap == null) { - innerMap = new HashMap<>(); - processorMap.put(nodeProcessor.getId(), innerMap); - } - + Map innerMap = processorMap.computeIfAbsent(nodeProcessor.getId(), id -> new HashMap<>()); innerMap.put(nodeResponse.getNodeId(), nodeProcessor); } for (final RemoteProcessGroupEntity nodeRemoteProcessGroup : nodeContents.getRemoteProcessGroups()) { - Map innerMap = remoteProcessGroupMap.get(nodeRemoteProcessGroup.getId()); - if (innerMap == null) { - innerMap = new HashMap<>(); - remoteProcessGroupMap.put(nodeRemoteProcessGroup.getId(), innerMap); - } - + Map innerMap = remoteProcessGroupMap.computeIfAbsent(nodeRemoteProcessGroup.getId(), id -> new HashMap<>()); innerMap.put(nodeResponse.getNodeId(), nodeRemoteProcessGroup); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ListFlowFilesEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ListFlowFilesEndpointMerger.java index 415334e9b58c..2d17c4946efe 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ListFlowFilesEndpointMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ListFlowFilesEndpointMerger.java @@ -35,7 +35,7 @@ import org.apache.nifi.web.api.dto.QueueSizeDTO; import org.apache.nifi.web.api.entity.ListingRequestEntity; -public class ListFlowFilesEndpointMerger extends AbstractSingleEntityEndpoint { +public class ListFlowFilesEndpointMerger extends AbstractSingleDTOEndpoint { public static final Pattern LISTING_REQUESTS_URI = Pattern.compile("/nifi-api/flowfile-queues/[a-f0-9\\-]{36}/listing-requests"); public static final Pattern LISTING_REQUEST_URI = Pattern.compile("/nifi-api/flowfile-queues/[a-f0-9\\-]{36}/listing-requests/[a-f0-9\\-]{36}"); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProcessorEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProcessorEndpointMerger.java index a892e8ad63f1..bd50eca4078d 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProcessorEndpointMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProcessorEndpointMerger.java @@ -25,11 +25,12 @@ import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger; 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.ProcessorDTO; import org.apache.nifi.web.api.entity.ProcessorEntity; -public class ProcessorEndpointMerger extends AbstractSingleEntityEndpoint implements EndpointResponseMerger { +public class ProcessorEndpointMerger extends AbstractSingleEntityEndpoint implements EndpointResponseMerger { public static final Pattern PROCESSORS_URI_PATTERN = Pattern.compile("/nifi-api/processors"); public static final Pattern PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}"); public static final Pattern CLUSTER_PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/cluster/processors/[a-f0-9\\-]{36}"); @@ -51,12 +52,7 @@ protected Class getEntityClass() { return ProcessorEntity.class; } - @Override - protected ProcessorDTO getDto(final ProcessorEntity entity) { - return entity.getComponent(); - } - @Override protected void mergeResponses(final ProcessorDTO clientDto, final Map dtoMap, final Set successfulResponses, final Set problematicResponses) { final Map> validationErrorMap = new HashMap<>(); @@ -73,6 +69,7 @@ protected void mergeResponses(final ProcessorDTO clientDto, final Map entityMap, final Set successfulResponses, final Set problematicResponses) { @@ -84,6 +81,14 @@ protected void mergeResponses(final ProcessorEntity clientEntity, final Map entry : entityMap.entrySet()) { + final NodeIdentifier nodeId = entry.getKey(); + final ProcessorEntity entity = entry.getValue(); + if (entity != clientEntity) { + StatusMerger.merge(clientEntity.getStatus(), entity.getStatus(), nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort()); + } + } + mergeResponses(clientDto, dtoMap, successfulResponses, problematicResponses); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProvenanceEventEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProvenanceEventEndpointMerger.java index 3f895bd8ab89..c05ca7ffee79 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProvenanceEventEndpointMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ProvenanceEventEndpointMerger.java @@ -27,7 +27,7 @@ import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO; import org.apache.nifi.web.api.entity.ProvenanceEventEntity; -public class ProvenanceEventEndpointMerger extends AbstractSingleEntityEndpoint { +public class ProvenanceEventEndpointMerger extends AbstractSingleDTOEndpoint { public static final Pattern PROVENANCE_EVENT_URI = Pattern.compile("/nifi-api/provenance/events/[0-9]+"); @Override diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/RemoteProcessGroupEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/RemoteProcessGroupEndpointMerger.java index 56636fbe627a..732d5274b348 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/RemoteProcessGroupEndpointMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/RemoteProcessGroupEndpointMerger.java @@ -33,7 +33,7 @@ import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO; import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity; -public class RemoteProcessGroupEndpointMerger extends AbstractSingleEntityEndpoint { +public class RemoteProcessGroupEndpointMerger extends AbstractSingleDTOEndpoint { public static final Pattern REMOTE_PROCESS_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/remote-process-groups"); public static final Pattern REMOTE_PROCESS_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/remote-process-groups/[a-f0-9\\-]{36}"); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ReportingTaskEndpointMerger.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ReportingTaskEndpointMerger.java index b45d8ecf3185..245d3bda8714 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ReportingTaskEndpointMerger.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/endpoints/ReportingTaskEndpointMerger.java @@ -28,7 +28,7 @@ import java.util.Set; import java.util.regex.Pattern; -public class ReportingTaskEndpointMerger extends AbstractSingleEntityEndpoint { +public class ReportingTaskEndpointMerger extends AbstractSingleDTOEndpoint { public static final String REPORTING_TASKS_URI = "/nifi-api/controller/reporting-tasks/node"; public static final Pattern REPORTING_TASK_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/node/[a-f0-9\\-]{36}"); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java index b64c7660d934..a4b762ae1a6f 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/RequestReplicator.java @@ -27,6 +27,7 @@ public interface RequestReplicator { public static final String REQUEST_TRANSACTION_ID_HEADER = "X-RequestTransactionId"; public static final String CLUSTER_ID_GENERATION_SEED_HEADER = "X-Cluster-Id-Generation-Seed"; + public static final String REPLICATION_INDICATOR_HEADER = "X-Request-Replicated"; /** * The HTTP header that the requestor specifies to ask a node if they are able to process a given request. The value @@ -37,21 +38,30 @@ public interface RequestReplicator { public static final String NODE_CONTINUE = "150-NodeContinue"; public static final int NODE_CONTINUE_STATUS_CODE = 150; - - /** - * Starts the instance for replicating requests. Calling this method on an already started instance has no effect. - */ - void start(); + public static final String CLAIM_CANCEL_HEADER = "X-Cancel-Claim"; /** * Stops the instance from replicating requests. Calling this method on a stopped instance has no effect. */ - void stop(); + void shutdown(); + /** - * @return true if the instance is started; false otherwise. + * Replicates a request to each node in the cluster. If the request attempts to modify the flow and there is a node + * that is not currently connected, an Exception will be thrown. Otherwise, the returned AsyncClusterResponse object + * will contain the results that are immediately available, as well as an identifier for obtaining an updated result + * later. + * + * @param method the HTTP method (e.g., POST, PUT) + * @param uri the base request URI (up to, but not including, the query string) + * @param entity an entity + * @param headers any HTTP headers + * @return an AsyncClusterResponse that indicates the current status of the request and provides an identifier for obtaining an updated response later + * + * @throws ConnectingNodeMutableRequestException if the request attempts to modify the flow and there is a node that is in the CONNECTING state + * @throws DisconnectedNodeMutableRequestException if the request attempts to modify the flow and there is a node that is in the DISCONNECTED state */ - boolean isRunning(); + AsyncClusterResponse replicate(String method, URI uri, Object entity, Map headers); /** * Requests are sent to each node in the given set of Node Identifiers. The returned AsyncClusterResponse object will contain diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java index dd4d2cedbf17..ed83159d3e1c 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/replication/ThreadPoolRequestReplicator.java @@ -47,14 +47,14 @@ import org.apache.nifi.cluster.coordination.http.HttpResponseMerger; import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMerger; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; -import org.apache.nifi.cluster.flow.DataFlowManagementService; -import org.apache.nifi.cluster.flow.PersistedFlowState; +import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; import org.apache.nifi.cluster.manager.NodeResponse; import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException; import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException; import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException; +import org.apache.nifi.cluster.manager.exception.NoConnectedNodesException; +import org.apache.nifi.cluster.manager.exception.UnknownNodeException; import org.apache.nifi.cluster.manager.exception.UriConstructionException; -import org.apache.nifi.cluster.manager.impl.WebClusterManager; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.events.EventReporter; import org.apache.nifi.logging.NiFiLog; @@ -76,14 +76,12 @@ public class ThreadPoolRequestReplicator implements RequestReplicator { private static final int MAX_CONCURRENT_REQUESTS = 100; private final Client client; // the client to use for issuing requests - private final int numThreads; // number of threads to use for request replication private final int connectionTimeoutMs; // connection timeout per node request private final int readTimeoutMs; // read timeout per node request private final HttpResponseMerger responseMerger; private final EventReporter eventReporter; private final RequestCompletionCallback callback; private final ClusterCoordinator clusterCoordinator; - private final DataFlowManagementService dfmService; private ExecutorService executorService; private ScheduledExecutorService maintenanceExecutor; @@ -101,8 +99,8 @@ public class ThreadPoolRequestReplicator implements RequestReplicator { * @param eventReporter an EventReporter that can be used to notify users of interesting events. May be null. */ public ThreadPoolRequestReplicator(final int numThreads, final Client client, final ClusterCoordinator clusterCoordinator, - final RequestCompletionCallback callback, final EventReporter eventReporter, final DataFlowManagementService dfmService) { - this(numThreads, client, clusterCoordinator, "3 sec", "3 sec", callback, eventReporter, null, dfmService); + final RequestCompletionCallback callback, final EventReporter eventReporter) { + this(numThreads, client, clusterCoordinator, "5 sec", "5 sec", callback, eventReporter); } /** @@ -117,36 +115,33 @@ public ThreadPoolRequestReplicator(final int numThreads, final Client client, fi * @param eventReporter an EventReporter that can be used to notify users of interesting events. May be null. */ public ThreadPoolRequestReplicator(final int numThreads, final Client client, final ClusterCoordinator clusterCoordinator, - final String connectionTimeout, final String readTimeout, final RequestCompletionCallback callback, final EventReporter eventReporter, - final WebClusterManager clusterManager, final DataFlowManagementService dfmService) { + final String connectionTimeout, final String readTimeout, final RequestCompletionCallback callback, final EventReporter eventReporter) { if (numThreads <= 0) { throw new IllegalArgumentException("The number of threads must be greater than zero."); } else if (client == null) { throw new IllegalArgumentException("Client may not be null."); } - this.numThreads = numThreads; this.client = client; this.clusterCoordinator = clusterCoordinator; this.connectionTimeoutMs = (int) FormatUtils.getTimeDuration(connectionTimeout, TimeUnit.MILLISECONDS); this.readTimeoutMs = (int) FormatUtils.getTimeDuration(readTimeout, TimeUnit.MILLISECONDS); - this.responseMerger = new StandardHttpResponseMerger(clusterManager); + this.responseMerger = new StandardHttpResponseMerger(); this.eventReporter = eventReporter; this.callback = callback; - this.dfmService = dfmService; client.getProperties().put(ClientConfig.PROPERTY_CONNECT_TIMEOUT, connectionTimeoutMs); client.getProperties().put(ClientConfig.PROPERTY_READ_TIMEOUT, readTimeoutMs); client.getProperties().put(ClientConfig.PROPERTY_FOLLOW_REDIRECTS, Boolean.TRUE); - } - @Override - public void start() { - if (isRunning()) { - return; - } + final AtomicInteger threadId = new AtomicInteger(0); + executorService = Executors.newFixedThreadPool(numThreads, r -> { + final Thread t = Executors.defaultThreadFactory().newThread(r); + t.setDaemon(true); + t.setName("Replicate Request Thread-" + threadId.incrementAndGet()); + return t; + }); - executorService = Executors.newFixedThreadPool(numThreads); maintenanceExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() { @Override public Thread newThread(final Runnable r) { @@ -161,25 +156,61 @@ public Thread newThread(final Runnable r) { } @Override - public boolean isRunning() { - return executorService != null && !executorService.isShutdown(); + public void shutdown() { + executorService.shutdown(); + maintenanceExecutor.shutdown(); } @Override - public void stop() { - if (!isRunning()) { - return; + public AsyncClusterResponse replicate(String method, URI uri, Object entity, Map headers) { + final Map> stateMap = clusterCoordinator.getConnectionStates(); + final boolean mutable = isMutableRequest(method, uri.getPath()); + + // If the request is mutable, ensure that all nodes are connected. + if (mutable) { + final List disconnected = stateMap.get(NodeConnectionState.DISCONNECTED); + if (disconnected != null && !disconnected.isEmpty()) { + if (disconnected.size() == 1) { + throw new DisconnectedNodeMutableRequestException("Node " + disconnected.iterator().next() + " is currently disconnected"); + } else { + throw new DisconnectedNodeMutableRequestException(disconnected.size() + " Nodes are currently disconnected"); + } + } + + final List disconnecting = stateMap.get(NodeConnectionState.DISCONNECTING); + if (disconnecting != null && !disconnecting.isEmpty()) { + if (disconnecting.size() == 1) { + throw new DisconnectedNodeMutableRequestException("Node " + disconnecting.iterator().next() + " is currently disconnecting"); + } else { + throw new DisconnectedNodeMutableRequestException(disconnecting.size() + " Nodes are currently disconnecting"); + } + } + + final List connecting = stateMap.get(NodeConnectionState.CONNECTING); + if (connecting != null && !connecting.isEmpty()) { + if (connecting.size() == 1) { + throw new ConnectingNodeMutableRequestException("Node " + connecting.iterator().next() + " is currently connecting"); + } else { + throw new ConnectingNodeMutableRequestException(connecting.size() + " Nodes are currently connecting"); + } + } } - executorService.shutdown(); - maintenanceExecutor.shutdown(); + final List nodeIds = stateMap.get(NodeConnectionState.CONNECTED); + if (nodeIds == null || nodeIds.isEmpty()) { + throw new NoConnectedNodesException(); + } + + final Set nodeIdSet = new HashSet<>(nodeIds); + return replicate(nodeIdSet, method, uri, entity, headers); } @Override public AsyncClusterResponse replicate(Set nodeIds, String method, URI uri, Object entity, Map headers) { - final Map headersPlusIdGenerationSeed = new HashMap<>(headers); - headersPlusIdGenerationSeed.put(RequestReplicator.CLUSTER_ID_GENERATION_SEED_HEADER, UUID.randomUUID().toString()); - return replicate(nodeIds, method, uri, entity, headersPlusIdGenerationSeed, true, null); + final Map updatedHeaders = new HashMap<>(headers); + updatedHeaders.put(RequestReplicator.CLUSTER_ID_GENERATION_SEED_HEADER, UUID.randomUUID().toString()); + updatedHeaders.put(RequestReplicator.REPLICATION_INDICATOR_HEADER, "true"); + return replicate(nodeIds, method, uri, entity, updatedHeaders, true, null); } /** @@ -209,6 +240,18 @@ private AsyncClusterResponse replicate(Set nodeIds, String metho throw new IllegalArgumentException("Cannot replicate request to 0 nodes"); } + // verify all of the nodes exist and are in the proper state + for (final NodeIdentifier nodeId : nodeIds) { + final NodeConnectionStatus status = clusterCoordinator.getConnectionStatus(nodeId); + if (status == null) { + throw new UnknownNodeException("Node " + nodeId + " does not exist in this cluster"); + } + + if (status.getState() != NodeConnectionState.CONNECTED) { + throw new IllegalClusterStateException("Cannot replicate request to Node " + nodeId + " because the node is not connected"); + } + } + logger.debug("Replicating request {} {} with entity {} to {}; response is {}", method, uri, entity, nodeIds, response); // Update headers to indicate the current revision so that we can @@ -238,7 +281,6 @@ private AsyncClusterResponse replicate(Set nodeIds, String metho } logger.debug("For Request ID {}, response object is {}", requestId, response); - // setRevision(updatedHeaders); // if mutable request, we have to do a two-phase commit where we ask each node to verify // that the request can take place and then, if all nodes agree that it can, we can actually @@ -259,7 +301,6 @@ private AsyncClusterResponse replicate(Set nodeIds, String metho finalResponse.add(nodeResponse); }; - // replicate the request to all nodes final Function requestFactory = nodeId -> new NodeHttpRequest(nodeId, method, createURI(uri, nodeId), entity, updatedHeaders, nodeCompletionCallback); @@ -309,6 +350,21 @@ public void onCompletion(final NodeResponse nodeResponse) { return; } + final Thread cancelClaimThread = new Thread(new Runnable() { + @Override + public void run() { + logger.debug("Found {} dissenting nodes for {} {}; canceling claim request", dissentingCount, method, uri.getPath()); + updatedHeaders.put(CLAIM_CANCEL_HEADER, "true"); + + final Function requestFactory = + nodeId -> new NodeHttpRequest(nodeId, method, createURI(uri, nodeId), entity, updatedHeaders, null); + + replicateRequest(nodeIds, uri.getScheme(), uri.getPath(), requestFactory, updatedHeaders); + } + }); + cancelClaimThread.setName("Cancel Claims"); + cancelClaimThread.start(); + // Add a NodeResponse for each node to the Cluster Response // Check that all nodes responded successfully. for (final NodeResponse response : nodeResponses) { @@ -354,9 +410,6 @@ public void onCompletion(final NodeResponse nodeResponse) { } }; - // notify dataflow management service that flow state is not known - dfmService.setPersistedFlowState(PersistedFlowState.UNKNOWN); - // Callback function for generating a NodeHttpRequestCallable that can be used to perform the work final Function requestFactory = nodeId -> new NodeHttpRequest(nodeId, method, createURI(uri, nodeId), entity, updatedHeaders, completionCallback); @@ -449,11 +502,7 @@ private void verifyState(final String httpMethod, final String uriPath) { * @param requestId the ID of the request that has been consumed by the client */ private void onResponseConsumed(final String requestId) { - final AsyncClusterResponse response = responseMap.remove(requestId); - - if (response != null && logger.isDebugEnabled()) { - logTimingInfo(response); - } + responseMap.remove(requestId); } /** @@ -466,12 +515,6 @@ private void onResponseConsumed(final String requestId) { private void onCompletedResponse(final String requestId) { final AsyncClusterResponse response = responseMap.get(requestId); - if (response != null) { - if (isMutableRequest(response.getMethod(), response.getURIPath())) { - dfmService.setPersistedFlowState(PersistedFlowState.STALE); - } - } - if (response != null && callback != null) { try { callback.afterRequest(response.getURIPath(), response.getMethod(), response.getCompletedNodeResponses()); @@ -482,6 +525,10 @@ private void onCompletedResponse(final String requestId) { } } + if (response != null && logger.isDebugEnabled()) { + logTimingInfo(response); + } + // If we have any nodes that are slow to respond, keep track of this. If the same node is slow 3 times in // a row, log a warning to indicate that the node is responding slowly. final Set slowResponseNodes = ResponseUtils.findLongResponseTimes(response, 1.5D); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/CuratorNodeProtocolSender.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/CuratorNodeProtocolSender.java new file mode 100644 index 000000000000..e893c3aacc55 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/CuratorNodeProtocolSender.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.cluster.coordination.node; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.charset.StandardCharsets; + +import org.apache.curator.RetryPolicy; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.retry.RetryNTimes; +import org.apache.nifi.cluster.protocol.AbstractNodeProtocolSender; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.controller.cluster.ZooKeeperClientConfig; +import org.apache.nifi.io.socket.SocketConfiguration; +import org.apache.nifi.util.NiFiProperties; +import org.apache.zookeeper.KeeperException.NoNodeException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Uses Apache Curator to determine the address of the current cluster coordinator + */ +public class CuratorNodeProtocolSender extends AbstractNodeProtocolSender { + private static final Logger logger = LoggerFactory.getLogger(CuratorNodeProtocolSender.class); + + private final String coordinatorPath; + private final ZooKeeperClientConfig zkConfig; + private InetSocketAddress coordinatorAddress; + + + public CuratorNodeProtocolSender(final SocketConfiguration socketConfig, final ProtocolContext protocolContext, final NiFiProperties properties) { + super(socketConfig, protocolContext); + zkConfig = ZooKeeperClientConfig.createConfig(properties); + coordinatorPath = zkConfig.resolvePath("cluster/nodes/coordinator"); + } + + @Override + protected synchronized InetSocketAddress getServiceAddress() throws IOException { + if (coordinatorAddress != null) { + return coordinatorAddress; + } + + final RetryPolicy retryPolicy = new RetryNTimes(0, 0); + final CuratorFramework curatorClient = CuratorFrameworkFactory.newClient(zkConfig.getConnectString(), + zkConfig.getSessionTimeoutMillis(), zkConfig.getConnectionTimeoutMillis(), retryPolicy); + curatorClient.start(); + + try { + // Get coordinator address and add watcher to change who we are heartbeating to if the value changes. + final byte[] coordinatorAddressBytes = curatorClient.getData().usingWatcher(new Watcher() { + @Override + public void process(final WatchedEvent event) { + coordinatorAddress = null; + } + }).forPath(coordinatorPath); + + final String address = new String(coordinatorAddressBytes, StandardCharsets.UTF_8); + + logger.info("Determined that Cluster Coordinator is located at {}; will use this address for sending heartbeat messages", address); + final String[] splits = address.split(":"); + if (splits.length != 2) { + final String message = String.format("Attempted to determine Cluster Coordinator address. Zookeeper indicates " + + "that address is %s, but this is not in the expected format of :", address); + logger.error(message); + throw new ProtocolException(message); + } + + final String hostname = splits[0]; + final int port; + try { + port = Integer.parseInt(splits[1]); + if (port < 1 || port > 65535) { + throw new NumberFormatException("Port must be in the range of 1 - 65535 but got " + port); + } + } catch (final NumberFormatException nfe) { + final String message = String.format("Attempted to determine Cluster Coordinator address. Zookeeper indicates " + + "that address is %s, but the port is not a valid port number", address); + logger.error(message); + throw new ProtocolException(message); + } + + final InetSocketAddress socketAddress = InetSocketAddress.createUnresolved(hostname, port); + coordinatorAddress = socketAddress; + return socketAddress; + } catch (final NoNodeException nne) { + logger.info("No node has yet been elected Cluster Coordinator. Cannot establish connection to cluster yet."); + throw new ProtocolException("No node has yet been elected Cluster Coordinator. Cannot establish connection to cluster yet."); + } catch (Exception e) { + throw new IOException("Unable to determine Cluster Coordinator from ZooKeeper", e); + } finally { + curatorClient.close(); + } + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java new file mode 100644 index 000000000000..6e918da39511 --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java @@ -0,0 +1,772 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.node; + +import java.io.IOException; +import java.util.ArrayList; +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.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import org.apache.commons.collections4.queue.CircularFifoQueue; +import org.apache.nifi.cluster.coordination.ClusterCoordinator; +import org.apache.nifi.cluster.coordination.http.HttpResponseMerger; +import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMerger; +import org.apache.nifi.cluster.coordination.http.replication.RequestCompletionCallback; +import org.apache.nifi.cluster.event.Event; +import org.apache.nifi.cluster.event.NodeEvent; +import org.apache.nifi.cluster.firewall.ClusterNodeFirewall; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.protocol.ComponentRevision; +import org.apache.nifi.cluster.protocol.ConnectionRequest; +import org.apache.nifi.cluster.protocol.ConnectionResponse; +import org.apache.nifi.cluster.protocol.DataFlow; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.StandardDataFlow; +import org.apache.nifi.cluster.protocol.impl.ClusterCoordinationProtocolSenderListener; +import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; +import org.apache.nifi.cluster.protocol.message.DisconnectMessage; +import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType; +import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage; +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.reporting.Severity; +import org.apache.nifi.services.FlowService; +import org.apache.nifi.web.revision.RevisionManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class NodeClusterCoordinator implements ClusterCoordinator, ProtocolHandler, RequestCompletionCallback { + private static final Logger logger = LoggerFactory.getLogger(NodeClusterCoordinator.class); + private static final String EVENT_CATEGORY = "Clustering"; + + private static final Pattern COUNTER_URI_PATTERN = Pattern.compile("/nifi-api/controller/counters/[a-f0-9\\-]{36}"); + + private final String instanceId = UUID.randomUUID().toString(); + private volatile NodeIdentifier nodeId; + + private final ClusterCoordinationProtocolSenderListener senderListener; + private final EventReporter eventReporter; + private final ClusterNodeFirewall firewall; + private final RevisionManager revisionManager; + private volatile FlowService flowService; + private volatile boolean connected; + + private final ConcurrentMap nodeStatuses = new ConcurrentHashMap<>(); + private final ConcurrentMap> nodeEvents = new ConcurrentHashMap<>(); + + public NodeClusterCoordinator(final ClusterCoordinationProtocolSenderListener senderListener, + final EventReporter eventReporter, final ClusterNodeFirewall firewall, final RevisionManager revisionManager) { + this.senderListener = senderListener; + this.flowService = null; + this.eventReporter = eventReporter; + this.firewall = firewall; + this.revisionManager = revisionManager; + senderListener.addHandler(this); + } + + @Override + public void shutdown() { + final NodeConnectionStatus shutdownStatus = new NodeConnectionStatus(getLocalNodeIdentifier(), DisconnectionCode.NODE_SHUTDOWN); + notifyOthersOfNodeStatusChange(shutdownStatus); + logger.info("Successfully notified other nodes that I am shutting down"); + } + + @Override + public void setLocalNodeIdentifier(final NodeIdentifier nodeId) { + this.nodeId = nodeId; + } + + NodeIdentifier getLocalNodeIdentifier() { + return nodeId; + } + + @Override + public void resetNodeStatuses(final Map statusMap) { + logger.info("Resetting cluster node statuses from {} to {}", nodeStatuses, statusMap); + + // For each proposed replacement, update the nodeStatuses map if and only if the replacement + // has a larger update id than the current value. + for (final Map.Entry entry : statusMap.entrySet()) { + final NodeIdentifier nodeId = entry.getKey(); + final NodeConnectionStatus proposedStatus = entry.getValue(); + + boolean updated = false; + while (!updated) { + final NodeConnectionStatus currentStatus = nodeStatuses.get(nodeId); + + if (currentStatus == null || proposedStatus.getUpdateIdentifier() > currentStatus.getUpdateIdentifier()) { + updated = replaceNodeStatus(nodeId, currentStatus, proposedStatus); + } else { + updated = true; + } + } + } + } + + /** + * Attempts to update the nodeStatuses map by changing the value for the given node id from the current status to the new status, as in + * ConcurrentMap.replace(nodeId, currentStatus, newStatus) but with the difference that this method can handle a null value + * for currentStatus + * + * @param nodeId the node id + * @param currentStatus the current status, or null if there is no value currently + * @param newStatus the new status to set + * @return true if the map was updated, false otherwise + */ + private boolean replaceNodeStatus(final NodeIdentifier nodeId, final NodeConnectionStatus currentStatus, final NodeConnectionStatus newStatus) { + if (newStatus == null) { + logger.error("Cannot change node status for {} from {} to {} because new status is null", nodeId, currentStatus, newStatus); + logger.error("", new NullPointerException()); + } + + if (currentStatus == null) { + final NodeConnectionStatus existingValue = nodeStatuses.putIfAbsent(nodeId, newStatus); + return existingValue == null; + } + + return nodeStatuses.replace(nodeId, currentStatus, newStatus); + } + + @Override + public void requestNodeConnect(final NodeIdentifier nodeId, final String userDn) { + if (userDn == null) { + reportEvent(nodeId, Severity.INFO, "Requesting that node connect to cluster"); + } else { + reportEvent(nodeId, Severity.INFO, "Requesting that node connect to cluster on behalf of " + userDn); + } + + updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTING, System.currentTimeMillis())); + + // create the request + final ReconnectionRequestMessage request = new ReconnectionRequestMessage(); + request.setNodeId(nodeId); + request.setInstanceId(instanceId); + + requestReconnectionAsynchronously(request, 10, 5); + } + + @Override + public void finishNodeConnection(final NodeIdentifier nodeId) { + final NodeConnectionState state = getConnectionState(nodeId); + if (state == null) { + logger.debug("Attempted to finish node connection for {} but node is not known. Requesting that node connect", nodeId); + requestNodeConnect(nodeId, null); + return; + } + + if (state == NodeConnectionState.CONNECTED) { + // already connected. Nothing to do. + return; + } + + if (state == NodeConnectionState.DISCONNECTED || state == NodeConnectionState.DISCONNECTING) { + logger.debug("Attempted to finish node connection for {} but node state was {}. Requesting that node connect", nodeId, state); + requestNodeConnect(nodeId, null); + return; + } + + logger.info("{} is now connected", nodeId); + final boolean updated = updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED)); + if (!updated) { + logger.error("Attempting to Finish Node Connection but could not find Node with Identifier {}", nodeId); + } + } + + @Override + public void requestNodeDisconnect(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) { + logger.info("Requesting that {} disconnect due to {}", nodeId, explanation == null ? disconnectionCode : explanation); + + updateNodeStatus(new NodeConnectionStatus(nodeId, disconnectionCode, explanation)); + + // There is no need to tell the node that it's disconnected if it is due to being + // shutdown, as we will not be able to connect to the node anyway. + if (disconnectionCode == DisconnectionCode.NODE_SHUTDOWN) { + return; + } + + final DisconnectMessage request = new DisconnectMessage(); + request.setNodeId(nodeId); + request.setExplanation(explanation); + + addNodeEvent(nodeId, "Disconnection requested due to " + explanation); + disconnectAsynchronously(request, 10, 5); + } + + @Override + public void disconnectionRequestedByNode(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) { + logger.info("{} requested disconnection from cluster due to {}", nodeId, explanation == null ? disconnectionCode : explanation); + updateNodeStatus(new NodeConnectionStatus(nodeId, disconnectionCode, explanation)); + + final Severity severity; + switch (disconnectionCode) { + case STARTUP_FAILURE: + case MISMATCHED_FLOWS: + case UNKNOWN: + severity = Severity.ERROR; + break; + default: + severity = Severity.INFO; + break; + } + + reportEvent(nodeId, severity, "Node disconnected from cluster due to " + explanation); + } + + @Override + public void removeNode(final NodeIdentifier nodeId, final String userDn) { + reportEvent(nodeId, Severity.INFO, "User " + userDn + " requested that node be removed from cluster"); + nodeStatuses.remove(nodeId); + nodeEvents.remove(nodeId); + notifyOthersOfNodeStatusChange(new NodeConnectionStatus(nodeId, NodeConnectionState.REMOVED)); + } + + @Override + public NodeConnectionStatus getConnectionStatus(final NodeIdentifier nodeId) { + return nodeStatuses.get(nodeId); + } + + private NodeConnectionState getConnectionState(final NodeIdentifier nodeId) { + final NodeConnectionStatus status = getConnectionStatus(nodeId); + return status == null ? null : status.getState(); + } + + + @Override + public Map> getConnectionStates() { + final Map> connectionStates = new HashMap<>(); + for (final Map.Entry entry : nodeStatuses.entrySet()) { + final NodeConnectionState state = entry.getValue().getState(); + final List nodeIds = connectionStates.computeIfAbsent(state, s -> new ArrayList()); + nodeIds.add(entry.getKey()); + } + + return connectionStates; + } + + @Override + public boolean isBlockedByFirewall(final String hostname) { + return firewall != null && !firewall.isPermissible(hostname); + } + + @Override + public void reportEvent(final NodeIdentifier nodeId, final Severity severity, final String event) { + eventReporter.reportEvent(severity, nodeId == null ? EVENT_CATEGORY : nodeId.toString(), event); + if (nodeId != null) { + addNodeEvent(nodeId, severity, event); + } + + final String message = nodeId == null ? event : "Event Reported for " + nodeId + ": " + event; + switch (severity) { + case ERROR: + logger.error(message); + break; + case WARNING: + logger.warn(message); + break; + case INFO: + logger.info(message); + break; + } + } + + @Override + public synchronized void updateNodeRoles(final NodeIdentifier nodeId, final Set roles) { + boolean updated = false; + while (!updated) { + final NodeConnectionStatus currentStatus = nodeStatuses.get(nodeId); + if (currentStatus == null) { + throw new IllegalStateException("Cannot update roles for " + nodeId + " to " + roles + " because the node is not part of this cluster"); + } + + if (currentStatus.getRoles().equals(roles)) { + logger.debug("Roles for {} already up-to-date as {}", nodeId, roles); + return; + } + + final NodeConnectionStatus updatedStatus = new NodeConnectionStatus(currentStatus, roles); + updated = replaceNodeStatus(nodeId, currentStatus, updatedStatus); + + if (updated) { + logger.info("Updated Roles of {} from {} to {}", nodeId, currentStatus, updatedStatus); + notifyOthersOfNodeStatusChange(updatedStatus); + } + } + + // If any other node contains any of the given roles, revoke the role from the other node. + for (final String role : roles) { + for (final Map.Entry entry : nodeStatuses.entrySet()) { + if (entry.getKey().equals(nodeId)) { + continue; + } + + updated = false; + while (!updated) { + final NodeConnectionStatus status = entry.getValue(); + if (status.getRoles().contains(role)) { + final Set newRoles = new HashSet<>(status.getRoles()); + newRoles.remove(role); + + final NodeConnectionStatus updatedStatus = new NodeConnectionStatus(status, newRoles); + updated = replaceNodeStatus(entry.getKey(), status, updatedStatus); + + if (updated) { + logger.info("Updated Roles of {} from {} to {}", nodeId, status, updatedStatus); + notifyOthersOfNodeStatusChange(updatedStatus); + } + } else { + updated = true; + } + } + } + } + } + + @Override + public NodeIdentifier getNodeIdentifier(final String uuid) { + for (final NodeIdentifier nodeId : nodeStatuses.keySet()) { + if (nodeId.getId().equals(uuid)) { + return nodeId; + } + } + + return null; + } + + + @Override + public Set getNodeIdentifiers(final NodeConnectionState... states) { + final Set statesOfInterest = new HashSet<>(); + if (states.length == 0) { + for (final NodeConnectionState state : NodeConnectionState.values()) { + statesOfInterest.add(state); + } + } else { + for (final NodeConnectionState state : states) { + statesOfInterest.add(state); + } + } + + return nodeStatuses.entrySet().stream() + .filter(entry -> statesOfInterest.contains(entry.getValue().getState())) + .map(entry -> entry.getKey()) + .collect(Collectors.toSet()); + } + + @Override + public NodeIdentifier getPrimaryNode() { + return nodeStatuses.values().stream() + .filter(status -> status.getRoles().contains(ClusterRoles.PRIMARY_NODE)) + .findFirst() + .map(status -> status.getNodeIdentifier()) + .orElse(null); + } + + @Override + public List getNodeEvents(final NodeIdentifier nodeId) { + final CircularFifoQueue eventQueue = nodeEvents.get(nodeId); + if (eventQueue == null) { + return Collections.emptyList(); + } + + synchronized (eventQueue) { + return new ArrayList<>(eventQueue); + } + } + + @Override + public void setFlowService(final FlowService flowService) { + if (this.flowService != null) { + throw new IllegalStateException("Flow Service has already been set"); + } + this.flowService = flowService; + } + + private void addNodeEvent(final NodeIdentifier nodeId, final String event) { + addNodeEvent(nodeId, Severity.INFO, event); + } + + private void addNodeEvent(final NodeIdentifier nodeId, final Severity severity, final String message) { + final NodeEvent event = new Event(nodeId.toString(), message, severity); + final CircularFifoQueue eventQueue = nodeEvents.computeIfAbsent(nodeId, id -> new CircularFifoQueue<>()); + synchronized (eventQueue) { + eventQueue.add(event); + } + } + + /** + * Updates the status of the node with the given ID to the given status and returns true + * if successful, false if no node exists with the given ID + * + * @param status the new status of the node + * @return true if the node exists and is updated, false if the node does not exist + */ + // visible for testing. + boolean updateNodeStatus(final NodeConnectionStatus status) { + final NodeIdentifier nodeId = status.getNodeIdentifier(); + + // In this case, we are using nodeStatuses.put() instead of getting the current value and + // comparing that to the new value and using the one with the largest update id. This is because + // this method is called when something occurs that causes this node to change the status of the + // node in question. We only use comparisons against the current value when we receive an update + // about a node status from a different node, since those may be received out-of-order. + final NodeConnectionStatus currentStatus = nodeStatuses.put(nodeId, status); + final NodeConnectionState currentState = currentStatus == null ? null : currentStatus.getState(); + logger.info("Status of {} changed from {} to {}", nodeId, currentStatus, status); + logger.debug("State of cluster nodes is now {}", nodeStatuses); + + if (currentState == null || currentState != status.getState()) { + notifyOthersOfNodeStatusChange(status); + } + + return true; + } + + + private void notifyOthersOfNodeStatusChange(final NodeConnectionStatus updatedStatus) { + final Set nodesToNotify = getNodeIdentifiers(NodeConnectionState.CONNECTED, NodeConnectionState.CONNECTING); + + // Do not notify ourselves because we already know about the status update. + nodesToNotify.remove(getLocalNodeIdentifier()); + + final NodeStatusChangeMessage message = new NodeStatusChangeMessage(); + message.setNodeId(updatedStatus.getNodeIdentifier()); + message.setNodeConnectionStatus(updatedStatus); + senderListener.notifyNodeStatusChange(nodesToNotify, message); + } + + private void disconnectAsynchronously(final DisconnectMessage request, final int attempts, final int retrySeconds) { + final Thread disconnectThread = new Thread(new Runnable() { + @Override + public void run() { + final NodeIdentifier nodeId = request.getNodeId(); + + for (int i = 0; i < attempts; i++) { + try { + senderListener.disconnect(request); + reportEvent(nodeId, Severity.INFO, "Node disconnected due to " + request.getExplanation()); + return; + } catch (final Exception e) { + logger.error("Failed to notify {} that it has been disconnected from the cluster due to {}", request.getNodeId(), request.getExplanation()); + + try { + Thread.sleep(retrySeconds * 1000L); + } catch (final InterruptedException ie) { + Thread.currentThread().interrupt(); + return; + } + } + } + } + }, "Disconnect " + request.getNodeId()); + + disconnectThread.start(); + } + + private void requestReconnectionAsynchronously(final ReconnectionRequestMessage request, final int reconnectionAttempts, final int retrySeconds) { + final Thread reconnectionThread = new Thread(new Runnable() { + @Override + public void run() { + // create the request + while (flowService == null) { + try { + Thread.sleep(100L); + } catch (final InterruptedException ie) { + logger.info("Could not send Reconnection request to {} because thread was " + + "interrupted before FlowService was made available", request.getNodeId()); + Thread.currentThread().interrupt(); + return; + } + } + + for (int i = 0; i < reconnectionAttempts; i++) { + try { + if (NodeConnectionState.CONNECTING != getConnectionState(request.getNodeId())) { + // the node status has changed. It's no longer appropriate to attempt reconnection. + return; + } + + request.setDataFlow(new StandardDataFlow(flowService.createDataFlow())); + request.setNodeConnectionStatuses(new ArrayList<>(nodeStatuses.values())); + request.setComponentRevisions(revisionManager.getAllRevisions().stream().map(rev -> ComponentRevision.fromRevision(rev)).collect(Collectors.toList())); + + // Issue a reconnection request to the node. + senderListener.requestReconnection(request); + + // successfully told node to reconnect -- we're done! + logger.info("Successfully requested that {} join the cluster", request.getNodeId()); + + return; + } catch (final Exception e) { + logger.warn("Problem encountered issuing reconnection request to node " + request.getNodeId(), e); + eventReporter.reportEvent(Severity.WARNING, EVENT_CATEGORY, "Problem encountered issuing reconnection request to node " + + request.getNodeId() + " due to: " + e); + } + + try { + Thread.sleep(1000L * retrySeconds); + } catch (final InterruptedException ie) { + break; + } + } + + // We failed to reconnect too many times. We must now mark node as disconnected. + if (NodeConnectionState.CONNECTING == getConnectionState(request.getNodeId())) { + requestNodeDisconnect(request.getNodeId(), DisconnectionCode.UNABLE_TO_COMMUNICATE, + "Attempted to request that node reconnect to cluster but could not communicate with node"); + } + } + }, "Reconnect " + request.getNodeId()); + + reconnectionThread.start(); + } + + @Override + public ProtocolMessage handle(final ProtocolMessage protocolMessage) throws ProtocolException { + switch (protocolMessage.getType()) { + case CONNECTION_REQUEST: + return handleConnectionRequest((ConnectionRequestMessage) protocolMessage); + case NODE_STATUS_CHANGE: + handleNodeStatusChange((NodeStatusChangeMessage) protocolMessage); + return null; + default: + throw new ProtocolException("Cannot handle Protocol Message " + protocolMessage + " because it is not of the correct type"); + } + } + + private void handleNodeStatusChange(final NodeStatusChangeMessage statusChangeMessage) { + final NodeConnectionStatus updatedStatus = statusChangeMessage.getNodeConnectionStatus(); + final NodeIdentifier nodeId = statusChangeMessage.getNodeId(); + logger.debug("Handling request {}", statusChangeMessage); + + boolean updated = false; + while (!updated) { + final NodeConnectionStatus oldStatus = nodeStatuses.get(statusChangeMessage.getNodeId()); + if (oldStatus == null || updatedStatus.getUpdateIdentifier() >= oldStatus.getUpdateIdentifier()) { + // Either remove the value from the map or update the map depending on the connection state + if (statusChangeMessage.getNodeConnectionStatus().getState() == NodeConnectionState.REMOVED) { + updated = nodeStatuses.remove(nodeId, oldStatus); + } else { + updated = replaceNodeStatus(nodeId, oldStatus, updatedStatus); + } + + if (updated) { + logger.info("Status of {} changed from {} to {}", statusChangeMessage.getNodeId(), oldStatus, updatedStatus); + + // Update our counter so that we are in-sync with the cluster on the + // most up-to-date version of the NodeConnectionStatus' Update Identifier. + // We do this so that we can accurately compare status updates that are generated + // locally against those generated from other nodes in the cluster. + NodeConnectionStatus.updateIdGenerator(updatedStatus.getUpdateIdentifier()); + } + } else { + updated = true; + logger.info("Received Node Status update that indicates that {} should change to {} but disregarding because the current state of {} is newer", + nodeId, updatedStatus, oldStatus); + } + } + } + + private NodeIdentifier resolveNodeId(final NodeIdentifier proposedIdentifier) { + final NodeConnectionStatus proposedConnectionStatus = new NodeConnectionStatus(proposedIdentifier, DisconnectionCode.NOT_YET_CONNECTED); + final NodeConnectionStatus existingStatus = nodeStatuses.putIfAbsent(proposedIdentifier, proposedConnectionStatus); + + NodeIdentifier resolvedNodeId = proposedIdentifier; + if (existingStatus == null) { + // there is no node with that ID + resolvedNodeId = proposedIdentifier; + logger.debug("No existing node with ID {}; resolved node ID is as-proposed", proposedIdentifier.getId()); + } else if (existingStatus.getNodeIdentifier().logicallyEquals(proposedIdentifier)) { + // there is a node with that ID but it's the same node. + resolvedNodeId = proposedIdentifier; + logger.debug("No existing node with ID {}; resolved node ID is as-proposed", proposedIdentifier.getId()); + } else { + // there is a node with that ID and it's a different node + resolvedNodeId = new NodeIdentifier(UUID.randomUUID().toString(), proposedIdentifier.getApiAddress(), proposedIdentifier.getApiPort(), + proposedIdentifier.getSocketAddress(), proposedIdentifier.getSocketPort(), + proposedIdentifier.getSiteToSiteAddress(), proposedIdentifier.getSiteToSitePort(), proposedIdentifier.isSiteToSiteSecure()); + logger.debug("A node already exists with ID {}. Proposed Node Identifier was {}; existing Node Identifier is {}; Resolved Node Identifier is {}", + proposedIdentifier.getId(), proposedIdentifier, getNodeIdentifier(proposedIdentifier.getId()), resolvedNodeId); + } + + return resolvedNodeId; + } + + private ConnectionResponseMessage handleConnectionRequest(final ConnectionRequestMessage requestMessage) { + final NodeIdentifier proposedIdentifier = requestMessage.getConnectionRequest().getProposedNodeIdentifier(); + final ConnectionRequest requestWithDn = new ConnectionRequest(addRequestorDn(proposedIdentifier, requestMessage.getRequestorDN())); + + // Resolve Node identifier. + final NodeIdentifier resolvedNodeId = resolveNodeId(proposedIdentifier); + final ConnectionResponse response = createConnectionResponse(requestWithDn, resolvedNodeId); + final ConnectionResponseMessage responseMessage = new ConnectionResponseMessage(); + responseMessage.setConnectionResponse(response); + return responseMessage; + } + + private ConnectionResponse createConnectionResponse(final ConnectionRequest request, final NodeIdentifier resolvedNodeIdentifier) { + if (isBlockedByFirewall(resolvedNodeIdentifier.getSocketAddress())) { + // if the socket address is not listed in the firewall, then return a null response + logger.info("Firewall blocked connection request from node " + resolvedNodeIdentifier); + return ConnectionResponse.createBlockedByFirewallResponse(); + } + + // Set node's status to 'CONNECTING' + NodeConnectionStatus status = getConnectionStatus(resolvedNodeIdentifier); + if (status == null) { + addNodeEvent(resolvedNodeIdentifier, "Connection requested from new node. Setting status to connecting."); + } else { + addNodeEvent(resolvedNodeIdentifier, "Connection requested from existing node. Setting status to connecting"); + } + + status = new NodeConnectionStatus(resolvedNodeIdentifier, NodeConnectionState.CONNECTING, System.currentTimeMillis()); + updateNodeStatus(status); + + DataFlow dataFlow = null; + if (flowService != null) { + try { + dataFlow = flowService.createDataFlow(); + } catch (final IOException ioe) { + logger.error("Unable to obtain current dataflow from FlowService in order to provide the flow to " + + resolvedNodeIdentifier + ". Will tell node to try again later", ioe); + } + } + + if (dataFlow == null) { + // Create try-later response based on flow retrieval delay to give + // the flow management service a chance to retrieve a current flow + final int tryAgainSeconds = 5; + addNodeEvent(resolvedNodeIdentifier, Severity.WARNING, "Connection requested from node, but manager was unable to obtain current flow. " + + "Instructing node to try again in " + tryAgainSeconds + " seconds."); + + // return try later response + return new ConnectionResponse(tryAgainSeconds); + } + + // TODO: Remove the 'null' values here from the ConnectionResponse all together. These + // will no longer be needed for site-to-site once the NCM is gone. + return new ConnectionResponse(resolvedNodeIdentifier, dataFlow, null, null, instanceId, + new ArrayList<>(nodeStatuses.values()), + revisionManager.getAllRevisions().stream().map(rev -> ComponentRevision.fromRevision(rev)).collect(Collectors.toList())); + } + + private NodeIdentifier addRequestorDn(final NodeIdentifier nodeId, final String dn) { + return new NodeIdentifier(nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort(), + nodeId.getSocketAddress(), nodeId.getSocketPort(), + nodeId.getSiteToSiteAddress(), nodeId.getSiteToSitePort(), nodeId.isSiteToSiteSecure(), dn); + } + + @Override + public boolean canHandle(final ProtocolMessage msg) { + return MessageType.CONNECTION_REQUEST == msg.getType() || MessageType.NODE_STATUS_CHANGE == msg.getType(); + } + + private boolean isMutableRequest(final String method) { + return "DELETE".equalsIgnoreCase(method) || "POST".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method); + } + + + /** + * Callback that is called after an HTTP Request has been replicated to nodes in the cluster. + * This allows us to disconnect nodes that did not complete the request, if applicable. + */ + @Override + public void afterRequest(final String uriPath, final String method, final Set nodeResponses) { + final boolean mutableRequest = isMutableRequest(method); + + /* + * Nodes that encountered issues handling the request are marked as + * disconnected for mutable requests (e.g., post, put, delete). For + * other requests (e.g., get, head), the nodes remain in their current + * state even if they had problems handling the request. + */ + if (mutableRequest) { + final HttpResponseMerger responseMerger = new StandardHttpResponseMerger(); + final Set problematicNodeResponses = responseMerger.getProblematicNodeResponses(nodeResponses); + + // all nodes failed + final boolean allNodesFailed = problematicNodeResponses.size() == nodeResponses.size(); + + // some nodes had a problematic response because of a missing counter, ensure the are not disconnected + final boolean someNodesFailedMissingCounter = !problematicNodeResponses.isEmpty() + && problematicNodeResponses.size() < nodeResponses.size() && isMissingCounter(problematicNodeResponses, uriPath); + + // ensure nodes stay connected in certain scenarios + if (allNodesFailed) { + logger.warn("All nodes failed to process URI {} {}. As a result, no node will be disconnected from cluster", method, uriPath); + return; + } + + if (someNodesFailedMissingCounter) { + return; + } + + // disconnect problematic nodes + if (!problematicNodeResponses.isEmpty() && problematicNodeResponses.size() < nodeResponses.size()) { + logger.warn(String.format("The following nodes failed to process URI %s '%s'. Requesting each node to disconnect from cluster: ", uriPath, problematicNodeResponses)); + for (final NodeResponse nodeResponse : problematicNodeResponses) { + requestNodeDisconnect(nodeResponse.getNodeId(), DisconnectionCode.FAILED_TO_SERVICE_REQUEST, "Failed to process URI to " + method + " " + uriPath); + } + } + } + } + + /** + * Determines if all problematic responses were due to 404 NOT_FOUND. Assumes that problematicNodeResponses is not empty and is not comprised of responses from all nodes in the cluster (at least + * one node contained the counter in question). + * + * @param problematicNodeResponses The problematic node responses + * @param uriPath The path of the URI for the request + * @return Whether all problematic node responses were due to a missing counter + */ + private boolean isMissingCounter(final Set problematicNodeResponses, final String uriPath) { + if (COUNTER_URI_PATTERN.matcher(uriPath).matches()) { + boolean notFound = true; + for (final NodeResponse problematicResponse : problematicNodeResponses) { + if (problematicResponse.getStatus() != 404) { + notFound = false; + break; + } + } + return notFound; + } + return false; + } + + @Override + public void setConnected(final boolean connected) { + this.connected = connected; + } + + @Override + public boolean isConnected() { + return connected; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/ClusterNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeStateManager.java similarity index 53% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/ClusterNode.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeStateManager.java index d04d144fe8ad..6bfdbe4844e3 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/ClusterNode.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeStateManager.java @@ -17,23 +17,25 @@ package org.apache.nifi.cluster.coordination.node; -import org.apache.nifi.cluster.protocol.NodeIdentifier; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; -public class ClusterNode { - private final NodeIdentifier nodeId; - private NodeConnectionStatus connectionStatus = new NodeConnectionStatus(NodeConnectionState.DISCONNECTED, DisconnectionCode.NOT_YET_CONNECTED); +import org.apache.nifi.cluster.protocol.NodeIdentifier; +public final class NodeStateManager { + private final ConcurrentMap nodeStatuses = new ConcurrentHashMap<>(); - public ClusterNode(final NodeIdentifier nodeId) { - this.nodeId = nodeId; + public NodeConnectionStatus getConnectionStatus(final NodeIdentifier nodeId) { + return nodeStatuses.get(nodeId); } - public NodeIdentifier getIdentifier() { - return nodeId; + public void setConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionStatus status) { + nodeStatuses.put(nodeId, status); + // TODO: Notify other nodes } - public NodeConnectionStatus getConnectionStatus() { - return connectionStatus; + public boolean transitionConnectionStatus(final NodeIdentifier nodeId, final NodeConnectionStatus currentStatus, final NodeConnectionStatus newStatus) { + return nodeStatuses.replace(nodeId, currentStatus, newStatus); + // TODO: Notify other nodes } - -} \ No newline at end of file +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/Event.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/Event.java index 9c61f953d8c1..8310857422c5 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/Event.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/Event.java @@ -19,6 +19,7 @@ import java.util.Date; import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.reporting.Severity; /** * Events describe the occurrence of something noteworthy. They record the event's source, a timestamp, a description, and a category. @@ -26,21 +27,11 @@ * * @Immutable */ -public class Event { - - public static enum Category { - - DEBUG, - INFO, - WARN - } +public class Event implements NodeEvent { private final String source; - private final long timestamp; - - private final Category category; - + private final Severity severity; private final String message; /** @@ -50,7 +41,7 @@ public static enum Category { * @param message the description */ public Event(final String source, final String message) { - this(source, message, Category.INFO); + this(source, message, Severity.INFO); } /** @@ -58,10 +49,10 @@ public Event(final String source, final String message) { * * @param source the source * @param message the description - * @param category the event category + * @param severity the event severity */ - public Event(final String source, final String message, final Category category) { - this(source, message, category, new Date().getTime()); + public Event(final String source, final String message, final Severity severity) { + this(source, message, severity, new Date().getTime()); } /** @@ -72,7 +63,7 @@ public Event(final String source, final String message, final Category category) * @param timestamp the time of occurrence */ public Event(final String source, final String message, final long timestamp) { - this(source, message, Category.INFO, timestamp); + this(source, message, Severity.INFO, timestamp); } /** @@ -80,16 +71,15 @@ public Event(final String source, final String message, final long timestamp) { * * @param source the source * @param message the description - * @param category the event category + * @param severity the event category * @param timestamp the time of occurrence */ - public Event(final String source, final String message, final Category category, final long timestamp) { - + public Event(final String source, final String message, final Severity severity, final long timestamp) { if (StringUtils.isBlank(source)) { throw new IllegalArgumentException("Source may not be empty or null."); } else if (StringUtils.isBlank(message)) { throw new IllegalArgumentException("Event message may not be empty or null."); - } else if (category == null) { + } else if (severity == null) { throw new IllegalArgumentException("Event category may not be null."); } else if (timestamp < 0) { throw new IllegalArgumentException("Timestamp may not be negative: " + timestamp); @@ -97,24 +87,27 @@ public Event(final String source, final String message, final Category category, this.source = source; this.message = message; - this.category = category; + this.severity = severity; this.timestamp = timestamp; } - public Category getCategory() { - return category; + @Override + public Severity getSeverity() { + return severity; } + @Override public String getMessage() { return message; } + @Override public String getSource() { return source; } + @Override public long getTimestamp() { return timestamp; } - } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java deleted file mode 100644 index 8411e5b8ef6e..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.event; - -import java.util.List; - -/** - * Manages an ordered list of events. The event history size dictates the total number of events to manage for a given source at a given time. When the size is exceeded, the oldest event for that - * source is evicted. - * - */ -public interface EventManager { - - /** - * Adds an event to the manager. - * - * @param event an Event - */ - void addEvent(Event event); - - /** - * Returns a list of events for a given source sorted by the event's timestamp where the most recent event is first in the list. - * - * @param eventSource the source - * - * @return the list of events - */ - List getEvents(String eventSource); - - /* - * Returns the most recent event for the source. If no events exist, then - * null is returned. - */ - Event getMostRecentEvent(String eventSource); - - /* - * Clears all events for the given source. - */ - void clearEventHistory(String eventSource); - - /** - * Returns the history size. - * - * @return the history size - */ - int getEventHistorySize(); - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java deleted file mode 100644 index d94910025829..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java +++ /dev/null @@ -1,141 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.event.impl; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.PriorityQueue; -import java.util.Queue; - -import org.apache.nifi.cluster.event.Event; -import org.apache.nifi.cluster.event.EventManager; - -/** - * Implements the EventManager. - * - */ -public class EventManagerImpl implements EventManager { - - /** - * associates the source ID with an ordered queue of events, ordered by most recent event - */ - private final Map> eventsMap = new HashMap<>(); - - /** - * the number of events to maintain for a given source - */ - private final int eventHistorySize; - - /** - * Creates an instance. - * - * @param eventHistorySize the number of events to manage for a given source. Value must be positive. - */ - public EventManagerImpl(final int eventHistorySize) { - if (eventHistorySize <= 0) { - throw new IllegalArgumentException("Event history size must be positive: " + eventHistorySize); - } - this.eventHistorySize = eventHistorySize; - } - - @Override - public void addEvent(final Event event) { - - if (event == null) { - throw new IllegalArgumentException("Event may not be null."); - } - - Queue events = eventsMap.get(event.getSource()); - if (events == null) { - // no events from this source, so add a new queue to the map - events = new PriorityQueue<>(eventHistorySize, createEventComparator()); - eventsMap.put(event.getSource(), events); - } - - // add event - events.add(event); - - // if we exceeded the history size, then evict the oldest event - if (events.size() > eventHistorySize) { - removeOldestEvent(events); - } - - } - - @Override - public List getEvents(final String eventSource) { - final Queue events = eventsMap.get(eventSource); - if (events == null) { - return Collections.EMPTY_LIST; - } else { - return Collections.unmodifiableList(new ArrayList<>(events)); - } - } - - @Override - public int getEventHistorySize() { - return eventHistorySize; - } - - @Override - public Event getMostRecentEvent(final String eventSource) { - final Queue events = eventsMap.get(eventSource); - if (events == null) { - return null; - } else { - return events.peek(); - } - } - - @Override - public void clearEventHistory(final String eventSource) { - eventsMap.remove(eventSource); - } - - private Comparator createEventComparator() { - return new Comparator() { - @Override - public int compare(final Event o1, final Event o2) { - // orders events by most recent first - return (int) (o2.getTimestamp() - o1.getTimestamp()); - } - }; - } - - private void removeOldestEvent(final Collection events) { - - if (events.isEmpty()) { - return; - } - - Event oldestEvent = null; - for (final Event event : events) { - if (oldestEvent == null || oldestEvent.getTimestamp() > event.getTimestamp()) { - oldestEvent = event; - } - } - - events.remove(oldestEvent); - - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java deleted file mode 100644 index a5e78bfdcac5..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.flow; - -import org.apache.nifi.cluster.protocol.NodeIdentifier; -import org.apache.nifi.cluster.protocol.StandardDataFlow; - -/** - * A dataflow with additional information about the cluster. - * - */ -public class ClusterDataFlow { - - private final StandardDataFlow dataFlow; - private final NodeIdentifier primaryNodeId; - private final byte[] controllerServices; - private final byte[] reportingTasks; - - public ClusterDataFlow(final StandardDataFlow dataFlow, final NodeIdentifier primaryNodeId, final byte[] controllerServices, final byte[] reportingTasks) { - this.dataFlow = dataFlow; - this.primaryNodeId = primaryNodeId; - this.controllerServices = controllerServices; - this.reportingTasks = reportingTasks; - } - - public byte[] getControllerServices() { - return controllerServices; - } - - public byte[] getReportingTasks() { - return reportingTasks; - } - - public NodeIdentifier getPrimaryNodeId() { - return primaryNodeId; - } - - public StandardDataFlow getDataFlow() { - return dataFlow; - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java deleted file mode 100644 index 861095f1422b..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.flow; - -/** - * A data access object for loading and saving the flow managed by the cluster. - * - */ -public interface DataFlowDao { - - /** - * Loads the cluster's dataflow. - * - * @return the dataflow or null if no dataflow exists - * - * @throws DaoException if the dataflow was unable to be loaded - */ - ClusterDataFlow loadDataFlow() throws DaoException; - - /** - * Saves the cluster's dataflow. - * - * - * @param dataFlow flow - * @throws DaoException if the dataflow was unable to be saved - */ - void saveDataFlow(ClusterDataFlow dataFlow) throws DaoException; - - /** - * Sets the state of the dataflow. If the dataflow does not exist, then an exception is thrown. - * - * @param flowState the state of the dataflow - * - * @throws DaoException if the state was unable to be updated - */ - void setPersistedFlowState(PersistedFlowState flowState) throws DaoException; - - /** - * Gets the state of the dataflow. - * - * @return the state of the dataflow - * - * @throws DaoException if the state was unable to be retrieved - */ - PersistedFlowState getPersistedFlowState() throws DaoException; -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java deleted file mode 100644 index 8a2af5430fa4..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.flow; - -import java.util.Set; - -import org.apache.nifi.cluster.protocol.NodeIdentifier; - -/** - * A service for managing the cluster's flow. The service will attempt to keep the cluster's dataflow current while respecting the value of the configured retrieval delay. - * - * The eligible retrieval time is reset with the configured delay every time the flow state is set to STALE. If the state is set to UNKNOWN or CURRENT, then the flow will not be retrieved. - * - * Clients must call start() and stop() to initialize and stop the instance. - * - */ -public interface DataFlowManagementService { - - /** - * Starts the instance. Start may only be called if the instance is not running. - */ - void start(); - - /** - * Stops the instance. Stop may only be called if the instance is running. - */ - void stop(); - - /** - * @return true if the instance is started; false otherwise. - */ - boolean isRunning(); - - /** - * Loads the dataflow. - * - * @return the dataflow or null if no dataflow exists - */ - ClusterDataFlow loadDataFlow(); - - /** - * Updates the dataflow with the given primary node identifier. - * - * @param nodeId the node identifier - * - * @throws DaoException if the update failed - */ - void updatePrimaryNode(NodeIdentifier nodeId) throws DaoException; - - /** - * Updates the dataflow with the given serialized form of the Controller Services that are to exist on the NCM. - * - * @param serializedControllerServices services - * @throws DaoException ex - */ - void updateControllerServices(byte[] serializedControllerServices) throws DaoException; - - /** - * Updates the dataflow with the given serialized form of Reporting Tasks that are to exist on the NCM. - * - * @param serializedReportingTasks tasks - * @throws DaoException ex - */ - void updateReportingTasks(byte[] serializedReportingTasks) throws DaoException; - - /** - * Sets the state of the flow. - * - * @param flowState the state - * - * @see PersistedFlowState - */ - void setPersistedFlowState(PersistedFlowState flowState); - - /** - * @return the state of the flow - */ - PersistedFlowState getPersistedFlowState(); - - /** - * @return true if the flow is current; false otherwise. - */ - boolean isFlowCurrent(); - - /** - * Sets the node identifiers to use when attempting to retrieve the flow. - * - * @param nodeIds the node identifiers - */ - void setNodeIds(Set nodeIds); - - /** - * Returns the set of node identifiers the service is using to retrieve the flow. - * - * @return the set of node identifiers the service is using to retrieve the flow. - */ - Set getNodeIds(); - - /** - * @return the retrieval delay in seconds - */ - int getRetrievalDelaySeconds(); - - /** - * Sets the retrieval delay. - * - * @param delay the retrieval delay in seconds - */ - void setRetrievalDelay(String delay); -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java deleted file mode 100644 index 0318e7b1f1f6..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.flow; - -/** - * Represents the exceptional case when a caller is requesting the current flow, but a current flow is not available. - * - */ -public class StaleFlowException extends RuntimeException { - - public StaleFlowException(String message, Throwable cause) { - super(message, cause); - } - - public StaleFlowException(String message) { - super(message); - } - - public StaleFlowException(Throwable cause) { - super(cause); - } - - public StaleFlowException() { - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java deleted file mode 100644 index 7c99c932d7ec..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java +++ /dev/null @@ -1,595 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.flow.impl; - -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.FilenameFilter; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; -import java.util.Arrays; -import java.util.UUID; - -import javax.xml.bind.JAXBContext; -import javax.xml.bind.JAXBException; -import javax.xml.bind.Marshaller; -import javax.xml.bind.Unmarshaller; -import javax.xml.bind.annotation.XmlRootElement; -import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; -import javax.xml.transform.OutputKeys; -import javax.xml.transform.Transformer; -import javax.xml.transform.TransformerFactory; -import javax.xml.transform.dom.DOMSource; -import javax.xml.transform.stream.StreamResult; - -import org.apache.commons.compress.archivers.ArchiveEntry; -import org.apache.commons.compress.archivers.tar.TarArchiveEntry; -import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; -import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream; -import org.apache.nifi.cluster.flow.ClusterDataFlow; -import org.apache.nifi.cluster.flow.DaoException; -import org.apache.nifi.cluster.flow.DataFlowDao; -import org.apache.nifi.cluster.flow.PersistedFlowState; -import org.apache.nifi.cluster.protocol.DataFlow; -import org.apache.nifi.cluster.protocol.NodeIdentifier; -import org.apache.nifi.cluster.protocol.StandardDataFlow; -import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter; -import org.apache.nifi.logging.NiFiLog; -import org.apache.nifi.stream.io.BufferedInputStream; -import org.apache.nifi.stream.io.BufferedOutputStream; -import org.apache.nifi.stream.io.ByteArrayInputStream; -import org.apache.nifi.stream.io.StreamUtils; -import org.apache.nifi.util.file.FileUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.w3c.dom.Document; -import org.w3c.dom.Element; - -/** - * Implements the FlowDao interface. The implementation tracks the state of the dataflow by annotating the filename of the flow state file. Specifically, the implementation correlates - * PersistedFlowState states to filename extensions. The correlation is as follows: - *
    - *
  • CURRENT maps to flow.xml
  • - *
  • STALE maps to flow.xml.stale
  • - *
  • UNKNOWN maps to flow.xml.unknown
  • - *
- * Whenever the flow state changes, the flow state file's name is updated to denote its state. - * - * The implementation also provides for a restore directory that may be configured for higher availability. At instance creation, if the primary or restore directories have multiple flow state files, - * an exception is thrown. If the primary directory has a current flow state file, but the restore directory does not, then the primary flow state file is copied to the restore directory. If the - * restore directory has a current flow state file, but the primary directory does not, then the restore flow state file is copied to the primary directory. If both the primary and restore directories - * have a current flow state file and the files are different, then an exception is thrown. - * - * When the flow state file is saved, it is always saved first to the restore directory followed by a save to the primary directory. When the flow state file is loaded, a check is made to verify that - * the primary and restore flow state files are both current. If either is not current, then an exception is thrown. The primary flow state file is always read when the load method is called. - * - */ -public class DataFlowDaoImpl implements DataFlowDao { - - private final File primaryDirectory; - private final File restoreDirectory; - private final boolean autoStart; - private final String generatedRootGroupId = UUID.randomUUID().toString(); - - public static final String STALE_EXT = ".stale"; - public static final String UNKNOWN_EXT = ".unknown"; - public static final String FLOW_PACKAGE = "flow.tar"; - public static final String FLOW_XML_FILENAME = "flow.xml"; - public static final String TEMPLATES_FILENAME = "templates.xml"; - public static final String SNIPPETS_FILENAME = "snippets.xml"; - public static final String CONTROLLER_SERVICES_FILENAME = "controller-services.xml"; - public static final String REPORTING_TASKS_FILENAME = "reporting-tasks.xml"; - public static final String CLUSTER_INFO_FILENAME = "cluster-info.xml"; - - private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(DataFlowDaoImpl.class)); - - public DataFlowDaoImpl(final File primaryDirectory) throws DaoException { - this(primaryDirectory, null, false); - } - - public DataFlowDaoImpl(final File primaryDirectory, final File restoreDirectory, final boolean autoStart) throws DaoException { - - // sanity check that primary directory is a directory, creating it if necessary - if (primaryDirectory == null) { - throw new IllegalArgumentException("Primary directory may not be null."); - } else if (!primaryDirectory.exists()) { - if (!primaryDirectory.mkdir()) { - throw new DaoException(String.format("Failed to create primary directory '%s'", primaryDirectory.getAbsolutePath())); - } - } else if (!primaryDirectory.isDirectory()) { - throw new IllegalArgumentException("Primary directory must be a directory."); - } - - this.autoStart = autoStart; - - try { - this.primaryDirectory = primaryDirectory; - this.restoreDirectory = restoreDirectory; - - if (restoreDirectory == null) { - // check that we have exactly one current flow state file - ensureSingleCurrentStateFile(primaryDirectory); - } else { - - // check that restore directory is a directory, creating it if necessary - FileUtils.ensureDirectoryExistAndCanAccess(restoreDirectory); - - // check that restore directory is not the same as the primary directory - if (primaryDirectory.getAbsolutePath().equals(restoreDirectory.getAbsolutePath())) { - throw new IllegalArgumentException(String.format("Primary directory '%s' is the same as restore directory '%s' ", - primaryDirectory.getAbsolutePath(), restoreDirectory.getAbsolutePath())); - } - - final File[] primaryFlowStateFiles = getFlowStateFiles(primaryDirectory); - final File[] restoreFlowStateFiles = getFlowStateFiles(restoreDirectory); - - // if more than one state file in either primary or restore, then throw exception - if (primaryFlowStateFiles.length > 1) { - throw new IllegalStateException(String.format("Found multiple dataflow state files in primary directory '%s'", primaryDirectory)); - } else if (restoreFlowStateFiles.length > 1) { - throw new IllegalStateException(String.format("Found multiple dataflow state files in restore directory '%s'", restoreDirectory)); - } - - // check that the single primary state file we found is current or create a new one - final File primaryFlowStateFile = ensureSingleCurrentStateFile(primaryDirectory); - - // check that the single restore state file we found is current or create a new one - final File restoreFlowStateFile = ensureSingleCurrentStateFile(restoreDirectory); - - // if there was a difference in flow state file directories, then copy the appropriate files - if (restoreFlowStateFiles.length == 0 && primaryFlowStateFiles.length != 0) { - // copy primary state file to restore - FileUtils.copyFile(primaryFlowStateFile, restoreFlowStateFile, false, false, logger); - } else if (primaryFlowStateFiles.length == 0 && restoreFlowStateFiles.length != 0) { - // copy restore state file to primary - FileUtils.copyFile(restoreFlowStateFile, primaryFlowStateFile, false, false, logger); - } else { - // sync the primary copy with the restore copy - syncWithRestore(primaryFlowStateFile, restoreFlowStateFile); - } - - } - } catch (final IOException | IllegalArgumentException | IllegalStateException | JAXBException ex) { - throw new DaoException(ex); - } - } - - private void syncWithRestore(final File primaryFile, final File restoreFile) throws IOException { - try (final FileInputStream primaryFis = new FileInputStream(primaryFile); - final TarArchiveInputStream primaryIn = new TarArchiveInputStream(primaryFis); - final FileInputStream restoreFis = new FileInputStream(restoreFile); - final TarArchiveInputStream restoreIn = new TarArchiveInputStream(restoreFis)) { - - final ArchiveEntry primaryEntry = primaryIn.getNextEntry(); - final ArchiveEntry restoreEntry = restoreIn.getNextEntry(); - - if (primaryEntry == null && restoreEntry == null) { - return; - } - - if (primaryEntry == null && restoreEntry != null || primaryEntry != null && restoreEntry == null) { - throw new IllegalStateException(String.format("Primary file '%s' is different than restore file '%s'", - primaryFile.getAbsoluteFile(), restoreFile.getAbsolutePath())); - } - - final byte[] primaryMd5 = calculateMd5(primaryIn); - final byte[] restoreMd5 = calculateMd5(restoreIn); - - if (!Arrays.equals(primaryMd5, restoreMd5)) { - throw new IllegalStateException(String.format("Primary file '%s' is different than restore file '%s'", - primaryFile.getAbsoluteFile(), restoreFile.getAbsolutePath())); - } - } - } - - private byte[] calculateMd5(final InputStream in) throws IOException { - final MessageDigest digest; - try { - digest = MessageDigest.getInstance("MD5"); - } catch (final NoSuchAlgorithmException nsae) { - throw new IOException(nsae); - } - - int len; - final byte[] buffer = new byte[8192]; - while ((len = in.read(buffer)) > -1) { - if (len > 0) { - digest.update(buffer, 0, len); - } - } - return digest.digest(); - } - - @Override - public ClusterDataFlow loadDataFlow() throws DaoException { - try { - return parseDataFlow(getExistingFlowStateFile(primaryDirectory)); - } catch (final IOException | JAXBException ex) { - throw new DaoException(ex); - } - } - - @Override - public void saveDataFlow(final ClusterDataFlow dataFlow) throws DaoException { - try { - - final File primaryStateFile = getFlowStateFile(primaryDirectory); - - // write to restore before writing to primary in case primary experiences problems - if (restoreDirectory != null) { - final File restoreStateFile = getFlowStateFile(restoreDirectory); - if (restoreStateFile == null) { - if (primaryStateFile == null) { - writeDataFlow(createNewFlowStateFile(restoreDirectory), dataFlow); - } else { - throw new DaoException(String.format("Unable to save dataflow because dataflow state file in primary directory " - + "'%s' exists, but it does not exist in the restore directory '%s'", - primaryDirectory.getAbsolutePath(), restoreDirectory.getAbsolutePath())); - } - } else { - if (primaryStateFile == null) { - throw new DaoException(String.format("Unable to save dataflow because dataflow state file in restore directory " - + "'%s' exists, but it does not exist in the primary directory '%s'", - restoreDirectory.getAbsolutePath(), primaryDirectory.getAbsolutePath())); - } else { - final PersistedFlowState primaryFlowState = getPersistedFlowState(primaryStateFile); - final PersistedFlowState restoreFlowState = getPersistedFlowState(restoreStateFile); - if (primaryFlowState == restoreFlowState) { - writeDataFlow(restoreStateFile, dataFlow); - } else { - throw new DaoException(String.format("Unable to save dataflow because state file in primary directory " - + "'%s' has state '%s', but the state file in the restore directory '%s' has state '%s'", - primaryDirectory.getAbsolutePath(), primaryFlowState, restoreDirectory.getAbsolutePath(), restoreFlowState)); - } - } - } - } - - // write dataflow to primary - if (primaryStateFile == null) { - writeDataFlow(createNewFlowStateFile(primaryDirectory), dataFlow); - } else { - writeDataFlow(primaryStateFile, dataFlow); - } - - } catch (final IOException | JAXBException ex) { - throw new DaoException(ex); - } - } - - @Override - public PersistedFlowState getPersistedFlowState() { - // trust restore over primary if configured for restore - if (restoreDirectory == null) { - return getPersistedFlowState(getExistingFlowStateFile(primaryDirectory)); - } else { - return getPersistedFlowState(getExistingFlowStateFile(restoreDirectory)); - } - } - - @Override - public void setPersistedFlowState(final PersistedFlowState flowState) throws DaoException { - // rename restore before primary if configured for restore - if (restoreDirectory != null) { - renameFlowStateFile(getExistingFlowStateFile(restoreDirectory), flowState); - } - renameFlowStateFile(getExistingFlowStateFile(primaryDirectory), flowState); - } - - private File ensureSingleCurrentStateFile(final File dir) throws IOException, JAXBException { - - // ensure that we have at most one state file and if we have one, it is current - final File[] directoryFlowStateFiles = getFlowStateFiles(dir); - if (directoryFlowStateFiles.length > 1) { - throw new DaoException(String.format("Found multiple dataflow state files in directory '%s'", dir)); - } else if (directoryFlowStateFiles.length == 0) { - // create a new file if none exist - return createNewFlowStateFile(dir); - } else { - // check that the single flow state file is current - final PersistedFlowState flowState = getPersistedFlowState(directoryFlowStateFiles[0]); - if (PersistedFlowState.CURRENT == flowState) { - return directoryFlowStateFiles[0]; - } else { - throw new DaoException(String.format("Dataflow state file '%s' must be current.", directoryFlowStateFiles[0].getAbsolutePath())); - } - } - - } - - private PersistedFlowState getPersistedFlowState(final File file) { - final String path = file.getAbsolutePath(); - if (path.endsWith(STALE_EXT)) { - return PersistedFlowState.STALE; - } else if (path.endsWith(UNKNOWN_EXT)) { - return PersistedFlowState.UNKNOWN; - } else { - return PersistedFlowState.CURRENT; - } - } - - private File getFlowStateFile(final File dir) { - final File[] files = getFlowStateFiles(dir); - if (files.length > 1) { - throw new IllegalStateException(String.format("Expected at most one dataflow state file, but found %s files.", files.length)); - } else if (files.length == 0) { - return null; - } else { - return files[0]; - } - } - - private File getExistingFlowStateFile(final File dir) { - final File file = getFlowStateFile(dir); - if (file == null) { - throw new IllegalStateException(String.format("Expected a dataflow state file, but none existed in directory '%s'", dir.getAbsolutePath())); - } - return file; - } - - private File[] getFlowStateFiles(final File dir) { - final File[] files = dir.listFiles(new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - return name.equals(FLOW_PACKAGE) || name.endsWith(STALE_EXT) || name.endsWith(UNKNOWN_EXT); - } - }); - - if (files == null) { - return new File[0]; - } else { - return files; - } - } - - private File removeStateFileExtension(final File file) { - - final String path = file.getAbsolutePath(); - final int stateFileExtIndex; - if (path.endsWith(STALE_EXT)) { - stateFileExtIndex = path.lastIndexOf(STALE_EXT); - } else if (path.endsWith(UNKNOWN_EXT)) { - stateFileExtIndex = path.lastIndexOf(UNKNOWN_EXT); - } else { - stateFileExtIndex = path.length(); - } - - return new File(path.substring(0, stateFileExtIndex)); - } - - private File addStateFileExtension(final File file, final PersistedFlowState state) { - switch (state) { - case CURRENT: { - return file; - } - case STALE: { - return new File(file.getAbsolutePath() + STALE_EXT); - } - case UNKNOWN: { - return new File(file.getAbsolutePath() + UNKNOWN_EXT); - } - default: { - throw new RuntimeException("Unsupported PersistedFlowState Enum value: " + state); - } - } - } - - private File createNewFlowStateFile(final File dir) throws IOException, JAXBException { - final File stateFile = new File(dir, FLOW_PACKAGE); - stateFile.createNewFile(); - - writeDataFlow(stateFile, new ClusterDataFlow(null, null, new byte[0], new byte[0]), new ClusterMetadata()); - - return stateFile; - } - - private byte[] getEmptyFlowBytes() throws IOException { - try { - final DocumentBuilderFactory docBuilderFactory = DocumentBuilderFactory.newInstance(); - docBuilderFactory.setNamespaceAware(true); - final DocumentBuilder docBuilder = docBuilderFactory.newDocumentBuilder(); - final Document document = docBuilder.newDocument(); - - final Element controller = document.createElement("flowController"); - document.appendChild(controller); - - controller.appendChild(createTextElement(document, "maxThreadCount", "15")); - - final Element rootGroup = document.createElement("rootGroup"); - rootGroup.appendChild(createTextElement(document, "id", generatedRootGroupId)); - rootGroup.appendChild(createTextElement(document, "name", "NiFi Flow")); - - // create the position element - final Element positionElement = createTextElement(document, "position", ""); - positionElement.setAttribute("x", "0.0"); - positionElement.setAttribute("y", "0.0"); - rootGroup.appendChild(positionElement); - - rootGroup.appendChild(createTextElement(document, "comment", "")); - controller.appendChild(rootGroup); - - final Transformer transformer = TransformerFactory.newInstance().newTransformer(); - transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2"); - transformer.setOutputProperty(OutputKeys.INDENT, "yes"); - - final DOMSource source = new DOMSource(document); - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - final StreamResult result = new StreamResult(baos); - transformer.transform(source, result); - - return baos.toByteArray(); - } catch (final Exception e) { - throw new IOException(e); - } - } - - private Element createTextElement(final Document document, final String elementName, final String value) { - final Element element = document.createElement(elementName); - element.setTextContent(value); - return element; - } - - private void renameFlowStateFile(final File flowStateFile, final PersistedFlowState newState) throws DaoException { - final PersistedFlowState existingState = getPersistedFlowState(flowStateFile); - if (existingState != newState) { - final File newFlowStateFile = addStateFileExtension(removeStateFileExtension(flowStateFile), newState); - if (flowStateFile.renameTo(newFlowStateFile) == false) { - throw new DaoException( - String.format("Failed to rename flow state file '%s' to new name '%s'", flowStateFile.getAbsolutePath(), newFlowStateFile.getAbsolutePath())); - } - } - } - - private ClusterDataFlow parseDataFlow(final File file) throws IOException, JAXBException, DaoException { - byte[] flowBytes = new byte[0]; - byte[] templateBytes = new byte[0]; - byte[] snippetBytes = new byte[0]; - byte[] clusterInfoBytes = new byte[0]; - byte[] controllerServiceBytes = new byte[0]; - byte[] reportingTaskBytes = new byte[0]; - - try (final InputStream inStream = new FileInputStream(file); - final TarArchiveInputStream tarIn = new TarArchiveInputStream(new BufferedInputStream(inStream))) { - TarArchiveEntry tarEntry; - while ((tarEntry = tarIn.getNextTarEntry()) != null) { - switch (tarEntry.getName()) { - case FLOW_XML_FILENAME: - flowBytes = new byte[(int) tarEntry.getSize()]; - StreamUtils.fillBuffer(tarIn, flowBytes, true); - break; - case TEMPLATES_FILENAME: - templateBytes = new byte[(int) tarEntry.getSize()]; - StreamUtils.fillBuffer(tarIn, templateBytes, true); - break; - case SNIPPETS_FILENAME: - snippetBytes = new byte[(int) tarEntry.getSize()]; - StreamUtils.fillBuffer(tarIn, snippetBytes, true); - break; - case CLUSTER_INFO_FILENAME: - clusterInfoBytes = new byte[(int) tarEntry.getSize()]; - StreamUtils.fillBuffer(tarIn, clusterInfoBytes, true); - break; - case CONTROLLER_SERVICES_FILENAME: - controllerServiceBytes = new byte[(int) tarEntry.getSize()]; - StreamUtils.fillBuffer(tarIn, controllerServiceBytes, true); - break; - case REPORTING_TASKS_FILENAME: - reportingTaskBytes = new byte[(int) tarEntry.getSize()]; - StreamUtils.fillBuffer(tarIn, reportingTaskBytes, true); - break; - default: - throw new DaoException("Found Unexpected file in dataflow configuration: " + tarEntry.getName()); - } - } - } - - final ClusterMetadata clusterMetadata; - if (clusterInfoBytes.length == 0) { - clusterMetadata = null; - } else { - final Unmarshaller clusterMetadataUnmarshaller = ClusterMetadata.jaxbCtx.createUnmarshaller(); - clusterMetadata = (ClusterMetadata) clusterMetadataUnmarshaller.unmarshal(new ByteArrayInputStream(clusterInfoBytes)); - } - - final StandardDataFlow dataFlow = new StandardDataFlow(flowBytes, snippetBytes); - dataFlow.setAutoStartProcessors(autoStart); - - return new ClusterDataFlow(dataFlow, clusterMetadata == null ? null : clusterMetadata.getPrimaryNodeId(), controllerServiceBytes, reportingTaskBytes); - } - - private void writeDataFlow(final File file, final ClusterDataFlow clusterDataFlow) throws IOException, JAXBException { - // setup the cluster metadata - final ClusterMetadata clusterMetadata = new ClusterMetadata(); - clusterMetadata.setPrimaryNodeId(clusterDataFlow.getPrimaryNodeId()); - - // write to disk - writeDataFlow(file, clusterDataFlow, clusterMetadata); - } - - private void writeTarEntry(final TarArchiveOutputStream tarOut, final String filename, final byte[] bytes) throws IOException { - final TarArchiveEntry flowEntry = new TarArchiveEntry(filename); - flowEntry.setSize(bytes.length); - tarOut.putArchiveEntry(flowEntry); - tarOut.write(bytes); - tarOut.closeArchiveEntry(); - } - - private void writeDataFlow(final File file, final ClusterDataFlow clusterDataFlow, final ClusterMetadata clusterMetadata) throws IOException, JAXBException { - - try (final OutputStream fos = new FileOutputStream(file); - final TarArchiveOutputStream tarOut = new TarArchiveOutputStream(new BufferedOutputStream(fos))) { - - final DataFlow dataFlow = clusterDataFlow.getDataFlow(); - if (dataFlow == null) { - writeTarEntry(tarOut, FLOW_XML_FILENAME, getEmptyFlowBytes()); - writeTarEntry(tarOut, SNIPPETS_FILENAME, new byte[0]); - } else { - writeTarEntry(tarOut, FLOW_XML_FILENAME, dataFlow.getFlow()); - writeTarEntry(tarOut, SNIPPETS_FILENAME, dataFlow.getSnippets()); - } - writeTarEntry(tarOut, CONTROLLER_SERVICES_FILENAME, clusterDataFlow.getControllerServices()); - writeTarEntry(tarOut, REPORTING_TASKS_FILENAME, clusterDataFlow.getReportingTasks()); - - final ByteArrayOutputStream baos = new ByteArrayOutputStream(256); - writeClusterMetadata(clusterMetadata, baos); - final byte[] clusterInfoBytes = baos.toByteArray(); - - writeTarEntry(tarOut, CLUSTER_INFO_FILENAME, clusterInfoBytes); - } - } - - private void writeClusterMetadata(final ClusterMetadata clusterMetadata, final OutputStream os) throws IOException, JAXBException { - // write cluster metadata to output stream - final Marshaller marshaller = ClusterMetadata.jaxbCtx.createMarshaller(); - marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true); - marshaller.setProperty(Marshaller.JAXB_FRAGMENT, true); - marshaller.setProperty(Marshaller.JAXB_ENCODING, "UTF-8"); - marshaller.marshal(clusterMetadata, os); - } - - @XmlRootElement(name = "clusterMetadata") - private static class ClusterMetadata { - - private NodeIdentifier primaryNodeId; - - private static final JAXBContext jaxbCtx; - - static { - try { - jaxbCtx = JAXBContext.newInstance(ClusterMetadata.class); - } catch (final JAXBException je) { - throw new RuntimeException(je); - } - } - - @XmlJavaTypeAdapter(NodeIdentifierAdapter.class) - public NodeIdentifier getPrimaryNodeId() { - return primaryNodeId; - } - - public void setPrimaryNodeId(final NodeIdentifier primaryNodeId) { - this.primaryNodeId = primaryNodeId; - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java deleted file mode 100644 index 22450dab14e8..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java +++ /dev/null @@ -1,403 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.flow.impl; - -import java.util.Collections; -import java.util.Date; -import java.util.Set; -import java.util.Timer; -import java.util.TimerTask; -import java.util.concurrent.CopyOnWriteArraySet; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; - -import org.apache.nifi.cluster.flow.ClusterDataFlow; -import org.apache.nifi.cluster.flow.DaoException; -import org.apache.nifi.cluster.flow.DataFlowDao; -import org.apache.nifi.cluster.flow.DataFlowManagementService; -import org.apache.nifi.cluster.flow.PersistedFlowState; -import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender; -import org.apache.nifi.cluster.protocol.NodeIdentifier; -import org.apache.nifi.cluster.protocol.StandardDataFlow; -import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; -import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; -import org.apache.nifi.logging.NiFiLog; -import org.apache.nifi.util.FormatUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Implements FlowManagementService interface. The service tries to keep the cluster's flow current with regards to the available nodes. - * - * The instance may be configured with a retrieval delay, which will reduce the number of retrievals performed by the service at the expense of increasing the chances that the service will not be able - * to provide a current flow to the caller. - * - * By default, the service will try to update the flow as quickly as possible. Configuring a delay enables a less aggressive retrieval strategy. Specifically, the eligible retrieval time is reset - * every time the flow state is set to STALE. If the state is set to UNKNOWN or CURRENT, then the flow will not be retrieved. - * - */ -public class DataFlowManagementServiceImpl implements DataFlowManagementService { - - /* - * Developer Note: - * - * This class maintains an ExecutorService and a Runnable. - * Although the class is not externally threadsafe, its internals are protected to - * accommodate multithread access between the ExecutorServer and the Runnable. - */ - private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(DataFlowManagementServiceImpl.class)); - - private final DataFlowDao flowDao; - - private final ClusterManagerProtocolSender sender; - - private final Set nodeIds = new CopyOnWriteArraySet<>(); - - private final AtomicBoolean stopRequested = new AtomicBoolean(false); - - private final AtomicLong lastRetrievalTime = new AtomicLong(-1); - - private Timer flowRetriever; - - private long retrievableAfterTime = 0L; - - private AtomicInteger retrievalDelaySeconds = new AtomicInteger(0); - - private final TimingReentrantLock resourceLock = new TimingReentrantLock(new ReentrantLock()); - - public DataFlowManagementServiceImpl(final DataFlowDao flowDao, final ClusterManagerProtocolSender sender) { - if (flowDao == null) { - throw new IllegalArgumentException("Flow DAO may not be null."); - } else if (sender == null) { - throw new IllegalArgumentException("Cluster Manager Protocol Sender may not be null."); - } - this.flowDao = flowDao; - this.sender = sender; - } - - @Override - public void start() { - - if (isRunning()) { - throw new IllegalArgumentException("Instance is already running."); - } - - // reset stop requested - stopRequested.set(false); - - // setup flow retreiver timer - flowRetriever = new Timer("Flow Management Service", /* is daemon */ true); - flowRetriever.schedule(new FlowRetrieverTimerTask(), 0, 500); - } - - @Override - public boolean isRunning() { - return (flowRetriever != null); - } - - @Override - public void stop() { - - if (isRunning() == false) { - throw new IllegalArgumentException("Instance is already stopped."); - } - - // record stop request - stopRequested.set(true); - - flowRetriever.cancel(); - flowRetriever = null; - - } - - @Override - public ClusterDataFlow loadDataFlow() throws DaoException { - resourceLock.lock(); - try { - return flowDao.loadDataFlow(); - } finally { - resourceLock.unlock("loadDataFlow"); - } - } - - @Override - public void updatePrimaryNode(final NodeIdentifier nodeId) { - resourceLock.lock(); - try { - final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow(); - - final StandardDataFlow dataFlow; - final byte[] controllerServiceBytes; - final byte[] reportingTaskBytes; - if (existingClusterDataFlow == null) { - dataFlow = null; - controllerServiceBytes = new byte[0]; - reportingTaskBytes = new byte[0]; - } else { - dataFlow = existingClusterDataFlow.getDataFlow(); - controllerServiceBytes = existingClusterDataFlow.getControllerServices(); - reportingTaskBytes = existingClusterDataFlow.getReportingTasks(); - } - - flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId, controllerServiceBytes, reportingTaskBytes)); - } finally { - resourceLock.unlock("updatePrimaryNode"); - } - } - - @Override - public void updateControllerServices(final byte[] controllerServiceBytes) throws DaoException { - resourceLock.lock(); - try { - final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow(); - - final StandardDataFlow dataFlow; - final byte[] reportingTaskBytes; - final NodeIdentifier nodeId; - if (existingClusterDataFlow == null) { - dataFlow = null; - nodeId = null; - reportingTaskBytes = new byte[0]; - } else { - dataFlow = existingClusterDataFlow.getDataFlow(); - nodeId = existingClusterDataFlow.getPrimaryNodeId(); - reportingTaskBytes = existingClusterDataFlow.getReportingTasks(); - } - - flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId, controllerServiceBytes, reportingTaskBytes)); - } finally { - resourceLock.unlock("updateControllerServices"); - } - } - - @Override - public void updateReportingTasks(final byte[] reportingTaskBytes) throws DaoException { - resourceLock.lock(); - try { - final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow(); - - final StandardDataFlow dataFlow; - final byte[] controllerServiceBytes; - final NodeIdentifier nodeId; - if (existingClusterDataFlow == null) { - dataFlow = null; - nodeId = null; - controllerServiceBytes = null; - } else { - dataFlow = existingClusterDataFlow.getDataFlow(); - nodeId = existingClusterDataFlow.getPrimaryNodeId(); - controllerServiceBytes = existingClusterDataFlow.getControllerServices(); - } - - flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId, controllerServiceBytes, reportingTaskBytes)); - } finally { - resourceLock.unlock("updateControllerServices"); - } - } - - @Override - public PersistedFlowState getPersistedFlowState() { - resourceLock.lock(); - try { - return flowDao.getPersistedFlowState(); - } finally { - resourceLock.unlock("getPersistedFlowState"); - } - } - - @Override - public boolean isFlowCurrent() { - return PersistedFlowState.CURRENT == getPersistedFlowState(); - } - - @Override - public void setPersistedFlowState(final PersistedFlowState flowState) { - // lock to ensure state change and retrievable time update are atomic - resourceLock.lock(); - try { - flowDao.setPersistedFlowState(flowState); - if (PersistedFlowState.STALE == flowState) { - retrievableAfterTime = new Date().getTime() + (getRetrievalDelaySeconds() * 1000); - } else if (PersistedFlowState.UNKNOWN == flowState || PersistedFlowState.CURRENT == flowState) { - retrievableAfterTime = Long.MAX_VALUE; - } - } finally { - resourceLock.unlock("setPersistedFlowState"); - } - } - - @Override - public Set getNodeIds() { - return Collections.unmodifiableSet(nodeIds); - } - - @Override - public void setNodeIds(final Set nodeIds) { - - if (nodeIds == null) { - throw new IllegalArgumentException("Node IDs may not be null."); - } - - resourceLock.lock(); - try { - - if (this.nodeIds.equals(nodeIds)) { - return; - } - - this.nodeIds.clear(); - this.nodeIds.addAll(nodeIds); - - } finally { - resourceLock.unlock("setNodeIds"); - } - - } - - @Override - public int getRetrievalDelaySeconds() { - return retrievalDelaySeconds.get(); - } - - @Override - public void setRetrievalDelay(final String retrievalDelay) { - this.retrievalDelaySeconds.set((int) FormatUtils.getTimeDuration(retrievalDelay, TimeUnit.SECONDS)); - } - - public ClusterManagerProtocolSender getSender() { - return sender; - } - - public long getLastRetrievalTime() { - return lastRetrievalTime.get(); - } - - /** - * A timer task for issuing FlowRequestMessage messages to nodes to retrieve an updated flow. - */ - private class FlowRetrieverTimerTask extends TimerTask { - - @Override - public void run() { - - resourceLock.lock(); - try { - // if flow is current, then we're done - if (isFlowCurrent()) { - return; - } - } catch (final Exception ex) { - logger.info("Encountered exception checking if flow is current caused by " + ex, ex); - } finally { - resourceLock.unlock("FlowRetrieverTimerTask - isFlowCurrent"); - } - - final FlowRequestMessage request = new FlowRequestMessage(); - for (final NodeIdentifier nodeId : getNodeIds()) { - try { - // setup request - request.setNodeId(nodeId); - - // record request time - final long requestSentTime = new Date().getTime(); - - resourceLock.lock(); - try { - // sanity checks before making request - if (stopRequested.get()) { // did we receive a stop request - logger.debug("Stopping runnable prematurely because a request to stop was issued."); - return; - } else if (requestSentTime < retrievableAfterTime) { - /* - * Retrievable after time was updated while obtaining - * the lock, so try again later - */ - return; - } - } finally { - resourceLock.unlock("FlowRetrieverTimerTask - check stopRequested"); - } - - // send request - final FlowResponseMessage response = sender.requestFlow(request); - - resourceLock.lock(); - try { - // check if the retrieved flow is still valid - if (requestSentTime > retrievableAfterTime) { - logger.info("Saving retrieved flow."); - - final StandardDataFlow dataFlow = response.getDataFlow(); - final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow(); - final ClusterDataFlow currentClusterDataFlow; - if (existingClusterDataFlow == null) { - currentClusterDataFlow = new ClusterDataFlow(dataFlow, null, new byte[0], new byte[0]); - } else { - currentClusterDataFlow = new ClusterDataFlow(dataFlow, existingClusterDataFlow.getPrimaryNodeId(), - existingClusterDataFlow.getControllerServices(), existingClusterDataFlow.getReportingTasks()); - } - flowDao.saveDataFlow(currentClusterDataFlow); - flowDao.setPersistedFlowState(PersistedFlowState.CURRENT); - lastRetrievalTime.set(new Date().getTime()); - } - - /* - * Retrievable after time was updated while requesting - * the flow, so try again later. - */ - } finally { - resourceLock.unlock("FlowRetrieverTimerTask - saveDataFlow"); - } - - } catch (final Throwable t) { - logger.info("Encountered exception retrieving flow from node " + nodeId + " caused by " + t, t); - } - } - } - } - - private static class TimingReentrantLock { - - private final Lock lock; - private static final Logger logger = LoggerFactory.getLogger("dataFlowManagementService.lock"); - - private final ThreadLocal lockTime = new ThreadLocal<>(); - - public TimingReentrantLock(final Lock lock) { - this.lock = lock; - } - - public void lock() { - lock.lock(); - lockTime.set(System.nanoTime()); - } - - public void unlock(final String task) { - final long nanosLocked = System.nanoTime() - lockTime.get(); - lock.unlock(); - - final long millisLocked = TimeUnit.MILLISECONDS.convert(nanosLocked, TimeUnit.NANOSECONDS); - if (millisLocked > 100L) { - logger.debug("Lock held for {} milliseconds for task: {}", millisLocked, task); - } - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java deleted file mode 100644 index de3c23ee1e4f..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.manager; - -import java.util.List; -import java.util.Set; - -import org.apache.nifi.cluster.event.Event; -import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException; -import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException; -import org.apache.nifi.cluster.manager.exception.IllegalNodeReconnectionException; -import org.apache.nifi.cluster.manager.exception.NodeDisconnectionException; -import org.apache.nifi.cluster.manager.exception.UnknownNodeException; -import org.apache.nifi.cluster.node.Node; -import org.apache.nifi.cluster.node.Node.Status; -import org.apache.nifi.cluster.protocol.ConnectionRequest; -import org.apache.nifi.cluster.protocol.ConnectionResponse; -import org.apache.nifi.cluster.protocol.NodeIdentifier; -import org.apache.nifi.remote.cluster.NodeInformant; -import org.apache.nifi.reporting.BulletinRepository; - -/** - * Defines the interface for a ClusterManager. The cluster manager is a threadsafe centralized manager for a cluster. Members of a cluster are nodes. A member becomes a node by issuing a connection - * request to the manager. The manager maintains the set of nodes. Nodes may be disconnected, reconnected, and deleted. - * - * Nodes are responsible for sending heartbeats to the manager to indicate their liveliness. A manager may disconnect a node if it does not receive a heartbeat within a configurable time period. A - * cluster manager instance may be configured with how often to monitor received heartbeats (getHeartbeatMonitoringIntervalSeconds()) and the maximum time that may elapse between node heartbeats - * before disconnecting the node (getMaxHeartbeatGapSeconds()). - * - * Since only a single node may execute isolated processors, the cluster manager maintains the notion of a primary node. The primary node is chosen at cluster startup and retains the role until a user - * requests a different node to be the primary node. - * - */ -public interface ClusterManager extends NodeInformant { - - /** - * @param statuses the statuses of the nodes - * @return the set of nodes - */ - Set getNodes(Status... statuses); - - /** - * @param nodeId node identifier - * @return returns the node with the given identifier or null if node does not exist - */ - Node getNode(String nodeId); - - /** - * @param statuses statuses - * @return the set of node identifiers with the given node status - */ - Set getNodeIds(Status... statuses); - - /** - * Deletes the node with the given node identifier. If the given node is the primary node, then a subsequent request may be made to the manager to set a new primary node. - * - * @param nodeId the node identifier - * @param userDn the Distinguished Name of the user requesting the node be deleted from the cluster - * - * @throws UnknownNodeException if the node does not exist - * @throws IllegalNodeDeletionException if the node is not in a disconnected state - */ - void deleteNode(String nodeId, String userDn) throws UnknownNodeException, IllegalNodeDeletionException; - - /** - * Requests a connection to the cluster. - * - * @param request the request - * - * @return the response - */ - ConnectionResponse requestConnection(ConnectionRequest request); - - /** - * Services reconnection requests for a given node. If the node indicates reconnection failure, then the node will be set to disconnected. Otherwise, a reconnection request will be sent to the - * node, initiating the connection handshake. - * - * @param nodeId a node identifier - * @param userDn the Distinguished Name of the user requesting the reconnection - * - * @throws UnknownNodeException if the node does not exist - * @throws IllegalNodeReconnectionException if the node is not disconnected - */ - void requestReconnection(String nodeId, String userDn) throws UnknownNodeException, IllegalNodeReconnectionException; - - /** - * Requests the node with the given identifier be disconnected. - * - * @param nodeId the node identifier - * @param userDn the Distinguished Name of the user requesting the disconnection - * - * @throws UnknownNodeException if the node does not exist - * @throws IllegalNodeDisconnectionException if the node cannot be disconnected due to the cluster's state (e.g., node is last connected node or node is primary) - * @throws UnknownNodeException if the node does not exist - * @throws IllegalNodeDisconnectionException if the node is not disconnected - * @throws NodeDisconnectionException if the disconnection failed - */ - void requestDisconnection(String nodeId, String userDn) throws UnknownNodeException, IllegalNodeDisconnectionException, NodeDisconnectionException; - - /** - * @return the time in seconds to wait between successive executions of heartbeat monitoring - */ - int getHeartbeatMonitoringIntervalSeconds(); - - /** - * @return the maximum time in seconds that is allowed between successive heartbeats of a node before disconnecting the node - */ - int getMaxHeartbeatGapSeconds(); - - /** - * Returns a list of node events for the node with the given identifier. The events will be returned in order of most recent to least recent according to the creation date of the event. - * - * @param nodeId the node identifier - * - * @return the list of events or an empty list if no node exists with the given identifier - */ - List getNodeEvents(final String nodeId); - - /** - * @return the primary node of the cluster or null if no primary node exists - */ - Node getPrimaryNode(); - - /** - * @return the bulletin repository - */ - BulletinRepository getBulletinRepository(); -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java deleted file mode 100644 index 0f9872fa8e2d..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.manager; - -import java.net.URI; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException; -import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException; -import org.apache.nifi.cluster.manager.exception.NoConnectedNodesException; -import org.apache.nifi.cluster.manager.exception.NoResponseFromNodesException; -import org.apache.nifi.cluster.manager.exception.SafeModeMutableRequestException; -import org.apache.nifi.cluster.manager.exception.UriConstructionException; -import org.apache.nifi.cluster.protocol.NodeIdentifier; - -/** - * Extends the ClusterManager interface to define how requests issued to the cluster manager are federated to the nodes. Specifically, the HTTP protocol is used for communicating requests to the - * cluster manager and to the nodes. - * - */ -public interface HttpClusterManager extends ClusterManager { - - /** - * Federates the HTTP request to all connected nodes in the cluster. The given URI's host and port will not be used and instead will be adjusted for each node's host and port. The node URIs are - * guaranteed to be constructed before issuing any requests, so if a UriConstructionException is thrown, then it is guaranteed that no request was issued. - * - * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD) - * @param uri the base request URI (up to, but not including, the query string) - * @param parameters the request parameters - * @param headers the request headers - * - * @return the client response - * - * @throws NoConnectedNodesException if no nodes are connected as results of the request - * @throws NoResponseFromNodesException if no response could be obtained - * @throws UriConstructionException if there was an issue constructing the URIs tailored for each individual node - * @throws ConnectingNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is connecting to the cluster - * @throws DisconnectedNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is disconnected from the cluster - * @throws SafeModeMutableRequestException if the request was a PUT, POST, DELETE and a the cluster is in safe mode - */ - NodeResponse applyRequest(String method, URI uri, Map> parameters, Map headers) - throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, - DisconnectedNodeMutableRequestException, SafeModeMutableRequestException; - - /** - * Federates the HTTP request to the nodes specified. The given URI's host and port will not be used and instead will be adjusted for each node's host and port. The node URIs are guaranteed to be - * constructed before issuing any requests, so if a UriConstructionException is thrown, then it is guaranteed that no request was issued. - * - * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD) - * @param uri the base request URI (up to, but not including, the query string) - * @param parameters the request parameters - * @param headers the request headers - * @param nodeIdentifiers the NodeIdentifier for each node that the request should be replaced to - * - * @return the client response - * - * @throws NoConnectedNodesException if no nodes are connected as results of the request - * @throws NoResponseFromNodesException if no response could be obtained - * @throws UriConstructionException if there was an issue constructing the URIs tailored for each individual node - * @throws ConnectingNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is connecting to the cluster - * @throws DisconnectedNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is disconnected from the cluster - * @throws SafeModeMutableRequestException if the request was a PUT, POST, DELETE and a the cluster is in safe mode - */ - NodeResponse applyRequest(String method, URI uri, Map> parameters, Map headers, - Set nodeIdentifiers) - throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, - DisconnectedNodeMutableRequestException, SafeModeMutableRequestException; - - /** - * Federates the HTTP request to all connected nodes in the cluster. The given URI's host and port will not be used and instead will be adjusted for each node's host and port. The node URIs are - * guaranteed to be constructed before issuing any requests, so if a UriConstructionException is thrown, then it is guaranteed that no request was issued. - * - * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD) - * @param uri the base request URI (up to, but not including, the query string) - * @param entity the HTTP request entity - * @param headers the request headers - * - * @return the client response - * - * @throws NoConnectedNodesException if no nodes are connected as results of the request - * @throws NoResponseFromNodesException if no response could be obtained - * @throws UriConstructionException if there was an issue constructing the URIs tailored for each individual node - * @throws ConnectingNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is connecting to the cluster - * @throws DisconnectedNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is disconnected from the cluster - * @throws SafeModeMutableRequestException if the request was a PUT, POST, DELETE and a the cluster is in safe mode - */ - NodeResponse applyRequest(String method, URI uri, Object entity, Map headers) - throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, - DisconnectedNodeMutableRequestException, SafeModeMutableRequestException; - - /** - * Federates the HTTP request to the nodes specified. The given URI's host and port will not be used and instead will be adjusted for each node's host and port. The node URIs are guaranteed to be - * constructed before issuing any requests, so if a UriConstructionException is thrown, then it is guaranteed that no request was issued. - * - * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD) - * @param uri the base request URI (up to, but not including, the query string) - * @param entity the HTTP request entity - * @param headers the request headers - * @param nodeIdentifiers the NodeIdentifier for each node that the request should be replaced to - * - * @return the client response - * - * @throws NoConnectedNodesException if no nodes are connected as results of the request - * @throws NoResponseFromNodesException if no response could be obtained - * @throws UriConstructionException if there was an issue constructing the URIs tailored for each individual node - * @throws ConnectingNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is connecting to the cluster - * @throws DisconnectedNodeMutableRequestException if the request was a PUT, POST, DELETE and a node is disconnected from the cluster - * @throws SafeModeMutableRequestException if the request was a PUT, POST, DELETE and a the cluster is in safe mode - */ - NodeResponse applyRequest(String method, URI uri, Object entity, Map headers, Set nodeIdentifiers) - throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, - DisconnectedNodeMutableRequestException, SafeModeMutableRequestException; -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java deleted file mode 100644 index 3a3d9760f450..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.manager; - -import java.net.URI; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.nifi.cluster.manager.exception.UriConstructionException; -import org.apache.nifi.cluster.protocol.NodeIdentifier; - -/** - * A service for managing the replication of requests to nodes. It is up to the implementing class to decide if requests are sent concurrently or serially. - * - * Clients must call start() and stop() to initialize and shutdown the instance. The instance must be started before issuing any replication requests. - * - */ -public interface HttpRequestReplicator { - - /** - * Starts the instance for replicating requests. Start may only be called if the instance is not running. - */ - void start(); - - /** - * Stops the instance from replicating requests. Stop may only be called if the instance is running. - */ - void stop(); - - /** - * @return true if the instance is started; false otherwise. - */ - boolean isRunning(); - - /** - * Requests are sent to each node in the cluster. If the request results in an exception, then the NodeResourceResponse will contain the exception. - * - * HTTP DELETE and OPTIONS methods must supply an empty parameters map or else and IllegalArgumentException is thrown. - * - * @param nodeIds the node identifiers - * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD, OPTIONS) - * @param uri the base request URI (up to, but not including, the query string) - * @param parameters any request parameters - * @param headers any HTTP headers - * - * @return the set of node responses - * - * @throws UriConstructionException if a request for a node failed to be constructed from the given prototype URI. If thrown, it is guaranteed that no request was sent. - */ - Set replicate(Set nodeIds, String method, URI uri, Map> parameters, Map headers) throws UriConstructionException; - - /** - * Requests are sent to each node in the cluster. If the request results in an exception, then the NodeResourceResponse will contain the exception. - * - * HTTP DELETE, GET, HEAD, and OPTIONS methods will throw an IllegalArgumentException if used. - * - * @param nodeIds the node identifiers - * @param method the HTTP method (e.g., POST, PUT) - * @param uri the base request URI (up to, but not including, the query string) - * @param entity an entity - * @param headers any HTTP headers - * - * @return the set of node responses - * - * @throws UriConstructionException if a request for a node failed to be constructed from the given prototype URI. If thrown, it is guaranteed that no request was sent. - */ - Set replicate(Set nodeIds, String method, URI uri, Object entity, Map headers) throws UriConstructionException; - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java deleted file mode 100644 index 8f56bbbfdbf6..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.manager; - -import java.net.URI; -import java.util.Map; -import java.util.Set; - -import org.apache.nifi.cluster.node.Node.Status; - -/** - * Maps a HTTP response to a node status. - * - */ -public interface HttpResponseMapper { - - /** - * Maps a HTTP response to a node response and the corresponding node status. - * - * @param requestURI the original request URI - * @param nodeResponses a set of node resource responses - * - * @return a map associating the node response to the node status - */ - Map map(URI requestURI, Set nodeResponses); - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java index ec365aac074a..308652e873b7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java @@ -131,9 +131,6 @@ public URI getRequestUri() { return requestUri; } - /** - * @return the HTTP response status code - */ public int getStatus() { if (hasThrowable()) { /* @@ -151,49 +148,25 @@ public int getStatus() { } } - /** - * @return true if the response status is 2xx, false otherwise. - */ public boolean is2xx() { final int statusCode = getStatus(); return (200 <= statusCode && statusCode <= 299); } - /** - * @return true if the response status is 5xx, false otherwise. - */ public boolean is5xx() { final int statusCode = getStatus(); return (500 <= statusCode && statusCode <= 599); } - /** - * Returns null if hasThrowable() is true; otherwise the client's response is returned. - * - * The ClientResponse's input stream can only be read once. - * - * @return the client's response - */ public ClientResponse getClientResponse() { return clientResponse; } - /** - * If this node response has been merged returns the updated entity, otherwise null. Also returns null if hasThrowable() is true. The intent of this method is to support getting the response - * entity when it was already consumed during the merge operation. In this case the client response rom getClientResponse() will not support a getEntity(...) or getEntityInputStream() call. - * - * @return If this node response has been merged returns the updated entity, otherwise null. Also returns null if hasThrowable() is true - */ + public Entity getUpdatedEntity() { return updatedEntity; } - /** - * Creates a Response by mapping the ClientResponse values to it. Since the ClientResponse's input stream can only be read once, this method should only be called once. Furthermore, the caller - * should not have already read the ClientResponse's input stream. - * - * @return the response - */ public Response getResponse() { // if the response encapsulates a throwable, then the input stream is never read and the below warning is irrelevant if (hasCreatedResponse && !hasThrowable()) { @@ -203,20 +176,11 @@ public Response getResponse() { return createResponse(); } - /** - * Returns the throwable or null if no throwable exists. - * - * @return the throwable or null if no throwable exists - */ + public Throwable getThrowable() { return throwable; } - /** - * Returns true if a throwable was thrown and a response was not able to be created; false otherwise. - * - * @return true if a throwable was thrown and a response was not able to be created; false otherwise - */ public boolean hasThrowable() { return getThrowable() != null; } @@ -313,10 +277,10 @@ public void write(final OutputStream output) throws IOException, WebApplicationE public String toString() { final StringBuilder sb = new StringBuilder(); sb.append("NodeResponse[nodeUri=").append(nodeId.getApiAddress()).append(":").append(nodeId.getApiPort()).append(",") - .append("method=").append(httpMethod) - .append(",URI=").append(requestUri) - .append(",ResponseCode=").append(getStatus()) - .append(",Duration=").append(TimeUnit.MILLISECONDS.convert(requestDurationNanos, TimeUnit.NANOSECONDS)).append(" ms]"); + .append("method=").append(httpMethod) + .append(",URI=").append(requestUri) + .append(",ResponseCode=").append(getStatus()) + .append(",Duration=").append(TimeUnit.MILLISECONDS.convert(requestDurationNanos, TimeUnit.NANOSECONDS)).append(" ms]"); return sb.toString(); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java index 49bcd35007f3..c13e9d95c6a3 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java @@ -22,7 +22,7 @@ * */ public class BlockedByFirewallException extends ClusterException { - + private static final long serialVersionUID = 1L; private final NodeIdentifier nodeId; private final boolean isExistingNode; diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java deleted file mode 100644 index 26b22f8cda07..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.manager.exception; - -/** - * Represents the exceptional case when the primary role cannot be assigned to a node because the node is ineligible for the role. - * - */ -public class IneligiblePrimaryNodeException extends IllegalClusterStateException { - - public IneligiblePrimaryNodeException() { - } - - public IneligiblePrimaryNodeException(String msg) { - super(msg); - } - - public IneligiblePrimaryNodeException(Throwable cause) { - super(cause); - } - - public IneligiblePrimaryNodeException(String msg, Throwable cause) { - super(msg, cause); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java deleted file mode 100644 index 57503826b00d..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.manager.exception; - -/** - * Represents the exceptional case when the cluster is unable to update the primary role of a node. - * - */ -public class PrimaryRoleAssignmentException extends IllegalClusterStateException { - - public PrimaryRoleAssignmentException() { - } - - public PrimaryRoleAssignmentException(String msg) { - super(msg); - } - - public PrimaryRoleAssignmentException(Throwable cause) { - super(cause); - } - - public PrimaryRoleAssignmentException(String msg, Throwable cause) { - super(msg, cause); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java deleted file mode 100644 index 05d24359a8db..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.manager.exception; - -/** - * Represents the exceptional case when a HTTP request that may change a node's dataflow is to be replicated while the cluster is in safe mode. - * - */ -public class SafeModeMutableRequestException extends MutableRequestException { - - public SafeModeMutableRequestException() { - } - - public SafeModeMutableRequestException(String msg) { - super(msg); - } - - public SafeModeMutableRequestException(Throwable cause) { - super(cause); - } - - public SafeModeMutableRequestException(String msg, Throwable cause) { - super(msg, cause); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java deleted file mode 100644 index 3966a31697b0..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.manager.impl; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -import org.apache.nifi.action.Action; -import org.apache.nifi.admin.service.AuditService; -import org.apache.nifi.controller.status.ProcessGroupStatus; -import org.apache.nifi.events.EventReporter; -import org.apache.nifi.history.History; -import org.apache.nifi.provenance.ProvenanceEventBuilder; -import org.apache.nifi.provenance.ProvenanceEventRecord; -import org.apache.nifi.provenance.ProvenanceEventRepository; -import org.apache.nifi.provenance.lineage.ComputeLineageSubmission; -import org.apache.nifi.provenance.search.Query; -import org.apache.nifi.provenance.search.QuerySubmission; -import org.apache.nifi.provenance.search.SearchableField; -import org.apache.nifi.reporting.EventAccess; - -public class ClusteredEventAccess implements EventAccess { - - private final WebClusterManager clusterManager; - private final AuditService auditService; - - public ClusteredEventAccess(final WebClusterManager clusterManager, final AuditService auditService) { - this.clusterManager = clusterManager; - this.auditService = auditService; - } - - @Override - public ProcessGroupStatus getControllerStatus() { - return new ProcessGroupStatus(); - } - - @Override - public List getProvenanceEvents(long arg0, int arg1) throws IOException { - return new ArrayList<>(); - } - - @Override - public ProvenanceEventRepository getProvenanceRepository() { - // NCM doesn't have provenance events, because it doesn't process FlowFiles. - // So we just use a Provenance Event Repository that does nothing. - return new ProvenanceEventRepository() { - @Override - public void close() throws IOException { - } - - @Override - public ProvenanceEventRecord getEvent(long eventId) throws IOException { - return null; - } - - @Override - public List getEvents(long startEventId, int maxEvents) throws IOException { - return new ArrayList<>(); - } - - @Override - public Long getMaxEventId() { - return null; - } - - @Override - public List getSearchableAttributes() { - return new ArrayList<>(); - } - - @Override - public List getSearchableFields() { - return new ArrayList<>(); - } - - @Override - public void registerEvent(final ProvenanceEventRecord event) { - } - - @Override - public void registerEvents(final Iterable events) { - } - - @Override - public ComputeLineageSubmission retrieveLineageSubmission(final String submissionId) { - return null; - } - - @Override - public QuerySubmission retrieveQuerySubmission(final String submissionId) { - return null; - } - - @Override - public ComputeLineageSubmission submitExpandChildren(final long eventId) { - return null; - } - - @Override - public ComputeLineageSubmission submitExpandParents(final long eventId) { - return null; - } - - @Override - public ComputeLineageSubmission submitLineageComputation(final String flowFileUuid) { - return null; - } - - @Override - public QuerySubmission submitQuery(final Query query) { - return null; - } - - @Override - public ProvenanceEventBuilder eventBuilder() { - return null; - } - - @Override - public void initialize(EventReporter eventReporter) throws IOException { - - } - }; - } - - @Override - public List getFlowChanges(int firstActionId, int maxActions) { - final History history = auditService.getActions(firstActionId, maxActions); - return new ArrayList<>(history.getActions()); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java deleted file mode 100644 index 7f176b03b49d..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java +++ /dev/null @@ -1,217 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.manager.impl; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - -import org.apache.nifi.attribute.expression.language.PreparedQuery; -import org.apache.nifi.attribute.expression.language.Query; -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.state.StateManager; -import org.apache.nifi.controller.ControllerService; -import org.apache.nifi.controller.ControllerServiceLookup; -import org.apache.nifi.controller.service.ControllerServiceProvider; -import org.apache.nifi.controller.status.PortStatus; -import org.apache.nifi.controller.status.ProcessGroupStatus; -import org.apache.nifi.controller.status.ProcessorStatus; -import org.apache.nifi.controller.status.RemoteProcessGroupStatus; -import org.apache.nifi.events.BulletinFactory; -import org.apache.nifi.reporting.Bulletin; -import org.apache.nifi.reporting.BulletinRepository; -import org.apache.nifi.reporting.ComponentType; -import org.apache.nifi.reporting.EventAccess; -import org.apache.nifi.reporting.ReportingContext; -import org.apache.nifi.reporting.Severity; - -public class ClusteredReportingContext implements ReportingContext { - - private final EventAccess eventAccess; - private final BulletinRepository bulletinRepository; - private final ControllerServiceProvider serviceProvider; - private final Map properties; - private final Map preparedQueries; - private final StateManager stateManager; - - public ClusteredReportingContext(final EventAccess eventAccess, final BulletinRepository bulletinRepository, final Map properties, - final ControllerServiceProvider serviceProvider, final StateManager stateManager) { - this.eventAccess = eventAccess; - this.bulletinRepository = bulletinRepository; - this.properties = Collections.unmodifiableMap(properties); - this.serviceProvider = serviceProvider; - this.stateManager = stateManager; - - preparedQueries = new HashMap<>(); - for (final Map.Entry entry : properties.entrySet()) { - final PropertyDescriptor desc = entry.getKey(); - String value = entry.getValue(); - if (value == null) { - value = desc.getDefaultValue(); - } - - final PreparedQuery pq = Query.prepare(value); - preparedQueries.put(desc, pq); - } - } - - @Override - public EventAccess getEventAccess() { - return eventAccess; - } - - @Override - public BulletinRepository getBulletinRepository() { - return bulletinRepository; - } - - @Override - public Bulletin createBulletin(final String category, final Severity severity, final String message) { - return BulletinFactory.createBulletin(category, severity.name(), message); - } - - @Override - public Bulletin createBulletin(final String componentId, final String category, final Severity severity, final String message) { - final ProcessGroupStatus rootGroupStatus = eventAccess.getControllerStatus(); - final String groupId = findGroupId(rootGroupStatus, componentId); - final String componentName = findComponentName(rootGroupStatus, componentId); - final ComponentType componentType = findComponentType(rootGroupStatus, componentId); - - return BulletinFactory.createBulletin(groupId, componentId, componentType, componentName, category, severity.name(), message); - } - - @Override - public Map getProperties() { - return Collections.unmodifiableMap(properties); - } - - @Override - public PropertyValue getProperty(final PropertyDescriptor property) { - final String configuredValue = properties.get(property); - return new StandardPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, serviceProvider, preparedQueries.get(property)); - } - - @Override - public ControllerServiceLookup getControllerServiceLookup() { - return serviceProvider; - } - - String findGroupId(final ProcessGroupStatus groupStatus, final String componentId) { - for (final ProcessorStatus procStatus : groupStatus.getProcessorStatus()) { - if (procStatus.getId().equals(componentId)) { - return groupStatus.getId(); - } - } - - for (final PortStatus portStatus : groupStatus.getInputPortStatus()) { - if (portStatus.getId().equals(componentId)) { - return groupStatus.getId(); - } - } - - for (final PortStatus portStatus : groupStatus.getOutputPortStatus()) { - if (portStatus.getId().equals(componentId)) { - return groupStatus.getId(); - } - } - - for (final ProcessGroupStatus childGroup : groupStatus.getProcessGroupStatus()) { - final String groupId = findGroupId(childGroup, componentId); - if (groupId != null) { - return groupId; - } - } - - return null; - } - - private ComponentType findComponentType(final ProcessGroupStatus groupStatus, final String componentId) { - for (final ProcessorStatus procStatus : groupStatus.getProcessorStatus()) { - if (procStatus.getId().equals(componentId)) { - return ComponentType.PROCESSOR; - } - } - - for (final PortStatus portStatus : groupStatus.getInputPortStatus()) { - if (portStatus.getId().equals(componentId)) { - return ComponentType.INPUT_PORT; - } - } - - for (final PortStatus portStatus : groupStatus.getOutputPortStatus()) { - if (portStatus.getId().equals(componentId)) { - return ComponentType.OUTPUT_PORT; - } - } - - for (final RemoteProcessGroupStatus remoteStatus : groupStatus.getRemoteProcessGroupStatus()) { - if (remoteStatus.getId().equals(componentId)) { - return ComponentType.REMOTE_PROCESS_GROUP; - } - } - - for (final ProcessGroupStatus childGroup : groupStatus.getProcessGroupStatus()) { - final ComponentType type = findComponentType(childGroup, componentId); - if (type != null) { - return type; - } - } - - final ControllerService service = serviceProvider.getControllerService(componentId); - if (service != null) { - return ComponentType.CONTROLLER_SERVICE; - } - - return null; - } - - private String findComponentName(final ProcessGroupStatus groupStatus, final String componentId) { - for (final ProcessorStatus procStatus : groupStatus.getProcessorStatus()) { - if (procStatus.getId().equals(componentId)) { - return procStatus.getName(); - } - } - - for (final PortStatus portStatus : groupStatus.getInputPortStatus()) { - if (portStatus.getId().equals(componentId)) { - return groupStatus.getName(); - } - } - - for (final PortStatus portStatus : groupStatus.getOutputPortStatus()) { - if (portStatus.getId().equals(componentId)) { - return groupStatus.getName(); - } - } - - for (final ProcessGroupStatus childGroup : groupStatus.getProcessGroupStatus()) { - final String componentName = findComponentName(childGroup, componentId); - if (componentName != null) { - return componentName; - } - } - - return null; - } - - @Override - public StateManager getStateManager() { - return stateManager; - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java deleted file mode 100644 index 8c645a924708..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java +++ /dev/null @@ -1,512 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.manager.impl; - -import java.net.URI; -import java.net.URISyntaxException; -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.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletionService; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorCompletionService; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; - -import javax.ws.rs.HttpMethod; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.MultivaluedMap; - -import org.apache.commons.lang3.StringUtils; -import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMerger; -import org.apache.nifi.cluster.manager.HttpRequestReplicator; -import org.apache.nifi.cluster.manager.NodeResponse; -import org.apache.nifi.cluster.manager.exception.UriConstructionException; -import org.apache.nifi.cluster.protocol.NodeIdentifier; -import org.apache.nifi.logging.NiFiLog; -import org.apache.nifi.util.FormatUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.sun.jersey.api.client.Client; -import com.sun.jersey.api.client.ClientResponse; -import com.sun.jersey.api.client.UniformInterfaceException; -import com.sun.jersey.api.client.WebResource; -import com.sun.jersey.api.client.config.ClientConfig; -import com.sun.jersey.api.client.filter.GZIPContentEncodingFilter; -import com.sun.jersey.core.util.MultivaluedMapImpl; - -/** - * An implementation of the HttpRequestReplicator interface. This implementation parallelizes the node HTTP requests using the given ExecutorService instance. Individual - * requests may have connection and read timeouts set, which may be set during instance construction. Otherwise, the default is not to timeout. - * - * If a node protocol scheme is provided during construction, then all requests will be replicated using the given scheme. If null is provided as the scheme (the default), then the requests will be - * replicated using the scheme of the original URI. - * - * Clients must call start() and stop() to initialize and shutdown the instance. The instance must be started before issuing any replication requests. - * - */ -public class HttpRequestReplicatorImpl implements HttpRequestReplicator { - - // defaults - private static final int DEFAULT_SHUTDOWN_REPLICATOR_SECONDS = 30; - - // logger - private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(HttpRequestReplicatorImpl.class)); - - // final members - private final Client client; // the client to use for issuing requests - private final int numThreads; // number of threads to use for request replication - private final int connectionTimeoutMs; // connection timeout per node request - private final int readTimeoutMs; // read timeout per node request - - // members - private ExecutorService executorService; - private int shutdownReplicatorSeconds = DEFAULT_SHUTDOWN_REPLICATOR_SECONDS; - - // guarded by synchronized method access in support of multithreaded replication - private String nodeProtocolScheme = null; - - /** - * Creates an instance. The connection timeout and read timeout will be infinite. - * - * @param numThreads the number of threads to use when parallelizing requests - * @param client a client for making requests - */ - public HttpRequestReplicatorImpl(final int numThreads, final Client client) { - this(numThreads, client, "0 sec", "0 sec"); - } - - /** - * Creates an instance. - * - * @param numThreads the number of threads to use when parallelizing requests - * @param client a client for making requests - * @param connectionTimeout the connection timeout specified in milliseconds - * @param readTimeout the read timeout specified in milliseconds - */ - public HttpRequestReplicatorImpl(final int numThreads, final Client client, final String connectionTimeout, final String readTimeout) { - - if (numThreads <= 0) { - throw new IllegalArgumentException("The number of threads must be greater than zero."); - } else if (client == null) { - throw new IllegalArgumentException("Client may not be null."); - } - - this.numThreads = numThreads; - this.client = client; - this.connectionTimeoutMs = (int) FormatUtils.getTimeDuration(connectionTimeout, TimeUnit.MILLISECONDS); - this.readTimeoutMs = (int) FormatUtils.getTimeDuration(readTimeout, TimeUnit.MILLISECONDS); - - client.getProperties().put(ClientConfig.PROPERTY_CONNECT_TIMEOUT, connectionTimeoutMs); - client.getProperties().put(ClientConfig.PROPERTY_READ_TIMEOUT, readTimeoutMs); - client.getProperties().put(ClientConfig.PROPERTY_FOLLOW_REDIRECTS, Boolean.TRUE); - } - - @Override - public void start() { - if (isRunning()) { - throw new IllegalStateException("Instance is already started."); - } - executorService = Executors.newFixedThreadPool(numThreads); - } - - @Override - public boolean isRunning() { - return executorService != null && !executorService.isShutdown(); - } - - @Override - public void stop() { - - if (!isRunning()) { - throw new IllegalStateException("Instance is already stopped."); - } - - // shutdown executor service - try { - if (getShutdownReplicatorSeconds() <= 0) { - executorService.shutdownNow(); - } else { - executorService.shutdown(); - } - executorService.awaitTermination(getShutdownReplicatorSeconds(), TimeUnit.SECONDS); - } catch (final InterruptedException ex) { - Thread.currentThread().interrupt(); - } finally { - if (executorService.isTerminated()) { - logger.info("HTTP Request Replicator has been terminated successfully."); - } else { - logger.warn("HTTP Request Replicator has not terminated properly. There exists an uninterruptable thread that will take an indeterminate amount of time to stop."); - } - } - } - - /** - * Sets the protocol scheme to use when issuing requests to nodes. - * - * @param nodeProtocolScheme the scheme. Valid values are "http", "https", or null. If null is specified, then the scheme of the originating request is used when replicating that request. - */ - public synchronized void setNodeProtocolScheme(final String nodeProtocolScheme) { - if (StringUtils.isNotBlank(nodeProtocolScheme)) { - if (!"http".equalsIgnoreCase(nodeProtocolScheme) && !"https".equalsIgnoreCase(nodeProtocolScheme)) { - throw new IllegalArgumentException("Node Protocol Scheme must be either HTTP or HTTPS"); - } - } - this.nodeProtocolScheme = nodeProtocolScheme; - } - - public synchronized String getNodeProtocolScheme() { - return nodeProtocolScheme; - } - - private synchronized String getNodeProtocolScheme(final URI uri) { - // if we are not configured to use a protocol scheme, then use the uri's scheme - if (StringUtils.isBlank(nodeProtocolScheme)) { - return uri.getScheme(); - } - return nodeProtocolScheme; - } - - public int getConnectionTimeoutMs() { - return connectionTimeoutMs; - } - - public int getReadTimeoutMs() { - return readTimeoutMs; - } - - public int getShutdownReplicatorSeconds() { - return shutdownReplicatorSeconds; - } - - public void setShutdownReplicatorSeconds(int shutdownReplicatorSeconds) { - this.shutdownReplicatorSeconds = shutdownReplicatorSeconds; - } - - @Override - public Set replicate(final Set nodeIds, final String method, - final URI uri, final Map> parameters, final Map headers) - throws UriConstructionException { - if (nodeIds == null) { - throw new IllegalArgumentException("Node IDs may not be null."); - } else if (method == null) { - throw new IllegalArgumentException("HTTP method may not be null."); - } else if (uri == null) { - throw new IllegalArgumentException("URI may not be null."); - } else if (parameters == null) { - throw new IllegalArgumentException("Parameters may not be null."); - } else if (headers == null) { - throw new IllegalArgumentException("HTTP headers map may not be null."); - } - return replicateHelper(nodeIds, method, getNodeProtocolScheme(uri), uri.getPath(), parameters, /* entity */ null, headers); - } - - @Override - public Set replicate(final Set nodeIds, final String method, final URI uri, - final Object entity, final Map headers) throws UriConstructionException { - if (nodeIds == null) { - throw new IllegalArgumentException("Node IDs may not be null."); - } else if (method == null) { - throw new IllegalArgumentException("HTTP method may not be null."); - } else if (method.equalsIgnoreCase(HttpMethod.DELETE) || method.equalsIgnoreCase(HttpMethod.GET) || method.equalsIgnoreCase(HttpMethod.HEAD) || method.equalsIgnoreCase(HttpMethod.OPTIONS)) { - throw new IllegalArgumentException("HTTP (DELETE | GET | HEAD | OPTIONS) requests cannot have a body containing an entity."); - } else if (uri == null) { - throw new IllegalArgumentException("URI may not be null."); - } else if (entity == null) { - throw new IllegalArgumentException("Entity may not be null."); - } else if (headers == null) { - throw new IllegalArgumentException("HTTP headers map may not be null."); - } - return replicateHelper(nodeIds, method, getNodeProtocolScheme(uri), uri.getPath(), /* parameters */ null, entity, headers); - } - - private Set replicateHelper(final Set nodeIds, final String method, final String scheme, - final String path, final Map> parameters, final Object entity, final Map headers) - throws UriConstructionException { - - if (nodeIds.isEmpty()) { - return new HashSet<>(); // return quickly for trivial case - } - - final CompletionService completionService = new ExecutorCompletionService<>(executorService); - - // keeps track of future requests so that failed requests can be tied back to the failing node - final Collection futureNodeHttpRequests = new ArrayList<>(); - - // construct the URIs for the nodes - final Map uriMap = new HashMap<>(); - try { - for (final NodeIdentifier nodeId : nodeIds) { - final URI nodeUri = new URI(scheme, null, nodeId.getApiAddress(), nodeId.getApiPort(), path, /* query */ null, /* fragment */ null); - uriMap.put(nodeId, nodeUri); - } - } catch (final URISyntaxException use) { - throw new UriConstructionException(use); - } - - // submit the requests to the nodes - for (final Map.Entry entry : uriMap.entrySet()) { - final NodeIdentifier nodeId = entry.getKey(); - final URI nodeUri = entry.getValue(); - final NodeHttpRequestCallable callable = (entity == null) - ? new NodeHttpRequestCallable(nodeId, method, nodeUri, parameters, headers) - : new NodeHttpRequestCallable(nodeId, method, nodeUri, entity, headers); - futureNodeHttpRequests.add(new NodeHttpRequestFutureWrapper(nodeId, method, nodeUri, completionService.submit(callable))); - } - - // get the node responses - final Set result = new HashSet<>(); - for (int i = 0; i < nodeIds.size(); i++) { - - // keeps track of the original request information in case we receive an exception - NodeHttpRequestFutureWrapper futureNodeHttpRequest = null; - try { - - // get the future resource response for the node - final Future futureNodeResourceResponse = completionService.take(); - - // find the original request by comparing the submitted future with the future returned by the completion service - for (final NodeHttpRequestFutureWrapper futureNodeHttpRequestElem : futureNodeHttpRequests) { - if (futureNodeHttpRequestElem.getFuture() == futureNodeResourceResponse) { - futureNodeHttpRequest = futureNodeHttpRequestElem; - } - } - - // try to retrieve the node response and add to result - final NodeResponse nodeResponse = futureNodeResourceResponse.get(); - result.add(nodeResponse); - - } catch (final InterruptedException | ExecutionException ex) { - - logger.warn("Node request for " + futureNodeHttpRequest.getNodeId() + " encountered exception: " + ex, ex); - - // create node response with the thrown exception and add to result - final NodeResponse nodeResponse = new NodeResponse( - futureNodeHttpRequest.getNodeId(), futureNodeHttpRequest.getHttpMethod(), futureNodeHttpRequest.getRequestUri(), ex); - result.add(nodeResponse); - - } - } - - if (logger.isDebugEnabled()) { - NodeResponse min = null; - NodeResponse max = null; - long nanosSum = 0L; - int nanosAdded = 0; - - for (final NodeResponse response : result) { - final long requestNanos = response.getRequestDuration(TimeUnit.NANOSECONDS); - final long minNanos = (min == null) ? -1 : min.getRequestDuration(TimeUnit.NANOSECONDS); - final long maxNanos = (max == null) ? -1 : max.getRequestDuration(TimeUnit.NANOSECONDS); - - if (requestNanos < minNanos || minNanos < 0L) { - min = response; - } - - if (requestNanos > maxNanos || maxNanos < 0L) { - max = response; - } - - if (requestNanos >= 0L) { - nanosSum += requestNanos; - nanosAdded++; - } - } - - final StringBuilder sb = new StringBuilder(); - sb.append("Node Responses for ").append(method).append(" ").append(path).append(":\n"); - for (final NodeResponse response : result) { - sb.append(response).append("\n"); - } - - final long averageNanos = (nanosAdded == 0) ? -1L : nanosSum / nanosAdded; - final long averageMillis = (averageNanos < 0) ? averageNanos : TimeUnit.MILLISECONDS.convert(averageNanos, TimeUnit.NANOSECONDS); - logger.debug("For {} {}, minimum response time = {}, max = {}, average = {} ms", - method, path, min, max, averageMillis); - logger.debug(sb.toString()); - } - - return result; - } - - /** - * Wraps a future node response with info from originating request. This coupling allows for futures that encountered exceptions to be linked back to the failing node and better reported. - */ - private class NodeHttpRequestFutureWrapper { - - private final NodeIdentifier nodeId; - - private final String httpMethod; - - private final URI requestUri; - - private final Future future; - - public NodeHttpRequestFutureWrapper(final NodeIdentifier nodeId, final String httpMethod, - final URI requestUri, final Future future) { - if (nodeId == null) { - throw new IllegalArgumentException("Node ID may not be null."); - } else if (StringUtils.isBlank(httpMethod)) { - throw new IllegalArgumentException("Http method may not be null or empty."); - } else if (requestUri == null) { - throw new IllegalArgumentException("Request URI may not be null."); - } else if (future == null) { - throw new IllegalArgumentException("Future may not be null."); - } - this.nodeId = nodeId; - this.httpMethod = httpMethod; - this.requestUri = requestUri; - this.future = future; - } - - public NodeIdentifier getNodeId() { - return nodeId; - } - - public String getHttpMethod() { - return httpMethod; - } - - public URI getRequestUri() { - return requestUri; - } - - public Future getFuture() { - return future; - } - } - - /** - * A Callable for making an HTTP request to a single node and returning its response. - */ - private class NodeHttpRequestCallable implements Callable { - - private final NodeIdentifier nodeId; - private final String method; - private final URI uri; - private final Object entity; - private final Map> parameters = new HashMap<>(); - private final Map headers = new HashMap<>(); - - private NodeHttpRequestCallable(final NodeIdentifier nodeId, final String method, - final URI uri, final Object entity, final Map headers) { - this.nodeId = nodeId; - this.method = method; - this.uri = uri; - this.entity = entity; - this.headers.putAll(headers); - } - - private NodeHttpRequestCallable(final NodeIdentifier nodeId, final String method, - final URI uri, final Map> parameters, final Map headers) { - this.nodeId = nodeId; - this.method = method; - this.uri = uri; - this.entity = null; - this.parameters.putAll(parameters); - this.headers.putAll(headers); - } - - @Override - public NodeResponse call() { - - try { - // create and send the request - final WebResource.Builder resourceBuilder = getResourceBuilder(); - final String requestId = headers.get("x-nifi-request-id"); - - final long startNanos = System.nanoTime(); - final ClientResponse clientResponse; - if (HttpMethod.DELETE.equalsIgnoreCase(method)) { - clientResponse = resourceBuilder.delete(ClientResponse.class); - } else if (HttpMethod.GET.equalsIgnoreCase(method)) { - clientResponse = resourceBuilder.get(ClientResponse.class); - } else if (HttpMethod.HEAD.equalsIgnoreCase(method)) { - clientResponse = resourceBuilder.head(); - } else if (HttpMethod.OPTIONS.equalsIgnoreCase(method)) { - clientResponse = resourceBuilder.options(ClientResponse.class); - } else if (HttpMethod.POST.equalsIgnoreCase(method)) { - clientResponse = resourceBuilder.post(ClientResponse.class); - } else if (HttpMethod.PUT.equalsIgnoreCase(method)) { - clientResponse = resourceBuilder.put(ClientResponse.class); - } else { - throw new IllegalArgumentException("HTTP Method '" + method + "' not supported for request replication."); - } - - // create and return the response - return new NodeResponse(nodeId, method, uri, clientResponse, System.nanoTime() - startNanos, requestId); - - } catch (final UniformInterfaceException | IllegalArgumentException t) { - return new NodeResponse(nodeId, method, uri, t); - } - - } - - private WebResource.Builder getResourceBuilder() { - - // convert parameters to a more convenient data structure - final MultivaluedMap map = new MultivaluedMapImpl(); - map.putAll(parameters); - - // create the resource - WebResource resource = client.resource(uri); - - if (new StandardHttpResponseMerger().isResponseInterpreted(uri, method)) { - resource.addFilter(new GZIPContentEncodingFilter(false)); - } - - // set the parameters as either query parameters or as request body - final WebResource.Builder builder; - if (HttpMethod.DELETE.equalsIgnoreCase(method) || HttpMethod.HEAD.equalsIgnoreCase(method) || HttpMethod.GET.equalsIgnoreCase(method) || HttpMethod.OPTIONS.equalsIgnoreCase(method)) { - resource = resource.queryParams(map); - builder = resource.getRequestBuilder(); - } else { - if (entity == null) { - builder = resource.entity(map); - } else { - builder = resource.entity(entity); - } - } - - // set headers - boolean foundContentType = false; - for (final Map.Entry entry : headers.entrySet()) { - builder.header(entry.getKey(), entry.getValue()); - if (entry.getKey().equalsIgnoreCase("content-type")) { - foundContentType = true; - } - } - - // set default content type - if (!foundContentType) { - // set default content type - builder.type(MediaType.APPLICATION_FORM_URLENCODED); - } - - return builder; - } - - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java deleted file mode 100644 index 098573d8bf48..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.manager.impl; - -import java.net.URI; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - -import org.apache.nifi.cluster.manager.HttpResponseMapper; -import org.apache.nifi.cluster.manager.NodeResponse; -import org.apache.nifi.cluster.node.Node; -import org.apache.nifi.cluster.node.Node.Status; -import org.apache.nifi.logging.NiFiLog; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Determines the status of nodes based on their HTTP response codes. - * - * The algorithm is as follows. - * - * If any HTTP responses were 2XX, then disconnect non-2XX responses. This is because 2XX may have changed a node's flow. - * - * If no 2XX responses were received, then the node's flow has not changed. Instead of disconnecting everything, we only disconnect the nodes with internal errors, i.e., 5XX responses. - * - */ -public class HttpResponseMapperImpl implements HttpResponseMapper { - - private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(HttpResponseMapperImpl.class)); - - @Override - public Map map(final URI requestURI, final Set nodeResponses) { - - final Map result = new HashMap<>(); - - // check if any responses were 2XX - boolean found2xx = false; - for (final NodeResponse nodeResponse : nodeResponses) { - if (nodeResponse.is2xx()) { - found2xx = true; - break; - } - } - - // determine the status of each node - for (final NodeResponse nodeResponse : nodeResponses) { - - final Node.Status status; - if (found2xx) { - // disconnect nodes with non-2XX responses - status = nodeResponse.is2xx() - ? Node.Status.CONNECTED - : Node.Status.DISCONNECTED; - } else { - // disconnect nodes with 5XX responses or exception - status = nodeResponse.is5xx() - ? Node.Status.DISCONNECTED - : Node.Status.CONNECTED; - } - - result.put(nodeResponse, status); - } - - return result; - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java deleted file mode 100644 index f7b5745e739a..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java +++ /dev/null @@ -1,2159 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.manager.impl; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.net.URI; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Date; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.CompletionService; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutorCompletionService; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import java.util.regex.Pattern; - -import javax.net.ssl.SSLContext; -import javax.ws.rs.HttpMethod; -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; -import javax.xml.parsers.ParserConfigurationException; -import javax.xml.transform.OutputKeys; -import javax.xml.transform.Transformer; -import javax.xml.transform.TransformerException; -import javax.xml.transform.TransformerFactory; -import javax.xml.transform.dom.DOMSource; -import javax.xml.transform.stream.StreamResult; - -import org.apache.nifi.admin.service.AuditService; -import org.apache.nifi.annotation.lifecycle.OnAdded; -import org.apache.nifi.annotation.lifecycle.OnConfigurationRestored; -import org.apache.nifi.annotation.lifecycle.OnRemoved; -import org.apache.nifi.cluster.coordination.heartbeat.ClusterProtocolHeartbeatMonitor; -import org.apache.nifi.cluster.coordination.heartbeat.NodeHeartbeat; -import org.apache.nifi.cluster.coordination.http.HttpResponseMerger; -import org.apache.nifi.cluster.coordination.http.StandardHttpResponseMerger; -import org.apache.nifi.cluster.coordination.http.replication.AsyncClusterResponse; -import org.apache.nifi.cluster.coordination.http.replication.RequestCompletionCallback; -import org.apache.nifi.cluster.coordination.http.replication.RequestReplicator; -import org.apache.nifi.cluster.coordination.http.replication.ThreadPoolRequestReplicator; -import org.apache.nifi.cluster.coordination.node.DisconnectionCode; -import org.apache.nifi.cluster.event.Event; -import org.apache.nifi.cluster.event.EventManager; -import org.apache.nifi.cluster.firewall.ClusterNodeFirewall; -import org.apache.nifi.cluster.flow.ClusterDataFlow; -import org.apache.nifi.cluster.flow.DaoException; -import org.apache.nifi.cluster.flow.DataFlowManagementService; -import org.apache.nifi.cluster.manager.HttpClusterManager; -import org.apache.nifi.cluster.manager.NodeResponse; -import org.apache.nifi.cluster.manager.exception.ConflictingNodeIdException; -import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException; -import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException; -import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException; -import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException; -import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException; -import org.apache.nifi.cluster.manager.exception.IllegalNodeReconnectionException; -import org.apache.nifi.cluster.manager.exception.NoConnectedNodesException; -import org.apache.nifi.cluster.manager.exception.NoResponseFromNodesException; -import org.apache.nifi.cluster.manager.exception.NodeDisconnectionException; -import org.apache.nifi.cluster.manager.exception.NodeReconnectionException; -import org.apache.nifi.cluster.manager.exception.SafeModeMutableRequestException; -import org.apache.nifi.cluster.manager.exception.UnknownNodeException; -import org.apache.nifi.cluster.manager.exception.UriConstructionException; -import org.apache.nifi.cluster.node.Node; -import org.apache.nifi.cluster.node.Node.Status; -import org.apache.nifi.cluster.protocol.ConnectionRequest; -import org.apache.nifi.cluster.protocol.ConnectionResponse; -import org.apache.nifi.cluster.protocol.NodeIdentifier; -import org.apache.nifi.cluster.protocol.ProtocolException; -import org.apache.nifi.cluster.protocol.ProtocolHandler; -import org.apache.nifi.cluster.protocol.StandardDataFlow; -import org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderListener; -import org.apache.nifi.cluster.protocol.impl.ClusterServicesBroadcaster; -import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; -import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; -import org.apache.nifi.cluster.protocol.message.DisconnectMessage; -import org.apache.nifi.cluster.protocol.message.ProtocolMessage; -import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType; -import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.components.state.StateManagerProvider; -import org.apache.nifi.controller.ConfiguredComponent; -import org.apache.nifi.controller.ControllerService; -import org.apache.nifi.controller.ReportingTaskNode; -import org.apache.nifi.controller.ScheduledState; -import org.apache.nifi.controller.StandardProcessorNode; -import org.apache.nifi.controller.ValidationContextFactory; -import org.apache.nifi.controller.exception.ComponentLifeCycleException; -import org.apache.nifi.controller.reporting.ClusteredReportingTaskNode; -import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException; -import org.apache.nifi.controller.reporting.ReportingTaskProvider; -import org.apache.nifi.controller.reporting.StandardReportingInitializationContext; -import org.apache.nifi.controller.scheduling.QuartzSchedulingAgent; -import org.apache.nifi.controller.scheduling.StandardProcessScheduler; -import org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent; -import org.apache.nifi.controller.serialization.StandardFlowSerializer; -import org.apache.nifi.controller.service.ControllerServiceLoader; -import org.apache.nifi.controller.service.ControllerServiceNode; -import org.apache.nifi.controller.service.ControllerServiceProvider; -import org.apache.nifi.controller.service.StandardControllerServiceProvider; -import org.apache.nifi.controller.state.manager.StandardStateManagerProvider; -import org.apache.nifi.encrypt.StringEncryptor; -import org.apache.nifi.engine.FlowEngine; -import org.apache.nifi.events.BulletinFactory; -import org.apache.nifi.events.EventReporter; -import org.apache.nifi.events.VolatileBulletinRepository; -import org.apache.nifi.framework.security.util.SslContextFactory; -import org.apache.nifi.io.socket.multicast.DiscoverableService; -import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.logging.ControllerServiceLogObserver; -import org.apache.nifi.logging.LogLevel; -import org.apache.nifi.logging.LogRepository; -import org.apache.nifi.logging.LogRepositoryFactory; -import org.apache.nifi.logging.NiFiLog; -import org.apache.nifi.logging.ReportingTaskLogObserver; -import org.apache.nifi.nar.ExtensionManager; -import org.apache.nifi.nar.NarCloseable; -import org.apache.nifi.processor.SimpleProcessLogger; -import org.apache.nifi.processor.StandardValidationContextFactory; -import org.apache.nifi.remote.RemoteResourceManager; -import org.apache.nifi.remote.RemoteSiteListener; -import org.apache.nifi.remote.SocketRemoteSiteListener; -import org.apache.nifi.remote.cluster.ClusterNodeInformation; -import org.apache.nifi.remote.cluster.NodeInformation; -import org.apache.nifi.remote.protocol.socket.ClusterManagerServerProtocol; -import org.apache.nifi.reporting.Bulletin; -import org.apache.nifi.reporting.BulletinRepository; -import org.apache.nifi.reporting.InitializationException; -import org.apache.nifi.reporting.ReportingInitializationContext; -import org.apache.nifi.reporting.ReportingTask; -import org.apache.nifi.reporting.Severity; -import org.apache.nifi.scheduling.SchedulingStrategy; -import org.apache.nifi.util.DomUtils; -import org.apache.nifi.util.FormatUtils; -import org.apache.nifi.util.NiFiProperties; -import org.apache.nifi.util.ReflectionUtils; -import org.apache.nifi.web.api.dto.BulletinDTO; -import org.apache.nifi.web.util.WebUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.w3c.dom.DOMException; -import org.w3c.dom.Document; -import org.w3c.dom.Element; -import org.w3c.dom.NodeList; -import org.xml.sax.SAXException; -import org.xml.sax.SAXParseException; - -import com.sun.jersey.api.client.Client; -import com.sun.jersey.api.client.config.DefaultClientConfig; - -/** - * Provides a cluster manager implementation. The manager federates incoming HTTP client requests to the nodes' external API using the HTTP protocol. The manager also communicates with nodes using the - * nodes' internal socket protocol. - * - * The manager's socket address may broadcasted using multicast if a MulticastServiceBroadcaster instance is set on this instance. The manager instance must be started after setting the broadcaster. - * - * The manager may be configured with an EventManager for recording noteworthy lifecycle events (e.g., first heartbeat received, node status change). - * - * The start() and stop() methods must be called to initialize and stop the instance. - * - */ -public class WebClusterManager implements HttpClusterManager, ProtocolHandler, ControllerServiceProvider, ReportingTaskProvider, RequestCompletionCallback { - - public static final String BULLETIN_CATEGORY = "Clustering"; - - private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(WebClusterManager.class)); - - - /** - * The default number of seconds to respond to a connecting node if the manager cannot provide it with a current data flow. - */ - private static final int DEFAULT_CONNECTION_REQUEST_TRY_AGAIN_SECONDS = 5; - - public static final Pattern COUNTER_URI_PATTERN = Pattern.compile("/nifi-api/controller/counters/[a-f0-9\\-]{36}"); - - private final NiFiProperties properties; - private final DataFlowManagementService dataFlowManagementService; - private final ClusterManagerProtocolSenderListener senderListener; - private final StringEncryptor encryptor; - private final ReentrantReadWriteLock resourceRWLock = new ReentrantReadWriteLock(true); - private final ClusterManagerLock readLock = new ClusterManagerLock(resourceRWLock.readLock(), "Read"); - private final ClusterManagerLock writeLock = new ClusterManagerLock(resourceRWLock.writeLock(), "Write"); - private final ClusterProtocolHeartbeatMonitor heartbeatMonitor; - private final WebClusterManagerCoordinator clusterCoordinator; - - private final Set nodes = new HashSet<>(); - private final ConcurrentMap reportingTasks = new ConcurrentHashMap<>(); - - // null means the dataflow should be read from disk - private StandardDataFlow cachedDataFlow = null; - private NodeIdentifier primaryNodeId = null; - private volatile ClusterServicesBroadcaster servicesBroadcaster = null; - private volatile EventManager eventManager = null; - private volatile ClusterNodeFirewall clusterFirewall = null; - private volatile AuditService auditService = null; - private volatile ControllerServiceProvider controllerServiceProvider = null; - - private final RemoteSiteListener remoteSiteListener; - private final Integer remoteInputPort; - private final Boolean remoteCommsSecure; - private final BulletinRepository bulletinRepository; - private final String instanceId; - private final FlowEngine reportingTaskEngine; - private final StandardProcessScheduler processScheduler; - private final StateManagerProvider stateManagerProvider; - - private final HttpResponseMerger responseMerger = new StandardHttpResponseMerger(this); - private final RequestReplicator httpRequestReplicator; - - public WebClusterManager( - final DataFlowManagementService dataFlowManagementService, final ClusterManagerProtocolSenderListener senderListener, - final NiFiProperties properties, final StringEncryptor encryptor) { - - if (dataFlowManagementService == null) { - throw new IllegalArgumentException("DataFlowManagementService may not be null."); - } else if (senderListener == null) { - throw new IllegalArgumentException("ClusterManagerProtocolSenderListener may not be null."); - } else if (properties == null) { - throw new IllegalArgumentException("NiFiProperties may not be null."); - } - - this.dataFlowManagementService = dataFlowManagementService; - this.properties = properties; - this.bulletinRepository = new VolatileBulletinRepository(); - this.instanceId = UUID.randomUUID().toString(); - this.senderListener = senderListener; - this.encryptor = encryptor; - senderListener.addHandler(this); - senderListener.setBulletinRepository(bulletinRepository); - - remoteInputPort = properties.getRemoteInputPort(); - if (remoteInputPort == null) { - remoteSiteListener = null; - remoteCommsSecure = null; - } else { - // Register the ClusterManagerServerProtocol as the appropriate resource for site-to-site Server Protocol - RemoteResourceManager.setServerProtocolImplementation(ClusterManagerServerProtocol.RESOURCE_NAME, ClusterManagerServerProtocol.class); - remoteCommsSecure = properties.isSiteToSiteSecure(); - if (remoteCommsSecure) { - final SSLContext sslContext = SslContextFactory.createSslContext(properties, false); - - if (sslContext == null) { - throw new IllegalStateException("NiFi Configured to allow Secure Site-to-Site communications but the Keystore/Truststore properties are not configured"); - } - - remoteSiteListener = new SocketRemoteSiteListener(remoteInputPort.intValue(), sslContext, this); - } else { - remoteSiteListener = new SocketRemoteSiteListener(remoteInputPort.intValue(), null, this); - } - } - - reportingTaskEngine = new FlowEngine(8, "Reporting Task Thread"); - - try { - this.stateManagerProvider = StandardStateManagerProvider.create(properties); - } catch (final IOException e) { - throw new RuntimeException(e); - } - - processScheduler = new StandardProcessScheduler(this, encryptor, stateManagerProvider); - - // When we construct the scheduling agents, we can pass null for a lot of the arguments because we are only - // going to be scheduling Reporting Tasks. Otherwise, it would not be okay. - processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, new TimerDrivenSchedulingAgent(null, reportingTaskEngine, null, encryptor)); - processScheduler.setSchedulingAgent(SchedulingStrategy.CRON_DRIVEN, new QuartzSchedulingAgent(null, reportingTaskEngine, null, encryptor)); - processScheduler.setMaxThreadCount(SchedulingStrategy.TIMER_DRIVEN, 10); - processScheduler.setMaxThreadCount(SchedulingStrategy.CRON_DRIVEN, 10); - - controllerServiceProvider = new StandardControllerServiceProvider(processScheduler, bulletinRepository, stateManagerProvider); - - clusterCoordinator = new WebClusterManagerCoordinator(this, senderListener, dataFlowManagementService); - heartbeatMonitor = new ClusterProtocolHeartbeatMonitor(clusterCoordinator, properties); - senderListener.addHandler(heartbeatMonitor); - httpRequestReplicator = createRequestReplicator(properties); - } - - private RequestReplicator createRequestReplicator(final NiFiProperties properties) { - final int numThreads = properties.getClusterManagerNodeApiRequestThreads(); - final String connectionTimeout = properties.getClusterManagerNodeApiConnectionTimeout(); - final String readTimeout = properties.getClusterManagerNodeApiReadTimeout(); - final EventReporter eventReporter = createEventReporter(); - - final Client jerseyClient = WebUtils.createClient(new DefaultClientConfig(), SslContextFactory.createSslContext(properties)); - return new ThreadPoolRequestReplicator(numThreads, jerseyClient, clusterCoordinator, connectionTimeout, readTimeout, this, - eventReporter, this, dataFlowManagementService); - } - - private EventReporter createEventReporter() { - return new EventReporter() { - private static final long serialVersionUID = 7770887158588031619L; - - @Override - public void reportEvent(Severity severity, String category, String message) { - final Bulletin bulletin = BulletinFactory.createBulletin(category, severity.name(), message); - getBulletinRepository().addBulletin(bulletin); - } - }; - } - - public void start() throws IOException { - writeLock.lock(); - try { - if (isRunning()) { - throw new IllegalStateException("Instance is already started."); - } - - try { - heartbeatMonitor.start(); - - // start request replication service - httpRequestReplicator.start(); - - // start protocol service - senderListener.start(); - - // start flow management service - dataFlowManagementService.start(); - - if (remoteSiteListener != null) { - remoteSiteListener.start(); - } - - // load flow - final ClusterDataFlow clusterDataFlow; - if (dataFlowManagementService.isFlowCurrent()) { - clusterDataFlow = dataFlowManagementService.loadDataFlow(); - cachedDataFlow = clusterDataFlow.getDataFlow(); - primaryNodeId = clusterDataFlow.getPrimaryNodeId(); - } else { - throw new IOException("Flow is not current."); - } - - final byte[] serializedServices = clusterDataFlow.getControllerServices(); - if (serializedServices != null && serializedServices.length > 0) { - ControllerServiceLoader.loadControllerServices(this, new ByteArrayInputStream(serializedServices), null, encryptor, bulletinRepository, properties.getAutoResumeState()); - } - - // start multicast broadcasting service, if configured - if (servicesBroadcaster != null) { - servicesBroadcaster.start(); - } - - // Load and start running Reporting Tasks - final byte[] serializedReportingTasks = clusterDataFlow.getReportingTasks(); - if (serializedReportingTasks != null && serializedReportingTasks.length > 0) { - loadReportingTasks(serializedReportingTasks); - } - - notifyComponentsConfigurationRestored(); - } catch (final IOException ioe) { - logger.warn("Failed to initialize cluster services due to: " + ioe, ioe); - stop(); - throw ioe; - } - - } finally { - writeLock.unlock("START"); - } - } - - public void stop() throws IOException { - writeLock.lock(); - try { - - // returns true if any service is running - if (isRunning() == false) { - throw new IllegalArgumentException("Instance is already stopped."); - } - - boolean encounteredException = false; - - heartbeatMonitor.stop(); - - // stop the HTTP request replicator service - if (httpRequestReplicator.isRunning()) { - httpRequestReplicator.stop(); - } - - // stop the flow management service - if (dataFlowManagementService.isRunning()) { - dataFlowManagementService.stop(); - } - - if (remoteSiteListener != null) { - remoteSiteListener.stop(); - } - - // stop the protocol listener service - if (senderListener.isRunning()) { - try { - senderListener.stop(); - } catch (final IOException ioe) { - encounteredException = true; - logger.warn("Failed to shutdown protocol service due to: " + ioe, ioe); - } - } - - // stop the service broadcaster - if (isBroadcasting()) { - servicesBroadcaster.stop(); - } - - if (processScheduler != null) { - processScheduler.shutdown(); - } - - if (encounteredException) { - throw new IOException("Failed to shutdown Cluster Manager because one or more cluster services failed to shutdown. Check the logs for details."); - } - - } finally { - writeLock.unlock("STOP"); - } - } - - public boolean isRunning() { - readLock.lock(); - try { - return httpRequestReplicator.isRunning() - || senderListener.isRunning() - || dataFlowManagementService.isRunning() - || isBroadcasting(); - } finally { - readLock.unlock("isRunning"); - } - } - - @Override - public boolean canHandle(ProtocolMessage msg) { - return MessageType.CONNECTION_REQUEST == msg.getType(); - } - - @Override - public ProtocolMessage handle(final ProtocolMessage protocolMessage) throws ProtocolException { - switch (protocolMessage.getType()) { - case CONNECTION_REQUEST: - return handleConnectionRequest((ConnectionRequestMessage) protocolMessage); - default: - throw new ProtocolException("No handler defined for message type: " + protocolMessage.getType()); - } - } - - - private void notifyComponentsConfigurationRestored() { - for (final ControllerServiceNode serviceNode : getAllControllerServices()) { - final ControllerService service = serviceNode.getControllerServiceImplementation(); - - try (final NarCloseable nc = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, service); - } - } - - for (final ReportingTaskNode taskNode : getAllReportingTasks()) { - final ReportingTask task = taskNode.getReportingTask(); - - try (final NarCloseable nc = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, task); - } - } - } - - /** - * Services connection requests. If the data flow management service is unable to provide a current copy of the data flow, then the returned connection response will indicate the node should try - * later. Otherwise, the connection response will contain the the flow and the node identifier. - * - * If this instance is configured with a firewall and the request is blocked, then the response will not contain a node identifier. - * - * @param request a connection request - * - * @return a connection response - */ - @Override - public ConnectionResponse requestConnection(final ConnectionRequest request) { - final boolean lockObtained = writeLock.tryLock(3, TimeUnit.SECONDS); - if (!lockObtained) { - // Create try-later response because we are too busy to service the request right now. We do not want - // to wait long because we want Node/NCM comms to be very responsive - final int tryAgainSeconds; - if (dataFlowManagementService.getRetrievalDelaySeconds() <= 0) { - tryAgainSeconds = DEFAULT_CONNECTION_REQUEST_TRY_AGAIN_SECONDS; - } else { - tryAgainSeconds = dataFlowManagementService.getRetrievalDelaySeconds(); - } - - // record event - final String msg = "Connection requested from node, but manager was too busy to service request. Instructing node to try again in " + tryAgainSeconds + " seconds."; - addEvent(request.getProposedNodeIdentifier(), msg); - addBulletin(request.getProposedNodeIdentifier(), Severity.INFO, msg); - - // return try later response - return new ConnectionResponse(tryAgainSeconds); - } - - try { - // resolve the proposed node identifier to a valid node identifier - final NodeIdentifier resolvedNodeIdentifier; - try { - resolvedNodeIdentifier = resolveProposedNodeIdentifier(request.getProposedNodeIdentifier()); - } catch (final ConflictingNodeIdException e) { - logger.info("Rejecting node {} from connecting to cluster because it provided a Node ID of {} but that Node ID already belongs to {}:{}", - request.getProposedNodeIdentifier().getSocketAddress(), request.getProposedNodeIdentifier().getId(), e.getConflictingNodeAddress(), e.getConflictingNodePort()); - return ConnectionResponse.createConflictingNodeIdResponse(e.getConflictingNodeAddress() + ":" + e.getConflictingNodePort()); - } - - if (isBlockedByFirewall(resolvedNodeIdentifier.getSocketAddress())) { - // if the socket address is not listed in the firewall, then return a null response - logger.info("Firewall blocked connection request from node " + resolvedNodeIdentifier); - return ConnectionResponse.createBlockedByFirewallResponse(); - } - - // get a raw reference to the node (if it doesn't exist, node will be null) - Node node = getRawNode(resolvedNodeIdentifier.getId()); - - // create a new node if necessary and set status to connecting - if (node == null) { - node = new Node(resolvedNodeIdentifier, Status.CONNECTING); - addEvent(node.getNodeId(), "Connection requested from new node. Setting status to connecting."); - nodes.add(node); - } else { - clusterCoordinator.updateNodeStatus(node, Status.CONNECTING); - addEvent(resolvedNodeIdentifier, "Connection requested from existing node. Setting status to connecting"); - } - - // record the time of the connection request - node.setConnectionRequestedTimestamp(new Date().getTime()); - - // try to obtain a current flow - if (dataFlowManagementService.isFlowCurrent()) { - // if a cached copy does not exist, load it from disk - if (cachedDataFlow == null) { - final ClusterDataFlow clusterDataFlow = dataFlowManagementService.loadDataFlow(); - cachedDataFlow = clusterDataFlow.getDataFlow(); - primaryNodeId = clusterDataFlow.getPrimaryNodeId(); - } - - return new ConnectionResponse(node.getNodeId(), cachedDataFlow, remoteInputPort, remoteCommsSecure, instanceId); - } - - /* - * The manager does not have a current copy of the data flow, - * so it will instruct the node to try connecting at a later - * time. Meanwhile, the flow will be locked down from user - * changes because the node is marked as connecting. - */ - - /* - * Create try-later response based on flow retrieval delay to give - * the flow management service a chance to retrieve a curren flow - */ - final int tryAgainSeconds; - if (dataFlowManagementService.getRetrievalDelaySeconds() <= 0) { - tryAgainSeconds = DEFAULT_CONNECTION_REQUEST_TRY_AGAIN_SECONDS; - } else { - tryAgainSeconds = dataFlowManagementService.getRetrievalDelaySeconds(); - } - - // record event - addEvent(node.getNodeId(), "Connection requested from node, but manager was unable to obtain current flow. Instructing node to try again in " + tryAgainSeconds + " seconds."); - - // return try later response - return new ConnectionResponse(tryAgainSeconds); - - } finally { - writeLock.unlock("requestConnection"); - } - } - - /** - * Services reconnection requests for a given node. If the node indicates reconnection failure, then the node will be set to disconnected and if the node has primary role, then the role will be - * revoked. Otherwise, a reconnection request will be sent to the node, initiating the connection handshake. - * - * @param nodeId a node identifier - * - * @throws UnknownNodeException if the node does not exist - * @throws IllegalNodeReconnectionException if the node cannot be reconnected because the node is not disconnected - * @throws NodeReconnectionException if the reconnection message failed to be sent or the cluster could not provide a current data flow for the reconnection request - */ - @Override - public void requestReconnection(final String nodeId, final String userDn) throws UnknownNodeException, IllegalNodeReconnectionException { - Node node = null; - - final boolean primaryRole; - final int tryAgainSeconds; - - writeLock.lock(); - try { - // check if we know about this node and that it is disconnected - node = getRawNode(nodeId); - logger.info("Request was made by {} to reconnect node {} to cluster", userDn, node == null ? nodeId : node); - - if (node == null) { - throw new UnknownNodeException("Node does not exist."); - } else if (Status.DISCONNECTED != node.getStatus()) { - throw new IllegalNodeReconnectionException("Node must be disconnected before it can reconnect."); - } - - // get the dataflow to send with the reconnection request - if (!dataFlowManagementService.isFlowCurrent()) { - /* node remains disconnected */ - final String msg = "Reconnection requested for node, but manager was unable to obtain current flow. Setting node to disconnected."; - addEvent(node.getNodeId(), msg); - addBulletin(node, Severity.WARNING, msg); - throw new NodeReconnectionException("Manager was unable to obtain current flow to provide in reconnection request to node. Try again in a few seconds."); - } - - // if a cached copy does not exist, load it from disk - if (cachedDataFlow == null) { - final ClusterDataFlow clusterDataFlow = dataFlowManagementService.loadDataFlow(); - cachedDataFlow = clusterDataFlow.getDataFlow(); - primaryNodeId = clusterDataFlow.getPrimaryNodeId(); - } - - clusterCoordinator.updateNodeStatus(node, Status.CONNECTING); - addEvent(node.getNodeId(), "Reconnection requested for node. Setting status to connecting."); - - // determine if this node should be assigned the primary role - if (primaryNodeId == null || primaryNodeId.logicallyEquals(node.getNodeId())) { - setPrimaryNodeId(node.getNodeId()); - addEvent(node.getNodeId(), "Setting primary role in reconnection request."); - primaryRole = true; - } else { - primaryRole = false; - } - - if (dataFlowManagementService.getRetrievalDelaySeconds() <= 0) { - tryAgainSeconds = DEFAULT_CONNECTION_REQUEST_TRY_AGAIN_SECONDS; - } else { - tryAgainSeconds = dataFlowManagementService.getRetrievalDelaySeconds(); - } - } catch (final UnknownNodeException | IllegalNodeReconnectionException | NodeReconnectionException une) { - throw une; - } catch (final Exception ex) { - logger.warn("Problem encountered issuing reconnection request to node " + node.getNodeId() + " due to: " + ex, ex); - - clusterCoordinator.updateNodeStatus(node, Status.DISCONNECTED); - final String eventMsg = "Problem encountered issuing reconnection request. Node will remain disconnected: " + ex; - addEvent(node.getNodeId(), eventMsg); - addBulletin(node, Severity.WARNING, eventMsg); - - // Exception thrown will include node ID but event/bulletin do not because the node/id is passed along with the message - throw new NodeReconnectionException("Problem encountered issuing reconnection request to " + node.getNodeId() + ". Node will remain disconnected: " + ex, ex); - } finally { - writeLock.unlock("requestReconnection"); - } - - // Asynchronously start attempting reconnection. This is not completely thread-safe, as - // we do this by releasing the write lock and then obtaining a read lock for each attempt, - // so we suffer from the ABA problem. However, we are willing to accept the consequences of - // this situation in order to avoid holding a lock for the entire duration. "The consequences" - // are that a second thread could potentially be doing the same thing, issuing a reconnection request. - // However, this is very unlikely to happen, based on the conditions under which we issue a reconnection - // request. And if we do, the node will simply reconnect multiple times, which is not a big deal. - requestReconnectionAsynchronously(node, primaryRole, 10, tryAgainSeconds); - } - - private void requestReconnectionAsynchronously(final Node node, final boolean primaryRole, final int reconnectionAttempts, final int retrySeconds) { - final Thread reconnectionThread = new Thread(new Runnable() { - @Override - public void run() { - for (int i = 0; i < reconnectionAttempts; i++) { - final ReconnectionRequestMessage request = new ReconnectionRequestMessage(); - - try { - readLock.lock(); - try { - if (Status.CONNECTING != node.getStatus()) { - // the node status has changed. It's no longer appropriate to attempt reconnection. - return; - } - - // create the request - request.setNodeId(node.getNodeId()); - request.setDataFlow(cachedDataFlow); - request.setPrimary(primaryRole); - request.setManagerRemoteSiteCommsSecure(remoteCommsSecure); - request.setManagerRemoteSiteListeningPort(remoteInputPort); - request.setInstanceId(instanceId); - } finally { - readLock.unlock("Reconnect " + node.getNodeId()); - } - - // Issue a reconnection request to the node. - senderListener.requestReconnection(request); - - node.setConnectionRequestedTimestamp(System.currentTimeMillis()); - - // successfully told node to reconnect -- we're done! - return; - } catch (final Exception e) { - logger.warn("Problem encountered issuing reconnection request to node " + node.getNodeId() + " due to: " + e); - if (logger.isDebugEnabled()) { - logger.warn("", e); - } - - addBulletin(node, Severity.WARNING, "Problem encountered issuing reconnection request to node " + node.getNodeId() + " due to: " + e); - } - - try { - Thread.sleep(1000L * retrySeconds); - } catch (final InterruptedException ie) { - break; - } - } - - // We failed to reconnect 10 times. We must now mark node as disconnected. - writeLock.lock(); - try { - if (Status.CONNECTING == node.getStatus()) { - requestDisconnectionQuietly(node.getNodeId(), "Failed to issue Reconnection Request " + reconnectionAttempts + " times"); - } - } finally { - writeLock.unlock("Mark node as Disconnected as a result of reconnection failure"); - } - } - }, "Reconnect " + node.getNodeId()); - - reconnectionThread.start(); - } - - private Map loadReportingTasks(final byte[] serialized) { - final Map tasks = new HashMap<>(); - - try { - final Document document = parse(serialized); - - final NodeList tasksNodes = document.getElementsByTagName("reportingTasks"); - final Element tasksElement = (Element) tasksNodes.item(0); - - //optional properties for all ReportingTasks - for (final Element taskElement : DomUtils.getChildElementsByTagName(tasksElement, "reportingTask")) { - //add global properties common to all tasks - final Map properties = new HashMap<>(); - - //get properties for the specific reporting task - id, name, class, - //and schedulingPeriod must be set - final String taskId = DomUtils.getChild(taskElement, "id").getTextContent().trim(); - final String taskName = DomUtils.getChild(taskElement, "name").getTextContent().trim(); - - final List schedulingStrategyNodeList = DomUtils.getChildElementsByTagName(taskElement, "schedulingStrategy"); - String schedulingStrategyValue = SchedulingStrategy.TIMER_DRIVEN.name(); - if (schedulingStrategyNodeList.size() == 1) { - final String specifiedValue = schedulingStrategyNodeList.get(0).getTextContent(); - - try { - schedulingStrategyValue = SchedulingStrategy.valueOf(specifiedValue).name(); - } catch (final Exception e) { - throw new RuntimeException("Cannot start Reporting Task with id " + taskId + " because its Scheduling Strategy does not have a valid value", e); - } - } - - final SchedulingStrategy schedulingStrategy = SchedulingStrategy.valueOf(schedulingStrategyValue); - final String taskSchedulingPeriod = DomUtils.getChild(taskElement, "schedulingPeriod").getTextContent().trim(); - final String taskClass = DomUtils.getChild(taskElement, "class").getTextContent().trim(); - - final String scheduleStateValue = DomUtils.getChild(taskElement, "scheduledState").getTextContent().trim(); - final ScheduledState scheduledState = ScheduledState.valueOf(scheduleStateValue); - - // Reporting Task Properties - for (final Element property : DomUtils.getChildElementsByTagName(taskElement, "property")) { - final String name = DomUtils.getChildText(property, "name"); - final String value = DomUtils.getChildText(property, "value"); - properties.put(name, value); - } - - //set the class to be used for the configured reporting task - final ReportingTaskNode reportingTaskNode; - try { - reportingTaskNode = createReportingTask(taskClass, taskId, false); - } catch (final ReportingTaskInstantiationException e) { - logger.error("Unable to load reporting task {} due to {}", new Object[]{taskId, e}); - if (logger.isDebugEnabled()) { - logger.error("", e); - } - continue; - } - - final ReportingTask reportingTask = reportingTaskNode.getReportingTask(); - - final ComponentLog componentLog = new SimpleProcessLogger(taskId, reportingTask); - final ReportingInitializationContext config = new StandardReportingInitializationContext(taskId, taskName, - schedulingStrategy, taskSchedulingPeriod, componentLog, this); - reportingTask.initialize(config); - - final String annotationData = DomUtils.getChildText(taskElement, "annotationData"); - if (annotationData != null) { - reportingTaskNode.setAnnotationData(annotationData.trim()); - } - - final Map resolvedProps; - try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { - resolvedProps = new HashMap<>(); - for (final Map.Entry entry : properties.entrySet()) { - final PropertyDescriptor descriptor = reportingTask.getPropertyDescriptor(entry.getKey()); - if (entry.getValue() == null) { - resolvedProps.put(descriptor, descriptor.getDefaultValue()); - } else { - resolvedProps.put(descriptor, entry.getValue()); - } - } - } - - for (final Map.Entry entry : resolvedProps.entrySet()) { - if (entry.getValue() != null) { - reportingTaskNode.setProperty(entry.getKey().getName(), entry.getValue()); - } - } - - final String comments = DomUtils.getChildText(taskElement, "comment"); - if (comments != null) { - reportingTaskNode.setComments(comments); - } - - reportingTaskNode.setScheduledState(scheduledState); - if (ScheduledState.RUNNING.equals(scheduledState)) { - if (reportingTaskNode.isValid()) { - try { - processScheduler.schedule(reportingTaskNode); - } catch (final Exception e) { - logger.error("Failed to start {} due to {}", reportingTaskNode, e); - if (logger.isDebugEnabled()) { - logger.error("", e); - } - } - } else { - logger.error("Failed to start {} because it is invalid due to {}", reportingTaskNode, reportingTaskNode.getValidationErrors()); - } - } - - tasks.put(reportingTaskNode.getIdentifier(), reportingTaskNode); - } - } catch (final SAXException | ParserConfigurationException | IOException | DOMException | NumberFormatException | InitializationException t) { - logger.error("Unable to load reporting tasks due to {}", new Object[]{t}); - if (logger.isDebugEnabled()) { - logger.error("", t); - } - } - - return tasks; - } - - @Override - public ReportingTaskNode createReportingTask(final String type, final String id, final boolean firstTimeAdded) throws ReportingTaskInstantiationException { - if (type == null) { - throw new NullPointerException(); - } - ReportingTask task = null; - final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader(); - try { - final ClassLoader detectedClassLoader = ExtensionManager.getClassLoader(type); - final Class rawClass; - if (detectedClassLoader == null) { - rawClass = Class.forName(type); - } else { - rawClass = Class.forName(type, false, detectedClassLoader); - } - - Thread.currentThread().setContextClassLoader(detectedClassLoader); - final Class reportingTaskClass = rawClass.asSubclass(ReportingTask.class); - final Object reportingTaskObj = reportingTaskClass.newInstance(); - task = reportingTaskClass.cast(reportingTaskObj); - } catch (final ClassNotFoundException | SecurityException | InstantiationException | IllegalAccessException | IllegalArgumentException t) { - throw new ReportingTaskInstantiationException(type, t); - } finally { - if (ctxClassLoader != null) { - Thread.currentThread().setContextClassLoader(ctxClassLoader); - } - } - - final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this); - final ReportingTaskNode taskNode = new ClusteredReportingTaskNode(task, id, processScheduler, - new ClusteredEventAccess(this, auditService), bulletinRepository, controllerServiceProvider, - validationContextFactory, stateManagerProvider.getStateManager(id)); - taskNode.setName(task.getClass().getSimpleName()); - - reportingTasks.put(id, taskNode); - if (firstTimeAdded) { - try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, task); - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, taskNode.getReportingTask()); - } catch (final Exception e) { - throw new ComponentLifeCycleException("Failed to invoke On-Added Lifecycle methods of " + task, e); - } - } - - // Register log observer to provide bulletins when reporting task logs anything at WARN level or above - final LogRepository logRepository = LogRepositoryFactory.getRepository(id); - logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN, - new ReportingTaskLogObserver(getBulletinRepository(), taskNode)); - - return taskNode; - } - - private Document parse(final byte[] serialized) throws SAXException, ParserConfigurationException, IOException { - final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance(); - docFactory.setNamespaceAware(true); - - final DocumentBuilder builder = docFactory.newDocumentBuilder(); - builder.setErrorHandler(new org.xml.sax.ErrorHandler() { - @Override - public void fatalError(final SAXParseException err) throws SAXException { - logger.error("Config file line " + err.getLineNumber() + ", col " + err.getColumnNumber() + ", uri " + err.getSystemId() + " :message: " + err.getMessage()); - if (logger.isDebugEnabled()) { - logger.error("Error Stack Dump", err); - } - throw err; - } - - @Override - public void error(final SAXParseException err) throws SAXParseException { - logger.error("Config file line " + err.getLineNumber() + ", col " + err.getColumnNumber() + ", uri " + err.getSystemId() + " :message: " + err.getMessage()); - if (logger.isDebugEnabled()) { - logger.error("Error Stack Dump", err); - } - throw err; - } - - @Override - public void warning(final SAXParseException err) throws SAXParseException { - logger.warn(" Config file line " + err.getLineNumber() + ", uri " + err.getSystemId() + " : message : " + err.getMessage()); - if (logger.isDebugEnabled()) { - logger.warn("Warning stack dump", err); - } - throw err; - } - }); - - // build the docuemnt - final Document document = builder.parse(new ByteArrayInputStream(serialized)); - return document; - } - - private void addBulletin(final Node node, final Severity severity, final String msg) { - addBulletin(node.getNodeId(), severity, msg); - } - - private void addBulletin(final NodeIdentifier nodeId, final Severity severity, final String msg) { - bulletinRepository.addBulletin(BulletinFactory.createBulletin(BULLETIN_CATEGORY, severity.toString(), - nodeId.getApiAddress() + ":" + nodeId.getApiPort() + " -- " + msg)); - } - - /** - * Services a disconnection request. - * - * @param nodeId a node identifier - * @param userDn the DN of the user requesting the disconnection - * - * @throws UnknownNodeException if the node does not exist - * @throws IllegalNodeDisconnectionException if the node cannot be disconnected due to the cluster's state (e.g., node is last connected node or node is primary) - * @throws NodeDisconnectionException if the disconnection message fails to be sent. - */ - @Override - public void requestDisconnection(final String nodeId, final String userDn) throws UnknownNodeException, IllegalNodeDisconnectionException, NodeDisconnectionException { - writeLock.lock(); - try { - // check that the node is known - final Node node = getNode(nodeId); - if (node == null) { - throw new UnknownNodeException("Node does not exist."); - } - - clusterCoordinator.requestNodeDisconnect(node.getNodeId(), DisconnectionCode.USER_DISCONNECTED, "User " + userDn + " Disconnected Node"); - } finally { - writeLock.unlock("requestDisconnection(String)"); - } - } - - /** - * Requests a disconnection to the node with the given node ID, but any exception thrown is suppressed. - * - * @param nodeId the node ID - */ - private void requestDisconnectionQuietly(final NodeIdentifier nodeId, final String explanation) { - try { - requestDisconnection(nodeId, /* ignore node check */ true, explanation); - } catch (final IllegalNodeDisconnectionException | NodeDisconnectionException ex) { /* suppress exception */ } - } - - /** - * Issues a disconnection message to the node identified by the given node ID. If the node is not known, then a UnknownNodeException is thrown. If the node cannot be disconnected due to the - * cluster's state and ignoreLastNodeCheck is false, then a IllegalNodeDisconnectionException is thrown. Otherwise, a disconnection message is issued to the node. - * - * Whether the disconnection message is successfully sent to the node, the node is marked as disconnected and if the node is the primary node, then the primary role is revoked. - * - * @param nodeId the ID of the node - * @param ignoreNodeChecks if false, checks will be made to ensure the cluster supports the node's disconnection (e.g., the node is not the last connected node in the cluster; the node is not the - * primary); otherwise, the request is made regardless of the cluster state - * @param explanation - * - * @throws IllegalNodeDisconnectionException if the node cannot be disconnected due to the cluster's state (e.g., node is last connected node or node is primary). Not thrown if ignoreNodeChecks is - * true. - * @throws NodeDisconnectionException if the disconnection message fails to be sent. - */ - void requestDisconnection(final NodeIdentifier nodeId, final boolean ignoreNodeChecks, final String explanation) - throws IllegalNodeDisconnectionException, NodeDisconnectionException { - - writeLock.lock(); - try { - - // check that the node is known - final Node node = getRawNode(nodeId.getId()); - if (node == null) { - if (ignoreNodeChecks) { - // issue the disconnection - final DisconnectMessage request = new DisconnectMessage(); - request.setNodeId(nodeId); - request.setExplanation(explanation); - - addEvent(nodeId, "Disconnection requested due to " + explanation); - senderListener.disconnect(request); - addEvent(nodeId, "Node disconnected due to " + explanation); - addBulletin(nodeId, Severity.INFO, "Node disconnected due to " + explanation); - return; - } else { - throw new UnknownNodeException("Node does not exist"); - } - } - - // if necessary, check that the node may be disconnected - if (!ignoreNodeChecks) { - final Set connectedNodes = getNodeIds(Status.CONNECTED); - // cannot disconnect the last connected node in the cluster - if (connectedNodes.size() == 1 && connectedNodes.iterator().next().equals(nodeId)) { - throw new IllegalNodeDisconnectionException("Node may not be disconnected because it is the only connected node in the cluster."); - } - } - - // update status - clusterCoordinator.updateNodeStatus(node, Status.DISCONNECTED); - notifyDataFlowManagementServiceOfNodeStatusChange(); - - // issue the disconnection - final DisconnectMessage request = new DisconnectMessage(); - request.setNodeId(nodeId); - request.setExplanation(explanation); - - addEvent(nodeId, "Disconnection requested due to " + explanation); - senderListener.disconnect(request); - addEvent(nodeId, "Node disconnected due to " + explanation); - addBulletin(node, Severity.INFO, "Node disconnected due to " + explanation); - - heartbeatMonitor.removeHeartbeat(nodeId); - } finally { - writeLock.unlock("requestDisconnection(NodeIdentifier, boolean)"); - } - } - - - private NodeIdentifier addRequestorDn(final NodeIdentifier nodeId, final String dn) { - return new NodeIdentifier(nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort(), - nodeId.getSocketAddress(), nodeId.getSocketPort(), - nodeId.getSiteToSiteAddress(), nodeId.getSiteToSitePort(), nodeId.isSiteToSiteSecure(), dn); - } - - private ConnectionResponseMessage handleConnectionRequest(final ConnectionRequestMessage requestMessage) { - final NodeIdentifier proposedIdentifier = requestMessage.getConnectionRequest().getProposedNodeIdentifier(); - final ConnectionRequest requestWithDn = new ConnectionRequest(addRequestorDn(proposedIdentifier, requestMessage.getRequestorDN())); - - final ConnectionResponse response = requestConnection(requestWithDn); - final ConnectionResponseMessage responseMessage = new ConnectionResponseMessage(); - responseMessage.setConnectionResponse(response); - return responseMessage; - } - - - @Override - public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) { - final ControllerServiceNode serviceNode = controllerServiceProvider.createControllerService(type, id, firstTimeAdded); - - // Register log observer to provide bulletins when reporting task logs anything at WARN level or above - final LogRepository logRepository = LogRepositoryFactory.getRepository(id); - logRepository.addObserver(StandardProcessorNode.BULLETIN_OBSERVER_ID, LogLevel.WARN, - new ControllerServiceLogObserver(getBulletinRepository(), serviceNode)); - - if (firstTimeAdded) { - final ControllerService service = serviceNode.getControllerServiceImplementation(); - - try (final NarCloseable nc = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, service); - } - } - - return serviceNode; - } - - public StateManagerProvider getStateManagerProvider() { - return stateManagerProvider; - } - - @Override - public ControllerService getControllerService(String serviceIdentifier) { - return controllerServiceProvider.getControllerService(serviceIdentifier); - } - - @Override - public ControllerServiceNode getControllerServiceNode(final String id) { - return controllerServiceProvider.getControllerServiceNode(id); - } - - @Override - public boolean isControllerServiceEnabled(final ControllerService service) { - return controllerServiceProvider.isControllerServiceEnabled(service); - } - - @Override - public boolean isControllerServiceEnabled(final String serviceIdentifier) { - return controllerServiceProvider.isControllerServiceEnabled(serviceIdentifier); - } - - @Override - public boolean isControllerServiceEnabling(final String serviceIdentifier) { - return controllerServiceProvider.isControllerServiceEnabling(serviceIdentifier); - } - - @Override - public String getControllerServiceName(final String serviceIdentifier) { - return controllerServiceProvider.getControllerServiceName(serviceIdentifier); - } - - @Override - public void removeControllerService(final ControllerServiceNode serviceNode) { - controllerServiceProvider.removeControllerService(serviceNode); - } - - @Override - public void enableControllerService(final ControllerServiceNode serviceNode) { - controllerServiceProvider.enableControllerService(serviceNode); - } - - @Override - public void enableControllerServices(final Collection serviceNodes) { - controllerServiceProvider.enableControllerServices(serviceNodes); - } - - @Override - public void disableControllerService(final ControllerServiceNode serviceNode) { - controllerServiceProvider.disableControllerService(serviceNode); - } - - @Override - public Set getAllControllerServices() { - return controllerServiceProvider.getAllControllerServices(); - } - - @Override - public Set disableReferencingServices(final ControllerServiceNode serviceNode) { - return controllerServiceProvider.disableReferencingServices(serviceNode); - } - - @Override - public Set enableReferencingServices(final ControllerServiceNode serviceNode) { - return controllerServiceProvider.enableReferencingServices(serviceNode); - } - - @Override - public Set scheduleReferencingComponents(final ControllerServiceNode serviceNode) { - return controllerServiceProvider.scheduleReferencingComponents(serviceNode); - } - - @Override - public Set unscheduleReferencingComponents(final ControllerServiceNode serviceNode) { - return controllerServiceProvider.unscheduleReferencingComponents(serviceNode); - } - - @Override - public void verifyCanEnableReferencingServices(final ControllerServiceNode serviceNode) { - controllerServiceProvider.verifyCanEnableReferencingServices(serviceNode); - } - - @Override - public void verifyCanScheduleReferencingComponents(final ControllerServiceNode serviceNode) { - controllerServiceProvider.verifyCanScheduleReferencingComponents(serviceNode); - } - - @Override - public void verifyCanDisableReferencingServices(final ControllerServiceNode serviceNode) { - controllerServiceProvider.verifyCanDisableReferencingServices(serviceNode); - } - - @Override - public void verifyCanStopReferencingComponents(final ControllerServiceNode serviceNode) { - controllerServiceProvider.verifyCanStopReferencingComponents(serviceNode); - } - - private byte[] serialize(final Document doc) throws TransformerException { - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - final DOMSource domSource = new DOMSource(doc); - final StreamResult streamResult = new StreamResult(baos); - - // configure the transformer and convert the DOM - final TransformerFactory transformFactory = TransformerFactory.newInstance(); - final Transformer transformer = transformFactory.newTransformer(); - transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2"); - transformer.setOutputProperty(OutputKeys.INDENT, "yes"); - - // transform the document to byte stream - transformer.transform(domSource, streamResult); - return baos.toByteArray(); - } - - - private byte[] serializeReportingTasks() throws ParserConfigurationException, TransformerException { - final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance(); - docFactory.setNamespaceAware(true); - - final DocumentBuilder docBuilder = docFactory.newDocumentBuilder(); - final Document document = docBuilder.newDocument(); - final Element rootElement = document.createElement("reportingTasks"); - document.appendChild(rootElement); - - for (final ReportingTaskNode taskNode : getAllReportingTasks()) { - StandardFlowSerializer.addReportingTask(rootElement, taskNode, encryptor); - } - - return serialize(document); - } - - - public void saveReportingTasks() { - try { - dataFlowManagementService.updateReportingTasks(serializeReportingTasks()); - } catch (final Exception e) { - logger.error("Failed to save changes to NCM's Reporting Tasks; changes may be lost on restart due to " + e); - if (logger.isDebugEnabled()) { - logger.error("", e); - } - - getBulletinRepository().addBulletin(BulletinFactory.createBulletin("Reporting Tasks", Severity.ERROR.name(), - "Failed to save changes to NCM's Reporting Tasks; changes may be lost on restart. See logs for more details.")); - } - } - - @Override - public Set getAllReportingTasks() { - readLock.lock(); - try { - return new HashSet<>(reportingTasks.values()); - } finally { - readLock.unlock("getReportingTasks"); - } - } - - @Override - public ReportingTaskNode getReportingTaskNode(final String taskId) { - readLock.lock(); - try { - return reportingTasks.get(taskId); - } finally { - readLock.unlock("getReportingTaskNode"); - } - } - - @Override - public void startReportingTask(final ReportingTaskNode reportingTaskNode) { - reportingTaskNode.verifyCanStart(); - processScheduler.schedule(reportingTaskNode); - } - - @Override - public void stopReportingTask(final ReportingTaskNode reportingTaskNode) { - reportingTaskNode.verifyCanStop(); - processScheduler.unschedule(reportingTaskNode); - } - - @Override - public void removeReportingTask(final ReportingTaskNode reportingTaskNode) { - writeLock.lock(); - try { - final ReportingTaskNode existing = reportingTasks.get(reportingTaskNode.getIdentifier()); - if (existing == null || existing != reportingTaskNode) { - throw new IllegalStateException("Reporting Task " + reportingTaskNode + " does not exist in this Flow"); - } - - reportingTaskNode.verifyCanDelete(); - - try (final NarCloseable x = NarCloseable.withNarLoader()) { - ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, reportingTaskNode.getReportingTask(), reportingTaskNode.getConfigurationContext()); - } - - for (final Map.Entry entry : reportingTaskNode.getProperties().entrySet()) { - final PropertyDescriptor descriptor = entry.getKey(); - if (descriptor.getControllerServiceDefinition() != null) { - final String value = entry.getValue() == null ? descriptor.getDefaultValue() : entry.getValue(); - if (value != null) { - final ControllerServiceNode serviceNode = controllerServiceProvider.getControllerServiceNode(value); - if (serviceNode != null) { - serviceNode.removeReference(reportingTaskNode); - } - } - } - } - - reportingTasks.remove(reportingTaskNode.getIdentifier()); - } finally { - writeLock.unlock("removeReportingTask"); - } - } - - @Override - public void disableReportingTask(final ReportingTaskNode reportingTask) { - reportingTask.verifyCanDisable(); - processScheduler.disableReportingTask(reportingTask); - } - - @Override - public void enableReportingTask(final ReportingTaskNode reportingTask) { - reportingTask.verifyCanEnable(); - processScheduler.enableReportingTask(reportingTask); - } - - - @Override - public Set getNodes(final Status... statuses) { - final Set desiredStatusSet = new HashSet<>(); - for (final Status status : statuses) { - desiredStatusSet.add(status); - } - - readLock.lock(); - try { - final Set clonedNodes = new HashSet<>(); - for (final Node node : nodes) { - if (desiredStatusSet.isEmpty() || desiredStatusSet.contains(node.getStatus())) { - clonedNodes.add(node.clone()); - } - } - return Collections.unmodifiableSet(clonedNodes); - } finally { - readLock.unlock("getNodes(Status...)"); - } - } - - @Override - public Node getNode(final String nodeId) { - readLock.lock(); - try { - for (final Node node : nodes) { - if (node.getNodeId().getId().equals(nodeId)) { - return node.clone(); - } - } - return null; - } finally { - readLock.unlock("getNode(String)"); - } - } - - @Override - public Node getPrimaryNode() { - readLock.lock(); - try { - if (primaryNodeId == null) { - return null; - } else { - return getNode(primaryNodeId.getId()); - } - } finally { - readLock.unlock("getPrimaryNode"); - } - } - - @Override - public void deleteNode(final String nodeId, final String userDn) throws UnknownNodeException, IllegalNodeDeletionException { - writeLock.lock(); - try { - final Node node = getNode(nodeId); - if (node == null) { - throw new UnknownNodeException("Node does not exist."); - } else if (Status.DISCONNECTED == node.getStatus()) { - nodes.remove(node); - - if (eventManager != null) { - eventManager.clearEventHistory(node.getNodeId().getId()); - } - - logger.info("Removing node {} from cluster because this action was requested by {}", node, userDn); - } else { - throw new IllegalNodeDeletionException("Node may not be deleted because it is not disconnected."); - } - } finally { - writeLock.unlock("deleteNode"); - } - } - - @Override - public Set getNodeIds(final Status... statuses) { - readLock.lock(); - try { - final Set nodeIds = new HashSet<>(); - for (final Node node : nodes) { - if (statuses == null || statuses.length == 0) { - nodeIds.add(node.getNodeId()); - } else { - for (final Node.Status status : statuses) { - if (node.getStatus() == status) { - nodeIds.add(node.getNodeId()); - break; - } - } - } - } - return nodeIds; - } finally { - readLock.unlock("getNodeIds(Status...)"); - } - } - - private int getClusterProtocolHeartbeatSeconds() { - return (int) FormatUtils.getTimeDuration(properties.getClusterProtocolHeartbeatInterval(), TimeUnit.SECONDS); - } - - @Override - public int getHeartbeatMonitoringIntervalSeconds() { - return 4 * getClusterProtocolHeartbeatSeconds(); - } - - @Override - public int getMaxHeartbeatGapSeconds() { - return 8 * getClusterProtocolHeartbeatSeconds(); - } - - @Override - public List getNodeEvents(final String nodeId) { - readLock.lock(); - try { - List events = null; - final EventManager eventMgr = eventManager; - if (eventMgr != null) { - events = eventMgr.getEvents(nodeId); - } - - if (events == null) { - return Collections.emptyList(); - } else { - return Collections.unmodifiableList(events); - } - } finally { - readLock.unlock("getNodeEvents"); - } - } - - @Override - public NodeResponse applyRequest(final String method, final URI uri, final Map> parameters, final Map headers) - throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, - ConnectingNodeMutableRequestException, DisconnectedNodeMutableRequestException, SafeModeMutableRequestException { - return applyRequest(method, uri, parameters, headers, getNodeIds(Status.CONNECTED)); - } - - @Override - public NodeResponse applyRequest(final String method, final URI uri, final Map> parameters, final Map headers, final Set nodeIdentifiers) - throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, - ConnectingNodeMutableRequestException, DisconnectedNodeMutableRequestException, SafeModeMutableRequestException { - - final boolean mutableRequest = canChangeNodeState(method, uri); - final ClusterManagerLock lock = mutableRequest ? writeLock : readLock; - - lock.lock(); - try { - final NodeResponse clientResponse = federateRequest(method, uri, parameters, null, headers, nodeIdentifiers); - if (clientResponse == null) { - if (mutableRequest) { - throw new NoConnectedNodesException(String.format("All nodes were disconnected as a result of applying request %s %s", method, uri)); - } else { - throw new NoResponseFromNodesException("No nodes were able to process this request."); - } - } else { - return clientResponse; - } - } finally { - lock.unlock("applyRequest(String, URI, Map>, Map, Set"); - } - } - - @Override - public NodeResponse applyRequest(final String method, final URI uri, final Object entity, final Map headers) - throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, - ConnectingNodeMutableRequestException, DisconnectedNodeMutableRequestException, SafeModeMutableRequestException { - return applyRequest(method, uri, entity, headers, getNodeIds(Status.CONNECTED)); - } - - @Override - public NodeResponse applyRequest(final String method, final URI uri, final Object entity, final Map headers, final Set nodeIdentifiers) - throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, - ConnectingNodeMutableRequestException, DisconnectedNodeMutableRequestException, SafeModeMutableRequestException { - - final boolean mutableRequest = canChangeNodeState(method, uri); - final ClusterManagerLock lock = mutableRequest ? writeLock : readLock; - - lock.lock(); - try { - // check that the request can be applied - if (mutableRequest) { - if (isInSafeMode()) { - throw new SafeModeMutableRequestException("Received a mutable request [" + method + " -- " + uri + "] while in safe mode"); - } else if (!getNodeIds(Status.DISCONNECTED, Status.DISCONNECTING).isEmpty()) { - throw new DisconnectedNodeMutableRequestException("Received a mutable request [" + method + " -- " + uri + "] while a node is disconnected from the cluster"); - } else if (!getNodeIds(Status.CONNECTING).isEmpty()) { - // if any node is connecting and a request can change the flow, then we throw an exception - throw new ConnectingNodeMutableRequestException("Received a mutable request [" + method + " -- " + uri + "] while a node is trying to connect to the cluster"); - } - } - - final NodeResponse clientResponse = federateRequest(method, uri, null, entity, headers, nodeIdentifiers); - if (clientResponse == null) { - if (mutableRequest) { - throw new NoConnectedNodesException(String.format("All nodes were disconnected as a result of applying request %s %s", method, uri)); - } else { - throw new NoResponseFromNodesException("No nodes were able to process this request."); - } - } else { - return clientResponse; - } - - } finally { - lock.unlock("applyRequest(String, URI, Object, Map, Set"); - } - } - - public void setServicesBroadcaster(final ClusterServicesBroadcaster servicesBroadcaster) { - writeLock.lock(); - try { - this.servicesBroadcaster = servicesBroadcaster; - } finally { - writeLock.unlock("setServicesBroadcaster"); - } - } - - public boolean addBroadcastedService(final DiscoverableService service) { - writeLock.lock(); - try { - final ClusterServicesBroadcaster broadcaster = this.servicesBroadcaster; - if (broadcaster == null) { - throw new IllegalStateException("Service broadcasting is not configured."); - } - return broadcaster.addService(service); - } finally { - writeLock.unlock("addBroadcastedService"); - } - } - - public boolean removeBroadcastedService(final String serviceName) { - writeLock.lock(); - try { - final ClusterServicesBroadcaster broadcaster = this.servicesBroadcaster; - if (broadcaster == null) { - throw new IllegalStateException("Service broadcasting is not configured."); - } - return broadcaster.removeService(serviceName); - } finally { - writeLock.unlock("removeBroadcastedService"); - } - } - - public boolean isBroadcastingConfigured() { - readLock.lock(); - try { - return servicesBroadcaster != null; - } finally { - readLock.unlock("isBroadcastingConfigured"); - } - } - - public boolean isBroadcasting() { - readLock.lock(); - try { - final ClusterServicesBroadcaster broadcaster = this.servicesBroadcaster; - return broadcaster != null && broadcaster.isRunning(); - } finally { - readLock.unlock("isBroadcasting"); - } - } - - public void addEvent(final NodeIdentifier nodeId, String eventMsg) { - writeLock.lock(); - try { - final Event event = new Event(nodeId.getId(), eventMsg); - final EventManager eventMgr = eventManager; - if (eventMgr != null) { - eventMgr.addEvent(event); - } - logger.info(String.format("Node Event: %s -- '%s'", nodeId, eventMsg)); - } finally { - writeLock.unlock("addEvent"); - } - } - - public void setEventManager(final EventManager eventManager) { - writeLock.lock(); - try { - this.eventManager = eventManager; - } finally { - writeLock.unlock("setEventManager"); - } - } - - public void setClusterFirewall(final ClusterNodeFirewall clusterFirewall) { - writeLock.lock(); - try { - this.clusterFirewall = clusterFirewall; - } finally { - writeLock.unlock("setClusterFirewall"); - } - } - - public boolean isFirewallConfigured() { - readLock.lock(); - try { - return clusterFirewall != null; - } finally { - readLock.unlock("isFirewallConfigured"); - } - } - - public void setAuditService(final AuditService auditService) { - writeLock.lock(); - try { - this.auditService = auditService; - } finally { - writeLock.unlock("setAuditService"); - } - } - - public boolean isAuditingConfigured() { - readLock.lock(); - try { - return auditService != null; - } finally { - readLock.unlock("isAuditingConfigured"); - } - } - - private boolean isInSafeMode() { - readLock.lock(); - try { - return primaryNodeId == null || getRawNode(primaryNodeId.getId()) == null; - } finally { - readLock.unlock("isInSafeMode"); - } - } - - void setPrimaryNodeId(final NodeIdentifier primaryNodeId) throws DaoException { - writeLock.lock(); - try { - dataFlowManagementService.updatePrimaryNode(primaryNodeId); - - // update the cached copy reference to minimize loading file from disk - this.primaryNodeId = primaryNodeId; - } finally { - writeLock.unlock("setPrimaryNodeId"); - } - } - - // requires write lock to already be acquired unless method cannot change node state - private NodeResponse federateRequest( - final String method, final URI uri, final Map> parameters, final Object entity, final Map headers, final Set nodeIds) - throws UriConstructionException { - // ensure some nodes are connected - if (nodeIds.isEmpty()) { - throw new NoConnectedNodesException("Cannot apply " + method + " request to " + uri + " because there are currently no connected Nodes"); - } - - logger.debug("Applying prototype request " + uri + " to nodes."); - - // replicate request - final AsyncClusterResponse clusterResponse = httpRequestReplicator.replicate(nodeIds, method, uri, entity == null ? parameters : entity, headers); - - final NodeResponse clientResponse; - try { - clientResponse = clusterResponse.awaitMergedResponse(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - logger.warn("Thread was interrupted while waiting for a response from one or more nodes", e); - final Set noResponses = clusterResponse.getNodesInvolved(); - noResponses.removeAll(clusterResponse.getCompletedNodeIdentifiers()); - throw new IllegalClusterStateException("Interrupted while waiting for a response from the following nodes: " + noResponses, e); - } - - return clientResponse; - } - - - private static boolean isCounterEndpoint(final String uriPath) { - return COUNTER_URI_PATTERN.matcher(uriPath).matches(); - } - - - public List mergeNCMBulletins(final List nodeBulletins, final List ncmBulletins) { - if (ncmBulletins == null || ncmBulletins.isEmpty()) { - return nodeBulletins; - } - - final List mergedBulletins = new ArrayList<>(nodeBulletins.size() + ncmBulletins.size()); - mergedBulletins.addAll(nodeBulletins); - mergedBulletins.addAll(createBulletinDtos(ncmBulletins)); - return mergedBulletins; - } - - - /** - * Creates BulletinDTOs for the specified Bulletins. - * - * @param bulletins bulletin - * @return dto - */ - public List createBulletinDtos(final List bulletins) { - final List bulletinDtos = new ArrayList<>(bulletins.size()); - for (final Bulletin bulletin : bulletins) { - bulletinDtos.add(createBulletinDto(bulletin)); - } - return bulletinDtos; - } - - /** - * Creates a BulletinDTO for the specified Bulletin. - * - * @param bulletin bulletin - * @return dto - */ - public BulletinDTO createBulletinDto(final Bulletin bulletin) { - final BulletinDTO dto = new BulletinDTO(); - dto.setId(bulletin.getId()); - dto.setNodeAddress(bulletin.getNodeAddress()); - dto.setTimestamp(bulletin.getTimestamp()); - dto.setGroupId(bulletin.getGroupId()); - dto.setSourceId(bulletin.getSourceId()); - dto.setSourceName(bulletin.getSourceName()); - dto.setCategory(bulletin.getCategory()); - dto.setLevel(bulletin.getLevel()); - dto.setMessage(bulletin.getMessage()); - return dto; - } - - - /** - * Merges the validation errors into the specified map, recording the corresponding node identifier. - * - * @param validationErrorMap map - * @param nodeId id - * @param nodeValidationErrors errors - */ - public void mergeValidationErrors(final Map> validationErrorMap, final NodeIdentifier nodeId, final Collection nodeValidationErrors) { - if (nodeValidationErrors != null) { - for (final String nodeValidationError : nodeValidationErrors) { - Set nodeSet = validationErrorMap.get(nodeValidationError); - if (nodeSet == null) { - nodeSet = new HashSet<>(); - validationErrorMap.put(nodeValidationError, nodeSet); - } - nodeSet.add(nodeId); - } - } - } - - /** - * Normalizes the validation errors by prepending the corresponding nodes when the error does not exist across all nodes. - * - * @param validationErrorMap map - * @param totalNodes total - * @return normalized errors - */ - public Set normalizedMergedValidationErrors(final Map> validationErrorMap, int totalNodes) { - final Set normalizedValidationErrors = new HashSet<>(); - for (final Map.Entry> validationEntry : validationErrorMap.entrySet()) { - final String msg = validationEntry.getKey(); - final Set nodeIds = validationEntry.getValue(); - - if (nodeIds.size() == totalNodes) { - normalizedValidationErrors.add(msg); - } else { - for (final NodeIdentifier nodeId : nodeIds) { - normalizedValidationErrors.add(nodeId.getApiAddress() + ":" + nodeId.getApiPort() + " -- " + msg); - } - } - } - return normalizedValidationErrors; - } - - - // TODO: This is temporary. Only here because while we have NCM, we must merge its bulletins. Once we get rid - // of the NCM, this goes away completely. - public static final Pattern CONTROLLER_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/controller/status"); - - public boolean isControllerStatusEndpoint(final URI uri, final String method) { - return "GET".equalsIgnoreCase(method) && CONTROLLER_STATUS_URI_PATTERN.matcher(uri.getPath()).matches(); - } - - - @Override - public void afterRequest(final String uriPath, final String method, final Set nodeResponses) { - final boolean mutableRequest = canChangeNodeState(method, null); - - /* - * Nodes that encountered issues handling the request are marked as - * disconnected for mutable requests (e.g., post, put, delete). For - * other requests (e.g., get, head), the nodes remain in their current - * state even if they had problems handling the request. - */ - if (mutableRequest) { - final Set problematicNodeResponses = responseMerger.getProblematicNodeResponses(nodeResponses); - - // all nodes failed - final boolean allNodesFailed = problematicNodeResponses.size() == nodeResponses.size(); - - // some nodes had a problematic response because of a missing counter, ensure the are not disconnected - final boolean someNodesFailedMissingCounter = !problematicNodeResponses.isEmpty() - && problematicNodeResponses.size() < nodeResponses.size() && isMissingCounter(problematicNodeResponses, uriPath); - - // ensure nodes stay connected in certain scenarios - if (allNodesFailed || someNodesFailedMissingCounter) { - for (final NodeResponse nodeResponse : nodeResponses) { - final Node node = getRawNode(nodeResponse.getNodeId().getId()); - - if (problematicNodeResponses.contains(nodeResponse)) { - node.setStatus(Status.CONNECTED); - problematicNodeResponses.remove(nodeResponse); - } - } - } - - // disconnect problematic nodes - if (!problematicNodeResponses.isEmpty()) { - if (problematicNodeResponses.size() < nodeResponses.size()) { - logger.warn(String.format("The following nodes failed to process URI %s '%s'. Requesting each node to disconnect from cluster: ", uriPath, problematicNodeResponses)); - disconnectNodes(problematicNodeResponses, "Failed to process URI " + method + " " + uriPath); - } else { - logger.warn("All nodes failed to process URI {} {}. As a result, no node will be disconnected from cluster", method, uriPath); - } - } - } - } - - - /** - * Determines if all problematic responses were due to 404 NOT_FOUND. Assumes that problematicNodeResponses is not empty and is not comprised of responses from all nodes in the cluster (at least - * one node contained the counter in question). - * - * @param problematicNodeResponses The problematic node responses - * @param uriPath The path of the URI for the request - * @return Whether all problematic node responses were due to a missing counter - */ - private boolean isMissingCounter(final Set problematicNodeResponses, final String uriPath) { - if (isCounterEndpoint(uriPath)) { - boolean notFound = true; - for (final NodeResponse problematicResponse : problematicNodeResponses) { - if (problematicResponse.getStatus() != 404) { - notFound = false; - break; - } - } - return notFound; - } - return false; - } - - - /** - * A helper method to disconnect nodes that returned unsuccessful HTTP responses because of a replicated request. Disconnection requests are sent concurrently. - * - */ - private void disconnectNodes(final Set nodeResponses, final String explanation) { - // return fast if nothing to do - if (nodeResponses == null || nodeResponses.isEmpty()) { - return; - } - - final ExecutorService executorService = Executors.newFixedThreadPool(properties.getClusterManagerProtocolThreads()); - final CompletionService completionService = new ExecutorCompletionService<>(executorService); - for (final NodeResponse nodeResponse : nodeResponses) { - completionService.submit(new Runnable() { - @Override - public void run() { - final NodeIdentifier nodeId = nodeResponse.getNodeId(); - final int responseStatus = nodeResponse.getStatus(); - final URI requestUri = nodeResponse.getRequestUri(); - final StringBuilder msgBuilder = new StringBuilder(); - msgBuilder - .append("Requesting disconnection for node ") - .append(nodeId) - .append(" for request URI ") - .append(requestUri); - if (nodeResponse.hasThrowable()) { - msgBuilder.append(" because manager encountered exception when issuing request: ") - .append(nodeResponse.getThrowable()); - // log stack trace anytime we have a throwable - ((NiFiLog) logger).getWrappedLog().info(msgBuilder.toString(), nodeResponse.getThrowable()); - addEvent(nodeId, "Manager encountered exception when issuing request for URI " + requestUri); - addBulletin(nodeId, Severity.ERROR, "Manager encountered exception when issuing request for URI " + requestUri + "; node will be disconnected"); - } else { - msgBuilder.append(" because HTTP response status was ") - .append(responseStatus); - logger.info(msgBuilder.toString()); - addEvent(nodeId, "HTTP response status was unsuccessful (" + responseStatus + ") for request URI " + requestUri); - addBulletin(nodeId, Severity.ERROR, "HTTP response status was unsuccessful (" + responseStatus + ") for request URI " + requestUri); - } - requestDisconnectionQuietly(nodeId, explanation); - } - }, null); - } - - executorService.shutdown(); - } - - /** - * Returns false if an internal protocol message was received by a node listed in the firewall. If no firewall is configured, then false is always returned. - * - * @param ip the IP of the remote machine - * - * @return false if the IP is listed in the firewall or if the firewall is not configured; true otherwise - */ - boolean isBlockedByFirewall(final String ip) { - if (isFirewallConfigured()) { - return !clusterFirewall.isPermissible(ip); - } else { - return false; - } - } - - private Set getRawNodes(final Status... statuses) { - readLock.lock(); - try { - final Set result = new HashSet<>(); - if (statuses == null || statuses.length == 0) { - result.addAll(nodes); - } else { - for (final Node node : nodes) { - for (final Node.Status status : statuses) { - if (node.getStatus() == status) { - result.add(node); - break; - } - } - } - } - return result; - } finally { - readLock.unlock("getRawNodes(Status...)"); - } - } - - Node getRawNode(final String nodeId) { - readLock.lock(); - try { - for (final Node node : nodes) { - if (node.getNodeId().getId().equals(nodeId)) { - return node; - } - } - return null; - } finally { - readLock.unlock("getRawNode(String)"); - } - } - - /** - * Resolves a proposed node identifier to a node identifier that the manager approves. If the proposed node identifier conflicts with an existing node identifier, then an approved node identifier - * is generated and returned to the caller. - * - * @param proposedNodeId a proposed identifier - * - * @return the node identifier that should be used - */ - private NodeIdentifier resolveProposedNodeIdentifier(final NodeIdentifier proposedNodeId) throws ConflictingNodeIdException { - readLock.lock(); - try { - for (final Node node : nodes) { - final NodeIdentifier nodeId = node.getNodeId(); - - // are the ids the same - final boolean sameId = nodeId.equals(proposedNodeId); - - // are the service coordinates the same - final boolean sameServiceCoordinates = nodeId.logicallyEquals(proposedNodeId); - - if (sameId && sameServiceCoordinates) { - // we know about this node and it has the same ID, so the proposal is fine - return proposedNodeId; - } else if (sameId && !sameServiceCoordinates) { - throw new ConflictingNodeIdException(nodeId.getId(), node.getNodeId().getApiAddress(), node.getNodeId().getApiPort()); - } else if (!sameId && sameServiceCoordinates) { - // we know about this node, so we'll use the existing ID - logger.debug(String.format("Using Node Identifier %s because proposed node identifier %s matches the service coordinates", nodeId, proposedNodeId)); - - // return a new Node Identifier that uses the existing Node UUID, Node Index, and ZooKeeper Port from the existing Node (because these are the - // elements that are assigned by the NCM), but use the other parameters from the proposed identifier, since these elements are determined by - // the node rather than the NCM. - return new NodeIdentifier(nodeId.getId(), - proposedNodeId.getApiAddress(), proposedNodeId.getApiPort(), - proposedNodeId.getSocketAddress(), proposedNodeId.getSocketPort(), - proposedNodeId.getSiteToSiteAddress(), proposedNodeId.getSiteToSitePort(), proposedNodeId.isSiteToSiteSecure()); - } - - } - - // proposal does not conflict with existing nodes - this is a new node. Assign a new Node Index to it - return new NodeIdentifier(proposedNodeId.getId(), proposedNodeId.getApiAddress(), proposedNodeId.getApiPort(), - proposedNodeId.getSocketAddress(), proposedNodeId.getSocketPort(), - proposedNodeId.getSiteToSiteAddress(), proposedNodeId.getSiteToSitePort(), proposedNodeId.isSiteToSiteSecure()); - } finally { - readLock.unlock("resolveProposedNodeIdentifier"); - } - } - - private boolean canChangeNodeState(final String method, final URI uri) { - return HttpMethod.DELETE.equalsIgnoreCase(method) || HttpMethod.POST.equalsIgnoreCase(method) || HttpMethod.PUT.equalsIgnoreCase(method); - } - - private void notifyDataFlowManagementServiceOfNodeStatusChange() { - writeLock.lock(); - try { - // tell service about the currently connected nodes - logger.debug("Notifying DataFlow Management Service of current set of connected nodes."); - dataFlowManagementService.setNodeIds(getNodeIds(Status.CONNECTED)); - } finally { - writeLock.unlock("notifyDataFlowManagementServiceOfNodeStatusChange"); - } - } - - public NodeHeartbeat getLatestHeartbeat(final NodeIdentifier nodeId) { - return heartbeatMonitor.getLatestHeartbeat(nodeId); - } - - @Override - public ClusterNodeInformation getNodeInformation() { - readLock.lock(); - try { - final Collection nodeInfos = new ArrayList<>(); - for (final Node node : getRawNodes(Status.CONNECTED)) { - final NodeIdentifier id = node.getNodeId(); - - final NodeHeartbeat nodeHeartbeat = heartbeatMonitor.getLatestHeartbeat(id); - if (nodeHeartbeat == null) { - continue; - } - - final Integer siteToSitePort = id.getSiteToSitePort(); - if (siteToSitePort == null) { - continue; - } - - final int flowFileCount = nodeHeartbeat.getFlowFileCount(); - final NodeInformation nodeInfo = new NodeInformation(id.getSiteToSiteAddress(), siteToSitePort, id.getApiPort(), - id.isSiteToSiteSecure(), flowFileCount); - nodeInfos.add(nodeInfo); - } - - final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation(); - clusterNodeInfo.setNodeInformation(nodeInfos); - return clusterNodeInfo; - } finally { - readLock.unlock("getNodeInformation"); - } - } - - @Override - public BulletinRepository getBulletinRepository() { - return bulletinRepository; - } - - - - private static class ClusterManagerLock { - - private final Lock lock; - private static final Logger logger = LoggerFactory.getLogger("cluster.lock"); - private long lockTime; - private final String name; - - public ClusterManagerLock(final Lock lock, final String name) { - this.lock = lock; - this.name = name; - } - - @SuppressWarnings("unused") - public boolean tryLock() { - logger.trace("Trying to obtain Cluster Manager Lock: {}", name); - final boolean success = lock.tryLock(); - if (!success) { - logger.trace("TryLock failed for Cluster Manager Lock: {}", name); - return false; - } - logger.trace("TryLock successful"); - return true; - } - - public boolean tryLock(final long timeout, final TimeUnit timeUnit) { - logger.trace("Trying to obtain Cluster Manager Lock {} with a timeout of {} {}", name, timeout, timeUnit); - final boolean success; - try { - success = lock.tryLock(timeout, timeUnit); - } catch (final InterruptedException ie) { - return false; - } - - if (!success) { - logger.trace("TryLock failed for Cluster Manager Lock {} with a timeout of {} {}", name, timeout, timeUnit); - return false; - } - logger.trace("TryLock successful"); - return true; - } - - public void lock() { - logger.trace("Obtaining Cluster Manager Lock {}", name); - lock.lock(); - lockTime = System.nanoTime(); - logger.trace("Obtained Cluster Manager Lock {}", name); - } - - public void unlock(final String task) { - logger.trace("Releasing Cluster Manager Lock {}", name); - final long nanosLocked = System.nanoTime() - lockTime; - lock.unlock(); - logger.trace("Released Cluster Manager Lock {}", name); - - final long millisLocked = TimeUnit.MILLISECONDS.convert(nanosLocked, TimeUnit.NANOSECONDS); - if (millisLocked > 100L) { - logger.debug("Cluster Manager Lock {} held for {} milliseconds for task: {}", name, millisLocked, task); - } - } - } - - @Override - public Set getControllerServiceIdentifiers(final Class serviceType, String groupId) { - return controllerServiceProvider.getControllerServiceIdentifiers(serviceType, groupId); - } - - public void reportEvent(final NodeIdentifier nodeId, final Severity severity, final String message) { - bulletinRepository.addBulletin(BulletinFactory.createBulletin(nodeId == null ? "Cluster" : nodeId.getId(), severity.name(), message)); - if (nodeId != null) { - addEvent(nodeId, message); - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManagerCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManagerCoordinator.java deleted file mode 100644 index 81e3e4c80a5f..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManagerCoordinator.java +++ /dev/null @@ -1,264 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.nifi.cluster.manager.impl; - -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; - -import org.apache.nifi.cluster.coordination.ClusterCoordinator; -import org.apache.nifi.cluster.coordination.node.DisconnectionCode; -import org.apache.nifi.cluster.coordination.node.NodeConnectionState; -import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; -import org.apache.nifi.cluster.flow.DataFlowManagementService; -import org.apache.nifi.cluster.node.Node; -import org.apache.nifi.cluster.node.Node.Status; -import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender; -import org.apache.nifi.cluster.protocol.ConnectionRequest; -import org.apache.nifi.cluster.protocol.NodeIdentifier; -import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage; -import org.apache.nifi.reporting.Severity; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class WebClusterManagerCoordinator implements ClusterCoordinator { - private static final Logger logger = LoggerFactory.getLogger(WebClusterManagerCoordinator.class); - private static final AtomicLong nodeStatusIdGenerator = new AtomicLong(0L); - - private final WebClusterManager clusterManager; - private final ClusterManagerProtocolSender protocolSender; - private final DataFlowManagementService dfmService; - - public WebClusterManagerCoordinator(final WebClusterManager clusterManager, final ClusterManagerProtocolSender protocolSender, final DataFlowManagementService dfmService) { - this.clusterManager = clusterManager; - this.protocolSender = protocolSender; - this.dfmService = dfmService; - } - - @Override - public void requestNodeConnect(final NodeIdentifier nodeId) { - final Node node = clusterManager.getRawNode(nodeId.getId()); - - if (node == null) { - final ConnectionRequest connectionRequest = new ConnectionRequest(nodeId); - clusterManager.requestConnection(connectionRequest); - } else { - updateNodeStatus(nodeId, new NodeConnectionStatus(DisconnectionCode.NOT_YET_CONNECTED, "Requesting that Node Connect to the Cluster")); - clusterManager.requestReconnection(nodeId.getId(), "Anonymous"); - } - } - - @Override - public void finishNodeConnection(final NodeIdentifier nodeId) { - final boolean updated = updateNodeStatus(nodeId, new NodeConnectionStatus(NodeConnectionState.CONNECTED)); - if (!updated) { - logger.error("Attempting to Finish Node Connection but could not find Node with Identifier {}", nodeId); - } - } - - @Override - public void requestNodeDisconnect(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) { - try { - clusterManager.requestDisconnection(nodeId, false, explanation); - - if (disconnectionCode == DisconnectionCode.LACK_OF_HEARTBEAT) { - final Node node = clusterManager.getRawNode(nodeId.getId()); - if (node != null) { - updateNodeStatus(node, Status.DISCONNECTED, true); - } - } - } catch (final Exception e) { - logger.error("Failed to request node {} disconnect from cluster due to {}", nodeId, e); - logger.error("", e); - } - } - - @Override - public void disconnectionRequestedByNode(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) { - updateNodeStatus(nodeId, new NodeConnectionStatus(disconnectionCode, explanation)); - - final Severity severity; - switch (disconnectionCode) { - case STARTUP_FAILURE: - case MISMATCHED_FLOWS: - case UNKNOWN: - severity = Severity.ERROR; - break; - default: - severity = Severity.INFO; - break; - } - - reportEvent(nodeId, severity, "Node disconnected from cluster due to " + explanation); - } - - @Override - public NodeConnectionStatus getConnectionStatus(final NodeIdentifier nodeId) { - final Node node = clusterManager.getNode(nodeId.getId()); - if (node == null) { - return null; - } - - final Status status = node.getStatus(); - final NodeConnectionState connectionState = NodeConnectionState.valueOf(status.name()); - return new NodeConnectionStatus(connectionState, node.getConnectionRequestedTimestamp()); - } - - @Override - public Set getNodeIdentifiers(final NodeConnectionState state) { - final Status status = Status.valueOf(state.name()); - final Set nodes = clusterManager.getNodes(status); - return nodes.stream() - .map(node -> node.getNodeId()) - .collect(Collectors.toSet()); - } - - @Override - public boolean isBlockedByFirewall(final String hostname) { - return clusterManager.isBlockedByFirewall(hostname); - } - - @Override - public void reportEvent(final NodeIdentifier nodeId, final Severity severity, final String event) { - final String messagePrefix = nodeId == null ? "" : nodeId.getApiAddress() + ":" + nodeId.getApiPort() + " -- "; - switch (severity) { - case INFO: - logger.info(messagePrefix + event); - break; - case WARNING: - logger.warn(messagePrefix + event); - break; - case ERROR: - logger.error(messagePrefix + event); - break; - } - - clusterManager.reportEvent(nodeId, severity, messagePrefix + event); - } - - @Override - public void setPrimaryNode(final NodeIdentifier nodeId) { - clusterManager.setPrimaryNodeId(nodeId); - } - - @Override - public NodeIdentifier getNodeIdentifier(final String uuid) { - final Node node = clusterManager.getNode(uuid); - return node == null ? null : node.getNodeId(); - } - - - /** - * Updates the status of the node with the given ID to the given status and returns true - * if successful, false if no node exists with the given ID - * - * @param nodeId the ID of the node whose status is changed - * @param status the new status of the node - * @return true if the node exists and is updated, false if the node does not exist - */ - private boolean updateNodeStatus(final NodeIdentifier nodeId, final NodeConnectionStatus status) { - final long statusUpdateId = nodeStatusIdGenerator.incrementAndGet(); - - final Node node = clusterManager.getRawNode(nodeId.getId()); - if (node == null) { - return false; - } - - final Status nodeStatus = Status.valueOf(status.getState().name()); - final Status oldStatus = node.setStatus(nodeStatus); - - if (nodeStatus != oldStatus) { - final Set nodesToNotify = clusterManager.getNodes(Status.CONNECTED, Status.CONNECTING).stream() - .map(toNotify -> toNotify.getNodeId()) - .collect(Collectors.toSet()); - - final NodeStatusChangeMessage message = new NodeStatusChangeMessage(); - message.setNodeId(nodeId); - message.setNodeConnectionStatus(status); - // TODO: When this is sent from one node to another, we need to ensure that we check the current - // 'revision number' on the node and include that as the Update ID because we need a way to indicate - // which status change event occurred first. I.e., when the status of a node is updated on any node - // that is not the elected leader, we need to ensure that our nodeStatusIdGenerator also is updated. - message.setStatusUpdateIdentifier(statusUpdateId); - - protocolSender.notifyNodeStatusChange(nodesToNotify, message); - } - - dfmService.setNodeIds(getNodeIdentifiers(NodeConnectionState.CONNECTED)); - - return true; - } - - /** - * Updates the status of the given node to the given new status. This method exists only because the NCM currently handles - * some of the status changing and we want it to call into this coordinator instead to change the status. - * - * @param rawNode the node whose status should be updated - * @param nodeStatus the new status of the node - */ - void updateNodeStatus(final Node rawNode, final Status nodeStatus) { - // TODO: Remove this method when NCM is removed - updateNodeStatus(rawNode, nodeStatus, false); - } - - - /** - * Updates the status of the given node to the given new status. This method exists only because the NCM currently handles - * some of the status changing and we want it to call into this coordinator instead to change the status. - * - * @param rawNode the node whose status should be updated - * @param nodeStatus the new status of the node - * @param heartbeatDisconnect indicates whether or not the node is being disconnected due to lack of heartbeat - */ - void updateNodeStatus(final Node rawNode, final Status nodeStatus, final boolean heartbeatDisconnect) { - // TODO: Remove this method when NCM is removed. - final long statusUpdateId = nodeStatusIdGenerator.incrementAndGet(); - final Status oldStatus; - if (heartbeatDisconnect) { - oldStatus = rawNode.setHeartbeatDisconnection(); - } else { - oldStatus = rawNode.setStatus(nodeStatus); - } - - if (nodeStatus != oldStatus) { - final Set nodesToNotify = clusterManager.getNodes(Status.CONNECTED, Status.CONNECTING).stream() - .map(toNotify -> toNotify.getNodeId()) - .collect(Collectors.toSet()); - - final NodeStatusChangeMessage message = new NodeStatusChangeMessage(); - message.setNodeId(rawNode.getNodeId()); - message.setNodeConnectionStatus(new NodeConnectionStatus(NodeConnectionState.valueOf(nodeStatus.name()))); - message.setStatusUpdateIdentifier(statusUpdateId); - - protocolSender.notifyNodeStatusChange(nodesToNotify, message); - dfmService.setNodeIds(getNodeIdentifiers(NodeConnectionState.CONNECTED)); - } - } - - @Override - public Map> getConnectionStates() { - final Set nodes = clusterManager.getNodes(); - final Map> connectionStatusMap = nodes.stream() - .map(node -> node.getNodeId()) - .collect(Collectors.groupingBy(nodeId -> getConnectionStatus(nodeId).getState())); - - return connectionStatusMap; - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/node/Node.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/node/Node.java deleted file mode 100644 index 6f71834dc9b3..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/node/Node.java +++ /dev/null @@ -1,190 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.node; - -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.nifi.cluster.protocol.NodeIdentifier; - -/** - * Represents a connected flow controller. Nodes always have an immutable identifier and a status. The status may be changed, but never null. - * - * A Node may be cloned, but the cloning is a shallow copy of the instance. - * - * This class overrides hashCode and equals and considers two instances to be equal if they have the equal NodeIdentifiers. - * - */ -public class Node implements Cloneable, Comparable { - - /** - * The semantics of a Node status are as follows: - *
    - *
  • CONNECTED -- a flow controller that is connected to the cluster. A connecting node transitions to connected after the cluster receives the flow controller's first heartbeat. A connected - * node can transition to disconnecting.
  • - *
  • CONNECTING -- a flow controller has issued a connection request to the cluster, but has not yet sent a heartbeat. A connecting node can transition to disconnecting or connected. The cluster - * will not accept any external requests to change the flow while any node is connecting.
  • - *
  • DISCONNECTED -- a flow controller that is not connected to the cluster. A disconnected node can transition to connecting.
  • - *
  • DISCONNECTING -- a flow controller that is in the process of disconnecting from the cluster. A disconnecting node will always transition to disconnected.
  • - *
- */ - public static enum Status { - - CONNECTED, - CONNECTING, - DISCONNECTED, - DISCONNECTING - } - - /** - * the node's unique identifier - */ - private final NodeIdentifier nodeId; - - /** - * the node statue - */ - private Status status; - - - /** - * the last time the connection for this node was requested - */ - private AtomicLong connectionRequestedTimestamp = new AtomicLong(0L); - - /** - * a flag to indicate this node was disconnected because of a lack of heartbeat - */ - private boolean heartbeatDisconnection; - - public Node(final NodeIdentifier id, final Status status) { - if (id == null) { - throw new IllegalArgumentException("ID may not be null."); - } else if (status == null) { - throw new IllegalArgumentException("Status may not be null."); - } - this.nodeId = id; - this.status = status; - } - - public NodeIdentifier getNodeId() { - return nodeId; - } - - /** - * Returns the time of the last received connection request for this node. - * - * @return the time when the connection request for this node was received. - */ - public long getConnectionRequestedTimestamp() { - return connectionRequestedTimestamp.get(); - } - - /** - * Sets the time when the connection request for this node was last received. - * - * This method is thread-safe and may be called without obtaining any lock. - * - * @param connectionRequestedTimestamp timestamp - */ - public void setConnectionRequestedTimestamp(long connectionRequestedTimestamp) { - this.connectionRequestedTimestamp.set(connectionRequestedTimestamp); - } - - /** - * Returns true if the node was disconnected due to lack of heartbeat; false otherwise. - * - * @return true if the node was disconnected due to lack of heartbeat; false otherwise. - */ - public boolean isHeartbeatDisconnection() { - return heartbeatDisconnection; - } - - /** - * Sets the status to disconnected and flags the node as being disconnected by lack of heartbeat. - */ - public Status setHeartbeatDisconnection() { - final Status oldStatus = setStatus(Status.DISCONNECTED); - heartbeatDisconnection = true; - return oldStatus; - } - - /** - * @return the status - */ - public synchronized Status getStatus() { - return status; - } - - /** - * Updates the status to the given value, returning the previous status - * - * @param status a status - * @return the previous status for the node - */ - public synchronized Status setStatus(final Status status) { - if (status == null) { - throw new IllegalArgumentException("Status may not be null."); - } - final Status oldStatus = this.status; - this.status = status; - heartbeatDisconnection = false; - return oldStatus; - } - - @Override - public Node clone() { - final Node clone = new Node(nodeId, status); - clone.heartbeatDisconnection = heartbeatDisconnection; - clone.connectionRequestedTimestamp = connectionRequestedTimestamp; - return clone; - } - - @Override - public boolean equals(Object obj) { - if (obj == null) { - return false; - } - if (getClass() != obj.getClass()) { - return false; - } - final Node other = (Node) obj; - if (this.nodeId != other.nodeId && (this.nodeId == null || !this.nodeId.equals(other.nodeId))) { - return false; - } - return true; - } - - @Override - public int hashCode() { - int hash = 7; - hash = 53 * hash + (this.nodeId != null ? this.nodeId.hashCode() : 0); - return hash; - } - - @Override - public String toString() { - return nodeId.toString(); - } - - @Override - public int compareTo(final Node o) { - if (o == null) { - return -1; - } - return getNodeId().getId().compareTo(o.getNodeId().getId()); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java deleted file mode 100644 index c51e06121529..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.spring; - -import java.net.InetSocketAddress; -import java.util.concurrent.TimeUnit; - -import org.apache.nifi.cluster.protocol.impl.ClusterServiceDiscovery; -import org.apache.nifi.cluster.protocol.impl.ClusterServiceLocator; -import org.apache.nifi.io.socket.multicast.DiscoverableService; -import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; -import org.apache.nifi.util.FormatUtils; -import org.apache.nifi.util.NiFiProperties; -import org.springframework.beans.BeansException; -import org.springframework.beans.factory.DisposableBean; -import org.springframework.beans.factory.FactoryBean; -import org.springframework.context.ApplicationContext; -import org.springframework.context.ApplicationContextAware; - -/** - * Factory bean for creating a singleton ClusterManagerProtocolServiceLocator instance. If the application is configured to act as the cluster manager, then null is always returned as the created - * instance. - * - * The cluster manager protocol service represents the socket endpoint for sending internal socket messages to the cluster manager. - */ -public class ClusterManagerProtocolServiceLocatorFactoryBean implements FactoryBean, ApplicationContextAware, DisposableBean { - - private ApplicationContext applicationContext; - - private ClusterServiceLocator locator; - - private NiFiProperties properties; - - @Override - public Object getObject() throws Exception { - /* - * If configured for the cluster manager, then the service locator is never used. - */ - if (properties.isClusterManager()) { - return null; - } else if (locator == null) { - - if (properties.getClusterProtocolUseMulticast()) { - - // get the service discovery instance - final ClusterServiceDiscovery serviceDiscovery = applicationContext.getBean("clusterManagerProtocolServiceDiscovery", ClusterServiceDiscovery.class); - - // create service location configuration - final ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig(); - config.setNumAttempts(properties.getClusterProtocolMulticastServiceLocatorAttempts()); - - final int delay = (int) FormatUtils.getTimeDuration(properties.getClusterProtocolMulticastServiceLocatorAttemptsDelay(), TimeUnit.SECONDS); - config.setTimeBetweenAttempts(delay); - config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS); - - locator = new ClusterServiceLocator(serviceDiscovery); - locator.setAttemptsConfig(config); - - } else { - final String serviceName = applicationContext.getBean("clusterManagerProtocolServiceName", String.class); - final InetSocketAddress serviceAddress = properties.getClusterNodeUnicastManagerProtocolAddress(); - final DiscoverableService service = new DiscoverableServiceImpl(serviceName, serviceAddress); - locator = new ClusterServiceLocator(service); - } - - // start the locator - locator.start(); - - } - return locator; - - } - - @Override - public Class getObjectType() { - return ClusterServiceLocator.class; - } - - @Override - public boolean isSingleton() { - return true; - } - - @Override - public void setApplicationContext(ApplicationContext applicationContext) throws BeansException { - this.applicationContext = applicationContext; - } - - @Override - public void destroy() throws Exception { - if (locator != null && locator.isRunning()) { - locator.stop(); - } - } - - public void setProperties(NiFiProperties properties) { - this.properties = properties; - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java index 9d064ebcc30c..a86fc79739ab 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java @@ -34,7 +34,7 @@ public class FileBasedClusterNodeFirewallFactoryBean implements FactoryBean { @Override public Object getObject() throws Exception { if (firewall == null) { - final File config = properties.getClusterManagerNodeFirewallFile(); + final File config = properties.getClusterNodeFirewallFile(); final File restoreDirectory = properties.getRestoreDirectory(); if (config != null) { firewall = new FileBasedClusterNodeFirewall(config, restoreDirectory); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ThreadPoolRequestReplicatorFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ThreadPoolRequestReplicatorFactoryBean.java new file mode 100644 index 000000000000..6a3e8cbae25c --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/ThreadPoolRequestReplicatorFactoryBean.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.cluster.spring; + +import org.apache.nifi.cluster.coordination.ClusterCoordinator; +import org.apache.nifi.cluster.coordination.http.replication.RequestCompletionCallback; +import org.apache.nifi.cluster.coordination.http.replication.ThreadPoolRequestReplicator; +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.framework.security.util.SslContextFactory; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.util.WebUtils; +import org.springframework.beans.factory.FactoryBean; + +import com.sun.jersey.api.client.Client; +import com.sun.jersey.api.client.config.DefaultClientConfig; + +public class ThreadPoolRequestReplicatorFactoryBean implements FactoryBean { + private NiFiProperties properties; + + private EventReporter eventReporter; + private ClusterCoordinator clusterCoordinator; + private RequestCompletionCallback requestCompletionCallback; + + @Override + public ThreadPoolRequestReplicator getObject() throws Exception { + final int numThreads = properties.getClusterNodeProtocolThreads(); + final Client jerseyClient = WebUtils.createClient(new DefaultClientConfig(), SslContextFactory.createSslContext(properties)); + final String connectionTimeout = properties.getClusterProtocolConnectionHandshakeTimeout(); + final String readTimeout = properties.getClusterProtocolSocketTimeout(); + + final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(numThreads, jerseyClient, clusterCoordinator, + connectionTimeout, readTimeout, requestCompletionCallback, eventReporter); + + return replicator; + } + + @Override + public Class getObjectType() { + return ThreadPoolRequestReplicator.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + + public void setEventReporter(final EventReporter eventReporter) { + this.eventReporter = eventReporter; + } + + public void setClusterCoordinator(final ClusterCoordinator clusterCoordinator) { + this.clusterCoordinator = clusterCoordinator; + } + + public void setRequestCompletionCallback(final RequestCompletionCallback callback) { + this.requestCompletionCallback = callback; + } +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java deleted file mode 100644 index 95263bdef4d5..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java +++ /dev/null @@ -1,118 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.spring; - -import org.apache.nifi.admin.service.AuditService; -import org.apache.nifi.cluster.event.EventManager; -import org.apache.nifi.cluster.firewall.ClusterNodeFirewall; -import org.apache.nifi.cluster.flow.DataFlowManagementService; -import org.apache.nifi.cluster.manager.impl.WebClusterManager; -import org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderListener; -import org.apache.nifi.cluster.protocol.impl.ClusterServicesBroadcaster; -import org.apache.nifi.encrypt.StringEncryptor; -import org.apache.nifi.io.socket.multicast.DiscoverableService; -import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; -import org.apache.nifi.util.NiFiProperties; -import org.springframework.beans.BeansException; -import org.springframework.beans.factory.FactoryBean; -import org.springframework.context.ApplicationContext; -import org.springframework.context.ApplicationContextAware; - -/** - * Factory bean for creating a singleton WebClusterManager instance. If the application is not configured to act as the cluster manager, then null is always returned as the created instance. - */ -public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationContextAware { - - private ApplicationContext applicationContext; - - private WebClusterManager clusterManager; - - private NiFiProperties properties; - - private StringEncryptor encryptor; - - @Override - public Object getObject() throws Exception { - if (properties.isClusterManager() && properties.isNode()) { - throw new IllegalStateException("Application may be configured as a cluster manager or a node, but not both."); - } else if (!properties.isClusterManager()) { - /* - * If not configured for the cluster manager, then the cluster manager is never used. - * null is returned so that we don't instantiate a thread pool or other resources. - */ - return null; - } else if (clusterManager == null) { - final DataFlowManagementService dataFlowService = applicationContext.getBean("dataFlowManagementService", DataFlowManagementService.class); - final ClusterManagerProtocolSenderListener senderListener = applicationContext.getBean("clusterManagerProtocolSenderListener", ClusterManagerProtocolSenderListener.class); - - // create the manager - clusterManager = new WebClusterManager( - dataFlowService, - senderListener, - properties, - encryptor - ); - - // set the service broadcaster - if (properties.getClusterProtocolUseMulticast()) { - - // create broadcaster - final ClusterServicesBroadcaster broadcaster = applicationContext.getBean("clusterServicesBroadcaster", ClusterServicesBroadcaster.class); - - // register the cluster manager protocol service - final String clusterManagerProtocolServiceName = applicationContext.getBean("clusterManagerProtocolServiceName", String.class); - final DiscoverableService clusterManagerProtocolService = new DiscoverableServiceImpl(clusterManagerProtocolServiceName, properties.getClusterManagerProtocolAddress()); - broadcaster.addService(clusterManagerProtocolService); - - clusterManager.setServicesBroadcaster(broadcaster); - } - - // set the event manager - clusterManager.setEventManager(applicationContext.getBean("nodeEventHistoryManager", EventManager.class)); - - // set the cluster firewall - clusterManager.setClusterFirewall(applicationContext.getBean("clusterFirewall", ClusterNodeFirewall.class)); - - // set the audit service - clusterManager.setAuditService(applicationContext.getBean("auditService", AuditService.class)); - } - return clusterManager; - } - - @Override - public Class getObjectType() { - return WebClusterManager.class; - } - - @Override - public boolean isSingleton() { - return true; - } - - @Override - public void setApplicationContext(ApplicationContext applicationContext) throws BeansException { - this.applicationContext = applicationContext; - } - - public void setProperties(NiFiProperties properties) { - this.properties = properties; - } - - public void setEncryptor(final StringEncryptor encryptor) { - this.encryptor = encryptor; - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java deleted file mode 100644 index 92d862998fc5..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.controller.reporting; - -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.cluster.manager.impl.ClusteredReportingContext; -import org.apache.nifi.components.state.StateManager; -import org.apache.nifi.controller.ProcessScheduler; -import org.apache.nifi.controller.ValidationContextFactory; -import org.apache.nifi.controller.service.ControllerServiceProvider; -import org.apache.nifi.reporting.BulletinRepository; -import org.apache.nifi.reporting.EventAccess; -import org.apache.nifi.reporting.ReportingContext; -import org.apache.nifi.reporting.ReportingTask; - -public class ClusteredReportingTaskNode extends AbstractReportingTaskNode { - - private final EventAccess eventAccess; - private final BulletinRepository bulletinRepository; - private final ControllerServiceProvider serviceProvider; - private final StateManager stateManager; - - public ClusteredReportingTaskNode(final ReportingTask reportingTask, final String id, final ProcessScheduler scheduler, - final EventAccess eventAccess, final BulletinRepository bulletinRepository, final ControllerServiceProvider serviceProvider, - final ValidationContextFactory validationContextFactory, final StateManager stateManager) { - super(reportingTask, id, serviceProvider, scheduler, validationContextFactory); - - this.eventAccess = eventAccess; - this.bulletinRepository = bulletinRepository; - this.serviceProvider = serviceProvider; - this.stateManager = stateManager; - } - - @Override - public Authorizable getParentAuthorizable() { - return new Authorizable() { - @Override - public Authorizable getParentAuthorizable() { - return null; - } - - @Override - public Resource getResource() { - return ResourceFactory.getControllerResource(); - } - }; - } - - @Override - public Resource getResource() { - return ResourceFactory.getComponentResource(ResourceType.ReportingTask, getIdentifier(), getName()); - } - - @Override - public ReportingContext getReportingContext() { - return new ClusteredReportingContext(eventAccess, bulletinRepository, getProperties(), serviceProvider, stateManager); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml index 559462ee5c7f..d910c19fcca7 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/resources/nifi-cluster-manager-context.xml @@ -25,68 +25,31 @@ http://www.springframework.org/schema/context http://www.springframework.org/schema/context/spring-context-3.1.xsd http://www.springframework.org/schema/aop http://www.springframework.org/schema/aop/spring-aop-3.1.xsd"> - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + - + + + - - - - - - - - - - - - - - - - - - + + + + + + + - - - + + + + + - diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java index 7d62b549fd76..d8c4e199c303 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestAbstractHeartbeatMonitor.java @@ -32,37 +32,30 @@ import java.util.UUID; import java.util.stream.Collectors; -import org.apache.curator.test.TestingServer; import org.apache.nifi.cluster.coordination.ClusterCoordinator; import org.apache.nifi.cluster.coordination.node.DisconnectionCode; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; +import org.apache.nifi.cluster.event.NodeEvent; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.reporting.Severity; +import org.apache.nifi.services.FlowService; import org.apache.nifi.util.NiFiProperties; import org.junit.After; import org.junit.Before; import org.junit.Test; public class TestAbstractHeartbeatMonitor { - private TestingServer zkServer; private NodeIdentifier nodeId; private TestFriendlyHeartbeatMonitor monitor; @Before public void setup() throws Exception { - zkServer = new TestingServer(true); - zkServer.start(); nodeId = new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 9999, "localhost", 8888, "localhost", null, false); } @After public void clear() throws IOException { - if (zkServer != null) { - zkServer.stop(); - zkServer.close(); - } - if (monitor != null) { monitor.stop(); } @@ -79,7 +72,7 @@ public void testNewConnectedHeartbeatFromUnknownNode() throws IOException, Inter final List requestedToConnect = Collections.synchronizedList(new ArrayList<>()); final ClusterCoordinatorAdapter coordinator = new ClusterCoordinatorAdapter() { @Override - public synchronized void requestNodeConnect(final NodeIdentifier nodeId) { + public synchronized void requestNodeConnect(final NodeIdentifier nodeId, String userDn) { requestedToConnect.add(nodeId); } }; @@ -164,51 +157,9 @@ public synchronized void finishNodeConnection(final NodeIdentifier nodeId) { } - @Test - public void testDisconnectedHeartbeatOnStartup() throws InterruptedException { - final Set requestedToConnect = Collections.synchronizedSet(new HashSet<>()); - final Set connected = Collections.synchronizedSet(new HashSet<>()); - final Set disconnected = Collections.synchronizedSet(new HashSet<>()); - final ClusterCoordinatorAdapter adapter = new ClusterCoordinatorAdapter() { - @Override - public synchronized void requestNodeConnect(final NodeIdentifier nodeId) { - super.requestNodeConnect(nodeId); - requestedToConnect.add(nodeId); - } - - @Override - public synchronized void finishNodeConnection(final NodeIdentifier nodeId) { - super.finishNodeConnection(nodeId); - connected.add(nodeId); - } - - @Override - public synchronized void requestNodeDisconnect(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) { - super.requestNodeDisconnect(nodeId, disconnectionCode, explanation); - disconnected.add(nodeId); - } - }; - - final TestFriendlyHeartbeatMonitor monitor = createMonitor(adapter); - - requestedToConnect.clear(); - - monitor.addHeartbeat(createHeartbeat(nodeId, DisconnectionCode.NODE_SHUTDOWN)); - monitor.waitForProcessed(); - - assertTrue(connected.isEmpty()); - assertTrue(requestedToConnect.isEmpty()); - assertTrue(disconnected.isEmpty()); - } - - private NodeHeartbeat createHeartbeat(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode) { - final NodeConnectionStatus status = new NodeConnectionStatus(disconnectionCode); - return new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(), status, false, 0, 0, 0, 0); - } - private NodeHeartbeat createHeartbeat(final NodeIdentifier nodeId, final NodeConnectionState state) { - final NodeConnectionStatus status = new NodeConnectionStatus(state); - return new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(), status, false, 0, 0, 0, 0); + final NodeConnectionStatus status = new NodeConnectionStatus(nodeId, state); + return new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(), status, Collections.emptySet(), 0, 0, 0, 0); } private TestFriendlyHeartbeatMonitor createMonitor(final ClusterCoordinator coordinator) { @@ -220,10 +171,6 @@ private TestFriendlyHeartbeatMonitor createMonitor(final ClusterCoordinator coor private Properties createProperties() { final Properties properties = new Properties(); properties.setProperty(NiFiProperties.CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL, "10 ms"); - properties.setProperty(NiFiProperties.ZOOKEEPER_CONNECT_STRING, zkServer.getConnectString()); - properties.setProperty(NiFiProperties.ZOOKEEPER_CONNECT_TIMEOUT, "3 secs"); - properties.setProperty(NiFiProperties.ZOOKEEPER_SESSION_TIMEOUT, "3 secs"); - properties.setProperty(NiFiProperties.ZOOKEEPER_ROOT_NODE, "/nifi"); return properties; } @@ -231,24 +178,33 @@ private static class ClusterCoordinatorAdapter implements ClusterCoordinator { private final Map statuses = new HashMap<>(); private final List events = new ArrayList<>(); - @Override public synchronized void requestNodeConnect(NodeIdentifier nodeId) { - statuses.put(nodeId, new NodeConnectionStatus(NodeConnectionState.CONNECTING)); + requestNodeConnect(nodeId, null); + } + + @Override + public synchronized void requestNodeConnect(NodeIdentifier nodeId, String userDn) { + statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTING)); + } + + @Override + public void removeNode(NodeIdentifier nodeId, String userDn) { + statuses.remove(nodeId); } @Override public synchronized void finishNodeConnection(NodeIdentifier nodeId) { - statuses.put(nodeId, new NodeConnectionStatus(NodeConnectionState.CONNECTED)); + statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED)); } @Override public synchronized void requestNodeDisconnect(NodeIdentifier nodeId, DisconnectionCode disconnectionCode, String explanation) { - statuses.put(nodeId, new NodeConnectionStatus(NodeConnectionState.DISCONNECTED)); + statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.DISCONNECTED)); } @Override public synchronized void disconnectionRequestedByNode(NodeIdentifier nodeId, DisconnectionCode disconnectionCode, String explanation) { - statuses.put(nodeId, new NodeConnectionStatus(NodeConnectionState.DISCONNECTED)); + statuses.put(nodeId, new NodeConnectionStatus(nodeId, NodeConnectionState.DISCONNECTED)); } @Override @@ -257,8 +213,16 @@ public synchronized NodeConnectionStatus getConnectionStatus(NodeIdentifier node } @Override - public synchronized Set getNodeIdentifiers(NodeConnectionState state) { - return statuses.entrySet().stream().filter(p -> p.getValue().getState() == state).map(p -> p.getKey()).collect(Collectors.toSet()); + public synchronized Set getNodeIdentifiers(NodeConnectionState... states) { + final Set stateSet = new HashSet<>(); + for (final NodeConnectionState state : states) { + stateSet.add(state); + } + + return statuses.entrySet().stream() + .filter(p -> stateSet.contains(p.getValue().getState())) + .map(p -> p.getKey()) + .collect(Collectors.toSet()); } @Override @@ -272,7 +236,7 @@ public synchronized void reportEvent(NodeIdentifier nodeId, Severity severity, S } @Override - public synchronized void setPrimaryNode(NodeIdentifier nodeId) { + public void updateNodeRoles(NodeIdentifier nodeId, Set roles) { } synchronized List getEvents() { @@ -288,6 +252,41 @@ public NodeIdentifier getNodeIdentifier(final String uuid) { public Map> getConnectionStates() { return statuses.keySet().stream().collect(Collectors.groupingBy(nodeId -> getConnectionStatus(nodeId).getState())); } + + @Override + public List getNodeEvents(NodeIdentifier nodeId) { + return null; + } + + @Override + public NodeIdentifier getPrimaryNode() { + return null; + } + + @Override + public void setFlowService(FlowService flowService) { + } + + @Override + public void resetNodeStatuses(Map statusMap) { + } + + @Override + public void shutdown() { + } + + @Override + public void setLocalNodeIdentifier(NodeIdentifier nodeId) { + } + + @Override + public boolean isConnected() { + return false; + } + + @Override + public void setConnected(boolean connected) { + } } public static class ReportedEvent { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestCuratorHeartbeatMonitor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestCuratorHeartbeatMonitor.java deleted file mode 100644 index 3652595a87e7..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/heartbeat/TestCuratorHeartbeatMonitor.java +++ /dev/null @@ -1,355 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.nifi.cluster.coordination.heartbeat; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import java.util.UUID; -import java.util.stream.Collectors; - -import org.apache.curator.test.TestingServer; -import org.apache.nifi.cluster.coordination.ClusterCoordinator; -import org.apache.nifi.cluster.coordination.node.DisconnectionCode; -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.reporting.Severity; -import org.apache.nifi.util.NiFiProperties; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -public class TestCuratorHeartbeatMonitor { - private TestingServer zkServer; - private NodeIdentifier nodeId; - private TestFriendlyHeartbeatMonitor monitor; - - @Before - public void setup() throws Exception { - zkServer = new TestingServer(true); - zkServer.start(); - nodeId = new NodeIdentifier(UUID.randomUUID().toString(), "localhost", 9999, "localhost", 8888, "localhost", null, false); - } - - @After - public void clear() throws IOException { - if (zkServer != null) { - zkServer.stop(); - zkServer.close(); - } - - if (monitor != null) { - monitor.stop(); - } - } - - /** - * Verifies that a node that sends a heartbeat that indicates that it is 'connected' is asked to connect to - * cluster if the cluster coordinator does not know about the node - * - * @throws InterruptedException if interrupted - */ - @Test - public void testNewConnectedHeartbeatFromUnknownNode() throws IOException, InterruptedException { - final List requestedToConnect = Collections.synchronizedList(new ArrayList<>()); - final ClusterCoordinatorAdapter coordinator = new ClusterCoordinatorAdapter() { - @Override - public synchronized void requestNodeConnect(final NodeIdentifier nodeId) { - requestedToConnect.add(nodeId); - } - }; - - final TestFriendlyHeartbeatMonitor monitor = createMonitor(coordinator); - - // Ensure that we request the Unknown Node connect to the cluster - final NodeHeartbeat heartbeat = createHeartbeat(nodeId, NodeConnectionState.CONNECTED); - monitor.addHeartbeat(heartbeat); - monitor.waitForProcessed(); - - assertEquals(1, requestedToConnect.size()); - assertEquals(nodeId, requestedToConnect.get(0)); - assertEquals(1, coordinator.getEvents().size()); - } - - /** - * Verifies that a node that sends a heartbeat that indicates that it is 'connected' if previously - * manually disconnected, will be asked to disconnect from the cluster again. - * - * @throws InterruptedException if interrupted - */ - @Test - public void testHeartbeatFromManuallyDisconnectedNode() throws InterruptedException { - final Set requestedToConnect = Collections.synchronizedSet(new HashSet<>()); - final Set requestedToDisconnect = Collections.synchronizedSet(new HashSet<>()); - final ClusterCoordinatorAdapter adapter = new ClusterCoordinatorAdapter() { - @Override - public synchronized void requestNodeConnect(final NodeIdentifier nodeId) { - super.requestNodeConnect(nodeId); - requestedToConnect.add(nodeId); - } - - @Override - public synchronized void requestNodeDisconnect(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) { - super.requestNodeDisconnect(nodeId, disconnectionCode, explanation); - requestedToDisconnect.add(nodeId); - } - }; - - final TestFriendlyHeartbeatMonitor monitor = createMonitor(adapter); - - adapter.requestNodeDisconnect(nodeId, DisconnectionCode.USER_DISCONNECTED, "Unit Testing"); - monitor.addHeartbeat(createHeartbeat(nodeId, NodeConnectionState.CONNECTED)); - monitor.waitForProcessed(); - - assertEquals(1, requestedToDisconnect.size()); - assertEquals(nodeId, requestedToDisconnect.iterator().next()); - assertTrue(requestedToConnect.isEmpty()); - } - - - @Test - public void testConnectingNodeMarkedConnectedWhenHeartbeatReceived() throws InterruptedException { - final Set requestedToConnect = Collections.synchronizedSet(new HashSet<>()); - final Set connected = Collections.synchronizedSet(new HashSet<>()); - final ClusterCoordinatorAdapter adapter = new ClusterCoordinatorAdapter() { - @Override - public synchronized void requestNodeConnect(final NodeIdentifier nodeId) { - super.requestNodeConnect(nodeId); - requestedToConnect.add(nodeId); - } - - @Override - public synchronized void finishNodeConnection(final NodeIdentifier nodeId) { - super.finishNodeConnection(nodeId); - connected.add(nodeId); - } - }; - - final TestFriendlyHeartbeatMonitor monitor = createMonitor(adapter); - - adapter.requestNodeConnect(nodeId); // set state to 'connecting' - requestedToConnect.clear(); - - monitor.addHeartbeat(createHeartbeat(nodeId, NodeConnectionState.CONNECTED)); - monitor.waitForProcessed(); - - assertEquals(1, connected.size()); - assertEquals(nodeId, connected.iterator().next()); - assertTrue(requestedToConnect.isEmpty()); - } - - - @Test - public void testDisconnectedHeartbeatOnStartup() throws InterruptedException { - final Set requestedToConnect = Collections.synchronizedSet(new HashSet<>()); - final Set connected = Collections.synchronizedSet(new HashSet<>()); - final Set disconnected = Collections.synchronizedSet(new HashSet<>()); - final ClusterCoordinatorAdapter adapter = new ClusterCoordinatorAdapter() { - @Override - public synchronized void requestNodeConnect(final NodeIdentifier nodeId) { - super.requestNodeConnect(nodeId); - requestedToConnect.add(nodeId); - } - - @Override - public synchronized void finishNodeConnection(final NodeIdentifier nodeId) { - super.finishNodeConnection(nodeId); - connected.add(nodeId); - } - - @Override - public synchronized void requestNodeDisconnect(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode, final String explanation) { - super.requestNodeDisconnect(nodeId, disconnectionCode, explanation); - disconnected.add(nodeId); - } - }; - - final TestFriendlyHeartbeatMonitor monitor = createMonitor(adapter); - - requestedToConnect.clear(); - - monitor.addHeartbeat(createHeartbeat(nodeId, DisconnectionCode.NODE_SHUTDOWN)); - monitor.waitForProcessed(); - - assertTrue(connected.isEmpty()); - assertTrue(requestedToConnect.isEmpty()); - assertTrue(disconnected.isEmpty()); - } - - private NodeHeartbeat createHeartbeat(final NodeIdentifier nodeId, final DisconnectionCode disconnectionCode) { - final NodeConnectionStatus status = new NodeConnectionStatus(disconnectionCode); - return new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(), status, false, 0, 0, 0, 0); - } - - private NodeHeartbeat createHeartbeat(final NodeIdentifier nodeId, final NodeConnectionState state) { - final NodeConnectionStatus status = new NodeConnectionStatus(state); - return new StandardNodeHeartbeat(nodeId, System.currentTimeMillis(), status, false, 0, 0, 0, 0); - } - - private TestFriendlyHeartbeatMonitor createMonitor(final ClusterCoordinator coordinator) { - monitor = new TestFriendlyHeartbeatMonitor(coordinator, createProperties()); - monitor.start(); - return monitor; - } - - private Properties createProperties() { - final Properties properties = new Properties(); - properties.setProperty(NiFiProperties.CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL, "10 ms"); - properties.setProperty(NiFiProperties.ZOOKEEPER_CONNECT_STRING, zkServer.getConnectString()); - properties.setProperty(NiFiProperties.ZOOKEEPER_CONNECT_TIMEOUT, "3 secs"); - properties.setProperty(NiFiProperties.ZOOKEEPER_SESSION_TIMEOUT, "3 secs"); - properties.setProperty(NiFiProperties.ZOOKEEPER_ROOT_NODE, "/nifi"); - return properties; - } - - private static class ClusterCoordinatorAdapter implements ClusterCoordinator { - private final Map statuses = new HashMap<>(); - private final List events = new ArrayList<>(); - - @Override - public synchronized void requestNodeConnect(NodeIdentifier nodeId) { - statuses.put(nodeId, new NodeConnectionStatus(NodeConnectionState.CONNECTING)); - } - - @Override - public synchronized void finishNodeConnection(NodeIdentifier nodeId) { - statuses.put(nodeId, new NodeConnectionStatus(NodeConnectionState.CONNECTED)); - } - - @Override - public synchronized void requestNodeDisconnect(NodeIdentifier nodeId, DisconnectionCode disconnectionCode, String explanation) { - statuses.put(nodeId, new NodeConnectionStatus(NodeConnectionState.DISCONNECTED)); - } - - @Override - public synchronized void disconnectionRequestedByNode(NodeIdentifier nodeId, DisconnectionCode disconnectionCode, String explanation) { - statuses.put(nodeId, new NodeConnectionStatus(NodeConnectionState.DISCONNECTED)); - } - - @Override - public synchronized NodeConnectionStatus getConnectionStatus(NodeIdentifier nodeId) { - return statuses.get(nodeId); - } - - @Override - public synchronized Set getNodeIdentifiers(NodeConnectionState state) { - return statuses.entrySet().stream().filter(p -> p.getValue().getState() == state).map(p -> p.getKey()).collect(Collectors.toSet()); - } - - @Override - public synchronized boolean isBlockedByFirewall(String hostname) { - return false; - } - - @Override - public synchronized void reportEvent(NodeIdentifier nodeId, Severity severity, String event) { - events.add(new ReportedEvent(nodeId, severity, event)); - } - - @Override - public synchronized void setPrimaryNode(NodeIdentifier nodeId) { - } - - synchronized List getEvents() { - return new ArrayList<>(events); - } - - @Override - public NodeIdentifier getNodeIdentifier(final String uuid) { - return statuses.keySet().stream().filter(p -> p.getId().equals(uuid)).findFirst().orElse(null); - } - - @Override - public Map> getConnectionStates() { - return null; - } - } - - public static class ReportedEvent { - private final NodeIdentifier nodeId; - private final Severity severity; - private final String event; - - public ReportedEvent(NodeIdentifier nodeId, Severity severity, String event) { - this.nodeId = nodeId; - this.severity = severity; - this.event = event; - } - - public NodeIdentifier getNodeId() { - return nodeId; - } - - public Severity getSeverity() { - return severity; - } - - public String getEvent() { - return event; - } - } - - - private static class TestFriendlyHeartbeatMonitor extends CuratorHeartbeatMonitor { - private Map heartbeats = new HashMap<>(); - private final Object mutex = new Object(); - - public TestFriendlyHeartbeatMonitor(ClusterCoordinator clusterCoordinator, Properties properties) { - super(clusterCoordinator, properties); - } - - @Override - synchronized Map fetchHeartbeats() { - return heartbeats; - } - - @Override - synchronized void monitorHeartbeats() { - super.monitorHeartbeats(); - - synchronized (mutex) { - mutex.notify(); - } - } - - synchronized void addHeartbeat(final NodeHeartbeat heartbeat) { - heartbeats.put(heartbeat.getNodeIdentifier(), heartbeat); - } - - @Override - public synchronized void removeHeartbeat(final NodeIdentifier nodeId) { - heartbeats.remove(nodeId); - } - - void waitForProcessed() throws InterruptedException { - synchronized (mutex) { - mutex.wait(); - } - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java index 1cc210e6e741..0bce52137bd3 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/http/replication/TestThreadPoolRequestReplicator.java @@ -26,17 +26,25 @@ import java.io.ByteArrayInputStream; import java.net.SocketTimeoutException; import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; +import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import javax.ws.rs.HttpMethod; +import org.apache.commons.collections4.map.MultiValueMap; import org.apache.nifi.cluster.coordination.ClusterCoordinator; -import org.apache.nifi.cluster.flow.DataFlowManagementService; +import org.apache.nifi.cluster.coordination.node.NodeConnectionState; +import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException; +import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException; import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException; import org.apache.nifi.cluster.protocol.NodeIdentifier; import org.apache.nifi.util.NiFiProperties; @@ -47,6 +55,8 @@ import org.junit.Test; import org.mockito.Mockito; import org.mockito.internal.util.reflection.Whitebox; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import com.sun.jersey.api.client.Client; import com.sun.jersey.api.client.ClientHandlerException; @@ -149,13 +159,14 @@ public void testCompleteOnError() { @Test(timeout = 15000) public void testMultipleRequestWithTwoPhaseCommit() { final Set nodeIds = new HashSet<>(); - nodeIds.add(new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, false)); + final NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, false); + nodeIds.add(nodeId); final ClusterCoordinator coordinator = Mockito.mock(ClusterCoordinator.class); + Mockito.when(coordinator.getConnectionStatus(Mockito.any(NodeIdentifier.class))).thenReturn(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED)); final AtomicInteger requestCount = new AtomicInteger(0); - final DataFlowManagementService dfmService = Mockito.mock(DataFlowManagementService.class); - final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null, null, dfmService) { + final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null) { @Override protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilder, final NodeIdentifier nodeId, final String method, final URI uri, final String requestId) { // the resource builder will not expose its headers to us, so we are using Mockito's Whitebox class to extract them. @@ -177,8 +188,6 @@ protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilde } }; - replicator.start(); - try { final AsyncClusterResponse clusterResponse = replicator.replicate(nodeIds, HttpMethod.POST, new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>()); @@ -191,7 +200,79 @@ protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilde e.printStackTrace(); Assert.fail(e.toString()); } finally { - replicator.stop(); + replicator.shutdown(); + } + } + + private ClusterCoordinator createClusterCoordinator() { + final ClusterCoordinator coordinator = Mockito.mock(ClusterCoordinator.class); + Mockito.when(coordinator.getConnectionStatus(Mockito.any(NodeIdentifier.class))).thenAnswer(new Answer() { + @Override + public NodeConnectionStatus answer(InvocationOnMock invocation) throws Throwable { + return new NodeConnectionStatus(invocation.getArgumentAt(0, NodeIdentifier.class), NodeConnectionState.CONNECTED); + } + }); + + return coordinator; + } + + @Test + public void testMutableRequestRequiresAllNodesConnected() throws URISyntaxException { + final ClusterCoordinator coordinator = createClusterCoordinator(); + + // build a map of connection state to node ids + final Map> nodeMap = new HashMap<>(); + final List connectedNodes = new ArrayList<>(); + connectedNodes.add(new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, false)); + connectedNodes.add(new NodeIdentifier("2", "localhost", 8200, "localhost", 8201, "localhost", 8202, false)); + nodeMap.put(NodeConnectionState.CONNECTED, connectedNodes); + + final List otherState = new ArrayList<>(); + otherState.add(new NodeIdentifier("3", "localhost", 8300, "localhost", 8301, "localhost", 8302, false)); + nodeMap.put(NodeConnectionState.CONNECTING, otherState); + + Mockito.when(coordinator.getConnectionStates()).thenReturn(nodeMap); + final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null) { + @Override + public AsyncClusterResponse replicate(Set nodeIds, String method, URI uri, Object entity, Map headers) { + return null; + } + }; + + try { + try { + replicator.replicate(HttpMethod.POST, new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>()); + Assert.fail("Expected ConnectingNodeMutableRequestException"); + } catch (final ConnectingNodeMutableRequestException e) { + // expected behavior + } + + nodeMap.remove(NodeConnectionState.CONNECTING); + nodeMap.put(NodeConnectionState.DISCONNECTED, otherState); + try { + replicator.replicate(HttpMethod.POST, new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>()); + Assert.fail("Expected DisconnectedNodeMutableRequestException"); + } catch (final DisconnectedNodeMutableRequestException e) { + // expected behavior + } + + nodeMap.remove(NodeConnectionState.DISCONNECTED); + nodeMap.put(NodeConnectionState.DISCONNECTING, otherState); + try { + replicator.replicate(HttpMethod.POST, new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>()); + Assert.fail("Expected DisconnectedNodeMutableRequestException"); + } catch (final DisconnectedNodeMutableRequestException e) { + // expected behavior + } + + // should not throw an Exception because it's a GET + replicator.replicate(HttpMethod.GET, new URI("http://localhost:80/processors/1"), new MultiValueMap<>(), new HashMap<>()); + + // should not throw an Exception because all nodes are now connected + nodeMap.remove(NodeConnectionState.DISCONNECTING); + replicator.replicate(HttpMethod.POST, new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>()); + } finally { + replicator.shutdown(); } } @@ -202,11 +283,9 @@ public void testOneNodeRejectsTwoPhaseCommit() { nodeIds.add(new NodeIdentifier("1", "localhost", 8100, "localhost", 8101, "localhost", 8102, false)); nodeIds.add(new NodeIdentifier("2", "localhost", 8200, "localhost", 8201, "localhost", 8202, false)); - final ClusterCoordinator coordinator = Mockito.mock(ClusterCoordinator.class); - + final ClusterCoordinator coordinator = createClusterCoordinator(); final AtomicInteger requestCount = new AtomicInteger(0); - final DataFlowManagementService dfmService = Mockito.mock(DataFlowManagementService.class); - final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null, null, dfmService) { + final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null) { @Override protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilder, final NodeIdentifier nodeId, final String method, final URI uri, final String requestId) { // the resource builder will not expose its headers to us, so we are using Mockito's Whitebox class to extract them. @@ -226,8 +305,6 @@ protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilde } }; - replicator.start(); - try { final AsyncClusterResponse clusterResponse = replicator.replicate(nodeIds, HttpMethod.POST, new URI("http://localhost:80/processors/1"), new ProcessorEntity(), new HashMap<>()); @@ -239,7 +316,7 @@ protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilde } catch (final Exception e) { Assert.fail(e.toString()); } finally { - replicator.stop(); + replicator.shutdown(); } } @@ -250,10 +327,8 @@ private void withReplicator(final WithReplicator function) { } private void withReplicator(final WithReplicator function, final Status status, final long delayMillis, final RuntimeException failure) { - final ClusterCoordinator coordinator = Mockito.mock(ClusterCoordinator.class); - - final DataFlowManagementService dfmService = Mockito.mock(DataFlowManagementService.class); - final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null, null, dfmService) { + final ClusterCoordinator coordinator = createClusterCoordinator(); + final ThreadPoolRequestReplicator replicator = new ThreadPoolRequestReplicator(2, new Client(), coordinator, "1 sec", "1 sec", null, null) { @Override protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilder, final NodeIdentifier nodeId, final String method, final URI uri, final String requestId) { if (delayMillis > 0L) { @@ -274,15 +349,13 @@ protected NodeResponse replicateRequest(final WebResource.Builder resourceBuilde } }; - replicator.start(); - try { function.withReplicator(replicator); } catch (final Exception e) { e.printStackTrace(); Assert.fail(e.toString()); } finally { - replicator.stop(); + replicator.shutdown(); } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java new file mode 100644 index 000000000000..994370cb666b --- /dev/null +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/coordination/node/TestNodeClusterCoordinator.java @@ -0,0 +1,497 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.node; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +import org.apache.nifi.cluster.protocol.ConnectionRequest; +import org.apache.nifi.cluster.protocol.ConnectionResponse; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.StandardDataFlow; +import org.apache.nifi.cluster.protocol.impl.ClusterCoordinationProtocolSenderListener; +import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; +import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage; +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.services.FlowService; +import org.apache.nifi.web.revision.RevisionManager; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +public class TestNodeClusterCoordinator { + private NodeClusterCoordinator coordinator; + private ClusterCoordinationProtocolSenderListener senderListener; + private List nodeStatusChangeMessages; + + @Before + @SuppressWarnings("unchecked") + public void setup() throws IOException { + senderListener = Mockito.mock(ClusterCoordinationProtocolSenderListener.class); + nodeStatusChangeMessages = Collections.synchronizedList(new ArrayList<>()); + + Mockito.doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + final NodeStatusChangeMessage statusChangeMessage = invocation.getArgumentAt(1, NodeStatusChangeMessage.class); + nodeStatusChangeMessages.add(statusChangeMessage); + return null; + } + }).when(senderListener).notifyNodeStatusChange(Mockito.any(Set.class), Mockito.any(NodeStatusChangeMessage.class)); + + final EventReporter eventReporter = Mockito.mock(EventReporter.class); + final RevisionManager revisionManager = Mockito.mock(RevisionManager.class); + Mockito.when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList()); + + coordinator = new NodeClusterCoordinator(senderListener, eventReporter, null, revisionManager); + + final FlowService flowService = Mockito.mock(FlowService.class); + final StandardDataFlow dataFlow = new StandardDataFlow(new byte[50], new byte[50]); + Mockito.when(flowService.createDataFlow()).thenReturn(dataFlow); + coordinator.setFlowService(flowService); + } + + @Test + public void testConnectionResponseIndicatesAllNodes() throws IOException { + // Add a disconnected node + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(1), DisconnectionCode.LACK_OF_HEARTBEAT)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(2), NodeConnectionState.DISCONNECTING)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(3), NodeConnectionState.CONNECTING)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(4), NodeConnectionState.CONNECTED)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(5), NodeConnectionState.CONNECTED)); + + // Create a connection request message and send to the coordinator + final NodeIdentifier requestedNodeId = createNodeId(6); + final ProtocolMessage protocolResponse = requestConnection(requestedNodeId, coordinator); + assertNotNull(protocolResponse); + assertTrue(protocolResponse instanceof ConnectionResponseMessage); + + final ConnectionResponse response = ((ConnectionResponseMessage) protocolResponse).getConnectionResponse(); + assertNotNull(response); + + assertEquals(requestedNodeId, response.getNodeIdentifier()); + assertNull(response.getRejectionReason()); + + final List statuses = response.getNodeConnectionStatuses(); + assertNotNull(statuses); + assertEquals(6, statuses.size()); + final Map statusMap = statuses.stream().collect( + Collectors.toMap(status -> status.getNodeIdentifier(), status -> status)); + + assertEquals(DisconnectionCode.LACK_OF_HEARTBEAT, statusMap.get(createNodeId(1)).getDisconnectCode()); + assertEquals(NodeConnectionState.DISCONNECTING, statusMap.get(createNodeId(2)).getState()); + assertEquals(NodeConnectionState.CONNECTING, statusMap.get(createNodeId(3)).getState()); + assertEquals(NodeConnectionState.CONNECTED, statusMap.get(createNodeId(4)).getState()); + assertEquals(NodeConnectionState.CONNECTED, statusMap.get(createNodeId(5)).getState()); + assertEquals(NodeConnectionState.CONNECTING, statusMap.get(createNodeId(6)).getState()); + } + + @Test + public void testTryAgainIfNoFlowServiceSet() { + final ClusterCoordinationProtocolSenderListener senderListener = Mockito.mock(ClusterCoordinationProtocolSenderListener.class); + final EventReporter eventReporter = Mockito.mock(EventReporter.class); + final RevisionManager revisionManager = Mockito.mock(RevisionManager.class); + Mockito.when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList()); + + final NodeClusterCoordinator coordinator = new NodeClusterCoordinator(senderListener, eventReporter, null, revisionManager); + + final NodeIdentifier requestedNodeId = createNodeId(6); + final ConnectionRequest request = new ConnectionRequest(requestedNodeId); + final ConnectionRequestMessage requestMsg = new ConnectionRequestMessage(); + requestMsg.setConnectionRequest(request); + + final ProtocolMessage protocolResponse = coordinator.handle(requestMsg); + assertNotNull(protocolResponse); + assertTrue(protocolResponse instanceof ConnectionResponseMessage); + + final ConnectionResponse response = ((ConnectionResponseMessage) protocolResponse).getConnectionResponse(); + assertNotNull(response); + assertEquals(5, response.getTryLaterSeconds()); + } + + @Test(timeout = 5000) + public void testUnknownNodeAskedToConnectOnAttemptedConnectionComplete() throws IOException, InterruptedException { + final ClusterCoordinationProtocolSenderListener senderListener = Mockito.mock(ClusterCoordinationProtocolSenderListener.class); + final AtomicReference requestRef = new AtomicReference<>(); + + Mockito.when(senderListener.requestReconnection(Mockito.any(ReconnectionRequestMessage.class))).thenAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + final ReconnectionRequestMessage msg = invocation.getArgumentAt(0, ReconnectionRequestMessage.class); + requestRef.set(msg); + return null; + } + }); + + final EventReporter eventReporter = Mockito.mock(EventReporter.class); + final RevisionManager revisionManager = Mockito.mock(RevisionManager.class); + Mockito.when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList()); + + final NodeClusterCoordinator coordinator = new NodeClusterCoordinator(senderListener, eventReporter, null, revisionManager); + + final FlowService flowService = Mockito.mock(FlowService.class); + final StandardDataFlow dataFlow = new StandardDataFlow(new byte[50], new byte[50]); + Mockito.when(flowService.createDataFlow()).thenReturn(dataFlow); + coordinator.setFlowService(flowService); + + final NodeIdentifier nodeId = createNodeId(1); + coordinator.finishNodeConnection(nodeId); + + while (requestRef.get() == null) { + Thread.sleep(10L); + } + + final ReconnectionRequestMessage msg = requestRef.get(); + assertEquals(nodeId, msg.getNodeId()); + + final StandardDataFlow df = msg.getDataFlow(); + assertNotNull(df); + assertTrue(Arrays.equals(dataFlow.getFlow(), df.getFlow())); + assertTrue(Arrays.equals(dataFlow.getSnippets(), df.getSnippets())); + } + + @Test(timeout = 5000) + public void testFinishNodeConnectionResultsInConnectedState() throws IOException, InterruptedException { + final NodeIdentifier nodeId = createNodeId(1); + + // Create a connection request message and send to the coordinator + requestConnection(createNodeId(1), coordinator); + + while (nodeStatusChangeMessages.isEmpty()) { + Thread.sleep(20L); + } + assertEquals(NodeConnectionState.CONNECTING, nodeStatusChangeMessages.get(0).getNodeConnectionStatus().getState()); + nodeStatusChangeMessages.clear(); + + // Finish connecting. This should notify all that the status is now 'CONNECTED' + coordinator.finishNodeConnection(nodeId); + + while (nodeStatusChangeMessages.isEmpty()) { + Thread.sleep(20L); + } + assertEquals(NodeConnectionState.CONNECTED, nodeStatusChangeMessages.get(0).getNodeConnectionStatus().getState()); + assertEquals(NodeConnectionState.CONNECTED, coordinator.getConnectionStatus(nodeId).getState()); + } + + @Test(timeout = 5000) + @SuppressWarnings("unchecked") + public void testStatusChangesReplicated() throws InterruptedException, IOException { + final ClusterCoordinationProtocolSenderListener senderListener = Mockito.mock(ClusterCoordinationProtocolSenderListener.class); + final List msgs = Collections.synchronizedList(new ArrayList<>()); + + Mockito.doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + final NodeStatusChangeMessage statusChangeMessage = invocation.getArgumentAt(1, NodeStatusChangeMessage.class); + msgs.add(statusChangeMessage); + return null; + } + }).when(senderListener).notifyNodeStatusChange(Mockito.any(Set.class), Mockito.any(NodeStatusChangeMessage.class)); + + final EventReporter eventReporter = Mockito.mock(EventReporter.class); + final RevisionManager revisionManager = Mockito.mock(RevisionManager.class); + Mockito.when(revisionManager.getAllRevisions()).thenReturn(Collections.emptyList()); + final NodeClusterCoordinator coordinator = new NodeClusterCoordinator(senderListener, eventReporter, null, revisionManager); + + final FlowService flowService = Mockito.mock(FlowService.class); + final StandardDataFlow dataFlow = new StandardDataFlow(new byte[50], new byte[50]); + Mockito.when(flowService.createDataFlow()).thenReturn(dataFlow); + coordinator.setFlowService(flowService); + + // Create a connection request message and send to the coordinator + final NodeIdentifier requestedNodeId = createNodeId(1); + requestConnection(requestedNodeId, coordinator); + + // The above connection request should trigger a 'CONNECTING' state transition to be replicated + while (msgs.isEmpty()) { + Thread.sleep(20L); + } + final NodeStatusChangeMessage connectingMsg = msgs.get(0); + assertEquals(NodeConnectionState.CONNECTING, connectingMsg.getNodeConnectionStatus().getState()); + assertEquals(requestedNodeId, connectingMsg.getNodeId()); + + // set node status to connected + coordinator.finishNodeConnection(requestedNodeId); + + // the above method will result in the node identifier becoming 'CONNECTED'. Wait for this to happen and clear the map + while (msgs.isEmpty()) { + Thread.sleep(20L); + } + msgs.clear(); + + coordinator.disconnectionRequestedByNode(requestedNodeId, DisconnectionCode.NODE_SHUTDOWN, "Unit Test"); + + while (msgs.isEmpty()) { + Thread.sleep(20L); + } + + assertEquals(1, msgs.size()); + final NodeStatusChangeMessage statusChangeMsg = msgs.get(0); + assertNotNull(statusChangeMsg); + assertEquals(createNodeId(1), statusChangeMsg.getNodeId()); + assertEquals(DisconnectionCode.NODE_SHUTDOWN, statusChangeMsg.getNodeConnectionStatus().getDisconnectCode()); + assertEquals("Unit Test", statusChangeMsg.getNodeConnectionStatus().getDisconnectReason()); + } + + + @Test + public void testGetConnectionStates() throws IOException { + // Add a disconnected node + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(1), DisconnectionCode.LACK_OF_HEARTBEAT)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(2), NodeConnectionState.DISCONNECTING)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(3), NodeConnectionState.CONNECTING)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(4), NodeConnectionState.CONNECTED)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(5), NodeConnectionState.CONNECTED)); + + final Map> stateMap = coordinator.getConnectionStates(); + assertEquals(4, stateMap.size()); + + final List connectedIds = stateMap.get(NodeConnectionState.CONNECTED); + assertEquals(2, connectedIds.size()); + assertTrue(connectedIds.contains(createNodeId(4))); + assertTrue(connectedIds.contains(createNodeId(5))); + + final List connectingIds = stateMap.get(NodeConnectionState.CONNECTING); + assertEquals(1, connectingIds.size()); + assertTrue(connectingIds.contains(createNodeId(3))); + + final List disconnectingIds = stateMap.get(NodeConnectionState.DISCONNECTING); + assertEquals(1, disconnectingIds.size()); + assertTrue(disconnectingIds.contains(createNodeId(2))); + + final List disconnectedIds = stateMap.get(NodeConnectionState.DISCONNECTED); + assertEquals(1, disconnectedIds.size()); + assertTrue(disconnectedIds.contains(createNodeId(1))); + } + + @Test + public void testGetNodeIdentifiers() throws IOException { + // Add a disconnected node + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(1), DisconnectionCode.LACK_OF_HEARTBEAT)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(2), NodeConnectionState.DISCONNECTING)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(3), NodeConnectionState.CONNECTING)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(4), NodeConnectionState.CONNECTED)); + coordinator.updateNodeStatus(new NodeConnectionStatus(createNodeId(5), NodeConnectionState.CONNECTED)); + + final Set connectedIds = coordinator.getNodeIdentifiers(NodeConnectionState.CONNECTED); + assertEquals(2, connectedIds.size()); + assertTrue(connectedIds.contains(createNodeId(4))); + assertTrue(connectedIds.contains(createNodeId(5))); + + final Set connectingIds = coordinator.getNodeIdentifiers(NodeConnectionState.CONNECTING); + assertEquals(1, connectingIds.size()); + assertTrue(connectingIds.contains(createNodeId(3))); + + final Set disconnectingIds = coordinator.getNodeIdentifiers(NodeConnectionState.DISCONNECTING); + assertEquals(1, disconnectingIds.size()); + assertTrue(disconnectingIds.contains(createNodeId(2))); + + final Set disconnectedIds = coordinator.getNodeIdentifiers(NodeConnectionState.DISCONNECTED); + assertEquals(1, disconnectedIds.size()); + assertTrue(disconnectedIds.contains(createNodeId(1))); + } + + + @Test(timeout = 5000) + public void testRequestNodeDisconnect() throws InterruptedException { + // Add a connected node + final NodeIdentifier nodeId = createNodeId(1); + coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.CONNECTED)); + + // wait for the status change message and clear it + while (nodeStatusChangeMessages.isEmpty()) { + Thread.sleep(10L); + } + nodeStatusChangeMessages.clear(); + + coordinator.requestNodeDisconnect(nodeId, DisconnectionCode.USER_DISCONNECTED, "Unit Test"); + assertEquals(NodeConnectionState.DISCONNECTED, coordinator.getConnectionStatus(nodeId).getState()); + + while (nodeStatusChangeMessages.isEmpty()) { + Thread.sleep(10L); + } + final NodeStatusChangeMessage msg = nodeStatusChangeMessages.get(0); + assertEquals(nodeId, msg.getNodeId()); + assertEquals(NodeConnectionState.DISCONNECTED, msg.getNodeConnectionStatus().getState()); + } + + + @Test(timeout = 5000) + public void testUpdateNodeStatusOutOfOrder() throws InterruptedException { + // Add a connected node + final NodeIdentifier nodeId1 = createNodeId(1); + final NodeIdentifier nodeId2 = createNodeId(2); + + coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId1, NodeConnectionState.CONNECTED)); + coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId2, NodeConnectionState.CONNECTED)); + + // wait for the status change message and clear it + while (nodeStatusChangeMessages.size() < 2) { + Thread.sleep(10L); + } + nodeStatusChangeMessages.clear(); + + final NodeConnectionStatus oldStatus = new NodeConnectionStatus(-1L, nodeId1, NodeConnectionState.DISCONNECTED, + DisconnectionCode.BLOCKED_BY_FIREWALL, null, 0L, null); + final NodeStatusChangeMessage msg = new NodeStatusChangeMessage(); + msg.setNodeId(nodeId1); + msg.setNodeConnectionStatus(oldStatus); + coordinator.handle(msg); + + // Ensure that no status change message was send + Thread.sleep(1000); + assertTrue(nodeStatusChangeMessages.isEmpty()); + + // Status should not have changed because our status id is too small. + NodeConnectionStatus curStatus = coordinator.getConnectionStatus(nodeId1); + assertEquals(NodeConnectionState.CONNECTED, curStatus.getState()); + + // Verify that resetMap updates only the newer statuses + final NodeConnectionStatus node2Disconnecting = new NodeConnectionStatus(nodeId2, NodeConnectionState.DISCONNECTING); + final Map resetMap = new HashMap<>(); + resetMap.put(nodeId1, oldStatus); + resetMap.put(nodeId2, node2Disconnecting); + coordinator.resetNodeStatuses(resetMap); + + curStatus = coordinator.getConnectionStatus(nodeId1); + assertEquals(NodeConnectionState.CONNECTED, curStatus.getState()); + assertEquals(NodeConnectionState.DISCONNECTING, coordinator.getConnectionStatus(nodeId2).getState()); + } + + @Test(timeout = 5000) + public void testUpdateNodeRoles() throws InterruptedException { + // Add a connected node + final NodeIdentifier nodeId1 = createNodeId(1); + final NodeIdentifier nodeId2 = createNodeId(2); + + coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId1, NodeConnectionState.CONNECTED)); + // wait for the status change message and clear it + while (nodeStatusChangeMessages.isEmpty()) { + Thread.sleep(10L); + } + nodeStatusChangeMessages.clear(); + + coordinator.updateNodeStatus(new NodeConnectionStatus(nodeId2, NodeConnectionState.CONNECTED)); + // wait for the status change message and clear it + while (nodeStatusChangeMessages.isEmpty()) { + Thread.sleep(10L); + } + nodeStatusChangeMessages.clear(); + + // Update role of node 1 to primary node + coordinator.updateNodeRoles(nodeId1, Collections.singleton(ClusterRoles.PRIMARY_NODE)); + + // wait for the status change message + while (nodeStatusChangeMessages.isEmpty()) { + Thread.sleep(10L); + } + // verify the message + final NodeStatusChangeMessage msg = nodeStatusChangeMessages.get(0); + assertNotNull(msg); + assertEquals(nodeId1, msg.getNodeId()); + assertEquals(NodeConnectionState.CONNECTED, msg.getNodeConnectionStatus().getState()); + assertEquals(Collections.singleton(ClusterRoles.PRIMARY_NODE), msg.getNodeConnectionStatus().getRoles()); + nodeStatusChangeMessages.clear(); + + // Update role of node 2 to primary node. This should trigger 2 status changes - + // node 1 should lose primary role & node 2 should gain it + coordinator.updateNodeRoles(nodeId2, Collections.singleton(ClusterRoles.PRIMARY_NODE)); + + // wait for the status change message + while (nodeStatusChangeMessages.size() < 2) { + Thread.sleep(10L); + } + + final NodeStatusChangeMessage msg1 = nodeStatusChangeMessages.get(0); + final NodeStatusChangeMessage msg2 = nodeStatusChangeMessages.get(1); + final NodeStatusChangeMessage id1Msg = (msg1.getNodeId().equals(nodeId1)) ? msg1 : msg2; + final NodeStatusChangeMessage id2Msg = (msg1.getNodeId().equals(nodeId2)) ? msg1 : msg2; + + assertNotSame(id1Msg, id2Msg); + + assertTrue(id1Msg.getNodeConnectionStatus().getRoles().isEmpty()); + assertEquals(Collections.singleton(ClusterRoles.PRIMARY_NODE), id2Msg.getNodeConnectionStatus().getRoles()); + } + + + @Test + public void testProposedIdentifierResolvedIfConflict() { + final NodeIdentifier id1 = new NodeIdentifier("1234", "localhost", 8000, "localhost", 9000, "localhost", 10000, false); + final NodeIdentifier conflictingId = new NodeIdentifier("1234", "localhost", 8001, "localhost", 9000, "localhost", 10000, false); + + final ConnectionRequest connectionRequest = new ConnectionRequest(id1); + final ConnectionRequestMessage crm = new ConnectionRequestMessage(); + crm.setConnectionRequest(connectionRequest); + + final ProtocolMessage response = coordinator.handle(crm); + assertNotNull(response); + assertTrue(response instanceof ConnectionResponseMessage); + final ConnectionResponseMessage responseMessage = (ConnectionResponseMessage) response; + final NodeIdentifier resolvedNodeId = responseMessage.getConnectionResponse().getNodeIdentifier(); + assertEquals(id1, resolvedNodeId); + + final ConnectionRequest conRequest2 = new ConnectionRequest(conflictingId); + final ConnectionRequestMessage crm2 = new ConnectionRequestMessage(); + crm2.setConnectionRequest(conRequest2); + + final ProtocolMessage conflictingResponse = coordinator.handle(crm2); + assertNotNull(conflictingResponse); + assertTrue(conflictingResponse instanceof ConnectionResponseMessage); + final ConnectionResponseMessage conflictingResponseMessage = (ConnectionResponseMessage) conflictingResponse; + final NodeIdentifier conflictingNodeId = conflictingResponseMessage.getConnectionResponse().getNodeIdentifier(); + assertNotSame(id1.getId(), conflictingNodeId.getId()); + assertEquals(conflictingId.getApiAddress(), conflictingNodeId.getApiAddress()); + assertEquals(conflictingId.getApiPort(), conflictingNodeId.getApiPort()); + assertEquals(conflictingId.getSiteToSiteAddress(), conflictingNodeId.getSiteToSiteAddress()); + assertEquals(conflictingId.getSiteToSitePort(), conflictingNodeId.getSiteToSitePort()); + assertEquals(conflictingId.getSocketAddress(), conflictingNodeId.getSocketAddress()); + assertEquals(conflictingId.getSocketPort(), conflictingNodeId.getSocketPort()); + } + + + private NodeIdentifier createNodeId(final int index) { + return new NodeIdentifier(String.valueOf(index), "localhost", 8000 + index, "localhost", 9000 + index, "localhost", 10000 + index, false); + } + + private ProtocolMessage requestConnection(final NodeIdentifier requestedNodeId, final NodeClusterCoordinator coordinator) { + final ConnectionRequest request = new ConnectionRequest(requestedNodeId); + final ConnectionRequestMessage requestMsg = new ConnectionRequestMessage(); + requestMsg.setConnectionRequest(request); + return coordinator.handle(requestMsg); + } + +} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java deleted file mode 100644 index 6487a20d7891..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.event.impl; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -import org.apache.nifi.cluster.event.Event; -import org.apache.nifi.cluster.event.Event.Category; -import org.apache.nifi.cluster.event.EventManager; -import org.junit.Test; - -/** - */ -public class EventManagerImplTest { - - @Test(expected = IllegalArgumentException.class) - public void testNonPositiveHistorySize() { - new EventManagerImpl(0); - } - - @Test - public void testGetEventsUnknownSource() { - EventManager manager = new EventManagerImpl(1); - assertEquals(Collections.EMPTY_LIST, manager.getEvents("unknown value")); - } - - @Test - public void testGetEvents() { - - EventManager manager = new EventManagerImpl(2); - - Event e1 = new Event("1", "Event1", Category.INFO, 0); - Event e2 = new Event("1", "Event2", Category.INFO, 1); - - manager.addEvent(e1); - manager.addEvent(e2); - - List events = manager.getEvents("1"); - - // assert newest to oldest - assertEquals(Arrays.asList(e2, e1), events); - } - - @Test - public void testGetMostRecentEventUnknownSource() { - EventManager manager = new EventManagerImpl(1); - assertNull(manager.getMostRecentEvent("unknown value")); - } - - @Test - public void testGetMostRecentEvent() { - - EventManager manager = new EventManagerImpl(2); - - Event e1 = new Event("1", "Event1", Category.INFO, 0); - Event e2 = new Event("1", "Event2", Category.INFO, 1); - - manager.addEvent(e1); - manager.addEvent(e2); - - // assert newest to oldest - assertEquals(e2, manager.getMostRecentEvent("1")); - } - - @Test - public void testAddEventExceedsHistorySize() { - - EventManager manager = new EventManagerImpl(1); - - Event e1 = new Event("1", "Event1", Category.INFO, 0); - Event e2 = new Event("1", "Event2", Category.INFO, 1); - - manager.addEvent(e1); - manager.addEvent(e2); - - List events = manager.getEvents("1"); - - // assert oldest evicted - assertEquals(Arrays.asList(e2), events); - - } - - @Test - public void testClearHistory() { - - EventManager manager = new EventManagerImpl(1); - - Event e1 = new Event("1", "Event1", Category.INFO, 0); - Event e2 = new Event("1", "Event2", Category.INFO, 1); - - manager.addEvent(e1); - manager.addEvent(e2); - - manager.clearEventHistory("1"); - - // assert oldest evicted - assertTrue(manager.getEvents("1").isEmpty()); - - } - -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java deleted file mode 100644 index 34189acb1723..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java +++ /dev/null @@ -1,344 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.flow.impl; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; - -import java.io.ByteArrayInputStream; -import java.io.File; -import java.io.IOException; -import java.util.Arrays; -import java.util.HashSet; -import java.util.Set; - -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; -import javax.xml.parsers.ParserConfigurationException; - -import org.apache.commons.io.FileUtils; -import org.apache.nifi.cluster.flow.DataFlowDao; -import org.apache.nifi.cluster.flow.PersistedFlowState; -import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender; -import org.apache.nifi.cluster.protocol.NodeIdentifier; -import org.apache.nifi.cluster.protocol.ProtocolContext; -import org.apache.nifi.cluster.protocol.ProtocolException; -import org.apache.nifi.cluster.protocol.ProtocolHandler; -import org.apache.nifi.cluster.protocol.StandardDataFlow; -import org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderImpl; -import org.apache.nifi.cluster.protocol.impl.SocketProtocolListener; -import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; -import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; -import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; -import org.apache.nifi.cluster.protocol.message.ProtocolMessage; -import org.apache.nifi.io.socket.ServerSocketConfiguration; -import org.apache.nifi.io.socket.SocketConfiguration; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.w3c.dom.Document; -import org.w3c.dom.Element; -import org.xml.sax.SAXException; - -/** - */ -public class DataFlowManagementServiceImplTest { - - private DataFlowManagementServiceImpl service; - private File restoreLocation; - private File primaryLocation; - private DataFlowDao dao; - private int apiDummyPort; - private int socketPort; - private SocketConfiguration socketConfig; - private ClusterManagerProtocolSender sender; - private ServerSocketConfiguration serverSocketConfig; - private SocketProtocolListener listener; - - @Before - public void setup() throws IOException { - - primaryLocation = new File(System.getProperty("java.io.tmpdir") + "/primary" + this.getClass().getSimpleName()); - restoreLocation = new File(System.getProperty("java.io.tmpdir") + "/restore" + this.getClass().getSimpleName()); - - FileUtils.deleteDirectory(primaryLocation); - FileUtils.deleteDirectory(restoreLocation); - - ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); - - socketConfig = new SocketConfiguration(); - socketConfig.setSocketTimeout(1000); - serverSocketConfig = new ServerSocketConfiguration(); - - dao = new DataFlowDaoImpl(primaryLocation, restoreLocation, false); - - sender = new ClusterManagerProtocolSenderImpl(socketConfig, protocolContext); - - service = new DataFlowManagementServiceImpl(dao, sender); - service.start(); - - listener = new SocketProtocolListener(1, 0, serverSocketConfig, protocolContext); - listener.start(); - - apiDummyPort = 7777; - socketPort = listener.getPort(); - } - - @After - public void teardown() throws IOException { - - if (service != null && service.isRunning()) { - service.stop(); - } - - if (listener != null && listener.isRunning()) { - try { - listener.stop(); - } catch (final Exception ex) { - ex.printStackTrace(System.out); - } - } - FileUtils.deleteDirectory(primaryLocation); - FileUtils.deleteDirectory(restoreLocation); - - } - - @Test - public void testLoadFlowWithNonExistentFlow() throws ParserConfigurationException, SAXException, IOException { - verifyFlow(); - } - - @Test - public void testLoadFlowWithNonExistentFlowWhenServiceStopped() throws IOException, SAXException, ParserConfigurationException { - service.stop(); - verifyFlow(); - } - - private void verifyFlow() throws ParserConfigurationException, SAXException, IOException { - final byte[] flowBytes = service.loadDataFlow().getDataFlow().getFlow(); - final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance(); - docFactory.setNamespaceAware(true); - - final DocumentBuilder docBuilder = docFactory.newDocumentBuilder(); - final Document doc = docBuilder.parse(new ByteArrayInputStream(flowBytes)); - final Element controller = (Element) doc.getElementsByTagName("flowController").item(0); - final Element rootGroup = (Element) controller.getElementsByTagName("rootGroup").item(0); - final String rootGroupName = rootGroup.getElementsByTagName("name").item(0).getTextContent(); - assertEquals("NiFi Flow", rootGroupName); - } - - @Test - public void testLoadFlowSingleNode() throws Exception { - String flowStr = ""; - byte[] flowBytes = flowStr.getBytes(); - listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0]))); - - NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false); - service.setNodeIds(new HashSet<>(Arrays.asList(nodeId))); - service.setPersistedFlowState(PersistedFlowState.STALE); - - assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); - - // sleep long enough for the flow retriever to run - waitForState(PersistedFlowState.CURRENT); - - assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState()); - assertArrayEquals(flowBytes, service.loadDataFlow().getDataFlow().getFlow()); - - } - - @Test - public void testLoadFlowWithSameNodeIds() throws Exception { - - String flowStr = ""; - listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0]))); - - NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false); - NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false); - service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2))); - service.setPersistedFlowState(PersistedFlowState.STALE); - - assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); - - // sleep long enough for the flow retriever to run - waitForState(PersistedFlowState.CURRENT); - - // verify that flow is current - assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState()); - - // add same ids in different order - service.setNodeIds(new HashSet<>(Arrays.asList(nodeId2, nodeId1))); - - // verify flow is still current - assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState()); - - } - - @Test - public void testLoadFlowWithABadNode() throws Exception { - - String flowStr = ""; - byte[] flowBytes = flowStr.getBytes(); - listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0]))); - - NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1, "localhost", 1234, false); - NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false); - service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2))); - service.setPersistedFlowState(PersistedFlowState.STALE); - - assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); - - // sleep long enough for the flow retriever to run - waitForState(PersistedFlowState.CURRENT); - - assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState()); - assertArrayEquals(flowBytes, service.loadDataFlow().getDataFlow().getFlow()); - - } - - @Test - public void testLoadFlowWithConstantNodeIdChanging() throws Exception { - String flowStr = ""; - byte[] flowBytes = flowStr.getBytes(); - listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0]))); - - NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1, "localhost", 1234, false); - NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false); - - for (int i = 0; i < 1000; i++) { - service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2))); - service.setPersistedFlowState(PersistedFlowState.STALE); - assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); - } - - // sleep long enough for the flow retriever to run - waitForState(PersistedFlowState.CURRENT); - - assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState()); - assertArrayEquals(flowBytes, service.loadDataFlow().getDataFlow().getFlow()); - } - - @Test - public void testLoadFlowWithConstantNodeIdChangingWithRetrievalDelay() throws Exception { - - String flowStr = ""; - listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0]))); - - NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1, "localhost", 1234, false); - NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false); - - service.setRetrievalDelay("5 sec"); - for (int i = 0; i < 1000; i++) { - service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2))); - service.setPersistedFlowState(PersistedFlowState.STALE); - assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); - } - - // sleep long enough for the flow retriever to run - waitForState(PersistedFlowState.STALE); - - assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); - - } - - @Test - public void testStopRequestedWhileRetrieving() throws Exception { - - String flowStr = ""; - listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0]))); - Set nodeIds = new HashSet<>(); - for (int i = 0; i < 1000; i++) { - nodeIds.add(new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1, "localhost", 1234, false)); - } - nodeIds.add(new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false)); - - long lastRetrievalTime = service.getLastRetrievalTime(); - - service.setNodeIds(nodeIds); - service.setPersistedFlowState(PersistedFlowState.STALE); - assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); - - // sleep long enough for the flow retriever to run - waitForState(PersistedFlowState.STALE); - - service.stop(); - - service.setPersistedFlowState(PersistedFlowState.STALE); - assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); - - assertEquals(lastRetrievalTime, service.getLastRetrievalTime()); - - } - - @Test - public void testLoadFlowUnknownState() throws Exception { - - String flowStr = ""; - byte[] flowBytes = flowStr.getBytes(); - listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0]))); - NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort, "localhost", 1234, false); - - service.setNodeIds(new HashSet<>(Arrays.asList(nodeId))); - service.setPersistedFlowState(PersistedFlowState.STALE); - assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); - - service.setPersistedFlowState(PersistedFlowState.UNKNOWN); - - assertEquals(PersistedFlowState.UNKNOWN, service.getPersistedFlowState()); - - service.setPersistedFlowState(PersistedFlowState.STALE); - assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); - - // sleep long enough for the flow retriever to run - waitForState(PersistedFlowState.CURRENT); - - assertArrayEquals(flowBytes, service.loadDataFlow().getDataFlow().getFlow()); - - } - - private class FlowRequestProtocolHandler implements ProtocolHandler { - - private StandardDataFlow dataFlow; - - public FlowRequestProtocolHandler(final StandardDataFlow dataFlow) { - this.dataFlow = dataFlow; - } - - @Override - public boolean canHandle(ProtocolMessage msg) { - return true; - } - - @Override - public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { - FlowResponseMessage response = new FlowResponseMessage(); - response.setDataFlow(dataFlow); - return response; - } - - } - - private void waitForState(PersistedFlowState state) throws InterruptedException { - for (int i = 0; i < 30; i++) { - if (service.getPersistedFlowState() == state) { - break; - } else { - Thread.sleep(1000); - } - } - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java deleted file mode 100644 index 6b93ef39b296..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java +++ /dev/null @@ -1,374 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.manager.impl; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import javax.ws.rs.HttpMethod; -import javax.ws.rs.core.MultivaluedMap; -import javax.ws.rs.core.Response; -import javax.ws.rs.core.Response.Status; -import javax.ws.rs.core.StreamingOutput; -import javax.xml.bind.annotation.XmlRootElement; - -import org.apache.nifi.cluster.manager.NodeResponse; -import org.apache.nifi.cluster.manager.testutils.HttpResponse; -import org.apache.nifi.cluster.manager.testutils.HttpResponseAction; -import org.apache.nifi.cluster.manager.testutils.HttpServer; -import org.apache.nifi.cluster.protocol.NodeIdentifier; -import org.apache.nifi.util.NiFiProperties; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import com.sun.jersey.api.client.Client; - -/** - */ -public class HttpRequestReplicatorImplTest { - - private Client client; - private HttpRequestReplicatorImpl replicator; - private int executorThreadCount; - private int serverThreadCount; - private int serverPort; - private HttpServer server; - private Map> expectedRequestParameters; - private Map expectedRequestHeaders; - private Map expectedResponseHeaders; - private Object expectedEntity; - private String expectedBody; - private URI prototypeUri; - - @Before - public void setUp() throws IOException, URISyntaxException { - System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, getClass().getResource("/conf/nifi.properties").getFile()); - - executorThreadCount = 5; - serverThreadCount = 3; - - client = Client.create(); - - replicator = new HttpRequestReplicatorImpl(executorThreadCount, client, "1 sec", "1 sec"); - replicator.start(); - - expectedRequestHeaders = new HashMap<>(); - expectedRequestHeaders.put("header1", "header value1"); - expectedRequestHeaders.put("header2", "header value2"); - - expectedRequestParameters = new HashMap<>(); - expectedRequestParameters.put("param1", Arrays.asList("p value1")); - expectedRequestParameters.put("param2", Arrays.asList("p value2")); - - expectedResponseHeaders = new HashMap<>(); - expectedResponseHeaders.put("header1", "header value1"); - expectedResponseHeaders.put("header2", "header value2"); - - expectedEntity = new Entity(); - - expectedBody = "some text"; - - prototypeUri = new URI("http://prototype.host/path/to/resource"); - - server = new HttpServer(serverThreadCount, 0); - server.start(); - serverPort = server.getPort(); - } - - @After - public void teardown() { - if (server.isRunning()) { - server.stop(); - } - if (replicator.isRunning()) { - replicator.stop(); - } - } - - @Test - public void testReplicateGetLessNodesThanReplicatorThreads() throws Throwable { - testReplicateXXX(executorThreadCount - 1, HttpMethod.GET); - } - - @Test - public void testReplicateGetMoreNodesThanReplicatorThreads() throws Throwable { - testReplicateXXX(executorThreadCount + 1, HttpMethod.GET); - } - - @Test - public void testReplicateGetWithUnresponsiveNode() throws Throwable { - - // nodes - Set nodeIds = createNodes(2, "localhost", serverPort); - - // response - HttpResponse expectedResponse = new HttpResponse(Status.OK, expectedBody); - - // first response normal, second response slow - server.addResponseAction(new HttpResponseAction(expectedResponse)); - server.addResponseAction(new HttpResponseAction(expectedResponse, 3500)); - - Set responses = replicator.replicate( - nodeIds, - HttpMethod.GET, - prototypeUri, - expectedRequestParameters, - expectedRequestHeaders); - - assertEquals(nodeIds.size(), responses.size()); - - Iterator nodeResponseItr = responses.iterator(); - - NodeResponse firstResponse = nodeResponseItr.next(); - NodeResponse secondResponse = nodeResponseItr.next(); - NodeResponse goodResponse; - NodeResponse badResponse; - if (firstResponse.hasThrowable()) { - goodResponse = secondResponse; - badResponse = firstResponse; - } else { - goodResponse = firstResponse; - badResponse = secondResponse; - } - - // good response - // check status - assertEquals(Status.OK.getStatusCode(), goodResponse.getStatus()); - - // check entity stream - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ((StreamingOutput) goodResponse.getResponse().getEntity()).write(baos); - assertEquals("some text", new String(baos.toByteArray())); - - // bad response - assertTrue(badResponse.hasThrowable()); - assertEquals(Status.INTERNAL_SERVER_ERROR.getStatusCode(), badResponse.getStatus()); - - } - - @Test(expected = IllegalArgumentException.class) - public void testReplicateGetWithEntity() throws Throwable { - testReplicateXXXEntity(HttpMethod.GET); - } - - @Test - public void testReplicatePost() throws Throwable { - testReplicateXXX(HttpMethod.POST); - } - - @Test - public void testReplicatePostWithEntity() throws Throwable { - testReplicateXXXEntity(HttpMethod.POST); - } - - @Test - public void testReplicatePut() throws Throwable { - testReplicateXXX(HttpMethod.PUT); - } - - @Test - public void testReplicatePutWithEntity() throws Throwable { - testReplicateXXXEntity(HttpMethod.PUT); - } - - @Test - public void testReplicateDelete() throws Throwable { - testReplicateXXX(HttpMethod.DELETE); - } - - @Test(expected = IllegalArgumentException.class) - public void testReplicateDeleteWithEntity() throws Throwable { - testReplicateXXXEntity(HttpMethod.DELETE); - } - - @Test - public void testReplicateHead() throws Throwable { - testReplicateXXX(HttpMethod.HEAD); - } - - @Test(expected = IllegalArgumentException.class) - public void testReplicateHeadWithEntity() throws Throwable { - testReplicateXXXEntity(HttpMethod.HEAD); - } - - @Test - public void testReplicateOptions() throws Throwable { - testReplicateXXX(HttpMethod.OPTIONS); - } - - @Test(expected = IllegalArgumentException.class) - public void testReplicateOptionsWithEntity() throws Throwable { - testReplicateXXXEntity(HttpMethod.OPTIONS); - } - - private void testReplicateXXX(final String method) throws Throwable { - testReplicateXXX(executorThreadCount, method); - } - - private void testReplicateXXX(final int numNodes, final String method) throws Throwable { - - // nodes - Set nodeIds = createNodes(numNodes, "localhost", serverPort); - - // set up responses - for (int i = 0; i < nodeIds.size(); i++) { - HttpResponse response = new HttpResponse(Status.OK, expectedBody); - response.addHeaders(expectedResponseHeaders); - server.addResponseAction(new HttpResponseAction(response)); - } - - // setup request parameters - server.addCheckedParameters(expectedRequestParameters); - - // request headers - server.addCheckedHeaders(expectedRequestHeaders); - - Set responses = replicator.replicate( - nodeIds, - method, - prototypeUri, - expectedRequestParameters, - expectedRequestHeaders); - - Set returnedNodeIds = new HashSet<>(); - for (NodeResponse response : responses) { - - // check if we received an exception - if (response.hasThrowable()) { - throw response.getThrowable(); - } - - // gather ids to verify later - returnedNodeIds.add(response.getNodeId()); - - // check status - assertEquals(Status.OK.getStatusCode(), response.getStatus()); - - Response serverResponse = response.getResponse(); - - // check response headers are copied - assertTrue(containsHeaders(expectedResponseHeaders, serverResponse.getMetadata())); - - // check entity stream - if (HttpMethod.HEAD.equalsIgnoreCase(method)) { - assertNull(serverResponse.getEntity()); - } else { - assertTrue(isEquals((StreamingOutput) serverResponse.getEntity(), expectedBody)); - } - - } - - // check node Ids - assertEquals(nodeIds, returnedNodeIds); - } - - private void testReplicateXXXEntity(final String method) throws Throwable { - testReplicateXXXEntity(executorThreadCount, method); - } - - private void testReplicateXXXEntity(final int numNodes, final String method) throws Throwable { - - // nodes - Set nodeIds = createNodes(numNodes, "localhost", serverPort); - - // set up responses - for (int i = 0; i < nodeIds.size(); i++) { - HttpResponse response = new HttpResponse(Status.OK, expectedBody); - response.addHeaders(expectedResponseHeaders); - server.addResponseAction(new HttpResponseAction(response)); - } - - // headers - expectedRequestHeaders.put("Content-Type", "application/xml"); - - Set responses = replicator.replicate( - nodeIds, - method, - prototypeUri, - expectedEntity, - expectedRequestHeaders); - - Set returnedNodeIds = new HashSet<>(); - for (NodeResponse response : responses) { - - // check if we received an exception - if (response.hasThrowable()) { - throw response.getThrowable(); - } - - // gather ids to verify later - returnedNodeIds.add(response.getNodeId()); - - // check status - assertEquals(Status.OK.getStatusCode(), response.getStatus()); - - Response serverResponse = response.getResponse(); - - // check response headers are copied - assertTrue(containsHeaders(expectedResponseHeaders, serverResponse.getMetadata())); - - // check entity stream - assertTrue(isEquals((StreamingOutput) serverResponse.getEntity(), expectedBody)); - - } - - // check node Ids - assertEquals(nodeIds, returnedNodeIds); - } - - private Set createNodes(int num, String host, int apiPort) { - Set result = new HashSet<>(); - for (int i = 0; i < num; i++) { - result.add(new NodeIdentifier(String.valueOf(i), host, apiPort, host, 1, "localhost", 1234, false)); - } - return result; - } - - private boolean isEquals(StreamingOutput so, String expectedText) throws IOException { - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - so.write(baos); - return expectedText.equals(new String(baos.toByteArray())); - } - - private boolean containsHeaders(Map expectedHeaders, MultivaluedMap metadata) { - for (Map.Entry expectedEntry : expectedHeaders.entrySet()) { - if (expectedEntry.getValue().equals(metadata.getFirst(expectedEntry.getKey())) == false) { - return false; - } - } - return true; - } - -} - -@XmlRootElement -class Entity { -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java deleted file mode 100644 index ba6ca829da24..000000000000 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.nifi.cluster.manager.impl; - -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.io.ByteArrayInputStream; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -import org.apache.nifi.cluster.manager.NodeResponse; -import org.apache.nifi.cluster.node.Node; -import org.apache.nifi.cluster.node.Node.Status; -import org.apache.nifi.cluster.protocol.NodeIdentifier; -import org.junit.Before; -import org.junit.Test; - -import com.sun.jersey.api.client.ClientResponse; -import com.sun.jersey.core.util.MultivaluedMapImpl; - -/** - */ -public class HttpResponseMapperImplTest { - - private HttpResponseMapperImpl mapper; - - private URI dummyUri; - - @Before - public void setup() throws URISyntaxException { - mapper = new HttpResponseMapperImpl(); - dummyUri = new URI("http://dummy.com"); - } - - @Test - public void testToNodeStatusWithNo2xxResponses() { - - Set nodeResponses = new HashSet<>(); - nodeResponses.add(createNodeResourceResponse("1", 400)); - nodeResponses.add(createNodeResourceResponse("2", 100)); - nodeResponses.add(createNodeResourceResponse("3", 300)); - nodeResponses.add(createNodeResourceResponse("4", 500)); - - Map map = mapper.map(dummyUri, nodeResponses); - - // since no 2xx responses, any 5xx is disconnected - for (Map.Entry entry : map.entrySet()) { - NodeResponse response = entry.getKey(); - Status status = entry.getValue(); - switch (response.getNodeId().getId()) { - case "1": - assertTrue(status == Node.Status.CONNECTED); - break; - case "2": - assertTrue(status == Node.Status.CONNECTED); - break; - case "3": - assertTrue(status == Node.Status.CONNECTED); - break; - case "4": - assertTrue(status == Node.Status.DISCONNECTED); - break; - } - } - } - - @Test - public void testToNodeStatusWith2xxResponses() { - - Set nodeResponses = new HashSet<>(); - nodeResponses.add(createNodeResourceResponse("1", 200)); - nodeResponses.add(createNodeResourceResponse("2", 100)); - nodeResponses.add(createNodeResourceResponse("3", 300)); - nodeResponses.add(createNodeResourceResponse("4", 500)); - - Map map = mapper.map(dummyUri, nodeResponses); - - // since there were 2xx responses, any non-2xx is disconnected - for (Map.Entry entry : map.entrySet()) { - NodeResponse response = entry.getKey(); - Status status = entry.getValue(); - switch (response.getNodeId().getId()) { - case "1": - assertTrue(status == Node.Status.CONNECTED); - break; - case "2": - assertTrue(status == Node.Status.DISCONNECTED); - break; - case "3": - assertTrue(status == Node.Status.DISCONNECTED); - break; - case "4": - assertTrue(status == Node.Status.DISCONNECTED); - break; - } - } - } - - private NodeResponse createNodeResourceResponse(String nodeId, int statusCode) { - - ClientResponse clientResponse = mock(ClientResponse.class); - when(clientResponse.getStatus()).thenReturn(statusCode); - when(clientResponse.getHeaders()).thenReturn(new MultivaluedMapImpl()); - when(clientResponse.getEntityInputStream()).thenReturn(new ByteArrayInputStream(new byte[0])); - - NodeIdentifier nodeIdentifier = new NodeIdentifier(nodeId, "localhost", 1, "localhost", 1, "localhost", 1234, false); - return new NodeResponse(nodeIdentifier, "GET", dummyUri, clientResponse, 1L, "111"); - } -} diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycle.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/lifecycle/LifeCycle.java similarity index 100% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycle.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/lifecycle/LifeCycle.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/lifecycle/LifeCycleException.java similarity index 100% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleException.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/lifecycle/LifeCycleException.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleStartException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/lifecycle/LifeCycleStartException.java similarity index 100% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleStartException.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/lifecycle/LifeCycleStartException.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleStopException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/lifecycle/LifeCycleStopException.java similarity index 100% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleStopException.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/lifecycle/LifeCycleStopException.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/services/FlowService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/services/FlowService.java similarity index 83% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/services/FlowService.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/services/FlowService.java index 08ea7da26008..be562102fabf 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/services/FlowService.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/services/FlowService.java @@ -22,10 +22,6 @@ import java.util.concurrent.TimeUnit; import org.apache.nifi.cluster.protocol.DataFlow; -import org.apache.nifi.controller.FlowController; -import org.apache.nifi.controller.UninheritableFlowException; -import org.apache.nifi.controller.serialization.FlowSerializationException; -import org.apache.nifi.controller.serialization.FlowSynchronizationException; import org.apache.nifi.lifecycle.LifeCycle; /** @@ -95,12 +91,8 @@ public interface FlowService extends LifeCycle { * @param proposedFlow the flow to load * * @throws IOException if flow configuration could not be retrieved from disk - * @throws FlowSerializationException if proposed flow is not a valid flow configuration file - * @throws UninheritableFlowException if the proposed flow cannot be loaded by the controller because in doing so would risk orphaning flow files - * @throws FlowSynchronizationException if updates to the controller failed. If this exception is thrown, then the controller should be considered unsafe to be used */ - void load(DataFlow proposedFlow) - throws IOException, FlowSerializationException, FlowSynchronizationException, UninheritableFlowException; + void load(DataFlow proposedFlow) throws IOException; /** * Copies the contents of the current flow.xml to the given OutputStream @@ -110,15 +102,19 @@ void load(DataFlow proposedFlow) */ void copyCurrentFlow(OutputStream os) throws IOException; - /** - * @return the managed controller - */ - FlowController getController(); - /** * Creates a copy of the current flow and saves it in the configured 'archive' directory * * @throws IOException if unable to write to the archive directory */ void archiveFlow() throws IOException; + + /** + * Creates a DataFlow object from the current flow + * + * @return the created DataFlow object + * + * @throws IOException if unable to read the flow from disk + */ + DataFlow createDataFlow() throws IOException; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/FlowModification.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/FlowModification.java similarity index 100% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/FlowModification.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/FlowModification.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/DeleteRevisionTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/DeleteRevisionTask.java similarity index 100% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/DeleteRevisionTask.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/DeleteRevisionTask.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/ExpiredRevisionClaimException.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/ExpiredRevisionClaimException.java similarity index 100% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/ExpiredRevisionClaimException.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/ExpiredRevisionClaimException.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/ReadOnlyRevisionCallback.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/ReadOnlyRevisionCallback.java similarity index 100% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/ReadOnlyRevisionCallback.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/ReadOnlyRevisionCallback.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/RevisionClaim.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/RevisionClaim.java similarity index 100% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/RevisionClaim.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/RevisionClaim.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/RevisionManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/RevisionManager.java similarity index 96% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/RevisionManager.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/RevisionManager.java index ae503a4f82c2..758529923b71 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/RevisionManager.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/RevisionManager.java @@ -18,6 +18,7 @@ package org.apache.nifi.web.revision; import java.util.Collection; +import java.util.List; import java.util.Set; import java.util.function.Supplier; @@ -207,4 +208,15 @@ public interface RevisionManager { * @return true if all claims were released, false otherwise */ boolean cancelClaims(Set revisions); + + /** + * Clears any revisions that are currently held and resets the Revision Manager so that the revisions + * present are those provided in the given collection + */ + void reset(Collection revisions); + + /** + * @return a List of all Revisions managed by this Revision Manager + */ + List getAllRevisions(); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/RevisionUpdate.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/RevisionUpdate.java similarity index 100% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/RevisionUpdate.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/RevisionUpdate.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/UpdateRevisionTask.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/UpdateRevisionTask.java similarity index 100% rename from nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-optimistic-locking/src/main/java/org/apache/nifi/web/revision/UpdateRevisionTask.java rename to nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/web/revision/UpdateRevisionTask.java diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 331140fe952a..162035f29ec6 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -16,7 +16,40 @@ */ package org.apache.nifi.controller; -import com.sun.jersey.api.client.ClientHandlerException; +import static java.util.Objects.requireNonNull; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.text.DateFormat; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Date; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.LockSupport; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import javax.net.ssl.SSLContext; + import org.apache.commons.lang3.StringUtils; import org.apache.nifi.action.Action; import org.apache.nifi.admin.service.AuditService; @@ -31,6 +64,8 @@ import org.apache.nifi.authorization.resource.Authorizable; import org.apache.nifi.authorization.resource.ResourceFactory; import org.apache.nifi.cluster.HeartbeatPayload; +import org.apache.nifi.cluster.coordination.heartbeat.HeartbeatMonitor; +import org.apache.nifi.cluster.coordination.node.ClusterRoles; import org.apache.nifi.cluster.coordination.node.DisconnectionCode; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; @@ -121,7 +156,6 @@ import org.apache.nifi.engine.FlowEngine; import org.apache.nifi.events.BulletinFactory; import org.apache.nifi.events.EventReporter; -import org.apache.nifi.events.VolatileBulletinRepository; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.framework.security.util.SslContextFactory; @@ -193,39 +227,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.net.ssl.SSLContext; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.text.DateFormat; -import java.text.SimpleDateFormat; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.Date; -import java.util.HashSet; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.Future; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.LockSupport; -import java.util.concurrent.locks.ReentrantReadWriteLock; - -import static java.util.Objects.requireNonNull; +import com.sun.jersey.api.client.ClientHandlerException; public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider, QueueProvider, Authorizable { @@ -243,7 +245,6 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R public static final String ROOT_GROUP_ID_ALIAS = "root"; public static final String DEFAULT_ROOT_GROUP_NAME = "NiFi Flow"; - public static final String PRIMARY_NODE_ROLE_NAME = "primary-node"; // default properties for scaling the positions of components from pre-1.0 flow encoding versions. public static final double DEFAULT_POSITION_SCALE_FACTOR_X = 1.5; @@ -258,7 +259,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R private final FlowFileRepository flowFileRepository; private final FlowFileEventRepository flowFileEventRepository; private final ProvenanceEventRepository provenanceEventRepository; - private final VolatileBulletinRepository bulletinRepository; + private final BulletinRepository bulletinRepository; private final StandardProcessScheduler processScheduler; private final SnippetManager snippetManager; private final long gracefulShutdownSeconds; @@ -295,7 +296,6 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R private final List startRemoteGroupPortsAfterInitialization; private final LeaderElectionManager leaderElectionManager; - /** * true if controller is configured to operate in a clustered environment */ @@ -312,7 +312,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R private final StringEncryptor encryptor; - private final ScheduledExecutorService clusterTaskExecutor = new FlowEngine(3, "Clustering Tasks"); + private final ScheduledExecutorService clusterTaskExecutor = new FlowEngine(3, "Clustering Tasks", true); private final ResourceClaimManager resourceClaimManager = new StandardResourceClaimManager(); // guarded by rwLock @@ -321,6 +321,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R */ private ScheduledFuture heartbeatSenderFuture; private final Heartbeater heartbeater; + private final HeartbeatMonitor heartbeatMonitor; // guarded by FlowController lock /** @@ -332,7 +333,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R /** * the node identifier; */ - private NodeIdentifier nodeId; + private volatile NodeIdentifier nodeId; // guarded by rwLock /** @@ -343,7 +344,6 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R // guarded by rwLock private NodeConnectionStatus connectionStatus; - private final ConcurrentMap nodeStatuses = new ConcurrentHashMap<>(); // guarded by rwLock private String instanceId; @@ -362,7 +362,8 @@ public static FlowController createStandaloneInstance( final NiFiProperties properties, final KeyService keyService, final AuditService auditService, - final StringEncryptor encryptor) { + final StringEncryptor encryptor, + final BulletinRepository bulletinRepo) { return new FlowController( flowFileEventRepo, properties, @@ -370,7 +371,9 @@ public static FlowController createStandaloneInstance( auditService, encryptor, /* configuredForClustering */ false, - /* NodeProtocolSender */ null); + /* NodeProtocolSender */ null, + bulletinRepo, + /* heartbeat monitor */ null); } public static FlowController createClusteredInstance( @@ -379,7 +382,9 @@ public static FlowController createClusteredInstance( final KeyService keyService, final AuditService auditService, final StringEncryptor encryptor, - final NodeProtocolSender protocolSender) { + final NodeProtocolSender protocolSender, + final BulletinRepository bulletinRepo, + final HeartbeatMonitor heartbeatMonitor) { final FlowController flowController = new FlowController( flowFileEventRepo, properties, @@ -387,7 +392,9 @@ public static FlowController createClusteredInstance( auditService, encryptor, /* configuredForClustering */ true, - /* NodeProtocolSender */ protocolSender); + protocolSender, + bulletinRepo, + heartbeatMonitor); flowController.setClusterManagerRemoteSiteInfo(properties.getRemoteInputPort(), properties.isSiteToSiteSecure()); @@ -401,13 +408,16 @@ private FlowController( final AuditService auditService, final StringEncryptor encryptor, final boolean configuredForClustering, - final NodeProtocolSender protocolSender) { + final NodeProtocolSender protocolSender, + final BulletinRepository bulletinRepo, + final HeartbeatMonitor heartbeatMonitor) { maxTimerDrivenThreads = new AtomicInteger(10); maxEventDrivenThreads = new AtomicInteger(5); this.encryptor = encryptor; this.properties = properties; + this.heartbeatMonitor = heartbeatMonitor; sslContext = SslContextFactory.createSslContext(properties, false); extensionManager = new ExtensionManager(); @@ -419,7 +429,7 @@ private FlowController( flowFileEventRepository = flowFileEventRepo; counterRepositoryRef = new AtomicReference(new StandardCounterRepository()); - bulletinRepository = new VolatileBulletinRepository(); + bulletinRepository = bulletinRepo; try { this.provenanceEventRepository = createProvenanceRepository(properties); @@ -533,11 +543,13 @@ public void run() { } }, snapshotMillis, snapshotMillis, TimeUnit.MILLISECONDS); - heartbeatBeanRef.set(new HeartbeatBean(rootGroup, false, new NodeConnectionStatus(NodeConnectionState.DISCONNECTED, DisconnectionCode.NOT_YET_CONNECTED))); + heartbeatBeanRef.set(new HeartbeatBean(rootGroup, false, new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED))); if (configuredForClustering) { leaderElectionManager = new CuratorLeaderElectionManager(4); heartbeater = new ClusterProtocolHeartbeater(protocolSender, properties); + registerForClusterCoordinator(); + leaderElectionManager.start(); } else { leaderElectionManager = null; heartbeater = null; @@ -554,6 +566,10 @@ public Resource getResource() { return ResourceFactory.getControllerResource(); } + public HeartbeatMonitor getHeartbeatMonitor() { + return heartbeatMonitor; + } + private static FlowFileRepository createFlowFileRepository(final NiFiProperties properties, final ResourceClaimManager contentClaimManager) { final String implementationClassName = properties.getProperty(NiFiProperties.FLOWFILE_REPOSITORY_IMPLEMENTATION, DEFAULT_FLOWFILE_REPO_IMPLEMENTATION); if (implementationClassName == null) { @@ -1188,10 +1204,6 @@ public void shutdown(final boolean kill) { throw new IllegalStateException("Controller already stopped or still stopping..."); } - if (heartbeater != null) { - sendShutdownNotification(); - } - if (leaderElectionManager != null) { leaderElectionManager.stop(); } @@ -1206,7 +1218,7 @@ public void shutdown(final boolean kill) { LOG.info("Initiated graceful shutdown of flow controller...waiting up to " + gracefulShutdownSeconds + " seconds"); } - clusterTaskExecutor.shutdown(); + clusterTaskExecutor.shutdownNow(); if (zooKeeperStateServer != null) { zooKeeperStateServer.shutdown(); @@ -1280,43 +1292,6 @@ public void shutdown(final boolean kill) { } } - /** - * Sends a notification to the cluster that the node was shut down. - */ - private void sendShutdownNotification() { - // Generate a heartbeat message and publish it, indicating that we are shutting down - final HeartbeatMessage heartbeatMsg = createHeartbeatMessage(); - if (heartbeatMsg == null) { - LOG.warn("Cannot sent Shutdown Notification Message because node's identifier is not known at this time"); - return; - } - - final Heartbeat heartbeat = heartbeatMsg.getHeartbeat(); - final byte[] hbPayload = heartbeatMsg.getHeartbeat().getPayload(); - final NodeConnectionStatus connectionStatus = new NodeConnectionStatus(DisconnectionCode.NODE_SHUTDOWN); - heartbeatMsg.setHeartbeat(new Heartbeat(heartbeat.getNodeIdentifier(), false, connectionStatus, hbPayload)); - final Runnable sendNotification = new Runnable() { - @Override - public void run() { - try { - heartbeater.send(heartbeatMsg); - } catch (IOException e) { - LOG.warn("Failed to send NODE_SHUTDOWN heartbeat message. Cluster may not be notified of " - + "shutdown and may have to wait for the heartbeats to time out before noticing that the node left the cluster"); - } - } - }; - - final Future hbFuture = processScheduler.submitFrameworkTask(sendNotification); - try { - hbFuture.get(3, TimeUnit.SECONDS); - LOG.info("Successfully sent Shutdown Notification to cluster"); - } catch (final Exception e) { - LOG.warn("Failed to send NODE_SHUTDOWN heartbeat message in time. Cluster may not be notified of " - + "shutdown and may have to wait for the heartbeats to time out before noticing that the node left the cluster"); - } - } - /** * Serializes the current state of the controller to the given OutputStream @@ -3036,12 +3011,7 @@ public int getHeartbeatDelaySeconds() { * @return the node identifier or null if no identifier is set */ public NodeIdentifier getNodeId() { - readLock.lock(); - try { - return nodeId; - } finally { - readLock.unlock(); - } + return nodeId; } /** @@ -3050,12 +3020,7 @@ public NodeIdentifier getNodeId() { * @param nodeId the node identifier, which may be null */ public void setNodeId(final NodeIdentifier nodeId) { - writeLock.lock(); - try { - this.nodeId = nodeId; - } finally { - writeLock.unlock(); - } + this.nodeId = nodeId; } /** @@ -3093,6 +3058,20 @@ public void setClustered(final boolean clustered, final String clusterInstanceId setClustered(clustered, clusterInstanceId, null); } + private void registerForClusterCoordinator() { + leaderElectionManager.register(ClusterRoles.CLUSTER_COORDINATOR, new LeaderElectionStateChangeListener() { + @Override + public void onLeaderRelinquish() { + heartbeatMonitor.stop(); + } + + @Override + public void onLeaderElection() { + heartbeatMonitor.start(); + } + }); + } + /** * Sets whether this instance is clustered. Clustered means that a node is either connected or trying to connect to the cluster. * @@ -3129,7 +3108,7 @@ public void setClustered(final boolean clustered, final String clusterInstanceId // update the bulletin repository if (isChanging) { if (clustered) { - leaderElectionManager.register(PRIMARY_NODE_ROLE_NAME, new LeaderElectionStateChangeListener() { + leaderElectionManager.register(ClusterRoles.PRIMARY_NODE, new LeaderElectionStateChangeListener() { @Override public void onLeaderElection() { setPrimary(true); @@ -3141,9 +3120,14 @@ public void onLeaderRelinquish() { } }); + // Participate in Leader Election for Heartbeat Monitor. Start the heartbeat monitor + // if/when we become leader and stop it when we lose leader role + registerForClusterCoordinator(); + leaderElectionManager.start(); stateManagerProvider.enableClusterProvider(); + // Start ZooKeeper State Server if necessary if (zooKeeperStateServer != null) { processScheduler.submitFrameworkTask(new Runnable() { @Override @@ -3179,8 +3163,11 @@ public void run() { // showing failures. This 1-second sleep is an attempt to at least make that occurrence rare. LockSupport.parkNanos(TimeUnit.SECONDS.toNanos(1L)); } + + heartbeat(); } else { - leaderElectionManager.unregister(PRIMARY_NODE_ROLE_NAME); + leaderElectionManager.unregister(ClusterRoles.PRIMARY_NODE); + leaderElectionManager.unregister(ClusterRoles.CLUSTER_COORDINATOR); if (zooKeeperStateServer != null) { zooKeeperStateServer.shutdown(); @@ -3207,7 +3194,7 @@ public void run() { * @return true if this instance is the primary node in the cluster; false otherwise */ public boolean isPrimary() { - return leaderElectionManager != null && leaderElectionManager.isLeader(PRIMARY_NODE_ROLE_NAME); + return leaderElectionManager != null && leaderElectionManager.isLeader(ClusterRoles.PRIMARY_NODE); } public void setPrimary(final boolean primary) { @@ -3663,8 +3650,16 @@ public void run() { final long sendNanos = System.nanoTime() - sendStart; final long sendMillis = TimeUnit.NANOSECONDS.toMillis(sendNanos); - heartbeatLogger.info("Heartbeat created at {} and sent at {}; send took {} millis", + String heartbeatAddress; + try { + heartbeatAddress = heartbeater.getHeartbeatAddress(); + } catch (final IOException ioe) { + heartbeatAddress = "Cluster Coordinator (could not determine socket address)"; + } + + heartbeatLogger.info("Heartbeat created at {} and sent to {} at {}; send took {} millis", dateFormatter.format(new Date(message.getHeartbeat().getCreatedTimestamp())), + heartbeatAddress, dateFormatter.format(new Date()), sendMillis); } catch (final UnknownServiceAddressException usae) { @@ -3683,7 +3678,7 @@ HeartbeatMessage createHeartbeatMessage() { if (bean == null) { readLock.lock(); try { - final NodeConnectionStatus connectionStatus = new NodeConnectionStatus(DisconnectionCode.NOT_YET_CONNECTED); + final NodeConnectionStatus connectionStatus = new NodeConnectionStatus(getNodeId(), DisconnectionCode.NOT_YET_CONNECTED); bean = new HeartbeatBean(getGroup(getRootGroupId()), isPrimary(), connectionStatus); } finally { readLock.unlock(); @@ -3706,7 +3701,8 @@ HeartbeatMessage createHeartbeatMessage() { return null; } - final Heartbeat heartbeat = new Heartbeat(nodeId, bean.isPrimary(), bean.getConnectionStatus(), hbPayload.marshal()); + final Set roles = bean.isPrimary() ? Collections.singleton(ClusterRoles.PRIMARY_NODE) : Collections.emptySet(); + final Heartbeat heartbeat = new Heartbeat(nodeId, roles, bean.getConnectionStatus(), hbPayload.marshal()); final HeartbeatMessage message = new HeartbeatMessage(); message.setHeartbeat(heartbeat); @@ -3837,43 +3833,8 @@ public Collection getAllQueues() { return queues; } - public void setNodeStatus(final NodeIdentifier nodeId, final NodeConnectionStatus nodeStatus, final Long updateId) { - // We keep a VersionedNodeConnectionStatus as the value in our map, rather than NodeConnectionStatus. - // We do this because we update this based on data that is coming from the network. It is possible that we will - // get these notifications out-of-order because they could be sent across different TCP connections. As a result, - // we need to ensure that we don't update the status to an older version. The VersionedNodeConnectionStatus - // allows us to do this by looking at an "Update ID" that is associated with the new node status. - final VersionedNodeConnectionStatus versionedStatus = new VersionedNodeConnectionStatus(nodeStatus, updateId); - - boolean updated = false; - while (!updated) { - VersionedNodeConnectionStatus curStatus = nodeStatuses.putIfAbsent(nodeId, versionedStatus); - if (curStatus == null) { - // There was no status before. - LOG.info("Status of Node {} set to {}", nodeId, nodeStatus); - return; - } - - if (updateId < curStatus.getUpdateId()) { - LOG.debug("Received notification that status of Node {} changed to {} but the status update was old. Ignoring update.", nodeId, nodeStatus); - return; - } - - updated = nodeStatuses.replace(nodeId, curStatus, versionedStatus); - if (updated) { - LOG.info("Status of {} changed from {} to {}", nodeId, curStatus.getStatus(), nodeStatus); - return; - } - } - } - - public NodeConnectionStatus getNodeStatus(final NodeIdentifier nodeId) { - final VersionedNodeConnectionStatus versionedStatus = nodeStatuses.get(nodeId); - return versionedStatus == null ? null : versionedStatus.getStatus(); - } private static class HeartbeatBean { - private final ProcessGroup rootGroup; private final boolean primary; private final NodeConnectionStatus connectionStatus; @@ -3896,52 +3857,4 @@ public NodeConnectionStatus getConnectionStatus() { return connectionStatus; } } - - - /** - * A simple wrapper around a Node Connection Status and an Update ID. This is used as a value in a map so that we - * ensure that we update that Map only with newer versions - */ - private static class VersionedNodeConnectionStatus { - private final NodeConnectionStatus status; - private final Long updateId; - - public VersionedNodeConnectionStatus(final NodeConnectionStatus status, final Long updateId) { - this.status = status; - this.updateId = updateId; - } - - public NodeConnectionStatus getStatus() { - return status; - } - - public Long getUpdateId() { - return updateId; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((status == null) ? 0 : status.hashCode()); - result = prime * result + ((updateId == null) ? 0 : updateId.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (getClass() != obj.getClass()) { - return false; - } - - VersionedNodeConnectionStatus other = (VersionedNodeConnectionStatus) obj; - return other.getStatus().equals(getStatus()) && other.getUpdateId().equals(getUpdateId()); - } - } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java index 296659caaeb9..5b91ce25cf0a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java @@ -43,12 +43,14 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; import java.util.zip.GZIPInputStream; import java.util.zip.GZIPOutputStream; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.cluster.ConnectionException; import org.apache.nifi.cluster.HeartbeatPayload; +import org.apache.nifi.cluster.coordination.ClusterCoordinator; import org.apache.nifi.cluster.coordination.node.DisconnectionCode; import org.apache.nifi.cluster.coordination.node.NodeConnectionState; import org.apache.nifi.cluster.coordination.node.NodeConnectionStatus; @@ -66,7 +68,6 @@ import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; -import org.apache.nifi.cluster.protocol.message.NodeStatusChangeMessage; import org.apache.nifi.cluster.protocol.message.ProtocolMessage; import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage; import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage; @@ -91,6 +92,7 @@ import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.file.FileUtils; import org.apache.nifi.web.api.dto.TemplateDTO; +import org.apache.nifi.web.revision.RevisionManager; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -107,7 +109,6 @@ public class StandardFlowService implements FlowService, ProtocolHandler { private final FlowConfigurationDAO dao; private final int gracefulShutdownSeconds; private final boolean autoResumeState; - private final int connectionRetryMillis; private final StringEncryptor encryptor; // Lock is used to protect the flow.xml file. @@ -118,6 +119,8 @@ public class StandardFlowService implements FlowService, ProtocolHandler { private final AtomicBoolean running = new AtomicBoolean(false); private final AtomicReference executor = new AtomicReference<>(null); private final AtomicReference saveHolder = new AtomicReference<>(null); + private final ClusterCoordinator clusterCoordinator; + private final RevisionManager revisionManager; /** * listener/sender for internal cluster communication @@ -141,36 +144,33 @@ public class StandardFlowService implements FlowService, ProtocolHandler { private static final Logger logger = LoggerFactory.getLogger(StandardFlowService.class); public static StandardFlowService createStandaloneInstance( - final FlowController controller, - final NiFiProperties properties, - final StringEncryptor encryptor) throws IOException { - return new StandardFlowService( - controller, - properties, - /* nodeProtocolSenderListener */ null, - encryptor, - /* configuredForClustering */ false); + final FlowController controller, + final NiFiProperties properties, + final StringEncryptor encryptor, + final RevisionManager revisionManager) throws IOException { + + return new StandardFlowService(controller, properties, null, encryptor, false, null, revisionManager); } public static StandardFlowService createClusteredInstance( - final FlowController controller, - final NiFiProperties properties, - final NodeProtocolSenderListener senderListener, - final StringEncryptor encryptor) throws IOException { - return new StandardFlowService( - controller, - properties, - senderListener, - encryptor, - /* configuredForClustering */ true); + final FlowController controller, + final NiFiProperties properties, + final NodeProtocolSenderListener senderListener, + final ClusterCoordinator coordinator, + final StringEncryptor encryptor, + final RevisionManager revisionManager) throws IOException { + + return new StandardFlowService(controller, properties, senderListener, encryptor, true, coordinator, revisionManager); } private StandardFlowService( - final FlowController controller, - final NiFiProperties properties, - final NodeProtocolSenderListener senderListener, - final StringEncryptor encryptor, - final boolean configuredForClustering) throws IOException { + final FlowController controller, + final NiFiProperties properties, + final NodeProtocolSenderListener senderListener, + final StringEncryptor encryptor, + final boolean configuredForClustering, + final ClusterCoordinator clusterCoordinator, + final RevisionManager revisionManager) throws IOException { this.controller = controller; this.encryptor = encryptor; @@ -178,12 +178,15 @@ private StandardFlowService( gracefulShutdownSeconds = (int) FormatUtils.getTimeDuration(properties.getProperty(NiFiProperties.FLOW_CONTROLLER_GRACEFUL_SHUTDOWN_PERIOD), TimeUnit.SECONDS); autoResumeState = properties.getAutoResumeState(); - connectionRetryMillis = (int) FormatUtils.getTimeDuration(properties.getClusterManagerFlowRetrievalDelay(), TimeUnit.MILLISECONDS); dao = new StandardXMLFlowConfigurationDAO(flowXml, encryptor); + this.clusterCoordinator = clusterCoordinator; + if (clusterCoordinator != null) { + clusterCoordinator.setFlowService(this); + } + this.revisionManager = revisionManager; if (configuredForClustering) { - this.configuredForClustering = configuredForClustering; this.senderListener = senderListener; @@ -252,7 +255,7 @@ public void saveFlowChanges(final OutputStream outStream) throws IOException { public void overwriteFlow(final InputStream is) throws IOException { writeLock.lock(); try (final OutputStream output = Files.newOutputStream(flowXml, StandardOpenOption.WRITE, StandardOpenOption.CREATE); - final OutputStream gzipOut = new GZIPOutputStream(output);) { + final OutputStream gzipOut = new GZIPOutputStream(output);) { FileUtils.copy(is, gzipOut); } finally { writeLock.unlock(); @@ -290,7 +293,6 @@ public boolean isRunning() { public void start() throws LifeCycleStartException { writeLock.lock(); try { - if (isRunning()) { return; } @@ -328,6 +330,18 @@ public void stop(final boolean force) { running.set(false); + if (clusterCoordinator != null) { + final Thread shutdownClusterCoordinator = new Thread(new Runnable() { + @Override + public void run() { + clusterCoordinator.shutdown(); + } + }); + shutdownClusterCoordinator.setDaemon(true); + shutdownClusterCoordinator.setName("Shutdown Cluster Coordinator"); + shutdownClusterCoordinator.start(); + } + if (!controller.isTerminated()) { controller.shutdown(force); } @@ -370,7 +384,6 @@ public boolean canHandle(final ProtocolMessage msg) { case RECONNECTION_REQUEST: case DISCONNECTION_REQUEST: case FLOW_REQUEST: - case NODE_STATUS_CHANGE: return true; default: return false; @@ -384,11 +397,7 @@ public ProtocolMessage handle(final ProtocolMessage request) throws ProtocolExce switch (request.getType()) { case FLOW_REQUEST: return handleFlowRequest((FlowRequestMessage) request); - case NODE_STATUS_CHANGE: - final NodeStatusChangeMessage statusChangeMsg = (NodeStatusChangeMessage) request; - controller.setNodeStatus(statusChangeMsg.getNodeId(), statusChangeMsg.getNodeConnectionStatus(), statusChangeMsg.getStatusUpdateIdentifier()); - return null; - case RECONNECTION_REQUEST: + case RECONNECTION_REQUEST: { // Suspend heartbeats until we've reconnected. Otherwise, // we may send a heartbeat while we are still in the process of // connecting, which will cause the Cluster Manager to mark us @@ -396,23 +405,29 @@ public ProtocolMessage handle(final ProtocolMessage request) throws ProtocolExce // may still be held, causing this node to take a long time to respond to requests. controller.suspendHeartbeats(); - new Thread(new Runnable() { + final Thread t = new Thread(new Runnable() { @Override public void run() { handleReconnectionRequest((ReconnectionRequestMessage) request); } - }, "Reconnect to Cluster").start(); + }, "Reconnect to Cluster"); + t.setDaemon(true); + t.start(); return new ReconnectionResponseMessage(); - case DISCONNECTION_REQUEST: - new Thread(new Runnable() { + } + case DISCONNECTION_REQUEST: { + final Thread t = new Thread(new Runnable() { @Override public void run() { handleDisconnectionRequest((DisconnectMessage) request); } - }, "Disconnect from Cluster").start(); + }, "Disconnect from Cluster"); + t.setDaemon(true); + t.start(); return null; + } default: throw new ProtocolException("Handler cannot handle message type: " + request.getType()); } @@ -426,21 +441,26 @@ public void run() { } @Override - public void load(final DataFlow proposedFlow) throws IOException, FlowSerializationException, - FlowSynchronizationException, UninheritableFlowException { - writeLock.lock(); - try { - if (configuredForClustering) { - /* - * Attempt to connect to the cluster. If the manager is able to - * provide a data flow, then the manager will send a connection - * response. If the manager was unable to be located, then - * the response will be null and we should load the local dataflow - * and heartbeat until a manager is located. - */ - final boolean localFlowEmpty = StandardFlowSynchronizer.isEmpty(proposedFlow, encryptor); - final ConnectionResponse response = connect(localFlowEmpty, localFlowEmpty); + public void load(final DataFlow dataFlow) throws IOException, FlowSerializationException, FlowSynchronizationException, UninheritableFlowException { + if (configuredForClustering) { + final DataFlow proposedFlow = (dataFlow == null) ? createDataFlow() : dataFlow; + /* + * Attempt to connect to the cluster. If the manager is able to + * provide a data flow, then the manager will send a connection + * response. If the manager was unable to be located, then + * the response will be null and we should load the local dataflow + * and heartbeat until a manager is located. + */ + final boolean localFlowEmpty = StandardFlowSynchronizer.isEmpty(proposedFlow, encryptor); + final ConnectionResponse response = connect(localFlowEmpty, localFlowEmpty); + + // obtain write lock while we are updating the controller. We need to ensure that we don't + // obtain the lock before calling connect(), though, or we will end up getting a deadlock + // because the node that is receiving the connection request won't be able to get the current + // flow, as that requires a read lock. + writeLock.lock(); + try { if (response == null) { logger.info("Flow controller will load local dataflow and suspend connection handshake until a cluster connection response is received."); @@ -449,16 +469,19 @@ public void load(final DataFlow proposedFlow) throws IOException, FlowSerializat // set node ID on controller before we start heartbeating because heartbeat needs node ID controller.setNodeId(nodeId); + clusterCoordinator.setLocalNodeIdentifier(nodeId); // set node as clustered, since it is trying to connect to a cluster controller.setClustered(true, null); + clusterCoordinator.setConnected(false); + controller.setClusterManagerRemoteSiteInfo(null, null); - controller.setConnectionStatus(new NodeConnectionStatus(DisconnectionCode.NOT_YET_CONNECTED)); + controller.setConnectionStatus(new NodeConnectionStatus(nodeId, DisconnectionCode.NOT_YET_CONNECTED)); /* - * Start heartbeating. Heartbeats will fail because we can't reach + * Start heartbeating. Heartbeats will fail because we can't reach * the manager, but when we locate the manager, the node will - * reconnect and establish a connection to the cluster. The + * reconnect and establish a connection to the cluster. The * heartbeat is the trigger that will cause the manager to * issue a reconnect request. */ @@ -482,13 +505,17 @@ public void load(final DataFlow proposedFlow) throws IOException, FlowSerializat throw new IOException(ce); } } - } else { + } finally { + writeLock.unlock(); + } + } else { + writeLock.lock(); + try { // operating in standalone mode, so load proposed flow - loadFromBytes(proposedFlow, true); + loadFromBytes(dataFlow, true); + } finally { + writeLock.unlock(); } - - } finally { - writeLock.unlock(); } } @@ -498,11 +525,11 @@ private void handleConnectionFailure(final Exception ex) { final HeartbeatMessage startupFailureMessage = new HeartbeatMessage(); final NodeConnectionStatus connectionStatus; if (ex instanceof UninheritableFlowException) { - connectionStatus = new NodeConnectionStatus(DisconnectionCode.MISMATCHED_FLOWS, ex.toString()); + connectionStatus = new NodeConnectionStatus(nodeId, DisconnectionCode.MISMATCHED_FLOWS, ex.toString()); } else if (ex instanceof FlowSynchronizationException) { - connectionStatus = new NodeConnectionStatus(DisconnectionCode.MISMATCHED_FLOWS, ex.toString()); + connectionStatus = new NodeConnectionStatus(nodeId, DisconnectionCode.MISMATCHED_FLOWS, ex.toString()); } else { - connectionStatus = new NodeConnectionStatus(DisconnectionCode.STARTUP_FAILURE, ex.toString()); + connectionStatus = new NodeConnectionStatus(nodeId, DisconnectionCode.STARTUP_FAILURE, ex.toString()); } final byte[] payload; @@ -510,7 +537,7 @@ private void handleConnectionFailure(final Exception ex) { HeartbeatPayload.marshal(new HeartbeatPayload(), baos); payload = baos.toByteArray(); - final Heartbeat failureHeartbeat = new Heartbeat(nodeId, false, connectionStatus, payload); + final Heartbeat failureHeartbeat = new Heartbeat(nodeId, Collections.emptySet(), connectionStatus, payload); startupFailureMessage.setHeartbeat(failureHeartbeat); heartbeater.send(startupFailureMessage); } catch (final Exception e) { @@ -524,19 +551,9 @@ private FlowResponseMessage handleFlowRequest(final FlowRequestMessage request) try { logger.info("Received flow request message from manager."); - // serialize the flow to the output stream - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - copyCurrentFlow(baos); - final byte[] flowBytes = baos.toByteArray(); - baos.reset(); - - final byte[] snippetBytes = controller.getSnippetManager().export(); - // create the response final FlowResponseMessage response = new FlowResponseMessage(); - - response.setDataFlow(new StandardDataFlow(flowBytes, snippetBytes)); - + response.setDataFlow(createDataFlow()); return response; } catch (final Exception ex) { throw new ProtocolException("Failed serializing flow controller state for flow request due to: " + ex, ex); @@ -545,18 +562,56 @@ private FlowResponseMessage handleFlowRequest(final FlowRequestMessage request) } } + @Override + public StandardDataFlow createDataFlow() throws IOException { + // Load the flow from disk + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + dao.load(baos); + final byte[] bytes = baos.toByteArray(); + final byte[] snippetBytes = controller.getSnippetManager().export(); + final StandardDataFlow fromDisk = new StandardDataFlow(bytes, snippetBytes); + + // Check if the flow from disk is empty. If not, use it. + if (!StandardFlowSynchronizer.isEmpty(fromDisk, encryptor)) { + return fromDisk; + } + + // Flow from disk is empty, so serialize the Flow Controller and use that. + // This is done because on startup, if there is no flow, the Flow Controller + // will automatically create a Root Process Group, and we need to ensure that + // we replicate that Process Group to all nodes in the cluster, so that they all + // end up with the same ID for the root Process Group. + baos.reset(); + dao.save(controller, baos); + final byte[] flowBytes = baos.toByteArray(); + baos.reset(); + + return new StandardDataFlow(flowBytes, snippetBytes); + } + + private NodeIdentifier getNodeId() { + readLock.lock(); + try { + return nodeId; + } finally { + readLock.unlock(); + } + } private void handleReconnectionRequest(final ReconnectionRequestMessage request) { - writeLock.lock(); try { logger.info("Processing reconnection request from manager."); // reconnect - final ConnectionResponse connectionResponse = new ConnectionResponse(nodeId, request.getDataFlow(), - request.getManagerRemoteSiteListeningPort(), request.isManagerRemoteSiteCommsSecure(), request.getInstanceId()); + final ConnectionResponse connectionResponse = new ConnectionResponse(getNodeId(), request.getDataFlow(), + request.getManagerRemoteSiteListeningPort(), request.isManagerRemoteSiteCommsSecure(), request.getInstanceId(), + request.getNodeConnectionStatuses(), request.getComponentRevisions()); + connectionResponse.setClusterManagerDN(request.getRequestorDN()); loadFromConnectionResponse(connectionResponse); + clusterCoordinator.resetNodeStatuses(connectionResponse.getNodeConnectionStatuses().stream() + .collect(Collectors.toMap(status -> status.getNodeIdentifier(), status -> status))); controller.resumeHeartbeats(); // we are now connected, so resume sending heartbeats. logger.info("Node reconnected."); @@ -568,19 +623,12 @@ private void handleReconnectionRequest(final ReconnectionRequestMessage request) logger.error("Handling reconnection request failed due to: " + ex, ex); handleConnectionFailure(ex); - } finally { - writeLock.unlock(); } } private void handleDisconnectionRequest(final DisconnectMessage request) { - writeLock.lock(); - try { - logger.info("Received disconnection request message from manager with explanation: " + request.getExplanation()); - disconnect(request.getExplanation()); - } finally { - writeLock.unlock(); - } + logger.info("Received disconnection request message from manager with explanation: " + request.getExplanation()); + disconnect(request.getExplanation()); } private void disconnect(final String explanation) { @@ -590,7 +638,7 @@ private void disconnect(final String explanation) { logger.info("Disconnecting node."); // mark node as not connected - controller.setConnectionStatus(new NodeConnectionStatus(DisconnectionCode.UNKNOWN, explanation)); + controller.setConnectionStatus(new NodeConnectionStatus(nodeId, DisconnectionCode.UNKNOWN, explanation)); // turn off primary flag controller.setPrimary(false); @@ -600,6 +648,7 @@ private void disconnect(final String explanation) { // set node to not clustered controller.setClustered(false, null); + clusterCoordinator.setConnected(false); logger.info("Node disconnected."); @@ -610,7 +659,7 @@ private void disconnect(final String explanation) { // write lock must already be acquired private void loadFromBytes(final DataFlow proposedFlow, final boolean allowEmptyFlow) - throws IOException, FlowSerializationException, FlowSynchronizationException, UninheritableFlowException { + throws IOException, FlowSerializationException, FlowSynchronizationException, UninheritableFlowException { logger.trace("Loading flow from bytes"); // resolve the given flow (null means load flow from disk) @@ -715,7 +764,7 @@ public List