You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2017/08/17 14:43:18 UTC
[1/9] nifi git commit: NIFI-4224: - Initial implementation of Process
Group level Variable Registry - Updated to incorporate PR Feedback - Changed
log message because slf4j-simple apparently has a memory leak;
passing a String instead of passing in the C
Repository: nifi
Updated Branches:
refs/heads/master c1b99d584 -> 5cd8e93be
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
index 9af57b6..258af72 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardProcessGroupDAO.java
@@ -16,6 +16,13 @@
*/
package org.apache.nifi.web.dao.impl;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
+
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.ConnectableType;
import org.apache.nifi.connectable.Port;
@@ -23,14 +30,15 @@ import org.apache.nifi.connectable.Position;
import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.web.ResourceNotFoundException;
import org.apache.nifi.web.api.dto.ProcessGroupDTO;
+import org.apache.nifi.web.api.dto.VariableRegistryDTO;
+import org.apache.nifi.web.api.entity.VariableEntity;
import org.apache.nifi.web.dao.ProcessGroupDAO;
-import java.util.HashSet;
-import java.util.Set;
-
public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGroupDAO {
private FlowController flowController;
@@ -64,6 +72,10 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
}
@Override
+ public void verifyUpdate(final ProcessGroupDTO processGroup) {
+ }
+
+ @Override
public ProcessGroup getProcessGroup(String groupId) {
return locateProcessGroup(flowController, groupId);
}
@@ -99,14 +111,32 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
}
@Override
- public void scheduleComponents(final String groupId, final ScheduledState state, final Set<String> componentIds) {
+ public void verifyActivateControllerServices(final String groupId, final ControllerServiceState state, final Set<String> serviceIds) {
final ProcessGroup group = locateProcessGroup(flowController, groupId);
+ group.findAllControllerServices().stream()
+ .filter(service -> serviceIds.contains(service.getIdentifier()))
+ .forEach(service -> {
+ if (state == ControllerServiceState.ENABLED) {
+ service.verifyCanEnable();
+ } else {
+ service.verifyCanDisable();
+ }
+ });
+ }
+
+ @Override
+ public CompletableFuture<Void> scheduleComponents(final String groupId, final ScheduledState state, final Set<String> componentIds) {
+ final ProcessGroup group = locateProcessGroup(flowController, groupId);
+
+ CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
+
for (final String componentId : componentIds) {
final Connectable connectable = group.findLocalConnectable(componentId);
if (ScheduledState.RUNNING.equals(state)) {
if (ConnectableType.PROCESSOR.equals(connectable.getConnectableType())) {
- connectable.getProcessGroup().startProcessor((ProcessorNode) connectable);
+ final CompletableFuture<?> processorFuture = connectable.getProcessGroup().startProcessor((ProcessorNode) connectable);
+ future = CompletableFuture.allOf(future, processorFuture);
} else if (ConnectableType.INPUT_PORT.equals(connectable.getConnectableType())) {
connectable.getProcessGroup().startInputPort((Port) connectable);
} else if (ConnectableType.OUTPUT_PORT.equals(connectable.getConnectableType())) {
@@ -114,7 +144,8 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
}
} else {
if (ConnectableType.PROCESSOR.equals(connectable.getConnectableType())) {
- connectable.getProcessGroup().stopProcessor((ProcessorNode) connectable);
+ final CompletableFuture<?> processorFuture = connectable.getProcessGroup().stopProcessor((ProcessorNode) connectable);
+ future = CompletableFuture.allOf(future, processorFuture);
} else if (ConnectableType.INPUT_PORT.equals(connectable.getConnectableType())) {
connectable.getProcessGroup().stopInputPort((Port) connectable);
} else if (ConnectableType.OUTPUT_PORT.equals(connectable.getConnectableType())) {
@@ -122,6 +153,27 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
}
}
}
+
+ return future;
+ }
+
+ @Override
+ public Future<Void> activateControllerServices(final String groupId, final ControllerServiceState state, final Set<String> serviceIds) {
+ final ProcessGroup group = locateProcessGroup(flowController, groupId);
+
+ CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
+ for (final String serviceId : serviceIds) {
+ final ControllerServiceNode serviceNode = group.findControllerService(serviceId);
+ if (ControllerServiceState.ENABLED.equals(state)) {
+ final CompletableFuture<Void> serviceFuture = flowController.enableControllerService(serviceNode);
+ future = CompletableFuture.allOf(future, serviceFuture);
+ } else {
+ final CompletableFuture<Void> serviceFuture = flowController.disableControllerService(serviceNode);
+ future = CompletableFuture.allOf(future, serviceFuture);
+ }
+ }
+
+ return future;
}
@Override
@@ -145,6 +197,22 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
}
@Override
+ public ProcessGroup updateVariableRegistry(final VariableRegistryDTO variableRegistry) {
+ final ProcessGroup group = locateProcessGroup(flowController, variableRegistry.getProcessGroupId());
+ if (group == null) {
+ throw new ResourceNotFoundException("Could not find Process Group with ID " + variableRegistry.getProcessGroupId());
+ }
+
+ final Map<String, String> variableMap = new HashMap<>();
+ variableRegistry.getVariables().stream() // have to use forEach here instead of using Collectors.toMap because value may be null
+ .map(VariableEntity::getVariable)
+ .forEach(var -> variableMap.put(var.getName(), var.getValue()));
+
+ group.setVariables(variableMap);
+ return group;
+ }
+
+ @Override
public void verifyDelete(String groupId) {
ProcessGroup group = locateProcessGroup(flowController, groupId);
group.verifyCanDelete();
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Pause.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Pause.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Pause.java
new file mode 100644
index 0000000..c2ef890
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/util/Pause.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.util;
+
+public interface Pause {
+
+ /**
+ * Waits up to given amount of time, and returns <code>true</code> if the action being performed
+ * should continue, <code>false</code> if the action being performed has been cancelled
+ *
+ * @return <code>true</code> if the action should continue, <code>false</code> otherwise
+ */
+ boolean pause();
+
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
index 2311b07..8ee39f0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
@@ -284,6 +284,7 @@
<property name="requestReplicator" ref="requestReplicator" />
<property name="authorizer" ref="authorizer"/>
<property name="flowController" ref="flowController" />
+ <property name="dtoFactory" ref="dtoFactory" />
</bean>
<bean id="processorResource" class="org.apache.nifi.web.api.ProcessorResource" scope="singleton">
<property name="serviceFacade" ref="serviceFacade"/>
[2/9] nifi git commit: NIFI-4224: - Initial implementation of Process
Group level Variable Registry - Updated to incorporate PR Feedback - Changed
log message because slf4j-simple apparently has a memory leak;
passing a String instead of passing in the C
Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
index a9167ae..0b634a1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ProcessGroupResource.java
@@ -16,14 +16,54 @@
*/
package org.apache.nifi.web.api;
-import com.sun.jersey.api.core.ResourceContext;
-import com.sun.jersey.multipart.FormDataParam;
-import com.wordnik.swagger.annotations.Api;
-import com.wordnik.swagger.annotations.ApiOperation;
-import com.wordnik.swagger.annotations.ApiParam;
-import com.wordnik.swagger.annotations.ApiResponse;
-import com.wordnik.swagger.annotations.ApiResponses;
-import com.wordnik.swagger.annotations.Authorization;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.HttpMethod;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+import javax.ws.rs.core.UriBuilder;
+import javax.ws.rs.core.UriInfo;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBElement;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.transform.stream.StreamSource;
+
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.AuthorizableLookup;
import org.apache.nifi.authorization.AuthorizeControllerServiceReference;
@@ -38,26 +78,41 @@ import org.apache.nifi.authorization.user.NiFiUser;
import org.apache.nifi.authorization.user.NiFiUserUtils;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.connectable.ConnectableType;
+import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.serialization.FlowEncodingVersion;
+import org.apache.nifi.controller.service.ControllerServiceState;
+import org.apache.nifi.framework.security.util.SslContextFactory;
+import org.apache.nifi.registry.variable.VariableRegistryUpdateRequest;
+import org.apache.nifi.registry.variable.VariableRegistryUpdateStep;
import org.apache.nifi.remote.util.SiteToSiteRestApiClient;
import org.apache.nifi.util.BundleUtils;
+import org.apache.nifi.util.FormatUtils;
+import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiServiceFacade;
import org.apache.nifi.web.ResourceNotFoundException;
import org.apache.nifi.web.Revision;
+import org.apache.nifi.web.api.dto.AffectedComponentDTO;
import org.apache.nifi.web.api.dto.BundleDTO;
import org.apache.nifi.web.api.dto.ConnectionDTO;
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
+import org.apache.nifi.web.api.dto.DtoFactory;
import org.apache.nifi.web.api.dto.FlowSnippetDTO;
import org.apache.nifi.web.api.dto.PositionDTO;
import org.apache.nifi.web.api.dto.ProcessGroupDTO;
import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
import org.apache.nifi.web.api.dto.ProcessorDTO;
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
+import org.apache.nifi.web.api.dto.RevisionDTO;
import org.apache.nifi.web.api.dto.TemplateDTO;
+import org.apache.nifi.web.api.dto.VariableRegistryDTO;
import org.apache.nifi.web.api.dto.flow.FlowDTO;
+import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO;
+import org.apache.nifi.web.api.dto.status.ProcessGroupStatusSnapshotDTO;
+import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
import org.apache.nifi.web.api.entity.ConnectionEntity;
import org.apache.nifi.web.api.entity.ConnectionsEntity;
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
+import org.apache.nifi.web.api.entity.ControllerServicesEntity;
import org.apache.nifi.web.api.entity.CopySnippetRequestEntity;
import org.apache.nifi.web.api.entity.CreateTemplateRequestEntity;
import org.apache.nifi.web.api.entity.FlowEntity;
@@ -70,47 +125,36 @@ import org.apache.nifi.web.api.entity.LabelsEntity;
import org.apache.nifi.web.api.entity.OutputPortsEntity;
import org.apache.nifi.web.api.entity.PortEntity;
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
+import org.apache.nifi.web.api.entity.ProcessGroupStatusEntity;
+import org.apache.nifi.web.api.entity.ProcessGroupStatusSnapshotEntity;
import org.apache.nifi.web.api.entity.ProcessGroupsEntity;
import org.apache.nifi.web.api.entity.ProcessorEntity;
import org.apache.nifi.web.api.entity.ProcessorsEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupsEntity;
+import org.apache.nifi.web.api.entity.ScheduleComponentsEntity;
import org.apache.nifi.web.api.entity.TemplateEntity;
+import org.apache.nifi.web.api.entity.VariableRegistryEntity;
+import org.apache.nifi.web.api.entity.VariableRegistryUpdateRequestEntity;
import org.apache.nifi.web.api.request.ClientIdParameter;
import org.apache.nifi.web.api.request.LongParameter;
+import org.apache.nifi.web.util.Pause;
+import org.apache.nifi.web.util.WebUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import javax.servlet.http.HttpServletRequest;
-import javax.ws.rs.Consumes;
-import javax.ws.rs.DELETE;
-import javax.ws.rs.DefaultValue;
-import javax.ws.rs.GET;
-import javax.ws.rs.HttpMethod;
-import javax.ws.rs.POST;
-import javax.ws.rs.PUT;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.QueryParam;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-import javax.ws.rs.core.UriBuilder;
-import javax.ws.rs.core.UriInfo;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.transform.stream.StreamSource;
-import java.io.InputStream;
-import java.net.URI;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Consumer;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.api.core.ResourceContext;
+import com.sun.jersey.multipart.FormDataParam;
+import com.wordnik.swagger.annotations.Api;
+import com.wordnik.swagger.annotations.ApiOperation;
+import com.wordnik.swagger.annotations.ApiParam;
+import com.wordnik.swagger.annotations.ApiResponse;
+import com.wordnik.swagger.annotations.ApiResponses;
+import com.wordnik.swagger.annotations.Authorization;
/**
* RESTful endpoint for managing a Group.
@@ -139,6 +183,22 @@ public class ProcessGroupResource extends ApplicationResource {
private ConnectionResource connectionResource;
private TemplateResource templateResource;
private ControllerServiceResource controllerServiceResource;
+ private DtoFactory dtoFactory;
+
+ private final ConcurrentMap<String, VariableRegistryUpdateRequest> varRegistryUpdateRequests = new ConcurrentHashMap<>();
+ private static final int MAX_VARIABLE_REGISTRY_UPDATE_REQUESTS = 100;
+ private static final long VARIABLE_REGISTRY_UPDATE_REQUEST_EXPIRATION = TimeUnit.MINUTES.toMillis(1L);
+ private final ExecutorService variableRegistryThreadPool = new ThreadPoolExecutor(1, 50, 5L, TimeUnit.SECONDS,
+ new ArrayBlockingQueue<Runnable>(MAX_VARIABLE_REGISTRY_UPDATE_REQUESTS),
+ new ThreadFactory() {
+ @Override
+ public Thread newThread(final Runnable r) {
+ final Thread thread = Executors.defaultThreadFactory().newThread(r);
+ thread.setName("Variable Registry Update Thread");
+ thread.setDaemon(true);
+ return thread;
+ }
+ });
/**
* Populates the remaining fields in the specified process groups.
@@ -164,6 +224,7 @@ public class ProcessGroupResource extends ApplicationResource {
return processGroupEntity;
}
+
/**
* Populates the remaining content of the specified snippet.
*/
@@ -238,6 +299,49 @@ public class ProcessGroupResource extends ApplicationResource {
return generateOkResponse(entity).build();
}
+
+ /**
+ * Retrieves the Variable Registry for the group with the given ID
+ *
+ * @param groupId the ID of the Process Group
+ * @return the Variable Registry for the group
+ */
+ @GET
+ @Consumes(MediaType.WILDCARD)
+ @Produces(MediaType.APPLICATION_JSON)
+ @Path("{id}/variable-registry")
+ @ApiOperation(value = "Gets a process group's variable registry",
+ response = VariableRegistryEntity.class,
+ notes = NON_GUARANTEED_ENDPOINT,
+ authorizations = {
+ @Authorization(value = "Read - /process-groups/{uuid}", type = "")
+ })
+ @ApiResponses(value = {
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 404, message = "The specified resource could not be found."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+ })
+ public Response getVariableRegistry(
+ @ApiParam(value = "The process group id.", required = true) @PathParam("id") final String groupId,
+ @ApiParam(value = "Whether or not to include ancestor groups", required = false) @QueryParam("includeAncestorGroups") @DefaultValue("true") final boolean includeAncestorGroups) {
+
+ if (isReplicateRequest()) {
+ return replicate(HttpMethod.GET);
+ }
+
+ // authorize access
+ serviceFacade.authorizeAccess(lookup -> {
+ final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
+ processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+ });
+
+ // get this process group's variable registry
+ final VariableRegistryEntity entity = serviceFacade.getVariableRegistry(groupId, includeAncestorGroups);
+ return generateOkResponse(entity).build();
+ }
+
/**
* Updates the specified process group.
*
@@ -314,7 +418,7 @@ public class ProcessGroupResource extends ApplicationResource {
Authorizable authorizable = lookup.getProcessGroup(id).getAuthorizable();
authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
},
- null,
+ () -> serviceFacade.verifyUpdateProcessGroup(requestProcessGroupDTO),
(revision, processGroupEntity) -> {
// update the process group
final ProcessGroupEntity entity = serviceFacade.updateProcessGroup(revision, processGroupEntity.getComponent());
@@ -325,6 +429,854 @@ public class ProcessGroupResource extends ApplicationResource {
);
}
+
+ @GET
+ @Consumes(MediaType.WILDCARD)
+ @Produces(MediaType.APPLICATION_JSON)
+ @Path("{groupId}/variable-registry/update-requests/{updateId}")
+ @ApiOperation(value = "Gets a process group's variable registry",
+ response = VariableRegistryUpdateRequestEntity.class,
+ notes = NON_GUARANTEED_ENDPOINT,
+ authorizations = {
+ @Authorization(value = "Read - /process-groups/{uuid}", type = "")
+ })
+ @ApiResponses(value = {
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 404, message = "The specified resource could not be found."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+ })
+ public Response getVariableRegistryUpdateRequest(
+ @ApiParam(value = "The process group id.", required = true) @PathParam("groupId") final String groupId,
+ @ApiParam(value = "The ID of the Variable Registry Update Request", required = true) @PathParam("updateId") final String updateId) {
+
+ if (groupId == null || updateId == null) {
+ throw new IllegalArgumentException("Group ID and Update ID must both be specified.");
+ }
+
+ if (isReplicateRequest()) {
+ return replicate(HttpMethod.GET);
+ }
+
+ // authorize access
+ serviceFacade.authorizeAccess(lookup -> {
+ final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
+ processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+ });
+
+ final VariableRegistryUpdateRequest request = varRegistryUpdateRequests.get(updateId);
+ if (request == null) {
+ throw new ResourceNotFoundException("Could not find a Variable Registry Update Request with identifier " + updateId);
+ }
+
+ if (!groupId.equals(request.getProcessGroupId())) {
+ throw new ResourceNotFoundException("Could not find a Variable Registry Update Request with identifier " + updateId + " for Process Group with identifier " + groupId);
+ }
+
+ final VariableRegistryUpdateRequestEntity entity = new VariableRegistryUpdateRequestEntity();
+ entity.setRequestDto(dtoFactory.createVariableRegistryUpdateRequestDto(request));
+ entity.getRequestDto().setUri(generateResourceUri("process-groups", groupId, "variable-registry", updateId));
+ return generateOkResponse(entity).build();
+ }
+
+
+ @DELETE
+ @Consumes(MediaType.WILDCARD)
+ @Produces(MediaType.APPLICATION_JSON)
+ @Path("{groupId}/variable-registry/update-requests/{updateId}")
+ @ApiOperation(value = "Deletes an update request for a process group's variable registry. If the request is not yet complete, it will automatically be cancelled.",
+ response = VariableRegistryUpdateRequestEntity.class,
+ notes = NON_GUARANTEED_ENDPOINT,
+ authorizations = {
+ @Authorization(value = "Read - /process-groups/{uuid}", type = "")
+ })
+ @ApiResponses(value = {
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 404, message = "The specified resource could not be found."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+ })
+ public Response deleteVariableRegistryUpdateRequest(
+ @ApiParam(value = "The process group id.", required = true) @PathParam("groupId") final String groupId,
+ @ApiParam(value = "The ID of the Variable Registry Update Request", required = true) @PathParam("updateId") final String updateId) {
+
+ if (groupId == null || updateId == null) {
+ throw new IllegalArgumentException("Group ID and Update ID must both be specified.");
+ }
+
+ if (isReplicateRequest()) {
+ return replicate(HttpMethod.DELETE);
+ }
+
+ // authorize access
+ serviceFacade.authorizeAccess(lookup -> {
+ final Authorizable processGroup = lookup.getProcessGroup(groupId).getAuthorizable();
+ processGroup.authorize(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser());
+ processGroup.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+ });
+
+ final VariableRegistryUpdateRequest request = varRegistryUpdateRequests.remove(updateId);
+ if (request == null) {
+ throw new ResourceNotFoundException("Could not find a Variable Registry Update Request with identifier " + updateId);
+ }
+
+ if (!groupId.equals(request.getProcessGroupId())) {
+ throw new ResourceNotFoundException("Could not find a Variable Registry Update Request with identifier " + updateId + " for Process Group with identifier " + groupId);
+ }
+
+ request.cancel();
+
+ final VariableRegistryUpdateRequestEntity entity = new VariableRegistryUpdateRequestEntity();
+ entity.setRequestDto(dtoFactory.createVariableRegistryUpdateRequestDto(request));
+ entity.getRequestDto().setUri(generateResourceUri("process-groups", groupId, "variable-registry", updateId));
+ return generateOkResponse(entity).build();
+ }
+
+
+ @PUT
+ @Consumes(MediaType.APPLICATION_JSON)
+ @Produces(MediaType.APPLICATION_JSON)
+ @Path("{id}/variable-registry")
+ @ApiOperation(value = "Updates the contents of a Process Group's variable Registry", response = VariableRegistryEntity.class, notes = NON_GUARANTEED_ENDPOINT, authorizations = {
+ @Authorization(value = "Write - /process-groups/{uuid}", type = "")
+ })
+ @ApiResponses(value = {
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 404, message = "The specified resource could not be found."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+ })
+ public Response updateVariableRegistry(
+ @Context final HttpServletRequest httpServletRequest,
+ @ApiParam(value = "The process group id.", required = true) @PathParam("id") final String groupId,
+ @ApiParam(value = "The process group configuration details.", required = true) final VariableRegistryEntity requestEntity) {
+
+ if (requestEntity == null || requestEntity.getVariableRegistry() == null) {
+ throw new IllegalArgumentException("Variable Registry details must be specified.");
+ }
+
+ if (requestEntity.getProcessGroupRevision() == null) {
+ throw new IllegalArgumentException("Process Group Revision must be specified.");
+ }
+
+ // ensure the same id is being used
+ final VariableRegistryDTO registryDto = requestEntity.getVariableRegistry();
+ if (!groupId.equals(registryDto.getProcessGroupId())) {
+ throw new IllegalArgumentException(String.format("The process group id (%s) in the request body does "
+ + "not equal the process group id of the requested resource (%s).", registryDto.getProcessGroupId(), groupId));
+ }
+
+ if (isReplicateRequest()) {
+ return replicate(HttpMethod.PUT, requestEntity);
+ }
+
+ // handle expects request (usually from the cluster manager)
+ final Revision requestRevision = getRevision(requestEntity.getProcessGroupRevision(), groupId);
+ return withWriteLock(
+ serviceFacade,
+ requestEntity,
+ requestRevision,
+ lookup -> {
+ Authorizable authorizable = lookup.getProcessGroup(groupId).getAuthorizable();
+ authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+ },
+ null,
+ (revision, processGroupEntity) -> {
+ // update the process group
+ final VariableRegistryEntity entity = serviceFacade.updateVariableRegistry(revision, registryDto);
+ return generateOkResponse(entity).build();
+ });
+ }
+
+
+ /**
+ * Updates the variable registry for the specified process group.
+ *
+ * @param httpServletRequest request
+ * @param groupId The id of the process group.
+ * @param requestEntity the Variable Registry Entity
+ * @return A Variable Registry Entry.
+ */
+ @POST
+ @Consumes(MediaType.APPLICATION_JSON)
+ @Produces(MediaType.APPLICATION_JSON)
+ @Path("{id}/variable-registry/update-requests")
+ @ApiOperation(value = "Submits a request to update a process group's variable registry",
+ response = VariableRegistryUpdateRequestEntity.class,
+ notes = NON_GUARANTEED_ENDPOINT,
+ authorizations = {
+ @Authorization(value = "Write - /process-groups/{uuid}", type = "")
+ })
+ @ApiResponses(value = {
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 404, message = "The specified resource could not be found."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+ })
+ public Response submitUpdateVariableRegistryRequest(
+ @Context final HttpServletRequest httpServletRequest,
+ @ApiParam(value = "The process group id.", required = true) @PathParam("id") final String groupId,
+ @ApiParam(value = "The process group configuration details.", required = true) final VariableRegistryEntity requestEntity) {
+
+ if (requestEntity == null || requestEntity.getVariableRegistry() == null) {
+ throw new IllegalArgumentException("Variable Registry details must be specified.");
+ }
+
+ if (requestEntity.getProcessGroupRevision() == null) {
+ throw new IllegalArgumentException("Process Group Revision must be specified.");
+ }
+
+ // In order to update variables in a variable registry, we have to perform the following steps:
+ // 1. Determine Affected Components (this includes any Processors and Controller Services and any components that reference an affected Controller Service).
+ // 1a. Determine ID's of components
+ // 1b. Determine Revision's of associated components
+ // 2. Stop All Affected Processors
+ // 3. Disable All Affected Controller Services
+ // 4. Update the Variables
+ // 5. Re-Enable all Affected Controller Services (services only, not dependent components)
+ // 6. Re-Enable all Processors that Depended on the Controller Services
+
+ // Determine the affected components (and their associated revisions)
+ final VariableRegistryEntity computedEntity = serviceFacade.populateAffectedComponents(requestEntity.getVariableRegistry());
+ final VariableRegistryDTO computedRegistryDto = computedEntity.getVariableRegistry();
+ if (computedRegistryDto == null) {
+ throw new ResourceNotFoundException(String.format("Unable to locate group with id '%s'.", groupId));
+ }
+
+ final Set<AffectedComponentDTO> affectedComponents = serviceFacade.getComponentsAffectedByVariableRegistryUpdate(requestEntity.getVariableRegistry());
+
+ final Map<String, List<AffectedComponentDTO>> affectedComponentsByType = affectedComponents.stream()
+ .collect(Collectors.groupingBy(comp -> comp.getComponentType()));
+
+ final List<AffectedComponentDTO> affectedProcessors = affectedComponentsByType.get(AffectedComponentDTO.COMPONENT_TYPE_PROCESSOR);
+ final List<AffectedComponentDTO> affectedServices = affectedComponentsByType.get(AffectedComponentDTO.COMPONENT_TYPE_CONTROLLER_SERVICE);
+
+
+ if (isReplicateRequest()) {
+ // update the variable registry
+ final VariableRegistryUpdateRequest updateRequest = createVariableRegistryUpdateRequest(groupId);
+ updateRequest.getIdentifyRelevantComponentsStep().setComplete(true);
+ final URI originalUri = getAbsolutePath();
+
+ // Submit the task to be run in the background
+ final Runnable taskWrapper = () -> {
+ try {
+ updateVariableRegistryReplicated(groupId, originalUri, affectedProcessors, affectedServices, updateRequest, requestEntity);
+ } catch (final Exception e) {
+ logger.error("Failed to update variable registry", e);
+ updateRequest.setFailureReason("An unexpected error has occurred: " + e);
+ }
+ };
+
+ variableRegistryThreadPool.submit(taskWrapper);
+
+ final VariableRegistryUpdateRequestEntity responseEntity = new VariableRegistryUpdateRequestEntity();
+ responseEntity.setRequestDto(dtoFactory.createVariableRegistryUpdateRequestDto(updateRequest));
+ responseEntity.getRequestDto().setUri(generateResourceUri("process-groups", groupId, "variable-registry", "update-requests", updateRequest.getRequestId()));
+
+ final URI location = URI.create(responseEntity.getRequestDto().getUri());
+ return Response.status(Status.ACCEPTED).location(location).entity(responseEntity).build();
+ }
+
+
+ final Revision requestRevision = getRevision(requestEntity.getProcessGroupRevision(), groupId);
+ return withWriteLock(
+ serviceFacade,
+ requestEntity,
+ requestRevision,
+ lookup -> {
+ final NiFiUser user = NiFiUserUtils.getNiFiUser();
+
+ final Authorizable groupAuthorizable = lookup.getProcessGroup(groupId).getAuthorizable();
+ groupAuthorizable.authorize(authorizer, RequestAction.WRITE, user);
+
+ // For every component that is affected, the user must have READ permissions and WRITE permissions
+ // (because this action requires stopping the component).
+ if (affectedProcessors != null) {
+ for (final AffectedComponentDTO affectedComponent : affectedProcessors) {
+ final Authorizable authorizable = lookup.getProcessor(affectedComponent.getComponentId()).getAuthorizable();
+ authorizable.authorize(authorizer, RequestAction.READ, user);
+ authorizable.authorize(authorizer, RequestAction.WRITE, user);
+ }
+ }
+
+ if (affectedServices != null) {
+ for (final AffectedComponentDTO affectedComponent : affectedServices) {
+ final Authorizable authorizable = lookup.getControllerService(affectedComponent.getComponentId()).getAuthorizable();
+ authorizable.authorize(authorizer, RequestAction.READ, user);
+ authorizable.authorize(authorizer, RequestAction.WRITE, user);
+ }
+ }
+ },
+ null,
+ (revision, varRegistryEntity) -> {
+ return updateVariableRegistryLocal(groupId, affectedProcessors, affectedServices, requestEntity);
+ });
+ }
+
+ private Pause createPause(final VariableRegistryUpdateRequest updateRequest) {
+ return new Pause() {
+ @Override
+ public boolean pause() {
+ if (updateRequest.isComplete()) {
+ return false;
+ }
+
+ try {
+ Thread.sleep(500);
+ } catch (final InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ return false;
+ }
+
+ return !updateRequest.isComplete();
+ }
+ };
+ }
+
+ private void updateVariableRegistryReplicated(final String groupId, final URI originalUri, final Collection<AffectedComponentDTO> affectedProcessors,
+ final Collection<AffectedComponentDTO> affectedServices,
+ final VariableRegistryUpdateRequest updateRequest, final VariableRegistryEntity requestEntity) {
+
+ final NiFiProperties properties = getProperties();
+ final Client jerseyClient = WebUtils.createClient(new DefaultClientConfig(), SslContextFactory.createSslContext(properties));
+ final int connectionTimeout = (int) FormatUtils.getTimeDuration(properties.getClusterNodeConnectionTimeout(), TimeUnit.MILLISECONDS);
+ final int readTimeout = (int) FormatUtils.getTimeDuration(properties.getClusterNodeReadTimeout(), TimeUnit.MILLISECONDS);
+ jerseyClient.getProperties().put(ClientConfig.PROPERTY_CONNECT_TIMEOUT, connectionTimeout);
+ jerseyClient.getProperties().put(ClientConfig.PROPERTY_READ_TIMEOUT, readTimeout);
+ jerseyClient.getProperties().put(ClientConfig.PROPERTY_FOLLOW_REDIRECTS, Boolean.TRUE);
+
+ final Pause pause = createPause(updateRequest);
+
+ // stop processors
+ if (affectedProcessors != null) {
+ logger.info("In order to update Variable Registry for Process Group with ID {}, "
+ + "replicating request to stop {} affected processors", groupId, affectedProcessors.size());
+
+ scheduleProcessors(groupId, originalUri, jerseyClient, updateRequest, pause,
+ affectedProcessors, ScheduledState.STOPPED, updateRequest.getStopProcessorsStep());
+ }
+
+ // disable controller services
+ if (affectedServices != null) {
+ logger.info("In order to update Variable Registry for Process Group with ID {}, "
+ + "replicating request to stop {} affected Controller Services", groupId, affectedServices.size());
+
+ activateControllerServices(groupId, originalUri, jerseyClient, updateRequest, pause,
+ affectedServices, ControllerServiceState.DISABLED, updateRequest.getDisableServicesStep());
+ }
+
+ // apply updates
+ logger.info("In order to update Variable Registry for Process Group with ID {}, "
+ + "replicating request to apply updates to variable registry", groupId);
+ applyVariableRegistryUpdate(groupId, originalUri, jerseyClient, updateRequest, requestEntity);
+
+ // re-enable controller services
+ if (affectedServices != null) {
+ logger.info("In order to update Variable Registry for Process Group with ID {}, "
+ + "replicating request to re-enable {} affected services", groupId, affectedServices.size());
+
+ activateControllerServices(groupId, originalUri, jerseyClient, updateRequest, pause,
+ affectedServices, ControllerServiceState.ENABLED, updateRequest.getEnableServicesStep());
+ }
+
+ // restart processors
+ if (affectedProcessors != null) {
+ logger.info("In order to update Variable Registry for Process Group with ID {}, "
+ + "replicating request to restart {} affected processors", groupId, affectedProcessors.size());
+
+ scheduleProcessors(groupId, originalUri, jerseyClient, updateRequest, pause,
+ affectedProcessors, ScheduledState.RUNNING, updateRequest.getStartProcessorsStep());
+ }
+
+ updateRequest.setComplete(true);
+ }
+
+ /**
+ * Periodically polls the process group with the given ID, waiting for all processors whose ID's are given to have the given Scheduled State.
+ *
+ * @param client the Jersey Client to use for making the request
+ * @param groupId the ID of the Process Group to poll
+ * @param processorIds the ID of all Processors whose state should be equal to the given desired state
+ * @param desiredState the desired state for all processors with the ID's given
+ * @param pause the Pause that can be used to wait between polling
+ * @return <code>true</code> if successful, <code>false</code> if unable to wait for processors to reach the desired state
+ */
+ private boolean waitForProcessorStatus(final Client client, final URI originalUri, final String groupId, final Set<String> processorIds, final ScheduledState desiredState, final Pause pause) {
+ URI groupUri;
+ try {
+ groupUri = new URI(originalUri.getScheme(), originalUri.getUserInfo(), originalUri.getHost(),
+ originalUri.getPort(), "/nifi-api/flow/process-groups/" + groupId + "/status", "recursive=true", originalUri.getFragment());
+ } catch (URISyntaxException e) {
+ throw new RuntimeException(e);
+ }
+
+ boolean continuePolling = true;
+ while (continuePolling) {
+ final ClientResponse response = client.resource(groupUri).header("Content-Type", "application/json").get(ClientResponse.class);
+ if (response.getStatus() != Status.OK.getStatusCode()) {
+ return false;
+ }
+
+ final ProcessGroupStatusEntity statusEntity = response.getEntity(ProcessGroupStatusEntity.class);
+ final ProcessGroupStatusDTO statusDto = statusEntity.getProcessGroupStatus();
+ final ProcessGroupStatusSnapshotDTO statusSnapshotDto = statusDto.getAggregateSnapshot();
+
+ if (isProcessorStatusEqual(statusSnapshotDto, processorIds, desiredState)) {
+ logger.debug("All {} processors of interest now have the desired state of {}", processorIds.size(), desiredState);
+ return true;
+ }
+
+ // Not all of the processors are in the desired state. Pause for a bit and poll again.
+ continuePolling = pause.pause();
+ }
+
+ return false;
+ }
+
+ /**
+ * Periodically polls the process group with the given ID, waiting for all processors whose ID's are given to have the given Scheduled State.
+ *
+ * @param groupId the ID of the Process Group to poll
+ * @param processorIds the ID of all Processors whose state should be equal to the given desired state
+ * @param desiredState the desired state for all processors with the ID's given
+ * @param pause the Pause that can be used to wait between polling
+ * @return <code>true</code> if successful, <code>false</code> if unable to wait for processors to reach the desired state
+ */
+ private boolean waitForLocalProcessorStatus(final String groupId, final Set<String> processorIds, final ScheduledState desiredState, final Pause pause) {
+ boolean continuePolling = true;
+ while (continuePolling) {
+ final ProcessGroupStatusEntity statusEntity = serviceFacade.getProcessGroupStatus(groupId, true);
+ final ProcessGroupStatusDTO statusDto = statusEntity.getProcessGroupStatus();
+ final ProcessGroupStatusSnapshotDTO statusSnapshotDto = statusDto.getAggregateSnapshot();
+
+ if (isProcessorStatusEqual(statusSnapshotDto, processorIds, desiredState)) {
+ logger.debug("All {} processors of interest now have the desired state of {}", processorIds.size(), desiredState);
+ return true;
+ }
+
+ // Not all of the processors are in the desired state. Pause for a bit and poll again.
+ continuePolling = pause.pause();
+ }
+
+ return false;
+ }
+
+ private boolean isProcessorStatusEqual(final ProcessGroupStatusSnapshotDTO statusSnapshot, final Set<String> processorIds, final ScheduledState desiredState) {
+ final String desiredStateName = desiredState.name();
+
+ final boolean allProcessorsMatch = statusSnapshot.getProcessorStatusSnapshots().stream()
+ .map(entity -> entity.getProcessorStatusSnapshot())
+ .filter(status -> processorIds.contains(status.getId()))
+ .allMatch(status -> {
+ final String runStatus = status.getRunStatus();
+ final boolean stateMatches = desiredStateName.equalsIgnoreCase(runStatus);
+ if (!stateMatches) {
+ return false;
+ }
+
+ if (desiredState == ScheduledState.STOPPED && status.getActiveThreadCount() != 0) {
+ return false;
+ }
+
+ return true;
+ });
+
+ if (!allProcessorsMatch) {
+ return false;
+ }
+
+ for (final ProcessGroupStatusSnapshotEntity childGroupEntity : statusSnapshot.getProcessGroupStatusSnapshots()) {
+ final ProcessGroupStatusSnapshotDTO childGroupStatus = childGroupEntity.getProcessGroupStatusSnapshot();
+ final boolean allMatchChildLevel = isProcessorStatusEqual(childGroupStatus, processorIds, desiredState);
+ if (!allMatchChildLevel) {
+ return false;
+ }
+ }
+
+ return true;
+ }
+
+
+
+ /**
+ * Periodically polls the process group with the given ID, waiting for all controller services whose ID's are given to have the given Controller Service State.
+ *
+ * @param client the Jersey Client to use for making the HTTP Request
+ * @param groupId the ID of the Process Group to poll
+ * @param serviceIds the ID of all Controller Services whose state should be equal to the given desired state
+ * @param desiredState the desired state for all services with the ID's given
+ * @param pause the Pause that can be used to wait between polling
+ * @return <code>true</code> if successful, <code>false</code> if unable to wait for services to reach the desired state
+ */
+ private boolean waitForControllerServiceStatus(final Client client, final URI originalUri, final String groupId, final Set<String> serviceIds, final ControllerServiceState desiredState,
+ final Pause pause) {
+ URI groupUri;
+ try {
+ groupUri = new URI(originalUri.getScheme(), originalUri.getUserInfo(), originalUri.getHost(),
+ originalUri.getPort(), "/nifi-api/flow/process-groups/" + groupId + "/controller-services", "includeAncestorGroups=false,includeDescendantGroups=true", originalUri.getFragment());
+ } catch (URISyntaxException e) {
+ throw new RuntimeException(e);
+ }
+
+ boolean continuePolling = true;
+ while (continuePolling) {
+ final ClientResponse response = client.resource(groupUri).header("Content-Type", "application/json").get(ClientResponse.class);
+ if (response.getStatus() != Status.OK.getStatusCode()) {
+ return false;
+ }
+
+ final ControllerServicesEntity controllerServicesEntity = response.getEntity(ControllerServicesEntity.class);
+ final Set<ControllerServiceEntity> serviceEntities = controllerServicesEntity.getControllerServices();
+
+ final String desiredStateName = desiredState.name();
+ final boolean allServicesMatch = serviceEntities.stream()
+ .map(entity -> entity.getComponent())
+ .filter(service -> serviceIds.contains(service.getId()))
+ .map(service -> service.getState())
+ .allMatch(state -> state.equals(desiredStateName));
+
+ if (allServicesMatch) {
+ logger.debug("All {} controller services of interest now have the desired state of {}", serviceIds.size(), desiredState);
+ return true;
+ }
+
+ // Not all of the processors are in the desired state. Pause for a bit and poll again.
+ continuePolling = pause.pause();
+ }
+
+ return false;
+ }
+
+
+ /**
+ * Periodically polls the process group with the given ID, waiting for all controller services whose ID's are given to have the given Controller Service State.
+ *
+ * @param groupId the ID of the Process Group to poll
+ * @param serviceIds the ID of all Controller Services whose state should be equal to the given desired state
+ * @param desiredState the desired state for all services with the ID's given
+ * @param pause the Pause that can be used to wait between polling
+ * @param user the user that is retrieving the controller services
+ * @return <code>true</code> if successful, <code>false</code> if unable to wait for services to reach the desired state
+ */
+ private boolean waitForLocalControllerServiceStatus(final String groupId, final Set<String> serviceIds, final ControllerServiceState desiredState, final Pause pause, final NiFiUser user) {
+ boolean continuePolling = true;
+ while (continuePolling) {
+ final Set<ControllerServiceEntity> serviceEntities = serviceFacade.getControllerServices(groupId, false, true, user);
+
+ final String desiredStateName = desiredState.name();
+ final boolean allServicesMatch = serviceEntities.stream()
+ .map(entity -> entity.getComponent())
+ .filter(service -> serviceIds.contains(service.getId()))
+ .map(service -> service.getState())
+ .allMatch(state -> desiredStateName.equals(state));
+
+ if (allServicesMatch) {
+ logger.debug("All {} controller services of interest now have the desired state of {}", serviceIds.size(), desiredState);
+ return true;
+ }
+
+ // Not all of the processors are in the desired state. Pause for a bit and poll again.
+ continuePolling = pause.pause();
+ }
+
+ return false;
+ }
+
+ private VariableRegistryUpdateRequest createVariableRegistryUpdateRequest(final String groupId) {
+ final VariableRegistryUpdateRequest updateRequest = new VariableRegistryUpdateRequest(UUID.randomUUID().toString(), groupId);
+
+ // before adding to the request map, purge any old requests. Must do this by creating a List of ID's
+ // and then removing those ID's one-at-a-time in order to avoid ConcurrentModificationException.
+ final Date oneMinuteAgo = new Date(System.currentTimeMillis() - VARIABLE_REGISTRY_UPDATE_REQUEST_EXPIRATION);
+ final List<String> completedRequestIds = varRegistryUpdateRequests.entrySet().stream()
+ .filter(entry -> entry.getValue().isComplete())
+ .filter(entry -> entry.getValue().getLastUpdated().before(oneMinuteAgo))
+ .map(Map.Entry::getKey)
+ .collect(Collectors.toList());
+
+ completedRequestIds.stream().forEach(id -> varRegistryUpdateRequests.remove(id));
+
+ final int requestCount = varRegistryUpdateRequests.size();
+ if (requestCount > MAX_VARIABLE_REGISTRY_UPDATE_REQUESTS) {
+ throw new IllegalStateException("There are already " + requestCount + " update requests for variable registries. "
+ + "Cannot issue any more requests until the older ones are deleted or expire");
+ }
+
+ this.varRegistryUpdateRequests.put(updateRequest.getRequestId(), updateRequest);
+ return updateRequest;
+ }
+
+ private Response updateVariableRegistryLocal(final String groupId, final List<AffectedComponentDTO> affectedProcessors, final List<AffectedComponentDTO> affectedServices,
+ final VariableRegistryEntity requestEntity) {
+
+ final Set<String> affectedProcessorIds = affectedProcessors == null ? Collections.emptySet() : affectedProcessors.stream()
+ .map(component -> component.getComponentId())
+ .collect(Collectors.toSet());
+ Map<String, Revision> processorRevisionMap = getRevisions(groupId, affectedProcessorIds);
+
+ final Set<String> affectedServiceIds = affectedServices == null ? Collections.emptySet() : affectedServices.stream()
+ .map(component -> component.getComponentId())
+ .collect(Collectors.toSet());
+ Map<String, Revision> serviceRevisionMap = getRevisions(groupId, affectedServiceIds);
+
+ // update the variable registry
+ final VariableRegistryUpdateRequest updateRequest = createVariableRegistryUpdateRequest(groupId);
+ updateRequest.getIdentifyRelevantComponentsStep().setComplete(true);
+ final Pause pause = createPause(updateRequest);
+
+ final Revision requestRevision = getRevision(requestEntity.getProcessGroupRevision(), groupId);
+
+ final NiFiUser user = NiFiUserUtils.getNiFiUser();
+ final Runnable updateTask = new Runnable() {
+ @Override
+ public void run() {
+ try {
+ // Stop processors
+ performUpdateVariableRegistryStep(groupId, updateRequest, updateRequest.getStopProcessorsStep(), "Stopping Processors",
+ () -> stopProcessors(user, updateRequest, groupId, processorRevisionMap, pause));
+
+ // Update revision map because this will have modified the revisions of our components.
+ final Map<String, Revision> updatedProcessorRevisionMap = getRevisions(groupId, affectedProcessorIds);
+
+ // Disable controller services
+ performUpdateVariableRegistryStep(groupId, updateRequest, updateRequest.getDisableServicesStep(), "Disabling Controller Services",
+ () -> disableControllerServices(user, updateRequest, groupId, serviceRevisionMap, pause));
+
+ // Update revision map because this will have modified the revisions of our components.
+ final Map<String, Revision> updatedServiceRevisionMap = getRevisions(groupId, affectedServiceIds);
+
+ // Apply the updates
+ performUpdateVariableRegistryStep(groupId, updateRequest, updateRequest.getApplyUpdatesStep(), "Applying updates to Variable Registry",
+ () -> serviceFacade.updateVariableRegistry(user, requestRevision, requestEntity.getVariableRegistry()));
+
+ // Re-enable the controller services
+ performUpdateVariableRegistryStep(groupId, updateRequest, updateRequest.getEnableServicesStep(), "Re-enabling Controller Services",
+ () -> enableControllerServices(user, groupId, updatedServiceRevisionMap, pause));
+
+ // Restart processors
+ performUpdateVariableRegistryStep(groupId, updateRequest, updateRequest.getStartProcessorsStep(), "Restarting Processors",
+ () -> startProcessors(user, groupId, updatedProcessorRevisionMap, pause));
+
+ // Set complete
+ updateRequest.setComplete(true);
+ updateRequest.setLastUpdated(new Date());
+ } catch (final Exception e) {
+ logger.error("Failed to update Variable Registry for Proces Group with ID " + groupId, e);
+ updateRequest.setFailureReason("An unexpected error has occurred: " + e);
+ }
+ }
+ };
+
+ // Submit the task to be run in the background
+ variableRegistryThreadPool.submit(updateTask);
+
+ final VariableRegistryUpdateRequestEntity responseEntity = new VariableRegistryUpdateRequestEntity();
+ responseEntity.setRequestDto(dtoFactory.createVariableRegistryUpdateRequestDto(updateRequest));
+ responseEntity.getRequestDto().setUri(generateResourceUri("process-groups", groupId, "variable-registry", "update-requests", updateRequest.getRequestId()));
+
+ final URI location = URI.create(responseEntity.getRequestDto().getUri());
+ return Response.status(Status.ACCEPTED).location(location).entity(responseEntity).build();
+ }
+
+ private Map<String, Revision> getRevisions(final String groupId, final Set<String> componentIds) {
+ final Set<Revision> processorRevisions = serviceFacade.getRevisionsFromGroup(groupId, group -> componentIds);
+ return processorRevisions.stream().collect(Collectors.toMap(revision -> revision.getComponentId(), Function.identity()));
+ }
+
+ private void performUpdateVariableRegistryStep(final String groupId, final VariableRegistryUpdateRequest request, final VariableRegistryUpdateStep step,
+ final String stepDescription, final Runnable action) {
+
+ if (request.isComplete()) {
+ logger.info("In updating Variable Registry for Process Group with ID {}"
+ + ", skipping the following step because the request has completed already: {}", groupId, stepDescription);
+ return;
+ }
+
+ try {
+ logger.info("In order to update Variable Registry for Process Group with ID {}, {}", groupId, stepDescription);
+
+ action.run();
+ step.setComplete(true);
+ } catch (final Exception e) {
+ request.setComplete(true);
+ logger.error("Failed to update variable registry for Process Group with ID {}", groupId, e);
+
+ step.setComplete(true);
+ step.setFailureReason(e.getMessage());
+ request.setFailureReason("Failed to update Variable Registry because failed while performing step: " + stepDescription);
+ }
+
+ request.setLastUpdated(new Date());
+ }
+
+ private void stopProcessors(final NiFiUser user, final VariableRegistryUpdateRequest updateRequest, final String processGroupId,
+ final Map<String, Revision> processorRevisions, final Pause pause) {
+
+ if (processorRevisions.isEmpty()) {
+ return;
+ }
+
+ serviceFacade.verifyScheduleComponents(processGroupId, ScheduledState.STOPPED, processorRevisions.keySet());
+ serviceFacade.scheduleComponents(user, processGroupId, ScheduledState.STOPPED, processorRevisions);
+ waitForLocalProcessorStatus(processGroupId, processorRevisions.keySet(), ScheduledState.STOPPED, pause);
+ }
+
+ private void startProcessors(final NiFiUser user, final String processGroupId, final Map<String, Revision> processorRevisions, final Pause pause) {
+ if (processorRevisions.isEmpty()) {
+ return;
+ }
+
+ serviceFacade.verifyScheduleComponents(processGroupId, ScheduledState.RUNNING, processorRevisions.keySet());
+ serviceFacade.scheduleComponents(user, processGroupId, ScheduledState.RUNNING, processorRevisions);
+ waitForLocalProcessorStatus(processGroupId, processorRevisions.keySet(), ScheduledState.RUNNING, pause);
+ }
+
+ private void disableControllerServices(final NiFiUser user, final VariableRegistryUpdateRequest updateRequest, final String processGroupId,
+ final Map<String, Revision> serviceRevisions, final Pause pause) {
+
+ if (serviceRevisions.isEmpty()) {
+ return;
+ }
+
+ serviceFacade.verifyActivateControllerServices(processGroupId, ControllerServiceState.DISABLED, serviceRevisions.keySet());
+ serviceFacade.activateControllerServices(user, processGroupId, ControllerServiceState.DISABLED, serviceRevisions);
+ waitForLocalControllerServiceStatus(processGroupId, serviceRevisions.keySet(), ControllerServiceState.DISABLED, pause, user);
+ }
+
+ private void enableControllerServices(final NiFiUser user, final String processGroupId, final Map<String, Revision> serviceRevisions, final Pause pause) {
+ if (serviceRevisions.isEmpty()) {
+ return;
+ }
+
+ serviceFacade.verifyActivateControllerServices(processGroupId, ControllerServiceState.ENABLED, serviceRevisions.keySet());
+ serviceFacade.activateControllerServices(user, processGroupId, ControllerServiceState.ENABLED, serviceRevisions);
+ waitForLocalControllerServiceStatus(processGroupId, serviceRevisions.keySet(), ControllerServiceState.ENABLED, pause, user);
+ }
+
+
+ private void scheduleProcessors(final String groupId, final URI originalUri, final Client jerseyClient, final VariableRegistryUpdateRequest updateRequest,
+ final Pause pause, final Collection<AffectedComponentDTO> affectedProcessors, final ScheduledState desiredState, final VariableRegistryUpdateStep updateStep) {
+ final Set<String> affectedProcessorIds = affectedProcessors.stream()
+ .map(component -> component.getComponentId())
+ .collect(Collectors.toSet());
+
+ final Map<String, Revision> processorRevisionMap = getRevisions(groupId, affectedProcessorIds);
+ final Map<String, RevisionDTO> processorRevisionDtoMap = processorRevisionMap.entrySet().stream().collect(
+ Collectors.toMap(Map.Entry::getKey, entry -> dtoFactory.createRevisionDTO(entry.getValue())));
+
+ final ScheduleComponentsEntity stopProcessorsEntity = new ScheduleComponentsEntity();
+ stopProcessorsEntity.setComponents(processorRevisionDtoMap);
+ stopProcessorsEntity.setId(groupId);
+ stopProcessorsEntity.setState(desiredState.name());
+
+ URI stopProcessorUri;
+ try {
+ stopProcessorUri = new URI(originalUri.getScheme(), originalUri.getUserInfo(), originalUri.getHost(),
+ originalUri.getPort(), "/nifi-api/flow/process-groups/" + groupId, null, originalUri.getFragment());
+ } catch (URISyntaxException e) {
+ throw new RuntimeException(e);
+ }
+
+ final ClientResponse stopProcessorResponse = jerseyClient.resource(stopProcessorUri)
+ .header("Content-Type", "application/json")
+ .entity(stopProcessorsEntity)
+ .put(ClientResponse.class);
+
+ final int stopProcessorStatus = stopProcessorResponse.getStatus();
+ if (stopProcessorStatus != Status.OK.getStatusCode()) {
+ updateRequest.getStopProcessorsStep().setFailureReason("Failed while " + updateStep.getDescription());
+ updateRequest.setFailureReason("Failed while " + updateStep.getDescription());
+ return;
+ }
+
+ updateRequest.setLastUpdated(new Date());
+ final boolean processorsTransitioned = waitForProcessorStatus(jerseyClient, originalUri, groupId, affectedProcessorIds, desiredState, pause);
+ if (processorsTransitioned) {
+ updateStep.setComplete(true);
+ } else {
+ updateStep.setFailureReason("Failed while " + updateStep.getDescription());
+ updateRequest.setFailureReason("Failed while " + updateStep.getDescription());
+ return;
+ }
+ }
+
+ private void activateControllerServices(final String groupId, final URI originalUri, final Client jerseyClient, final VariableRegistryUpdateRequest updateRequest,
+ final Pause pause, final Collection<AffectedComponentDTO> affectedServices, final ControllerServiceState desiredState, final VariableRegistryUpdateStep updateStep) {
+
+ final Set<String> affectedServiceIds = affectedServices.stream()
+ .map(component -> component.getComponentId())
+ .collect(Collectors.toSet());
+
+ final Map<String, Revision> serviceRevisionMap = getRevisions(groupId, affectedServiceIds);
+ final Map<String, RevisionDTO> serviceRevisionDtoMap = serviceRevisionMap.entrySet().stream().collect(
+ Collectors.toMap(Map.Entry::getKey, entry -> dtoFactory.createRevisionDTO(entry.getValue())));
+
+ final ActivateControllerServicesEntity disableServicesEntity = new ActivateControllerServicesEntity();
+ disableServicesEntity.setComponents(serviceRevisionDtoMap);
+ disableServicesEntity.setId(groupId);
+ disableServicesEntity.setState(desiredState.name());
+
+ URI disableServicesUri;
+ try {
+ disableServicesUri = new URI(originalUri.getScheme(), originalUri.getUserInfo(), originalUri.getHost(),
+ originalUri.getPort(), "/nifi-api/flow/process-groups/" + groupId + "/controller-services", null, originalUri.getFragment());
+ } catch (URISyntaxException e) {
+ throw new RuntimeException(e);
+ }
+
+ final ClientResponse disableServicesResponse = jerseyClient.resource(disableServicesUri)
+ .header("Content-Type", "application/json")
+ .entity(disableServicesEntity)
+ .put(ClientResponse.class);
+
+ final int disableServicesStatus = disableServicesResponse.getStatus();
+ if (disableServicesStatus != Status.OK.getStatusCode()) {
+ updateStep.setFailureReason("Failed while " + updateStep.getDescription());
+ updateRequest.setFailureReason("Failed while " + updateStep.getDescription());
+ return;
+ }
+
+ updateRequest.setLastUpdated(new Date());
+ if (waitForControllerServiceStatus(jerseyClient, originalUri, groupId, affectedServiceIds, desiredState, pause)) {
+ updateStep.setComplete(true);
+ } else {
+ updateStep.setFailureReason("Failed while " + updateStep.getDescription());
+ updateRequest.setFailureReason("Failed while " + updateStep.getDescription());
+ return;
+ }
+ }
+
+
+ private void applyVariableRegistryUpdate(final String groupId, final URI originalUri, final Client jerseyClient, final VariableRegistryUpdateRequest updateRequest,
+ final VariableRegistryEntity updateEntity) {
+
+ URI applyUpdatesUri;
+ try {
+ applyUpdatesUri = new URI(originalUri.getScheme(), originalUri.getUserInfo(), originalUri.getHost(),
+ originalUri.getPort(), "/nifi-api/process-groups/" + groupId + "/variable-registry", null, originalUri.getFragment());
+ } catch (URISyntaxException e) {
+ throw new RuntimeException(e);
+ }
+
+ final ClientResponse applyUpdatesResponse = jerseyClient.resource(applyUpdatesUri)
+ .header("Content-Type", "application/json")
+ .entity(updateEntity)
+ .put(ClientResponse.class);
+
+ final int applyUpdatesStatus = applyUpdatesResponse.getStatus();
+ updateRequest.setLastUpdated(new Date());
+ if (applyUpdatesStatus != Status.OK.getStatusCode()) {
+ updateRequest.getApplyUpdatesStep().setFailureReason("Failed to apply updates to the Variable Registry");
+ updateRequest.setFailureReason("Failed to apply updates to the Variable Registry");
+ return;
+ }
+ }
+
/**
* Removes the specified process group reference.
*
@@ -2426,4 +3378,8 @@ public class ProcessGroupResource extends ApplicationResource {
public void setAuthorizer(Authorizer authorizer) {
this.authorizer = authorizer;
}
+
+ public void setDtoFactory(DtoFactory dtoFactory) {
+ this.dtoFactory = dtoFactory;
+ }
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index 4332a0c..a445e49 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -16,6 +16,33 @@
*/
package org.apache.nifi.web.api.dto;
+import java.text.Collator;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TimeZone;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import javax.ws.rs.WebApplicationException;
+
import org.apache.commons.lang3.ClassUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.action.Action;
@@ -47,6 +74,7 @@ import org.apache.nifi.authorization.Resource;
import org.apache.nifi.authorization.User;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.resource.ComponentAuthorizable;
+import org.apache.nifi.authorization.user.NiFiUser;
import org.apache.nifi.authorization.user.NiFiUserUtils;
import org.apache.nifi.bundle.Bundle;
import org.apache.nifi.bundle.BundleCoordinate;
@@ -110,6 +138,9 @@ import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
import org.apache.nifi.provenance.lineage.LineageEdge;
import org.apache.nifi.provenance.lineage.LineageNode;
import org.apache.nifi.provenance.lineage.ProvenanceEventLineageNode;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.registry.variable.VariableRegistryUpdateRequest;
+import org.apache.nifi.registry.variable.VariableRegistryUpdateStep;
import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.reporting.Bulletin;
@@ -161,35 +192,10 @@ import org.apache.nifi.web.api.entity.ProcessGroupStatusSnapshotEntity;
import org.apache.nifi.web.api.entity.ProcessorStatusSnapshotEntity;
import org.apache.nifi.web.api.entity.RemoteProcessGroupStatusSnapshotEntity;
import org.apache.nifi.web.api.entity.TenantEntity;
+import org.apache.nifi.web.api.entity.VariableEntity;
import org.apache.nifi.web.controller.ControllerFacade;
import org.apache.nifi.web.revision.RevisionManager;
-import javax.ws.rs.WebApplicationException;
-import java.text.Collator;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TimeZone;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
public final class DtoFactory {
@SuppressWarnings("rawtypes")
@@ -1712,9 +1718,34 @@ public final class DtoFactory {
* @return dto
*/
public PermissionsDTO createPermissionsDto(final Authorizable authorizable) {
+ return createPermissionsDto(authorizable, NiFiUserUtils.getNiFiUser());
+ }
+
+ /**
+ * Creates the PermissionsDTO based on the specified Authorizable for the given user
+ *
+ * @param authorizable authorizable
+ * @param user the NiFi User for which the Permissions are being created
+ * @return dto
+ */
+ public PermissionsDTO createPermissionsDto(final Authorizable authorizable, final NiFiUser user) {
final PermissionsDTO dto = new PermissionsDTO();
- dto.setCanRead(authorizable.isAuthorized(authorizer, RequestAction.READ, NiFiUserUtils.getNiFiUser()));
- dto.setCanWrite(authorizable.isAuthorized(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()));
+ dto.setCanRead(authorizable.isAuthorized(authorizer, RequestAction.READ, user));
+ dto.setCanWrite(authorizable.isAuthorized(authorizer, RequestAction.WRITE, user));
+ return dto;
+ }
+
+ public AffectedComponentDTO createAffectedComponentDto(final ConfiguredComponent component) {
+ final AffectedComponentDTO dto = new AffectedComponentDTO();
+ dto.setComponentId(component.getIdentifier());
+ dto.setParentGroupId(component.getProcessGroupIdentifier());
+
+ if (component instanceof ProcessorNode) {
+ dto.setComponentType(AffectedComponentDTO.COMPONENT_TYPE_PROCESSOR);
+ } else if (component instanceof ControllerServiceNode) {
+ dto.setComponentType(AffectedComponentDTO.COMPONENT_TYPE_CONTROLLER_SERVICE);
+ }
+
return dto;
}
@@ -1998,6 +2029,10 @@ public final class DtoFactory {
dto.setComments(group.getComments());
dto.setName(group.getName());
+ final Map<String, String> variables = group.getVariableRegistry().getVariableMap().entrySet().stream()
+ .collect(Collectors.toMap(entry -> entry.getKey().getName(), entry -> entry.getValue()));
+ group.setVariables(variables);
+
final ProcessGroup parentGroup = group.getParent();
if (parentGroup != null) {
dto.setParentGroupId(parentGroup.getIdentifier());
@@ -2079,6 +2114,128 @@ public final class DtoFactory {
return deprecationNotice == null ? null : deprecationNotice.reason();
}
+
+ public VariableRegistryDTO createVariableRegistryDto(final ProcessGroup processGroup) {
+ final ComponentVariableRegistry variableRegistry = processGroup.getVariableRegistry();
+
+ final List<String> variableNames = variableRegistry.getVariableMap().keySet().stream()
+ .map(descriptor -> descriptor.getName())
+ .collect(Collectors.toList());
+
+ final Set<VariableEntity> variableEntities = new LinkedHashSet<>();
+
+ for (final String variableName : variableNames) {
+ final VariableDTO variableDto = new VariableDTO();
+ variableDto.setName(variableName);
+ variableDto.setValue(variableRegistry.getVariableValue(variableName));
+ variableDto.setProcessGroupId(processGroup.getIdentifier());
+
+ final Set<ConfiguredComponent> affectedComponents = processGroup.getComponentsAffectedByVariable(variableName);
+ final Set<AffectedComponentDTO> affectedComponentDtos = affectedComponents.stream()
+ .map(component -> createAffectedComponentDto(component))
+ .collect(Collectors.toSet());
+
+ boolean canWrite = true;
+ for (final ConfiguredComponent component : affectedComponents) {
+ final PermissionsDTO permissions = createPermissionsDto(component);
+ if (!permissions.getCanRead() || !permissions.getCanWrite()) {
+ canWrite = false;
+ break;
+ }
+ }
+
+ variableDto.setAffectedComponents(affectedComponentDtos);
+
+ final VariableEntity variableEntity = new VariableEntity();
+ variableEntity.setVariable(variableDto);
+ variableEntity.setCanWrite(canWrite);
+
+ variableEntities.add(variableEntity);
+ }
+
+ final VariableRegistryDTO registryDto = new VariableRegistryDTO();
+ registryDto.setProcessGroupId(processGroup.getIdentifier());
+ registryDto.setVariables(variableEntities);
+
+ return registryDto;
+ }
+
+ public VariableRegistryUpdateRequestDTO createVariableRegistryUpdateRequestDto(final VariableRegistryUpdateRequest request) {
+ final VariableRegistryUpdateRequestDTO dto = new VariableRegistryUpdateRequestDTO();
+ dto.setComplete(request.isComplete());
+ dto.setFailureReason(request.getFailureReason());
+ dto.setLastUpdated(request.getLastUpdated());
+ dto.setProcessGroupId(request.getProcessGroupId());
+ dto.setRequestId(request.getRequestId());
+ dto.setSubmissionTime(request.getSubmissionTime());
+
+ final List<VariableRegistryUpdateStepDTO> updateSteps = new ArrayList<>();
+ updateSteps.add(createVariableRegistryUpdateStepDto(request.getIdentifyRelevantComponentsStep()));
+ updateSteps.add(createVariableRegistryUpdateStepDto(request.getStopProcessorsStep()));
+ updateSteps.add(createVariableRegistryUpdateStepDto(request.getDisableServicesStep()));
+ updateSteps.add(createVariableRegistryUpdateStepDto(request.getApplyUpdatesStep()));
+ updateSteps.add(createVariableRegistryUpdateStepDto(request.getEnableServicesStep()));
+ updateSteps.add(createVariableRegistryUpdateStepDto(request.getStartProcessorsStep()));
+ dto.setUpdateSteps(updateSteps);
+
+ return dto;
+ }
+
+ public VariableRegistryUpdateStepDTO createVariableRegistryUpdateStepDto(final VariableRegistryUpdateStep step) {
+ final VariableRegistryUpdateStepDTO dto = new VariableRegistryUpdateStepDTO();
+ dto.setComplete(step.isComplete());
+ dto.setDescription(step.getDescription());
+ dto.setFailureReason(step.getFailureReason());
+ return dto;
+ }
+
+
+ public VariableRegistryDTO populateAffectedComponents(final VariableRegistryDTO variableRegistry, final ProcessGroup group) {
+ if (!group.getIdentifier().equals(variableRegistry.getProcessGroupId())) {
+ throw new IllegalArgumentException("Variable Registry does not have the same Group ID as the given Process Group");
+ }
+
+ final Set<VariableEntity> variableEntities = new LinkedHashSet<>();
+
+ for (final VariableEntity inputEntity : variableRegistry.getVariables()) {
+ final VariableEntity entity = new VariableEntity();
+
+ final VariableDTO inputDto = inputEntity.getVariable();
+ final VariableDTO variableDto = new VariableDTO();
+ variableDto.setName(inputDto.getName());
+ variableDto.setValue(inputDto.getValue());
+ variableDto.setProcessGroupId(group.getIdentifier());
+
+ final Set<ConfiguredComponent> affectedComponents = group.getComponentsAffectedByVariable(variableDto.getName());
+ final Set<AffectedComponentDTO> affectedComponentDtos = affectedComponents.stream()
+ .map(component -> createAffectedComponentDto(component))
+ .collect(Collectors.toSet());
+
+ boolean canWrite = true;
+ for (final ConfiguredComponent component : affectedComponents) {
+ final PermissionsDTO permissions = createPermissionsDto(component);
+ if (!permissions.getCanRead() || !permissions.getCanWrite()) {
+ canWrite = false;
+ break;
+ }
+ }
+
+ variableDto.setAffectedComponents(affectedComponentDtos);
+
+ entity.setCanWrite(canWrite);
+ entity.setVariable(inputDto);
+
+ variableEntities.add(entity);
+ }
+
+ final VariableRegistryDTO registryDto = new VariableRegistryDTO();
+ registryDto.setProcessGroupId(group.getIdentifier());
+ registryDto.setVariables(variableEntities);
+
+ return registryDto;
+ }
+
+
/**
* Gets the capability description from the specified class.
*/
@@ -3016,6 +3173,10 @@ public final class DtoFactory {
copy.setActiveRemotePortCount(original.getActiveRemotePortCount());
copy.setInactiveRemotePortCount(original.getInactiveRemotePortCount());
+ if (original.getVariables() != null) {
+ copy.setVariables(new HashMap<>(original.getVariables()));
+ }
+
return copy;
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
index 41249ba..a7f370a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/EntityFactory.java
@@ -65,6 +65,7 @@ import org.apache.nifi.web.api.entity.StatusHistoryEntity;
import org.apache.nifi.web.api.entity.TenantEntity;
import org.apache.nifi.web.api.entity.UserEntity;
import org.apache.nifi.web.api.entity.UserGroupEntity;
+import org.apache.nifi.web.api.entity.VariableRegistryEntity;
import java.util.Date;
import java.util.List;
@@ -431,6 +432,18 @@ public final class EntityFactory {
return entity;
}
+ public VariableRegistryEntity createVariableRegistryEntity(final VariableRegistryDTO dto, final RevisionDTO revision, final PermissionsDTO permissions) {
+ final VariableRegistryEntity entity = new VariableRegistryEntity();
+ entity.setProcessGroupRevision(revision);
+ if (dto != null) {
+ if (permissions != null && permissions.getCanRead()) {
+ entity.setVariableRegistry(dto);
+ }
+ }
+
+ return entity;
+ }
+
public ControllerServiceEntity createControllerServiceEntity(final ControllerServiceDTO dto, final RevisionDTO revision, final PermissionsDTO permissions, final List<BulletinEntity> bulletins) {
final ControllerServiceEntity entity = new ControllerServiceEntity();
entity.setRevision(revision);
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java
index fa92425..0409e95 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java
@@ -63,7 +63,7 @@ public interface ControllerServiceDAO {
*
* @return The controller services
*/
- Set<ControllerServiceNode> getControllerServices(String groupId);
+ Set<ControllerServiceNode> getControllerServices(String groupId, boolean includeAncestorGroups, boolean includeDescendantGroups);
/**
* Updates the specified controller service.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java
index 155b36e..d7ca806 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ProcessGroupDAO.java
@@ -16,11 +16,14 @@
*/
package org.apache.nifi.web.dao;
+import java.util.Set;
+import java.util.concurrent.Future;
+
import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.web.api.dto.ProcessGroupDTO;
-
-import java.util.Set;
+import org.apache.nifi.web.api.dto.VariableRegistryDTO;
public interface ProcessGroupDAO {
@@ -65,12 +68,32 @@ public interface ProcessGroupDAO {
void verifyScheduleComponents(String groupId, ScheduledState state, Set<String> componentIds);
/**
+ * Verifies the specified controller services can be modified
+ *
+ * @param groupId the ID of the process group
+ * @param state the desired state
+ * @param serviceIds the ID's of the controller services
+ */
+ void verifyActivateControllerServices(String groupId, ControllerServiceState state, Set<String> serviceIds);
+
+ /**
* Schedules the components in the specified process group.
*
* @param groupId id
* @param state scheduled state
+ *
+ * @return a Future that can be used to wait for the services to finish starting or stopping
*/
- void scheduleComponents(String groupId, ScheduledState state, Set<String> componentIds);
+ Future<Void> scheduleComponents(String groupId, ScheduledState state, Set<String> componentIds);
+
+ /**
+ * Enables or disables the controller services in the specified process group
+ *
+ * @param groupId the id of the group
+ * @param state the desired state
+ * @param serviceIds the ID's of the services to enable or disable
+ */
+ Future<Void> activateControllerServices(String groupId, ControllerServiceState state, Set<String> serviceIds);
/**
* Updates the specified process group.
@@ -81,6 +104,21 @@ public interface ProcessGroupDAO {
ProcessGroup updateProcessGroup(ProcessGroupDTO processGroup);
/**
+ * Updates the specified variable registry
+ *
+ * @param variableRegistry the Variable Registry
+ * @return the Process Group that was updated
+ */
+ ProcessGroup updateVariableRegistry(VariableRegistryDTO variableRegistry);
+
+ /**
+ * Verifies that the specified updates to a current Process Group can be applied at this time
+ *
+ * @param processGroup the DTO That describes the changes to occur
+ */
+ void verifyUpdate(ProcessGroupDTO processGroup);
+
+ /**
* Verifies the specified process group can be removed.
*
* @param groupId id
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
index 36cf85b..0f9ec7a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
@@ -124,7 +124,7 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
}
@Override
- public Set<ControllerServiceNode> getControllerServices(final String groupId) {
+ public Set<ControllerServiceNode> getControllerServices(final String groupId, final boolean includeAncestorGroups, final boolean includeDescendantGroups) {
if (groupId == null) {
return flowController.getRootControllerServices();
} else {
@@ -134,7 +134,12 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
throw new ResourceNotFoundException("Could not find Process Group with ID " + groupId);
}
- return procGroup.getControllerServices(true);
+ final Set<ControllerServiceNode> serviceNodes = procGroup.getControllerServices(includeAncestorGroups);
+ if (includeDescendantGroups) {
+ serviceNodes.addAll(procGroup.findAllControllerServices());
+ }
+
+ return serviceNodes;
}
}
[8/9] nifi git commit: NIFI-4224: - Initial implementation of Process
Group level Variable Registry - Updated to incorporate PR Feedback - Changed
log message because slf4j-simple apparently has a memory leak;
passing a String instead of passing in the C
Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
index 4b1ce59..fe2c060 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java
@@ -16,209 +16,21 @@
*/
package org.apache.nifi.attribute.expression.language;
-import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionLexer;
-import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser;
-import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator;
+import org.antlr.runtime.tree.Tree;
+import org.apache.nifi.attribute.expression.language.compile.CompiledExpression;
+import org.apache.nifi.attribute.expression.language.compile.ExpressionCompiler;
import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.BooleanCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.DateCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.DecimalCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.NumberCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.StringCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.cast.WholeNumberCastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.AndEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.AppendEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.AttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ContainsEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.DivideEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.EndsWithEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.EqualsEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.EqualsIgnoreCaseEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.CharSequenceTranslatorEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.FindEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.FormatEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.FromRadixEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.GetDelimitedFieldEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.GetStateVariableEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.GreaterThanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.GreaterThanOrEqualEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.HostnameEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.IPEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.IfElseEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.InEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.IndexOfEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.IsEmptyEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.IsNullEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.JsonPathEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.LastIndexOfEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.LengthEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.LessThanEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.LessThanOrEqualEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.MatchesEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.MathEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.MinusEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ModEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.MultiplyEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.NotEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.NotNullEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.NowEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.NumberToDateEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.OneUpSequenceEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.OrEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.PlusEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.PrependEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.RandomNumberGeneratorEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceAllEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceEmptyEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceFirstEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceNullEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.StartsWithEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.StringToDateEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringAfterEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringAfterLastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringBeforeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringBeforeLastEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ToLowerEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ToRadixEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ToStringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.ToUpperEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.TrimEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.UrlDecodeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.UrlEncodeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.Base64DecodeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.Base64EncodeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.functions.UuidEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.BooleanLiteralEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.DecimalLiteralEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.ToLiteralEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.literals.WholeNumberLiteralEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.reduce.CountEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.reduce.JoinEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.reduce.ReduceEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.AllAttributesEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.AnyAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.DelineatedAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.IteratingEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiMatchAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiNamedAttributeEvaluator;
-import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
import org.apache.nifi.expression.AttributeExpression.ResultType;
import org.apache.nifi.expression.AttributeValueDecorator;
-import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.exception.ProcessException;
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CharStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.tree.Tree;
-
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FROM_RADIX;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IF_ELSE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MATH;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_ATTRIBUTES;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_DELINEATED_VALUES;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_MATCHING_ATTRIBUTES;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.AND;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_ATTRIBUTE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_DELINEATED_VALUE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_MATCHING_ATTRIBUTE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.APPEND;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ATTRIBUTE_REFERENCE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ATTR_NAME;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.CONTAINS;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.DECIMAL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IN;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.COUNT;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.DIVIDE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ENDS_WITH;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EQUALS;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EQUALS_IGNORE_CASE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EXPRESSION;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FALSE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FIND;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FORMAT;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GET_DELIMITED_FIELD;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GET_STATE_VALUE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GREATER_THAN;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GREATER_THAN_OR_EQUAL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.HOSTNAME;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.INDEX_OF;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IP;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IS_EMPTY;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IS_NULL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.JOIN;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.JSON_PATH;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LAST_INDEX_OF;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LENGTH;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LESS_THAN;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LESS_THAN_OR_EQUAL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MATCHES;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MINUS;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MOD;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MULTIPLY;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MULTI_ATTRIBUTE_REFERENCE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NEXT_INT;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOT;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOT_NULL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOW;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_DECIMAL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.WHOLE_NUMBER;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.OR;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PLUS;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PREPEND;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.RANDOM;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_ALL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_EMPTY;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_FIRST;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_NULL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.STARTS_WITH;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.STRING_LITERAL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_AFTER;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_AFTER_LAST;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE_LAST;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_DATE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_LITERAL;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_LOWER;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_NUMBER;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_RADIX;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_STRING;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_UPPER;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TRIM;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TRUE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.URL_DECODE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.URL_ENCODE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.BASE64_DECODE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.BASE64_ENCODE;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_JSON;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_CSV;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_HTML3;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_HTML4;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_XML;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_JSON;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_CSV;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_HTML3;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_HTML4;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_XML;
-import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UUID;
-
-import org.apache.nifi.attribute.expression.language.evaluation.selection.MappingEvaluator;
/**
* Class used for creating and evaluating NiFi Expression Language. Once a Query
@@ -413,18 +225,6 @@ public class Query {
return Query.prepare(rawValue).evaluateExpressions(valueLookup, decorator);
}
- private static Evaluator<?> getRootSubjectEvaluator(final Evaluator<?> evaluator) {
- if (evaluator == null) {
- return null;
- }
-
- final Evaluator<?> subject = evaluator.getSubjectEvaluator();
- if (subject == null) {
- return evaluator;
- }
-
- return getRootSubjectEvaluator(subject);
- }
/**
* Un-escapes ${...} patterns that were escaped
@@ -436,28 +236,11 @@ public class Query {
return value.replaceAll("\\$\\$(?=\\$*\\{.*?\\})", "\\$");
}
-
public static Query fromTree(final Tree tree, final String text) {
- return new Query(text, tree, buildEvaluator(tree));
+ final ExpressionCompiler compiler = new ExpressionCompiler();
+ return new Query(text, tree, compiler.buildEvaluator(tree));
}
- public static Tree compileTree(final String query) throws AttributeExpressionLanguageParsingException {
- try {
- final CommonTokenStream lexerTokenStream = createTokenStream(query);
- final AttributeExpressionParser parser = new AttributeExpressionParser(lexerTokenStream);
- final Tree ast = (Tree) parser.query().getTree();
- final Tree tree = ast.getChild(0);
-
- // ensure that we are able to build the evaluators, so that we validate syntax
- final Evaluator<?> evaluator = buildEvaluator(tree);
- verifyMappingEvaluatorReduced(evaluator);
- return tree;
- } catch (final AttributeExpressionLanguageParsingException e) {
- throw e;
- } catch (final Exception e) {
- throw new AttributeExpressionLanguageParsingException(e);
- }
- }
public static PreparedQuery prepare(final String query) throws AttributeExpressionLanguageParsingException {
if (query == null) {
@@ -470,9 +253,11 @@ public class Query {
return new EmptyPreparedQuery(query.replace("$$", "$"));
}
+ final ExpressionCompiler compiler = new ExpressionCompiler();
+
try {
final List<String> substrings = new ArrayList<>();
- final Map<String, Tree> trees = new HashMap<>();
+ final Map<String, CompiledExpression> compiledExpressions = new HashMap<>();
int lastIndex = 0;
for (final Range range : ranges) {
@@ -483,7 +268,10 @@ public class Query {
final String treeText = query.substring(range.getStart(), range.getEnd() + 1).replace("$$", "$");
substrings.add(treeText);
- trees.put(treeText, Query.compileTree(treeText));
+
+ final CompiledExpression compiledExpression = compiler.compile(treeText);
+
+ compiledExpressions.put(treeText, compiledExpression);
lastIndex = range.getEnd() + 1;
}
@@ -493,7 +281,7 @@ public class Query {
substrings.add(treeText);
}
- return new StandardPreparedQuery(substrings, trees);
+ return new StandardPreparedQuery(substrings, compiledExpressions);
} catch (final AttributeExpressionLanguageParsingException e) {
return new InvalidPreparedQuery(query, e.getMessage());
}
@@ -501,15 +289,10 @@ public class Query {
public static Query compile(final String query) throws AttributeExpressionLanguageParsingException {
try {
- final CommonTokenStream lexerTokenStream = createTokenStream(query);
- final AttributeExpressionParser parser = new AttributeExpressionParser(lexerTokenStream);
- final Tree ast = (Tree) parser.query().getTree();
- final Tree tree = ast.getChild(0);
+ final ExpressionCompiler compiler = new ExpressionCompiler();
+ final CompiledExpression compiledExpression = compiler.compile(query);
- final Evaluator<?> evaluator = buildEvaluator(tree);
- verifyMappingEvaluatorReduced(evaluator);
-
- return new Query(query, tree, evaluator);
+ return new Query(compiledExpression.getExpression(), compiledExpression.getTree(), compiledExpression.getRootEvaluator());
} catch (final AttributeExpressionLanguageParsingException e) {
throw e;
} catch (final Exception e) {
@@ -517,57 +300,6 @@ public class Query {
}
}
- private static void verifyMappingEvaluatorReduced(final Evaluator<?> evaluator) {
- final Evaluator<?> rightMostEvaluator;
- if (evaluator instanceof IteratingEvaluator) {
- rightMostEvaluator = ((IteratingEvaluator<?>) evaluator).getLogicEvaluator();
- } else {
- rightMostEvaluator = evaluator;
- }
-
- Evaluator<?> eval = rightMostEvaluator.getSubjectEvaluator();
- Evaluator<?> lastEval = rightMostEvaluator;
- while (eval != null) {
- if (eval instanceof ReduceEvaluator) {
- throw new AttributeExpressionLanguageParsingException("Expression attempts to call function '" + lastEval.getToken() + "' on the result of '" + eval.getToken() +
- "'. This is not allowed. Instead, use \"${literal( ${<embedded expression>} ):" + lastEval.getToken() + "(...)}\"");
- }
-
- lastEval = eval;
- eval = eval.getSubjectEvaluator();
- }
-
- // if the result type of the evaluator is BOOLEAN, then it will always
- // be reduced when evaluator.
- final ResultType resultType = evaluator.getResultType();
- if (resultType == ResultType.BOOLEAN) {
- return;
- }
-
- final Evaluator<?> rootEvaluator = getRootSubjectEvaluator(evaluator);
- if (rootEvaluator != null && rootEvaluator instanceof MultiAttributeEvaluator) {
- final MultiAttributeEvaluator multiAttrEval = (MultiAttributeEvaluator) rootEvaluator;
- switch (multiAttrEval.getEvaluationType()) {
- case ALL_ATTRIBUTES:
- case ALL_MATCHING_ATTRIBUTES:
- case ALL_DELINEATED_VALUES: {
- if (!(evaluator instanceof ReduceEvaluator)) {
- throw new AttributeExpressionLanguageParsingException("Cannot evaluate expression because it attempts to reference multiple attributes but does not use a reducing function");
- }
- break;
- }
- default:
- throw new AttributeExpressionLanguageParsingException("Cannot evaluate expression because it attempts to reference multiple attributes but does not use a reducing function");
- }
- }
- }
-
- private static CommonTokenStream createTokenStream(final String expression) throws AttributeExpressionLanguageParsingException {
- final CharStream input = new ANTLRStringStream(expression);
- final AttributeExpressionLexer lexer = new AttributeExpressionLexer(input);
- return new CommonTokenStream(lexer);
- }
-
public ResultType getResultType() {
return evaluator.getResultType();
}
@@ -598,789 +330,7 @@ public class Query {
return "Query [" + query + "]";
}
- private static Evaluator<String> newStringLiteralEvaluator(final String literalValue) {
- if (literalValue == null || literalValue.length() < 2) {
- return new StringLiteralEvaluator(literalValue);
- }
-
- final List<Range> ranges = extractExpressionRanges(literalValue);
- if (ranges.isEmpty()) {
- return new StringLiteralEvaluator(literalValue);
- }
-
- final List<Evaluator<?>> evaluators = new ArrayList<>();
-
- int lastIndex = 0;
- for (final Range range : ranges) {
- if (range.getStart() > lastIndex) {
- evaluators.add(newStringLiteralEvaluator(literalValue.substring(lastIndex, range.getStart())));
- }
-
- final String treeText = literalValue.substring(range.getStart(), range.getEnd() + 1);
- evaluators.add(buildEvaluator(compileTree(treeText)));
- lastIndex = range.getEnd() + 1;
- }
-
- final Range lastRange = ranges.get(ranges.size() - 1);
- if (lastRange.getEnd() + 1 < literalValue.length()) {
- final String treeText = literalValue.substring(lastRange.getEnd() + 1);
- evaluators.add(newStringLiteralEvaluator(treeText));
- }
-
- if (evaluators.size() == 1) {
- return toStringEvaluator(evaluators.get(0));
- }
-
- Evaluator<String> lastEvaluator = toStringEvaluator(evaluators.get(0));
- for (int i = 1; i < evaluators.size(); i++) {
- lastEvaluator = new AppendEvaluator(lastEvaluator, toStringEvaluator(evaluators.get(i)));
- }
-
- return lastEvaluator;
- }
-
- private static Evaluator<?> buildEvaluator(final Tree tree) {
- switch (tree.getType()) {
- case EXPRESSION: {
- return buildExpressionEvaluator(tree);
- }
- case ATTRIBUTE_REFERENCE: {
- final Evaluator<?> childEvaluator = buildEvaluator(tree.getChild(0));
- if (childEvaluator instanceof MultiAttributeEvaluator) {
- return childEvaluator;
- }
- return new AttributeEvaluator(toStringEvaluator(childEvaluator));
- }
- case MULTI_ATTRIBUTE_REFERENCE: {
-
- final Tree functionTypeTree = tree.getChild(0);
- final int multiAttrType = functionTypeTree.getType();
- if (multiAttrType == ANY_DELINEATED_VALUE || multiAttrType == ALL_DELINEATED_VALUES) {
- final Evaluator<String> delineatedValueEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(1)));
- final Evaluator<String> delimiterEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(2)));
-
- return new DelineatedAttributeEvaluator(delineatedValueEvaluator, delimiterEvaluator, multiAttrType);
- }
-
- final List<String> attributeNames = new ArrayList<>();
- for (int i = 1; i < tree.getChildCount(); i++) { // skip the first child because that's the name of the multi-attribute function
- attributeNames.add(newStringLiteralEvaluator(tree.getChild(i).getText()).evaluate(null).getValue());
- }
-
- switch (multiAttrType) {
- case ALL_ATTRIBUTES:
- for (final String attributeName : attributeNames) {
- try {
- FlowFile.KeyValidator.validateKey(attributeName);
- } catch (final IllegalArgumentException iae) {
- throw new AttributeExpressionLanguageParsingException("Invalid Attribute Name: " + attributeName + ". " + iae.getMessage());
- }
- }
-
- return new MultiNamedAttributeEvaluator(attributeNames, ALL_ATTRIBUTES);
- case ALL_MATCHING_ATTRIBUTES:
- return new MultiMatchAttributeEvaluator(attributeNames, ALL_MATCHING_ATTRIBUTES);
- case ANY_ATTRIBUTE:
- for (final String attributeName : attributeNames) {
- try {
- FlowFile.KeyValidator.validateKey(attributeName);
- } catch (final IllegalArgumentException iae) {
- throw new AttributeExpressionLanguageParsingException("Invalid Attribute Name: " + attributeName + ". " + iae.getMessage());
- }
- }
-
- return new MultiNamedAttributeEvaluator(attributeNames, ANY_ATTRIBUTE);
- case ANY_MATCHING_ATTRIBUTE:
- return new MultiMatchAttributeEvaluator(attributeNames, ANY_MATCHING_ATTRIBUTE);
- default:
- throw new AssertionError("Illegal Multi-Attribute Reference: " + functionTypeTree.toString());
- }
- }
- case ATTR_NAME: {
- return newStringLiteralEvaluator(tree.getChild(0).getText());
- }
- case WHOLE_NUMBER: {
- return new WholeNumberLiteralEvaluator(tree.getText());
- }
- case STRING_LITERAL: {
- return newStringLiteralEvaluator(tree.getText());
- }
- case DECIMAL: {
- return new DecimalLiteralEvaluator(tree.getText());
- }
- case TRUE:
- case FALSE:
- return buildBooleanEvaluator(tree);
- case UUID: {
- return new UuidEvaluator();
- }
- case NOW: {
- return new NowEvaluator();
- }
- case TO_LITERAL: {
- final Evaluator<?> argEvaluator = buildEvaluator(tree.getChild(0));
- return new ToLiteralEvaluator(argEvaluator);
- }
- case IP: {
- try {
- return new IPEvaluator();
- } catch (final UnknownHostException e) {
- throw new AttributeExpressionLanguageException(e);
- }
- }
- case HOSTNAME: {
- if (tree.getChildCount() == 0) {
- try {
- return new HostnameEvaluator(false);
- } catch (final UnknownHostException e) {
- throw new AttributeExpressionLanguageException(e);
- }
- } else if (tree.getChildCount() == 1) {
- final Tree childTree = tree.getChild(0);
- try {
- switch (childTree.getType()) {
- case TRUE:
- return new HostnameEvaluator(true);
- case FALSE:
- return new HostnameEvaluator(false);
- default:
- throw new AttributeExpressionLanguageParsingException("Call to hostname() must take 0 or 1 (boolean) parameter");
- }
- } catch (final UnknownHostException e) {
- throw new AttributeExpressionLanguageException(e);
- }
- } else {
- throw new AttributeExpressionLanguageParsingException("Call to hostname() must take 0 or 1 (boolean) parameter");
- }
- }
- case NEXT_INT: {
- return new OneUpSequenceEvaluator();
- }
- case RANDOM: {
- return new RandomNumberGeneratorEvaluator();
- }
- case MATH: {
- if (tree.getChildCount() == 1) {
- return addToken(new MathEvaluator(null, toStringEvaluator(buildEvaluator(tree.getChild(0))), null), "math");
- } else {
- throw new AttributeExpressionLanguageParsingException("Call to math() as the subject must take exactly 1 parameter");
- }
- }
- case GET_STATE_VALUE: {
- final Tree childTree = tree.getChild(0);
- final Evaluator<?> argEvaluator = buildEvaluator(childTree);
- final Evaluator<String> stringEvaluator = toStringEvaluator(argEvaluator);
- return new GetStateVariableEvaluator(stringEvaluator);
- }
- default:
- throw new AttributeExpressionLanguageParsingException("Unexpected token: " + tree.toString());
- }
- }
-
- private static <T> Evaluator<T> addToken(final Evaluator<T> evaluator, final String token) {
- evaluator.setToken(token);
- return evaluator;
- }
-
- private static Evaluator<Boolean> buildBooleanEvaluator(final Tree tree) {
- switch (tree.getType()) {
- case TRUE:
- return addToken(new BooleanLiteralEvaluator(true), "true");
- case FALSE:
- return addToken(new BooleanLiteralEvaluator(false), "true");
- }
- throw new AttributeExpressionLanguageParsingException("Cannot build Boolean evaluator from tree " + tree.toString());
- }
-
- @SuppressWarnings({ "rawtypes", "unchecked" })
- private static Evaluator<?> buildExpressionEvaluator(final Tree tree) {
- if (tree.getChildCount() == 0) {
- throw new AttributeExpressionLanguageParsingException("EXPRESSION tree node has no children");
- }
-
- final Evaluator<?> evaluator;
- if (tree.getChildCount() == 1) {
- evaluator = buildEvaluator(tree.getChild(0));
- } else {
- // we can chain together functions in the form of:
- // ${x:trim():substring(1,2):trim()}
- // in this case, the subject of the right-most function is the function to its left; its
- // subject is the function to its left (the first trim()), and its subject is the value of
- // the 'x' attribute. We accomplish this logic by iterating over all of the children of the
- // tree from the right-most child going left-ward.
- evaluator = buildFunctionExpressionEvaluator(tree, 0);
- }
-
- Evaluator<?> chosenEvaluator = evaluator;
- final Evaluator<?> rootEvaluator = getRootSubjectEvaluator(evaluator);
- if (rootEvaluator != null) {
- if (rootEvaluator instanceof MultiAttributeEvaluator) {
- final MultiAttributeEvaluator multiAttrEval = (MultiAttributeEvaluator) rootEvaluator;
-
- switch (multiAttrEval.getEvaluationType()) {
- case ANY_ATTRIBUTE:
- case ANY_MATCHING_ATTRIBUTE:
- case ANY_DELINEATED_VALUE:
- chosenEvaluator = new AnyAttributeEvaluator((BooleanEvaluator) evaluator, multiAttrEval);
- break;
- case ALL_ATTRIBUTES:
- case ALL_MATCHING_ATTRIBUTES:
- case ALL_DELINEATED_VALUES: {
- final ResultType resultType = evaluator.getResultType();
- if (resultType == ResultType.BOOLEAN) {
- chosenEvaluator = new AllAttributesEvaluator((BooleanEvaluator) evaluator, multiAttrEval);
- } else if (evaluator instanceof ReduceEvaluator) {
- chosenEvaluator = new MappingEvaluator((ReduceEvaluator) evaluator, multiAttrEval);
- } else {
- throw new AttributeExpressionLanguageException("Cannot evaluate Expression because it attempts to reference multiple attributes but does not use a reducing function");
- }
- break;
- }
- }
-
- switch (multiAttrEval.getEvaluationType()) {
- case ANY_ATTRIBUTE:
- chosenEvaluator.setToken("anyAttribute");
- break;
- case ANY_MATCHING_ATTRIBUTE:
- chosenEvaluator.setToken("anyMatchingAttribute");
- break;
- case ANY_DELINEATED_VALUE:
- chosenEvaluator.setToken("anyDelineatedValue");
- break;
- case ALL_ATTRIBUTES:
- chosenEvaluator.setToken("allAttributes");
- break;
- case ALL_MATCHING_ATTRIBUTES:
- chosenEvaluator.setToken("allMatchingAttributes");
- break;
- case ALL_DELINEATED_VALUES:
- chosenEvaluator.setToken("allDelineatedValues");
- break;
- }
- }
- }
-
- return chosenEvaluator;
- }
-
- private static Evaluator<?> buildFunctionExpressionEvaluator(final Tree tree, final int offset) {
- if (tree.getChildCount() == 0) {
- throw new AttributeExpressionLanguageParsingException("EXPRESSION tree node has no children");
- }
- final int firstChildIndex = tree.getChildCount() - offset - 1;
- if (firstChildIndex == 0) {
- return buildEvaluator(tree.getChild(0));
- }
- final Tree functionTree = tree.getChild(firstChildIndex);
- final Evaluator<?> subjectEvaluator = buildFunctionExpressionEvaluator(tree, offset + 1);
-
- final Tree functionNameTree = functionTree.getChild(0);
- final List<Evaluator<?>> argEvaluators = new ArrayList<>();
- for (int i = 1; i < functionTree.getChildCount(); i++) {
- argEvaluators.add(buildEvaluator(functionTree.getChild(i)));
- }
- return buildFunctionEvaluator(functionNameTree, subjectEvaluator, argEvaluators);
- }
-
- private static List<Evaluator<?>> verifyArgCount(final List<Evaluator<?>> args, final int count, final String functionName) {
- if (args.size() != count) {
- throw new AttributeExpressionLanguageParsingException(functionName + "() function takes " + count + " arguments");
- }
- return args;
- }
-
- private static Evaluator<String> toStringEvaluator(final Evaluator<?> evaluator) {
- return toStringEvaluator(evaluator, null);
- }
-
- private static Evaluator<String> toStringEvaluator(final Evaluator<?> evaluator, final String location) {
- if (evaluator.getResultType() == ResultType.STRING) {
- return (StringEvaluator) evaluator;
- }
-
- return addToken(new StringCastEvaluator(evaluator), evaluator.getToken());
- }
-
- @SuppressWarnings("unchecked")
- private static Evaluator<Boolean> toBooleanEvaluator(final Evaluator<?> evaluator, final String location) {
- switch (evaluator.getResultType()) {
- case BOOLEAN:
- return (Evaluator<Boolean>) evaluator;
- case STRING:
- return addToken(new BooleanCastEvaluator((StringEvaluator) evaluator), evaluator.getToken());
- default:
- throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.BOOLEAN
- + (location == null ? "" : " at location [" + location + "]"));
- }
-
- }
-
- private static Evaluator<Boolean> toBooleanEvaluator(final Evaluator<?> evaluator) {
- return toBooleanEvaluator(evaluator, null);
- }
-
- private static Evaluator<Long> toWholeNumberEvaluator(final Evaluator<?> evaluator) {
- return toWholeNumberEvaluator(evaluator, null);
- }
-
- @SuppressWarnings("unchecked")
- private static Evaluator<Long> toWholeNumberEvaluator(final Evaluator<?> evaluator, final String location) {
- switch (evaluator.getResultType()) {
- case WHOLE_NUMBER:
- return (Evaluator<Long>) evaluator;
- case STRING:
- case DATE:
- case DECIMAL:
- case NUMBER:
- return addToken(new WholeNumberCastEvaluator(evaluator), evaluator.getToken());
- default:
- throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.WHOLE_NUMBER
- + (location == null ? "" : " at location [" + location + "]"));
- }
- }
-
- private static Evaluator<Double> toDecimalEvaluator(final Evaluator<?> evaluator) {
- return toDecimalEvaluator(evaluator, null);
- }
-
- @SuppressWarnings("unchecked")
- private static Evaluator<Double> toDecimalEvaluator(final Evaluator<?> evaluator, final String location) {
- switch (evaluator.getResultType()) {
- case DECIMAL:
- return (Evaluator<Double>) evaluator;
- case WHOLE_NUMBER:
- case STRING:
- case DATE:
- case NUMBER:
- return addToken(new DecimalCastEvaluator(evaluator), evaluator.getToken());
- default:
- throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.DECIMAL
- + (location == null ? "" : " at location [" + location + "]"));
- }
- }
-
- private static Evaluator<Number> toNumberEvaluator(final Evaluator<?> evaluator) {
- return toNumberEvaluator(evaluator, null);
- }
-
- @SuppressWarnings("unchecked")
- private static Evaluator<Number> toNumberEvaluator(final Evaluator<?> evaluator, final String location) {
- switch (evaluator.getResultType()) {
- case NUMBER:
- return (Evaluator<Number>) evaluator;
- case STRING:
- case DATE:
- case DECIMAL:
- case WHOLE_NUMBER:
- return addToken(new NumberCastEvaluator(evaluator), evaluator.getToken());
- default:
- throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.WHOLE_NUMBER
- + (location == null ? "" : " at location [" + location + "]"));
- }
- }
-
- private static DateEvaluator toDateEvaluator(final Evaluator<?> evaluator) {
- return toDateEvaluator(evaluator, null);
- }
-
- private static DateEvaluator toDateEvaluator(final Evaluator<?> evaluator, final String location) {
- if (evaluator.getResultType() == ResultType.DATE) {
- return (DateEvaluator) evaluator;
- }
-
- return new DateCastEvaluator(evaluator);
- }
-
- private static Evaluator<?> buildFunctionEvaluator(final Tree tree, final Evaluator<?> subjectEvaluator, final List<Evaluator<?>> argEvaluators) {
- switch (tree.getType()) {
- case TRIM: {
- verifyArgCount(argEvaluators, 0, "trim");
- return addToken(new TrimEvaluator(toStringEvaluator(subjectEvaluator)), "trim");
- }
- case TO_STRING: {
- verifyArgCount(argEvaluators, 0, "toString");
- return addToken(new ToStringEvaluator(subjectEvaluator), "toString");
- }
- case TO_LOWER: {
- verifyArgCount(argEvaluators, 0, "toLower");
- return addToken(new ToLowerEvaluator(toStringEvaluator(subjectEvaluator)), "toLower");
- }
- case TO_UPPER: {
- verifyArgCount(argEvaluators, 0, "toUpper");
- return addToken(new ToUpperEvaluator(toStringEvaluator(subjectEvaluator)), "toUpper");
- }
- case URL_ENCODE: {
- verifyArgCount(argEvaluators, 0, "urlEncode");
- return addToken(new UrlEncodeEvaluator(toStringEvaluator(subjectEvaluator)), "urlEncode");
- }
- case URL_DECODE: {
- verifyArgCount(argEvaluators, 0, "urlDecode");
- return addToken(new UrlDecodeEvaluator(toStringEvaluator(subjectEvaluator)), "urlDecode");
- }
- case BASE64_ENCODE: {
- verifyArgCount(argEvaluators, 0, "base64Encode");
- return addToken(new Base64EncodeEvaluator(toStringEvaluator(subjectEvaluator)), "base64Encode");
- }
- case BASE64_DECODE: {
- verifyArgCount(argEvaluators, 0, "base64Decode");
- return addToken(new Base64DecodeEvaluator(toStringEvaluator(subjectEvaluator)), "base64Decode");
- }
- case ESCAPE_CSV: {
- verifyArgCount(argEvaluators, 0, "escapeCsv");
- return addToken(CharSequenceTranslatorEvaluator.csvEscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
- }
- case ESCAPE_HTML3: {
- verifyArgCount(argEvaluators, 0, "escapeHtml3");
- return addToken(CharSequenceTranslatorEvaluator.html3EscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
- }
- case ESCAPE_HTML4: {
- verifyArgCount(argEvaluators, 0, "escapeHtml4");
- return addToken(CharSequenceTranslatorEvaluator.html4EscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
- }
- case ESCAPE_JSON: {
- verifyArgCount(argEvaluators, 0, "escapeJson");
- return addToken(CharSequenceTranslatorEvaluator.jsonEscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
- }
- case ESCAPE_XML: {
- verifyArgCount(argEvaluators, 0, "escapeXml");
- return addToken(CharSequenceTranslatorEvaluator.xmlEscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
- }
- case UNESCAPE_CSV: {
- verifyArgCount(argEvaluators, 0, "unescapeCsv");
- return addToken(CharSequenceTranslatorEvaluator.csvUnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
- }
- case UNESCAPE_HTML3: {
- verifyArgCount(argEvaluators, 0, "unescapeHtml3");
- return addToken(CharSequenceTranslatorEvaluator.html3UnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
- }
- case UNESCAPE_HTML4: {
- verifyArgCount(argEvaluators, 0, "unescapeHtml4");
- return addToken(CharSequenceTranslatorEvaluator.html4UnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
- }
- case UNESCAPE_JSON: {
- verifyArgCount(argEvaluators, 0, "unescapeJson");
- return addToken(CharSequenceTranslatorEvaluator.jsonUnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
- }
- case UNESCAPE_XML: {
- verifyArgCount(argEvaluators, 0, "unescapeXml");
- return addToken(CharSequenceTranslatorEvaluator.xmlUnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
- }
- case SUBSTRING_BEFORE: {
- verifyArgCount(argEvaluators, 1, "substringBefore");
- return addToken(new SubstringBeforeEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to substringBefore")), "substringBefore");
- }
- case SUBSTRING_BEFORE_LAST: {
- verifyArgCount(argEvaluators, 1, "substringBeforeLast");
- return addToken(new SubstringBeforeLastEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to substringBeforeLast")), "substringBeforeLast");
- }
- case SUBSTRING_AFTER: {
- verifyArgCount(argEvaluators, 1, "substringAfter");
- return addToken(new SubstringAfterEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to substringAfter")), "substringAfter");
- }
- case SUBSTRING_AFTER_LAST: {
- verifyArgCount(argEvaluators, 1, "substringAfterLast");
- return addToken(new SubstringAfterLastEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to substringAfterLast")), "substringAfterLast");
- }
- case REPLACE_NULL: {
- verifyArgCount(argEvaluators, 1, "replaceNull");
- return addToken(new ReplaceNullEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to replaceNull")), "replaceNull");
- }
- case REPLACE_EMPTY: {
- verifyArgCount(argEvaluators, 1, "replaceEmtpy");
- return addToken(new ReplaceEmptyEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument to replaceEmpty")), "replaceEmpty");
- }
- case REPLACE: {
- verifyArgCount(argEvaluators, 2, "replace");
- return addToken(new ReplaceEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to replace"),
- toStringEvaluator(argEvaluators.get(1), "second argument to replace")), "replace");
- }
- case REPLACE_FIRST: {
- verifyArgCount(argEvaluators, 2, "replaceFirst");
- return addToken(new ReplaceFirstEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to replaceFirst"),
- toStringEvaluator(argEvaluators.get(1), "second argument to replaceFirst")), "replaceFirst");
- }
- case REPLACE_ALL: {
- verifyArgCount(argEvaluators, 2, "replaceAll");
- return addToken(new ReplaceAllEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to replaceAll"),
- toStringEvaluator(argEvaluators.get(1), "second argument to replaceAll")), "replaceAll");
- }
- case APPEND: {
- verifyArgCount(argEvaluators, 1, "append");
- return addToken(new AppendEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to append")), "append");
- }
- case PREPEND: {
- verifyArgCount(argEvaluators, 1, "prepend");
- return addToken(new PrependEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to prepend")), "prepend");
- }
- case SUBSTRING: {
- final int numArgs = argEvaluators.size();
- if (numArgs == 1) {
- return addToken(new SubstringEvaluator(toStringEvaluator(subjectEvaluator),
- toWholeNumberEvaluator(argEvaluators.get(0), "first argument to substring")), "substring");
- } else if (numArgs == 2) {
- return addToken(new SubstringEvaluator(toStringEvaluator(subjectEvaluator),
- toWholeNumberEvaluator(argEvaluators.get(0), "first argument to substring"),
- toWholeNumberEvaluator(argEvaluators.get(1), "second argument to substring")), "substring");
- } else {
- throw new AttributeExpressionLanguageParsingException("substring() function can take either 1 or 2 arguments but cannot take " + numArgs + " arguments");
- }
- }
- case JOIN: {
- verifyArgCount(argEvaluators, 1, "join");
- return addToken(new JoinEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0))), "join");
- }
- case COUNT: {
- verifyArgCount(argEvaluators, 0, "count");
- return addToken(new CountEvaluator(subjectEvaluator), "count");
- }
- case IS_NULL: {
- verifyArgCount(argEvaluators, 0, "isNull");
- return addToken(new IsNullEvaluator(toStringEvaluator(subjectEvaluator)), "isNull");
- }
- case IS_EMPTY: {
- verifyArgCount(argEvaluators, 0, "isEmpty");
- return addToken(new IsEmptyEvaluator(toStringEvaluator(subjectEvaluator)), "isEmpty");
- }
- case NOT_NULL: {
- verifyArgCount(argEvaluators, 0, "notNull");
- return addToken(new NotNullEvaluator(toStringEvaluator(subjectEvaluator)), "notNull");
- }
- case STARTS_WITH: {
- verifyArgCount(argEvaluators, 1, "startsWith");
- return addToken(new StartsWithEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to startsWith")), "startsWith");
- }
- case ENDS_WITH: {
- verifyArgCount(argEvaluators, 1, "endsWith");
- return addToken(new EndsWithEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to endsWith")), "endsWith");
- }
- case CONTAINS: {
- verifyArgCount(argEvaluators, 1, "contains");
- return addToken(new ContainsEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to contains")), "contains");
- }
- case IN: {
- List<Evaluator<String>> list = new ArrayList<Evaluator<String>>();
- for(int i = 0; i < argEvaluators.size(); i++) {
- list.add(toStringEvaluator(argEvaluators.get(i), i + "th argument to in"));
- }
- return addToken(new InEvaluator(toStringEvaluator(subjectEvaluator), list), "in");
- }
- case FIND: {
- verifyArgCount(argEvaluators, 1, "find");
- return addToken(new FindEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to find")), "find");
- }
- case MATCHES: {
- verifyArgCount(argEvaluators, 1, "matches");
- return addToken(new MatchesEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to matches")), "matches");
- }
- case EQUALS: {
- verifyArgCount(argEvaluators, 1, "equals");
- return addToken(new EqualsEvaluator(subjectEvaluator, argEvaluators.get(0)), "equals");
- }
- case EQUALS_IGNORE_CASE: {
- verifyArgCount(argEvaluators, 1, "equalsIgnoreCase");
- return addToken(new EqualsIgnoreCaseEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to equalsIgnoreCase")), "equalsIgnoreCase");
- }
- case GREATER_THAN: {
- verifyArgCount(argEvaluators, 1, "gt");
- return addToken(new GreaterThanEvaluator(toNumberEvaluator(subjectEvaluator),
- toNumberEvaluator(argEvaluators.get(0), "first argument to gt")), "gt");
- }
- case GREATER_THAN_OR_EQUAL: {
- verifyArgCount(argEvaluators, 1, "ge");
- return addToken(new GreaterThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator),
- toNumberEvaluator(argEvaluators.get(0), "first argument to ge")), "ge");
- }
- case LESS_THAN: {
- verifyArgCount(argEvaluators, 1, "lt");
- return addToken(new LessThanEvaluator(toNumberEvaluator(subjectEvaluator),
- toNumberEvaluator(argEvaluators.get(0), "first argument to lt")), "lt");
- }
- case LESS_THAN_OR_EQUAL: {
- verifyArgCount(argEvaluators, 1, "le");
- return addToken(new LessThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator),
- toNumberEvaluator(argEvaluators.get(0), "first argument to le")), "le");
- }
- case LENGTH: {
- verifyArgCount(argEvaluators, 0, "length");
- return addToken(new LengthEvaluator(toStringEvaluator(subjectEvaluator)), "length");
- }
- case TO_DATE: {
- if (argEvaluators.isEmpty()) {
- return addToken(new NumberToDateEvaluator(toWholeNumberEvaluator(subjectEvaluator)), "toDate");
- } else if (subjectEvaluator.getResultType() == ResultType.STRING && argEvaluators.size() == 1) {
- return addToken(new StringToDateEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), null), "toDate");
- } else if (subjectEvaluator.getResultType() == ResultType.STRING && argEvaluators.size() == 2) {
- return addToken(new StringToDateEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), toStringEvaluator(argEvaluators.get(1))), "toDate");
- } else {
- return addToken(new NumberToDateEvaluator(toWholeNumberEvaluator(subjectEvaluator)), "toDate");
- }
- }
- case TO_NUMBER: {
- verifyArgCount(argEvaluators, 0, "toNumber");
- switch (subjectEvaluator.getResultType()) {
- case STRING:
- case WHOLE_NUMBER:
- case DECIMAL:
- case NUMBER:
- case DATE:
- return addToken(toWholeNumberEvaluator(subjectEvaluator), "toNumber");
- default:
- throw new AttributeExpressionLanguageParsingException(subjectEvaluator + " returns type " + subjectEvaluator.getResultType() + " but expected to get " + ResultType.STRING +
- ", " + ResultType.DECIMAL + ", or " + ResultType.DATE);
- }
- }
- case TO_DECIMAL: {
- verifyArgCount(argEvaluators, 0, "toDecimal");
- switch (subjectEvaluator.getResultType()) {
- case WHOLE_NUMBER:
- case DECIMAL:
- case STRING:
- case NUMBER:
- case DATE:
- return addToken(toDecimalEvaluator(subjectEvaluator), "toDecimal");
- default:
- throw new AttributeExpressionLanguageParsingException(subjectEvaluator + " returns type " + subjectEvaluator.getResultType() + " but expected to get " + ResultType.STRING +
- ", " + ResultType.WHOLE_NUMBER + ", or " + ResultType.DATE);
- }
- }
- case TO_RADIX: {
- if (argEvaluators.size() == 1) {
- return addToken(new ToRadixEvaluator(toWholeNumberEvaluator(subjectEvaluator),
- toWholeNumberEvaluator(argEvaluators.get(0))), "toRadix");
- } else {
- return addToken(new ToRadixEvaluator(toWholeNumberEvaluator(subjectEvaluator),
- toWholeNumberEvaluator(argEvaluators.get(0)), toWholeNumberEvaluator(argEvaluators.get(1))), "toRadix");
- }
- }
- case FROM_RADIX: {
- return addToken(new FromRadixEvaluator(toStringEvaluator(subjectEvaluator),
- toWholeNumberEvaluator(argEvaluators.get(0))), "fromRadix");
- }
- case MOD: {
- return addToken(new ModEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "mod");
- }
- case PLUS: {
- return addToken(new PlusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "plus");
- }
- case MINUS: {
- return addToken(new MinusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "minus");
- }
- case MULTIPLY: {
- return addToken(new MultiplyEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "multiply");
- }
- case DIVIDE: {
- return addToken(new DivideEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "divide");
- }
- case MATH: {
- if (argEvaluators.size() == 1) {
- return addToken(new MathEvaluator(toNumberEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), null), "math");
- } else if (argEvaluators.size() == 2){
- return addToken(new MathEvaluator(toNumberEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), toNumberEvaluator(argEvaluators.get(1))), "math");
- } else {
- throw new AttributeExpressionLanguageParsingException("math() function takes 1 or 2 arguments");
- }
- }
- case RANDOM : {
- return addToken(new RandomNumberGeneratorEvaluator(), "random");
- }
- case INDEX_OF: {
- verifyArgCount(argEvaluators, 1, "indexOf");
- return addToken(new IndexOfEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to indexOf")), "indexOf");
- }
- case LAST_INDEX_OF: {
- verifyArgCount(argEvaluators, 1, "lastIndexOf");
- return addToken(new LastIndexOfEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to lastIndexOf")), "lastIndexOf");
- }
- case FORMAT: {
- if(argEvaluators.size() == 1) {
- return addToken(new FormatEvaluator(toDateEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument of format"), null), "format");
- } else if (argEvaluators.size() == 2) {
- return addToken(new FormatEvaluator(toDateEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), toStringEvaluator(argEvaluators.get(1))), "format");
- } else {
- throw new AttributeExpressionLanguageParsingException("format() function takes 1 or 2 arguments");
- }
- }
- case OR: {
- return addToken(new OrEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0))), "or");
- }
- case AND: {
- return addToken(new AndEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0))), "and");
- }
- case NOT: {
- return addToken(new NotEvaluator(toBooleanEvaluator(subjectEvaluator)), "not");
- }
- case GET_DELIMITED_FIELD: {
- if (argEvaluators.size() == 1) {
- // Only a single argument - the index to return.
- return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
- toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField")), "getDelimitedField");
- } else if (argEvaluators.size() == 2) {
- // two arguments - index and delimiter.
- return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
- toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
- toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField")),
- "getDelimitedField");
- } else if (argEvaluators.size() == 3) {
- // 3 arguments - index, delimiter, quote char.
- return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
- toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
- toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField"),
- toStringEvaluator(argEvaluators.get(2), "third argument of getDelimitedField")),
- "getDelimitedField");
- } else if (argEvaluators.size() == 4) {
- // 4 arguments - index, delimiter, quote char, escape char
- return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
- toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
- toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField"),
- toStringEvaluator(argEvaluators.get(2), "third argument of getDelimitedField"),
- toStringEvaluator(argEvaluators.get(3), "fourth argument of getDelimitedField")),
- "getDelimitedField");
- } else {
- // 5 arguments - index, delimiter, quote char, escape char, strip escape/quote chars flag
- return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
- toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
- toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField"),
- toStringEvaluator(argEvaluators.get(2), "third argument of getDelimitedField"),
- toStringEvaluator(argEvaluators.get(3), "fourth argument of getDelimitedField"),
- toBooleanEvaluator(argEvaluators.get(4), "fifth argument of getDelimitedField")),
- "getDelimitedField");
- }
- }
- case JSON_PATH: {
- verifyArgCount(argEvaluators, 1, "jsonPath");
- return addToken(new JsonPathEvaluator(toStringEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "first argument to jsonPath")), "jsonPath");
- }
- case IF_ELSE: {
- verifyArgCount(argEvaluators, 2, "ifElse");
- return addToken(new IfElseEvaluator(toBooleanEvaluator(subjectEvaluator),
- toStringEvaluator(argEvaluators.get(0), "argument to return if true"),
- toStringEvaluator(argEvaluators.get(1), "argument to return if false")), "ifElse");
- }
- default:
- throw new AttributeExpressionLanguageParsingException("Expected a Function-type expression but got " + tree.toString());
- }
- }
public static class Range {
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
index 9f12c92..cdf5a2d 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java
@@ -16,36 +16,45 @@
*/
package org.apache.nifi.attribute.expression.language;
-import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Collections;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Set;
+import org.apache.nifi.attribute.expression.language.compile.CompiledExpression;
+import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.AllAttributesEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.AnyAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.AttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MappingEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiMatchAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiNamedAttributeEvaluator;
import org.apache.nifi.expression.AttributeValueDecorator;
import org.apache.nifi.processor.exception.ProcessException;
-import org.antlr.runtime.tree.Tree;
-
public class StandardPreparedQuery implements PreparedQuery {
private final List<String> queryStrings;
- private final Map<String, Tree> trees;
+ private final Map<String, CompiledExpression> expressions;
+ private volatile VariableImpact variableImpact;
- public StandardPreparedQuery(final List<String> queryStrings, final Map<String, Tree> trees) {
- this.queryStrings = new ArrayList<>(queryStrings);
- this.trees = new HashMap<>(trees);
+ public StandardPreparedQuery(final List<String> queryStrings, final Map<String, CompiledExpression> expressions) {
+ this.queryStrings = queryStrings;
+ this.expressions = expressions;
}
-
@Override
public String evaluateExpressions(final Map<String, String> valMap, final AttributeValueDecorator decorator, final Map<String, String> stateVariables) throws ProcessException {
final StringBuilder sb = new StringBuilder();
for (final String val : queryStrings) {
- final Tree tree = trees.get(val);
- if (tree == null) {
+ final CompiledExpression expression = expressions.get(val);
+ if (expression == null) {
sb.append(val);
} else {
- final String evaluated = Query.evaluateExpression(tree, val, valMap, decorator, stateVariables);
+ final String evaluated = Query.evaluateExpression(expression.getTree(), val, valMap, decorator, stateVariables);
if (evaluated != null) {
sb.append(evaluated);
}
@@ -62,6 +71,56 @@ public class StandardPreparedQuery implements PreparedQuery {
@Override
public boolean isExpressionLanguagePresent() {
- return !trees.isEmpty();
+ return !expressions.isEmpty();
+ }
+
+ @Override
+ public VariableImpact getVariableImpact() {
+ final VariableImpact existing = this.variableImpact;
+ if (existing != null) {
+ return existing;
+ }
+
+ final Set<String> variables = new HashSet<>();
+
+ for (final CompiledExpression expression : expressions.values()) {
+ for (final Evaluator<?> evaluator : expression.getAllEvaluators()) {
+ if (evaluator instanceof AttributeEvaluator) {
+ final AttributeEvaluator attributeEval = (AttributeEvaluator) evaluator;
+ final Evaluator<String> nameEval = attributeEval.getNameEvaluator();
+
+ if (nameEval instanceof StringLiteralEvaluator) {
+ final String referencedVar = nameEval.evaluate(Collections.emptyMap()).getValue();
+ variables.add(referencedVar);
+ }
+ } else if (evaluator instanceof AllAttributesEvaluator) {
+ final AllAttributesEvaluator allAttrsEval = (AllAttributesEvaluator) evaluator;
+ final MultiAttributeEvaluator iteratingEval = allAttrsEval.getVariableIteratingEvaluator();
+ if (iteratingEval instanceof MultiNamedAttributeEvaluator) {
+ variables.addAll(((MultiNamedAttributeEvaluator) iteratingEval).getAttributeNames());
+ } else if (iteratingEval instanceof MultiMatchAttributeEvaluator) {
+ return VariableImpact.ALWAYS_IMPACTED;
+ }
+ } else if (evaluator instanceof AnyAttributeEvaluator) {
+ final AnyAttributeEvaluator allAttrsEval = (AnyAttributeEvaluator) evaluator;
+ final MultiAttributeEvaluator iteratingEval = allAttrsEval.getVariableIteratingEvaluator();
+ if (iteratingEval instanceof MultiNamedAttributeEvaluator) {
+ variables.addAll(((MultiNamedAttributeEvaluator) iteratingEval).getAttributeNames());
+ } else if (iteratingEval instanceof MultiMatchAttributeEvaluator) {
+ return VariableImpact.ALWAYS_IMPACTED;
+ }
+ } else if (evaluator instanceof MappingEvaluator) {
+ final MappingEvaluator<?> allAttrsEval = (MappingEvaluator<?>) evaluator;
+ final MultiAttributeEvaluator iteratingEval = allAttrsEval.getVariableIteratingEvaluator();
+ if (iteratingEval instanceof MultiNamedAttributeEvaluator) {
+ variables.addAll(((MultiNamedAttributeEvaluator) iteratingEval).getAttributeNames());
+ }
+ }
+ }
+ }
+
+ final VariableImpact impact = new NamedVariableImpact(variables);
+ this.variableImpact = impact;
+ return impact;
}
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ValueLookup.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ValueLookup.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ValueLookup.java
index 5b0cdda..06c1877 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ValueLookup.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/ValueLookup.java
@@ -49,6 +49,7 @@ final class ValueLookup implements Map<String, String> {
* @param flowFile the flowFile to pull attributes from; may be null
* @param additionalMaps the maps to pull values from; may be null or empty
*/
+ @SuppressWarnings("unchecked")
ValueLookup(final VariableRegistry registry, final FlowFile flowFile, final Map<String, String>... additionalMaps) {
for (final Map<String, String> map : additionalMaps) {
if (map != null && !map.isEmpty()) {
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/VariableImpact.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/VariableImpact.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/VariableImpact.java
new file mode 100644
index 0000000..4a66c87
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/VariableImpact.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.attribute.expression.language;
+
+public interface VariableImpact {
+ boolean isImpacted(String variableName);
+
+ public static final VariableImpact NEVER_IMPACTED = var -> false;
+
+ public static final VariableImpact ALWAYS_IMPACTED = var -> true;
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/CompiledExpression.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/CompiledExpression.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/CompiledExpression.java
new file mode 100644
index 0000000..d234445
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/CompiledExpression.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.attribute.expression.language.compile;
+
+import java.util.Set;
+
+import org.antlr.runtime.tree.Tree;
+import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
+
+public class CompiledExpression {
+ private final Evaluator<?> rootEvaluator;
+ private final Tree tree;
+ private final String expression;
+ private final Set<Evaluator<?>> allEvaluators;
+
+ public CompiledExpression(final String expression, final Evaluator<?> rootEvaluator, final Tree tree, final Set<Evaluator<?>> allEvaluators) {
+ this.rootEvaluator = rootEvaluator;
+ this.tree = tree;
+ this.expression = expression;
+ this.allEvaluators = allEvaluators;
+ }
+
+ public Evaluator<?> getRootEvaluator() {
+ return rootEvaluator;
+ }
+
+ public Tree getTree() {
+ return tree;
+ }
+
+ public String getExpression() {
+ return expression;
+ }
+
+ public Set<Evaluator<?>> getAllEvaluators() {
+ return allEvaluators;
+ }
+}
[3/9] nifi git commit: NIFI-4224: - Initial implementation of Process
Group level Variable Registry - Updated to incorporate PR Feedback - Changed
log message because slf4j-simple apparently has a memory leak;
passing a String instead of passing in the C
Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
index 36a9524..35686a5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -16,7 +16,31 @@
*/
package org.apache.nifi.web;
-import com.google.common.collect.Sets;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Response;
+
import org.apache.nifi.action.Action;
import org.apache.nifi.action.Component;
import org.apache.nifi.action.FlowChangeAction;
@@ -84,6 +108,7 @@ import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.history.History;
import org.apache.nifi.history.HistoryQuery;
import org.apache.nifi.history.PreviousValue;
+import org.apache.nifi.registry.ComponentVariableRegistry;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.reporting.Bulletin;
import org.apache.nifi.reporting.BulletinQuery;
@@ -92,6 +117,7 @@ import org.apache.nifi.reporting.ComponentType;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.api.dto.AccessPolicyDTO;
import org.apache.nifi.web.api.dto.AccessPolicySummaryDTO;
+import org.apache.nifi.web.api.dto.AffectedComponentDTO;
import org.apache.nifi.web.api.dto.BulletinBoardDTO;
import org.apache.nifi.web.api.dto.BulletinDTO;
import org.apache.nifi.web.api.dto.BulletinQueryDTO;
@@ -137,6 +163,7 @@ import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO;
import org.apache.nifi.web.api.dto.TemplateDTO;
import org.apache.nifi.web.api.dto.UserDTO;
import org.apache.nifi.web.api.dto.UserGroupDTO;
+import org.apache.nifi.web.api.dto.VariableRegistryDTO;
import org.apache.nifi.web.api.dto.action.HistoryDTO;
import org.apache.nifi.web.api.dto.action.HistoryQueryDTO;
import org.apache.nifi.web.api.dto.flow.FlowDTO;
@@ -157,6 +184,7 @@ import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
import org.apache.nifi.web.api.entity.AccessPolicyEntity;
import org.apache.nifi.web.api.entity.AccessPolicySummaryEntity;
import org.apache.nifi.web.api.entity.ActionEntity;
+import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
import org.apache.nifi.web.api.entity.BulletinEntity;
import org.apache.nifi.web.api.entity.ComponentReferenceEntity;
import org.apache.nifi.web.api.entity.ConnectionEntity;
@@ -190,6 +218,8 @@ import org.apache.nifi.web.api.entity.TemplateEntity;
import org.apache.nifi.web.api.entity.TenantEntity;
import org.apache.nifi.web.api.entity.UserEntity;
import org.apache.nifi.web.api.entity.UserGroupEntity;
+import org.apache.nifi.web.api.entity.VariableEntity;
+import org.apache.nifi.web.api.entity.VariableRegistryEntity;
import org.apache.nifi.web.controller.ControllerFacade;
import org.apache.nifi.web.dao.AccessPolicyDAO;
import org.apache.nifi.web.dao.ConnectionDAO;
@@ -217,28 +247,7 @@ import org.apache.nifi.web.util.SnippetUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import javax.ws.rs.WebApplicationException;
-import javax.ws.rs.core.Response;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.UUID;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
+import com.google.common.collect.Sets;
/**
* Implementation of NiFiServiceFacade that performs revision checking.
@@ -423,6 +432,11 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
}
@Override
+ public void verifyActivateControllerServices(final String groupId, final ControllerServiceState state, final Set<String> serviceIds) {
+ processGroupDAO.verifyActivateControllerServices(groupId, state, serviceIds);
+ }
+
+ @Override
public void verifyDeleteProcessGroup(final String groupId) {
processGroupDAO.verifyDelete(groupId);
}
@@ -624,6 +638,10 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
*/
private <D, C> RevisionUpdate<D> updateComponent(final Revision revision, final Authorizable authorizable, final Supplier<C> daoUpdate, final Function<C, D> dtoCreation) {
final NiFiUser user = NiFiUserUtils.getNiFiUser();
+ return updateComponent(user, revision, authorizable, daoUpdate, dtoCreation);
+ }
+
+ private <D, C> RevisionUpdate<D> updateComponent(final NiFiUser user, final Revision revision, final Authorizable authorizable, final Supplier<C> daoUpdate, final Function<C, D> dtoCreation) {
try {
final RevisionUpdate<D> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(revision), user, new UpdateRevisionTask<D>() {
@Override
@@ -774,6 +792,81 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
}
@Override
+ public Set<AffectedComponentDTO> getComponentsAffectedByVariableRegistryUpdate(final VariableRegistryDTO variableRegistryDto) {
+ final ProcessGroup group = processGroupDAO.getProcessGroup(variableRegistryDto.getProcessGroupId());
+ if (group == null) {
+ throw new ResourceNotFoundException("Could not find Process Group with ID " + variableRegistryDto.getProcessGroupId());
+ }
+
+ final Map<String, String> variableMap = new HashMap<>();
+ variableRegistryDto.getVariables().stream() // have to use forEach here instead of using Collectors.toMap because value may be null
+ .map(VariableEntity::getVariable)
+ .forEach(var -> variableMap.put(var.getName(), var.getValue()));
+
+ final Set<AffectedComponentDTO> affectedComponentDtos = new HashSet<>();
+
+ final Set<String> updatedVariableNames = getUpdatedVariables(group, variableMap);
+ for (final String variableName : updatedVariableNames) {
+ final Set<ConfiguredComponent> affectedComponents = group.getComponentsAffectedByVariable(variableName);
+
+ for (final ConfiguredComponent component : affectedComponents) {
+ if (component instanceof ProcessorNode) {
+ final ProcessorNode procNode = (ProcessorNode) component;
+ if (procNode.isRunning()) {
+ affectedComponentDtos.add(dtoFactory.createAffectedComponentDto(procNode));
+ }
+ } else if (component instanceof ControllerServiceNode) {
+ final ControllerServiceNode serviceNode = (ControllerServiceNode) component;
+ if (serviceNode.isActive()) {
+ affectedComponentDtos.add(dtoFactory.createAffectedComponentDto(serviceNode));
+ }
+ } else {
+ throw new RuntimeException("Found unexpected type of Component [" + component.getCanonicalClassName() + "] dependending on variable");
+ }
+ }
+ }
+
+ return affectedComponentDtos;
+ }
+
+ private Set<String> getUpdatedVariables(final ProcessGroup group, final Map<String, String> newVariableValues) {
+ final Set<String> updatedVariableNames = new HashSet<>();
+
+ final ComponentVariableRegistry registry = group.getVariableRegistry();
+ for (final Map.Entry<String, String> entry : newVariableValues.entrySet()) {
+ final String varName = entry.getKey();
+ final String newValue = entry.getValue();
+
+ final String curValue = registry.getVariableValue(varName);
+ if (!Objects.equals(newValue, curValue)) {
+ updatedVariableNames.add(varName);
+ }
+ }
+
+ return updatedVariableNames;
+ }
+
+
+ @Override
+ public VariableRegistryEntity updateVariableRegistry(Revision revision, VariableRegistryDTO variableRegistryDto) {
+ return updateVariableRegistry(NiFiUserUtils.getNiFiUser(), revision, variableRegistryDto);
+ }
+
+ @Override
+ public VariableRegistryEntity updateVariableRegistry(NiFiUser user, Revision revision, VariableRegistryDTO variableRegistryDto) {
+ final ProcessGroup processGroupNode = processGroupDAO.getProcessGroup(variableRegistryDto.getProcessGroupId());
+ final RevisionUpdate<VariableRegistryDTO> snapshot = updateComponent(user, revision,
+ processGroupNode,
+ () -> processGroupDAO.updateVariableRegistry(variableRegistryDto),
+ processGroup -> dtoFactory.createVariableRegistryDto(processGroup));
+
+ final PermissionsDTO permissions = dtoFactory.createPermissionsDto(processGroupNode);
+ final RevisionDTO updatedRevision = dtoFactory.createRevisionDTO(snapshot.getLastModification());
+ return entityFactory.createVariableRegistryEntity(snapshot.getComponent(), updatedRevision, permissions);
+ }
+
+
+ @Override
public ProcessGroupEntity updateProcessGroup(final Revision revision, final ProcessGroupDTO processGroupDTO) {
final ProcessGroup processGroupNode = processGroupDAO.getProcessGroup(processGroupDTO.getId());
final RevisionUpdate<ProcessGroupDTO> snapshot = updateComponent(revision,
@@ -790,14 +883,27 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
}
@Override
+ public void verifyUpdateProcessGroup(ProcessGroupDTO processGroupDTO) {
+ if (processGroupDAO.hasProcessGroup(processGroupDTO.getId())) {
+ processGroupDAO.verifyUpdate(processGroupDTO);
+ }
+ }
+
+ @Override
public ScheduleComponentsEntity scheduleComponents(final String processGroupId, final ScheduledState state, final Map<String, Revision> componentRevisions) {
final NiFiUser user = NiFiUserUtils.getNiFiUser();
+ return scheduleComponents(user, processGroupId, state, componentRevisions);
+ }
+
+ @Override
+ public ScheduleComponentsEntity scheduleComponents(final NiFiUser user, final String processGroupId, final ScheduledState state, final Map<String, Revision> componentRevisions) {
+
final RevisionUpdate<ScheduleComponentsEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(componentRevisions.values()), user, new
UpdateRevisionTask<ScheduleComponentsEntity>() {
@Override
public RevisionUpdate<ScheduleComponentsEntity> update() {
// schedule the components
- processGroupDAO.scheduleComponents(processGroupId, state, componentRevisions.keySet());
+ processGroupDAO.scheduleComponents(processGroupId, state, componentRevisions.keySet());
// update the revisions
final Map<String, Revision> updatedRevisions = new HashMap<>();
@@ -821,6 +927,46 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
}
@Override
+ public ActivateControllerServicesEntity activateControllerServices(final String processGroupId, final ControllerServiceState state, final Map<String, Revision> serviceRevisions) {
+
+ final NiFiUser user = NiFiUserUtils.getNiFiUser();
+ return activateControllerServices(user, processGroupId, state, serviceRevisions);
+ }
+
+ @Override
+ public ActivateControllerServicesEntity activateControllerServices(final NiFiUser user, final String processGroupId, final ControllerServiceState state,
+ final Map<String, Revision> serviceRevisions) {
+
+ final RevisionUpdate<ActivateControllerServicesEntity> updatedComponent = revisionManager.updateRevision(new StandardRevisionClaim(serviceRevisions.values()), user,
+ new UpdateRevisionTask<ActivateControllerServicesEntity>() {
+ @Override
+ public RevisionUpdate<ActivateControllerServicesEntity> update() {
+ // schedule the components
+ processGroupDAO.activateControllerServices(processGroupId, state, serviceRevisions.keySet());
+
+ // update the revisions
+ final Map<String, Revision> updatedRevisions = new HashMap<>();
+ for (final Revision revision : serviceRevisions.values()) {
+ final Revision currentRevision = revisionManager.getRevision(revision.getComponentId());
+ updatedRevisions.put(revision.getComponentId(), currentRevision.incrementRevision(revision.getClientId()));
+ }
+
+ // save
+ controllerFacade.save();
+
+ // gather details for response
+ final ActivateControllerServicesEntity entity = new ActivateControllerServicesEntity();
+ entity.setId(processGroupId);
+ entity.setState(state.name());
+ return new StandardRevisionUpdate<>(entity, null, new HashSet<>(updatedRevisions.values()));
+ }
+ });
+
+ return updatedComponent.getComponent();
+ }
+
+
+ @Override
public ControllerConfigurationEntity updateControllerConfiguration(final Revision revision, final ControllerConfigurationDTO controllerConfigurationDTO) {
final RevisionUpdate<ControllerConfigurationDTO> updatedComponent = updateComponent(
revision,
@@ -3062,7 +3208,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
return createProcessGroupEntity(processGroup);
}
- private ControllerServiceEntity createControllerServiceEntity(final ControllerServiceNode serviceNode, final Set<String> serviceIds) {
+ private ControllerServiceEntity createControllerServiceEntity(final ControllerServiceNode serviceNode, final Set<String> serviceIds, final NiFiUser user) {
final ControllerServiceDTO dto = dtoFactory.createControllerServiceDto(serviceNode);
final ControllerServiceReference ref = serviceNode.getReferences();
@@ -3070,26 +3216,77 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
dto.setReferencingComponents(referencingComponentsEntity.getControllerServiceReferencingComponents());
final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(serviceNode.getIdentifier()));
- final PermissionsDTO permissions = dtoFactory.createPermissionsDto(serviceNode);
+ final PermissionsDTO permissions = dtoFactory.createPermissionsDto(serviceNode, user);
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(serviceNode.getIdentifier()));
final List<BulletinEntity> bulletinEntities = bulletins.stream().map(bulletin -> entityFactory.createBulletinEntity(bulletin, permissions.getCanRead())).collect(Collectors.toList());
return entityFactory.createControllerServiceEntity(dto, revision, permissions, bulletinEntities);
}
@Override
- public Set<ControllerServiceEntity> getControllerServices(final String groupId) {
- final Set<ControllerServiceNode> serviceNodes = controllerServiceDAO.getControllerServices(groupId);
+ public VariableRegistryEntity getVariableRegistry(final String groupId, final boolean includeAncestorGroups) {
+ final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
+ if (processGroup == null) {
+ throw new ResourceNotFoundException("Could not find group with ID " + groupId);
+ }
+
+ return createVariableRegistryEntity(processGroup, includeAncestorGroups);
+ }
+
+ private VariableRegistryEntity createVariableRegistryEntity(final ProcessGroup processGroup, final boolean includeAncestorGroups) {
+ final VariableRegistryDTO registryDto = dtoFactory.createVariableRegistryDto(processGroup);
+ final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(processGroup.getIdentifier()));
+ final PermissionsDTO permissions = dtoFactory.createPermissionsDto(processGroup);
+
+ if (includeAncestorGroups) {
+ ProcessGroup parent = processGroup.getParent();
+ while (parent != null) {
+ final PermissionsDTO parentPerms = dtoFactory.createPermissionsDto(processGroup);
+ if (Boolean.TRUE.equals(parentPerms.getCanRead())) {
+ final VariableRegistryDTO parentRegistryDto = dtoFactory.createVariableRegistryDto(parent);
+ final Set<VariableEntity> parentVariables = parentRegistryDto.getVariables();
+ registryDto.getVariables().addAll(parentVariables);
+ }
+
+ parent = parent.getParent();
+ }
+ }
+
+ return entityFactory.createVariableRegistryEntity(registryDto, revision, permissions);
+ }
+
+ @Override
+ public VariableRegistryEntity populateAffectedComponents(final VariableRegistryDTO variableRegistryDto) {
+ final String groupId = variableRegistryDto.getProcessGroupId();
+ final ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
+ if (processGroup == null) {
+ throw new ResourceNotFoundException("Could not find group with ID " + groupId);
+ }
+
+ final VariableRegistryDTO registryDto = dtoFactory.populateAffectedComponents(variableRegistryDto, processGroup);
+ final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(processGroup.getIdentifier()));
+ final PermissionsDTO permissions = dtoFactory.createPermissionsDto(processGroup);
+ return entityFactory.createVariableRegistryEntity(registryDto, revision, permissions);
+ }
+
+ @Override
+ public Set<ControllerServiceEntity> getControllerServices(final String groupId, final boolean includeAncestorGroups, final boolean includeDescendantGroups) {
+ return getControllerServices(groupId, includeAncestorGroups, includeDescendantGroups, NiFiUserUtils.getNiFiUser());
+ }
+
+ @Override
+ public Set<ControllerServiceEntity> getControllerServices(final String groupId, final boolean includeAncestorGroups, final boolean includeDescendantGroups, final NiFiUser user) {
+ final Set<ControllerServiceNode> serviceNodes = controllerServiceDAO.getControllerServices(groupId, includeAncestorGroups, includeDescendantGroups);
final Set<String> serviceIds = serviceNodes.stream().map(service -> service.getIdentifier()).collect(Collectors.toSet());
return serviceNodes.stream()
- .map(serviceNode -> createControllerServiceEntity(serviceNode, serviceIds))
+ .map(serviceNode -> createControllerServiceEntity(serviceNode, serviceIds, user))
.collect(Collectors.toSet());
}
@Override
public ControllerServiceEntity getControllerService(final String controllerServiceId) {
final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerServiceId);
- return createControllerServiceEntity(controllerService, Sets.newHashSet(controllerServiceId));
+ return createControllerServiceEntity(controllerService, Sets.newHashSet(controllerServiceId), NiFiUserUtils.getNiFiUser());
}
@Override
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
index 3d78741..1a50d04 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ApplicationResource.java
@@ -608,6 +608,11 @@ public abstract class ApplicationResource {
serviceFacade.authorizeAccess(authorizer);
serviceFacade.verifyRevision(revision, user);
+ // verify if necessary
+ if (verifier != null) {
+ verifier.run();
+ }
+
return action.apply(revision, entity);
}
}
@@ -657,6 +662,11 @@ public abstract class ApplicationResource {
serviceFacade.authorizeAccess(authorizer);
serviceFacade.verifyRevisions(revisions, user);
+ // verify if necessary
+ if (verifier != null) {
+ verifier.run();
+ }
+
return action.apply(revisions, entity);
}
}
@@ -820,16 +830,16 @@ public abstract class ApplicationResource {
}
}
- /**
- * Replicates the request to the given node
- *
- * @param method the HTTP method
- * @param nodeUuid the UUID of the node to replicate the request to
- * @return the response from the node
- * @throws UnknownNodeException if the nodeUuid given does not map to any node in the cluster
- */
- protected Response replicate(final String method, final String nodeUuid) {
- return replicate(method, getRequestParameters(), nodeUuid);
+
+ private void ensureFlowInitialized() {
+ if (!flowController.isInitialized()) {
+ throw new IllegalClusterStateException("Cluster is still in the process of voting on the appropriate Data Flow.");
+ }
+ }
+
+ protected Response replicate(final String method, final Object entity, final String nodeUuid, final Map<String, String> headersToOverride) {
+ final URI path = getAbsolutePath();
+ return replicate(path, method, entity, nodeUuid, headersToOverride);
}
/**
@@ -845,22 +855,16 @@ public abstract class ApplicationResource {
return replicate(method, entity, nodeUuid, null);
}
- private void ensureFlowInitialized() {
- if (!flowController.isInitialized()) {
- throw new IllegalClusterStateException("Cluster is still in the process of voting on the appropriate Data Flow.");
- }
- }
-
/**
* Replicates the request to the given node
*
- * @param method the HTTP method
- * @param entity the Entity to replicate
+ * @param method the HTTP method
+ * @param entity the Entity to replicate
* @param nodeUuid the UUID of the node to replicate the request to
* @return the response from the node
* @throws UnknownNodeException if the nodeUuid given does not map to any node in the cluster
*/
- protected Response replicate(final String method, final Object entity, final String nodeUuid, final Map<String, String> headersToOverride) {
+ protected Response replicate(final URI path, final String method, final Object entity, final String nodeUuid, final Map<String, String> headersToOverride) {
// since we're cluster we must specify the cluster node identifier
if (nodeUuid == null) {
throw new IllegalArgumentException("The cluster node identifier must be specified.");
@@ -873,7 +877,6 @@ public abstract class ApplicationResource {
ensureFlowInitialized();
- final URI path = getAbsolutePath();
try {
final Map<String, String> headers = headersToOverride == null ? getHeaders() : getHeaders(headersToOverride);
@@ -996,6 +999,12 @@ public abstract class ApplicationResource {
}
}
+
+ protected NodeResponse replicateNodeResponse(final String method, final Object entity, final Map<String, String> headersToOverride) throws InterruptedException {
+ final URI path = getAbsolutePath();
+ return replicateNodeResponse(path, method, entity, headersToOverride);
+ }
+
/**
* Replicates the request to all nodes in the cluster using the provided method and entity. The headers
* used will be those provided by the {@link #getHeaders()} method. The URI that will be used will be
@@ -1009,10 +1018,9 @@ public abstract class ApplicationResource {
* @throws InterruptedException if interrupted while replicating the request
* @see #replicate(String, Object, Map)
*/
- protected NodeResponse replicateNodeResponse(final String method, final Object entity, final Map<String, String> headersToOverride) throws InterruptedException {
+ protected NodeResponse replicateNodeResponse(final URI path, final String method, final Object entity, final Map<String, String> headersToOverride) throws InterruptedException {
ensureFlowInitialized();
- final URI path = getAbsolutePath();
final Map<String, String> headers = headersToOverride == null ? getHeaders() : getHeaders(headersToOverride);
// Determine whether we should replicate only to the cluster coordinator, or if we should replicate directly
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
index 889676c..5d5a796 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
@@ -36,6 +36,8 @@ import org.apache.nifi.cluster.coordination.node.NodeConnectionState;
import org.apache.nifi.cluster.manager.NodeResponse;
import org.apache.nifi.cluster.protocol.NodeIdentifier;
import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.nar.NarClassLoaders;
import org.apache.nifi.util.NiFiProperties;
@@ -61,6 +63,7 @@ import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
import org.apache.nifi.web.api.entity.AboutEntity;
import org.apache.nifi.web.api.entity.ActionEntity;
+import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
import org.apache.nifi.web.api.entity.BannerEntity;
import org.apache.nifi.web.api.entity.BulletinBoardEntity;
import org.apache.nifi.web.api.entity.ClusteSummaryEntity;
@@ -118,6 +121,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.function.Predicate;
import java.util.stream.Collectors;
/**
@@ -388,7 +392,7 @@ public class FlowResource extends ApplicationResource {
}
// get all the controller services
- final Set<ControllerServiceEntity> controllerServices = serviceFacade.getControllerServices(null);
+ final Set<ControllerServiceEntity> controllerServices = serviceFacade.getControllerServices(null, false, false);
controllerServiceResource.populateRemainingControllerServiceEntitiesContent(controllerServices);
// create the response entity
@@ -426,11 +430,10 @@ public class FlowResource extends ApplicationResource {
}
)
public Response getControllerServicesFromGroup(
- @ApiParam(
- value = "The process group id.",
- required = true
- )
- @PathParam("id") String groupId) throws InterruptedException {
+ @ApiParam(value = "The process group id.", required = true) @PathParam("id") String groupId,
+ @ApiParam("Whether or not to include parent/ancestory process groups") @QueryParam("includeAncestorGroups") @DefaultValue("true") boolean includeAncestorGroups,
+ @ApiParam("Whether or not to include descendant process groups") @QueryParam("includeDescendantGroups") @DefaultValue("false") boolean includeDescendantGroups
+ ) throws InterruptedException {
authorizeFlow();
@@ -439,7 +442,7 @@ public class FlowResource extends ApplicationResource {
}
// get all the controller services
- final Set<ControllerServiceEntity> controllerServices = serviceFacade.getControllerServices(groupId);
+ final Set<ControllerServiceEntity> controllerServices = serviceFacade.getControllerServices(groupId, includeAncestorGroups, includeDescendantGroups);
controllerServiceResource.populateRemainingControllerServiceEntitiesContent(controllerServices);
// create the response entity
@@ -512,7 +515,7 @@ public class FlowResource extends ApplicationResource {
@Produces(MediaType.APPLICATION_JSON)
@Path("process-groups/{id}")
@ApiOperation(
- value = "Schedule or unschedule comopnents in the specified Process Group.",
+ value = "Schedule or unschedule components in the specified Process Group.",
response = ScheduleComponentsEntity.class,
authorizations = {
@Authorization(value = "Read - /flow", type = ""),
@@ -570,7 +573,7 @@ public class FlowResource extends ApplicationResource {
// ensure authorized for each processor we will attempt to schedule
group.findAllProcessors().stream()
- .filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PROCESSORS : ProcessGroup.UNSCHEDULABLE_PROCESSORS)
+ .filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PROCESSORS : ProcessGroup.UNSCHEDULABLE_PROCESSORS)
.filter(processor -> processor.isAuthorized(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()))
.forEach(processor -> {
componentIds.add(processor.getIdentifier());
@@ -578,7 +581,7 @@ public class FlowResource extends ApplicationResource {
// ensure authorized for each input port we will attempt to schedule
group.findAllInputPorts().stream()
- .filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PORTS : ProcessGroup.UNSCHEDULABLE_PORTS)
+ .filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PORTS : ProcessGroup.UNSCHEDULABLE_PORTS)
.filter(inputPort -> inputPort.isAuthorized(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()))
.forEach(inputPort -> {
componentIds.add(inputPort.getIdentifier());
@@ -586,7 +589,7 @@ public class FlowResource extends ApplicationResource {
// ensure authorized for each output port we will attempt to schedule
group.findAllOutputPorts().stream()
- .filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PORTS : ProcessGroup.UNSCHEDULABLE_PORTS)
+ .filter(ScheduledState.RUNNING.equals(state) ? ProcessGroup.SCHEDULABLE_PORTS : ProcessGroup.UNSCHEDULABLE_PORTS)
.filter(outputPort -> outputPort.isAuthorized(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()))
.forEach(outputPort -> {
componentIds.add(outputPort.getIdentifier());
@@ -640,7 +643,129 @@ public class FlowResource extends ApplicationResource {
componentsToSchedule.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> getRevision(e.getValue(), e.getKey())));
// update the process group
- final ScheduleComponentsEntity entity = serviceFacade.scheduleComponents(id, scheduledState, componentRevisions);
+ final ScheduleComponentsEntity entity = serviceFacade.scheduleComponents(id, scheduledState, componentRevisions);
+ return generateOkResponse(entity).build();
+ }
+ );
+ }
+
+
+ @PUT
+ @Consumes(MediaType.APPLICATION_JSON)
+ @Produces(MediaType.APPLICATION_JSON)
+ @Path("process-groups/{id}/controller-services")
+ @ApiOperation(value = "Enable or disable Controller Services in the specified Process Group.",
+ response = ActivateControllerServicesEntity.class,
+ authorizations = {
+ @Authorization(value = "Read - /flow", type = ""),
+ @Authorization(value = "Write - /{component-type}/{uuid} - For every service being enabled/disabled", type = "")
+ })
+ @ApiResponses(
+ value = {
+ @ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
+ @ApiResponse(code = 401, message = "Client could not be authenticated."),
+ @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
+ @ApiResponse(code = 404, message = "The specified resource could not be found."),
+ @ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
+ }
+ )
+ public Response activateControllerServices(
+ @Context HttpServletRequest httpServletRequest,
+ @ApiParam(value = "The process group id.", required = true)
+ @PathParam("id") String id,
+ @ApiParam(value = "The request to schedule or unschedule. If the comopnents in the request are not specified, all authorized components will be considered.", required = true)
+ final ActivateControllerServicesEntity requestEntity) {
+
+ // ensure the same id is being used
+ if (!id.equals(requestEntity.getId())) {
+ throw new IllegalArgumentException(String.format("The process group id (%s) in the request body does "
+ + "not equal the process group id of the requested resource (%s).", requestEntity.getId(), id));
+ }
+
+ final ControllerServiceState state;
+ if (requestEntity.getState() == null) {
+ throw new IllegalArgumentException("The controller service state must be specified.");
+ } else {
+ try {
+ state = ControllerServiceState.valueOf(requestEntity.getState());
+ } catch (final IllegalArgumentException iae) {
+ throw new IllegalArgumentException(String.format("The controller service state must be one of [%s].",
+ StringUtils.join(EnumSet.of(ControllerServiceState.ENABLED, ControllerServiceState.DISABLED), ", ")));
+ }
+ }
+
+ // ensure its a supported scheduled state
+ if (ControllerServiceState.DISABLING.equals(state) || ControllerServiceState.ENABLING.equals(state)) {
+ throw new IllegalArgumentException(String.format("The scheduled must be one of [%s].",
+ StringUtils.join(EnumSet.of(ControllerServiceState.ENABLED, ControllerServiceState.DISABLED), ", ")));
+ }
+
+ // if the components are not specified, gather all components and their current revision
+ if (requestEntity.getComponents() == null) {
+ // get the current revisions for the components being updated
+ final Set<Revision> revisions = serviceFacade.getRevisionsFromGroup(id, group -> {
+ final Set<String> componentIds = new HashSet<>();
+
+ final Predicate<ControllerServiceNode> filter;
+ if (ControllerServiceState.ENABLED.equals(state)) {
+ filter = service -> !service.isActive() && service.isValid();
+ } else {
+ filter = service -> service.isActive();
+ }
+
+ group.findAllControllerServices().stream()
+ .filter(filter)
+ .filter(service -> service.isAuthorized(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser()))
+ .forEach(service -> componentIds.add(service.getIdentifier()));
+ return componentIds;
+ });
+
+ // build the component mapping
+ final Map<String, RevisionDTO> componentsToSchedule = new HashMap<>();
+ revisions.forEach(revision -> {
+ final RevisionDTO dto = new RevisionDTO();
+ dto.setClientId(revision.getClientId());
+ dto.setVersion(revision.getVersion());
+ componentsToSchedule.put(revision.getComponentId(), dto);
+ });
+
+ // set the components and their current revision
+ requestEntity.setComponents(componentsToSchedule);
+ }
+
+ if (isReplicateRequest()) {
+ return replicate(HttpMethod.PUT, requestEntity);
+ }
+
+ final Map<String, RevisionDTO> requestComponentsToSchedule = requestEntity.getComponents();
+ final Map<String, Revision> requestComponentRevisions =
+ requestComponentsToSchedule.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> getRevision(e.getValue(), e.getKey())));
+ final Set<Revision> requestRevisions = new HashSet<>(requestComponentRevisions.values());
+
+ return withWriteLock(
+ serviceFacade,
+ requestEntity,
+ requestRevisions,
+ lookup -> {
+ // ensure access to the flow
+ authorizeFlow();
+
+ // ensure access to every component being scheduled
+ requestComponentsToSchedule.keySet().forEach(componentId -> {
+ final Authorizable authorizable = lookup.getControllerService(componentId).getAuthorizable();
+ authorizable.authorize(authorizer, RequestAction.WRITE, NiFiUserUtils.getNiFiUser());
+ });
+ },
+ () -> serviceFacade.verifyActivateControllerServices(id, state, requestComponentRevisions.keySet()),
+ (revisions, scheduleComponentsEntity) -> {
+ final ControllerServiceState serviceState = ControllerServiceState.valueOf(scheduleComponentsEntity.getState());
+
+ final Map<String, RevisionDTO> componentsToSchedule = scheduleComponentsEntity.getComponents();
+ final Map<String, Revision> componentRevisions =
+ componentsToSchedule.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> getRevision(e.getValue(), e.getKey())));
+
+ // update the controller services
+ final ActivateControllerServicesEntity entity = serviceFacade.activateControllerServices(id, serviceState, componentRevisions);
return generateOkResponse(entity).build();
}
);
[9/9] nifi git commit: NIFI-4224: - Initial implementation of Process
Group level Variable Registry - Updated to incorporate PR Feedback - Changed
log message because slf4j-simple apparently has a memory leak;
passing a String instead of passing in the C
Posted by mc...@apache.org.
NIFI-4224:
- Initial implementation of Process Group level Variable Registry
- Updated to incorporate PR Feedback
- Changed log message because slf4j-simple apparently has a memory leak; passing a String instead of passing in the Controller Service object as an argument addresses this.
- This closes #2051
Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/5cd8e93b
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/5cd8e93b
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/5cd8e93b
Branch: refs/heads/master
Commit: 5cd8e93beb30075ebe8623c2a278a5c9cf1afc75
Parents: c1b99d5
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Jul 25 13:46:10 2017 -0400
Committer: Matt Gilman <ma...@gmail.com>
Committed: Thu Aug 17 10:42:29 2017 -0400
----------------------------------------------------------------------
.../registry/ComponentVariableRegistry.java | 24 +
.../nifi/registry/VariableDescriptor.java | 77 +-
.../expression/language/EmptyPreparedQuery.java | 6 +
.../language/InvalidPreparedQuery.java | 6 +
.../language/NamedVariableImpact.java | 34 +
.../language/NoVariablesImpacted.java | 25 +
.../expression/language/PreparedQuery.java | 10 +
.../attribute/expression/language/Query.java | 1082 +----------------
.../language/StandardPreparedQuery.java | 85 +-
.../expression/language/ValueLookup.java | 1 +
.../expression/language/VariableImpact.java | 26 +
.../language/compile/CompiledExpression.java | 53 +
.../language/compile/ExpressionCompiler.java | 1122 ++++++++++++++++++
.../functions/AttributeEvaluator.java | 45 -
.../selection/AllAttributesEvaluator.java | 4 +
.../selection/AnyAttributeEvaluator.java | 4 +
.../selection/AttributeEvaluator.java | 49 +
.../evaluation/selection/MappingEvaluator.java | 4 +
.../selection/MultiNamedAttributeEvaluator.java | 4 +
.../expression/language/TestQuery.java | 44 +-
.../language/TestStandardPreparedQuery.java | 41 +
.../expression/language/TestValueLookup.java | 4 +-
.../nifi/web/api/dto/AffectedComponentDTO.java | 59 +
.../nifi/web/api/dto/ProcessGroupDTO.java | 16 +
.../apache/nifi/web/api/dto/VariableDTO.java | 69 ++
.../nifi/web/api/dto/VariableRegistryDTO.java | 50 +
.../dto/VariableRegistryUpdateRequestDTO.java | 115 ++
.../api/dto/VariableRegistryUpdateStepDTO.java | 59 +
.../ActivateControllerServicesEntity.java | 66 ++
.../api/entity/ScheduleComponentsEntity.java | 12 +-
.../nifi/web/api/entity/VariableEntity.java | 51 +
.../web/api/entity/VariableRegistryEntity.java | 50 +
.../VariableRegistryUpdateRequestEntity.java | 49 +
.../http/StandardHttpResponseMapper.java | 19 +-
.../controller/AbstractConfiguredComponent.java | 42 +-
.../nifi/controller/ConfiguredComponent.java | 7 +
.../nifi/controller/ProcessScheduler.java | 9 +-
.../apache/nifi/controller/ProcessorNode.java | 19 +-
.../service/ControllerServiceNode.java | 2 +-
.../service/ControllerServiceProvider.java | 6 +-
.../service/ControllerServiceReference.java | 10 +
.../org/apache/nifi/groups/ProcessGroup.java | 51 +-
.../apache/nifi/controller/FlowController.java | 46 +-
.../controller/StandardFlowSynchronizer.java | 70 +-
.../nifi/controller/StandardProcessorNode.java | 77 +-
.../reporting/AbstractReportingTaskNode.java | 22 +-
.../reporting/StandardReportingTaskNode.java | 8 +-
.../scheduling/EventDrivenSchedulingAgent.java | 9 +-
.../scheduling/QuartzSchedulingAgent.java | 8 +-
.../scheduling/StandardProcessScheduler.java | 42 +-
.../scheduling/TimerDrivenSchedulingAgent.java | 6 +-
.../serialization/FlowFromDOMFactory.java | 10 +
.../serialization/StandardFlowSerializer.java | 16 +-
.../service/ServiceStateTransition.java | 68 ++
.../service/StandardControllerServiceNode.java | 71 +-
.../StandardControllerServiceProvider.java | 76 +-
.../StandardControllerServiceReference.java | 33 +
.../nifi/fingerprint/FingerprintFactory.java | 69 +-
.../nifi/groups/StandardProcessGroup.java | 189 ++-
.../nifi/processor/StandardProcessContext.java | 12 +-
.../processor/StandardValidationContext.java | 7 +-
.../variable/FileBasedVariableRegistry.java | 89 ++
.../variable/MutableVariableRegistry.java | 52 +
.../StandardComponentVariableRegistry.java | 94 ++
.../nifi/util/FileBasedVariableRegistry.java | 88 --
.../src/main/resources/FlowConfiguration.xsd | 6 +
.../src/main/resources/nifi-context.xml | 2 +-
.../controller/StandardFlowServiceTest.java | 2 +-
.../nifi/controller/TestFlowController.java | 2 +-
.../controller/TestStandardProcessorNode.java | 48 +-
.../reporting/TestStandardReportingContext.java | 20 +-
.../scheduling/TestProcessorLifecycle.java | 2 +-
.../TestStandardProcessScheduler.java | 10 +-
.../StandardFlowSerializerTest.java | 2 +-
.../StandardControllerServiceProviderTest.java | 2 +-
.../TestStandardControllerServiceProvider.java | 41 +-
.../service/mock/MockProcessGroup.java | 32 +-
.../util/TestFileBasedVariableRegistry.java | 1 +
.../apache/nifi/audit/ProcessGroupAuditor.java | 63 +-
.../variable/VariableRegistryUpdateRequest.java | 109 ++
.../variable/VariableRegistryUpdateStep.java | 48 +
.../org/apache/nifi/web/NiFiServiceFacade.java | 126 +-
.../apache/nifi/web/NiFiServiceFacadeLock.java | 12 +
.../nifi/web/StandardNiFiServiceFacade.java | 257 +++-
.../nifi/web/api/ApplicationResource.java | 52 +-
.../org/apache/nifi/web/api/FlowResource.java | 149 ++-
.../nifi/web/api/ProcessGroupResource.java | 1034 +++++++++++++++-
.../org/apache/nifi/web/api/dto/DtoFactory.java | 217 +++-
.../apache/nifi/web/api/dto/EntityFactory.java | 13 +
.../nifi/web/dao/ControllerServiceDAO.java | 2 +-
.../apache/nifi/web/dao/ProcessGroupDAO.java | 44 +-
.../dao/impl/StandardControllerServiceDAO.java | 9 +-
.../web/dao/impl/StandardProcessGroupDAO.java | 80 +-
.../java/org/apache/nifi/web/util/Pause.java | 30 +
.../src/main/resources/nifi-web-api-context.xml | 1 +
95 files changed, 5258 insertions(+), 1831 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-api/src/main/java/org/apache/nifi/registry/ComponentVariableRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/registry/ComponentVariableRegistry.java b/nifi-api/src/main/java/org/apache/nifi/registry/ComponentVariableRegistry.java
new file mode 100644
index 0000000..7284b77
--- /dev/null
+++ b/nifi-api/src/main/java/org/apache/nifi/registry/ComponentVariableRegistry.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.registry;
+
+public interface ComponentVariableRegistry extends VariableRegistry {
+ VariableRegistry getParent();
+
+ void setParent(VariableRegistry parentRegistry);
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-api/src/main/java/org/apache/nifi/registry/VariableDescriptor.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/registry/VariableDescriptor.java b/nifi-api/src/main/java/org/apache/nifi/registry/VariableDescriptor.java
index 5b1e88f..f9e9ce2 100644
--- a/nifi-api/src/main/java/org/apache/nifi/registry/VariableDescriptor.java
+++ b/nifi-api/src/main/java/org/apache/nifi/registry/VariableDescriptor.java
@@ -66,6 +66,44 @@ public final class VariableDescriptor implements Comparable<VariableDescriptor>
return getName().compareTo(o.getName());
}
+ public String getName() {
+ return name;
+ }
+
+ public String getDescription() {
+ return description;
+ }
+
+ public boolean isSensitive() {
+ return sensitive;
+ }
+
+ @Override
+ public boolean equals(final Object other) {
+ if (other == null) {
+ return false;
+ }
+ if (this == other) {
+ return true;
+ }
+ if (!(other instanceof VariableDescriptor)) {
+ return false;
+ }
+
+ final VariableDescriptor desc = (VariableDescriptor) other;
+ return this.name.equals(desc.name);
+ }
+
+ @Override
+ public int hashCode() {
+ return 797 + this.name.hashCode() * 97;
+ }
+
+ @Override
+ public String toString() {
+ return getClass().getSimpleName() + "[" + name + "]";
+ }
+
public static final class Builder {
private String name = null;
@@ -112,43 +150,4 @@ public final class VariableDescriptor implements Comparable<VariableDescriptor>
return new VariableDescriptor(this);
}
}
-
- public String getName() {
- return name;
- }
-
- public String getDescription() {
- return description;
- }
-
- public boolean isSensitive() {
- return sensitive;
- }
-
- @Override
- public boolean equals(final Object other) {
- if (other == null) {
- return false;
- }
- if (!(other instanceof VariableDescriptor)) {
- return false;
- }
- if (this == other) {
- return true;
- }
-
- final VariableDescriptor desc = (VariableDescriptor) other;
- return this.name.equals(desc.name);
- }
-
- @Override
- public int hashCode() {
- return 797 + this.name.hashCode() * 97;
- }
-
- @Override
- public String toString() {
- return getClass().getSimpleName() + "[" + name + "]";
- }
-
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
index a435b08..e9ac03b 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language;
import java.util.Map;
+
import org.apache.nifi.expression.AttributeValueDecorator;
import org.apache.nifi.processor.exception.ProcessException;
@@ -43,4 +44,9 @@ public class EmptyPreparedQuery implements PreparedQuery {
public boolean isExpressionLanguagePresent() {
return false;
}
+
+ @Override
+ public VariableImpact getVariableImpact() {
+ return VariableImpact.NEVER_IMPACTED;
+ }
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
index ce0dec3..8ef996b 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language;
import java.util.Map;
+
import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
import org.apache.nifi.expression.AttributeValueDecorator;
import org.apache.nifi.processor.exception.ProcessException;
@@ -52,4 +53,9 @@ public class InvalidPreparedQuery implements PreparedQuery {
public boolean isExpressionLanguagePresent() {
return false;
}
+
+ @Override
+ public VariableImpact getVariableImpact() {
+ return VariableImpact.NEVER_IMPACTED;
+ }
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NamedVariableImpact.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NamedVariableImpact.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NamedVariableImpact.java
new file mode 100644
index 0000000..730c6ec
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NamedVariableImpact.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.attribute.expression.language;
+
+import java.util.Set;
+
+public class NamedVariableImpact implements VariableImpact {
+ private final Set<String> impactedVariableNames;
+
+ public NamedVariableImpact(final Set<String> impactedVariableNames) {
+ this.impactedVariableNames = impactedVariableNames;
+ }
+
+ @Override
+ public boolean isImpacted(final String variableName) {
+ return impactedVariableNames.contains(variableName);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NoVariablesImpacted.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NoVariablesImpacted.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NoVariablesImpacted.java
new file mode 100644
index 0000000..669a532
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/NoVariablesImpacted.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.attribute.expression.language;
+
+public class NoVariablesImpacted implements VariableImpact {
+ @Override
+ public boolean isImpacted(final String variableName) {
+ return false;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
index 5552cac..c51656d 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java
@@ -18,6 +18,7 @@ package org.apache.nifi.attribute.expression.language;
import java.util.Map;
+
import org.apache.nifi.expression.AttributeValueDecorator;
import org.apache.nifi.processor.exception.ProcessException;
@@ -28,4 +29,13 @@ public interface PreparedQuery {
String evaluateExpressions(final Map<String, String> valueLookup, final AttributeValueDecorator decorator, final Map<String, String> stateVariables) throws ProcessException;
boolean isExpressionLanguagePresent();
+
+ /**
+ * Returns a {@link VariableImpact} that can be used to determine whether or not a given
+ * variable impacts this Expression.
+ *
+ * @return a {@link VariableImpact} that can be used to determine whether or not a given
+ * variable impacts this Expression.
+ */
+ VariableImpact getVariableImpact();
}
[5/9] nifi git commit: NIFI-4224: - Initial implementation of Process
Group level Variable Registry - Updated to incorporate PR Feedback - Changed
log message because slf4j-simple apparently has a memory leak;
passing a String instead of passing in the C
Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
index 9611b73..07923c6 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/AbstractReportingTaskNode.java
@@ -16,6 +16,14 @@
*/
package org.apache.nifi.controller.reporting;
+import java.net.URL;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
import org.apache.nifi.annotation.configuration.DefaultSchedule;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.ConfigurableComponent;
@@ -33,7 +41,7 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.controller.service.StandardConfigurationContext;
import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.ComponentVariableRegistry;
import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.util.CharacterFilterUtils;
@@ -42,14 +50,6 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.core.annotation.AnnotationUtils;
-import java.net.URL;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
public abstract class AbstractReportingTaskNode extends AbstractConfiguredComponent implements ReportingTaskNode {
private static final Logger LOG = LoggerFactory.getLogger(AbstractReportingTaskNode.class);
@@ -66,7 +66,7 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
public AbstractReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id,
final ControllerServiceProvider controllerServiceProvider, final ProcessScheduler processScheduler,
- final ValidationContextFactory validationContextFactory, final VariableRegistry variableRegistry,
+ final ValidationContextFactory validationContextFactory, final ComponentVariableRegistry variableRegistry,
final ReloadComponent reloadComponent) {
this(reportingTask, id, controllerServiceProvider, processScheduler, validationContextFactory,
@@ -77,7 +77,7 @@ public abstract class AbstractReportingTaskNode extends AbstractConfiguredCompon
public AbstractReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final ControllerServiceProvider controllerServiceProvider,
final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory,
- final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry,
+ final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
super(id, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
index 53b7d15..40bdf8b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/reporting/StandardReportingTaskNode.java
@@ -22,13 +22,13 @@ 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.controller.ReloadComponent;
import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.LoggableComponent;
import org.apache.nifi.controller.ProcessScheduler;
+import org.apache.nifi.controller.ReloadComponent;
import org.apache.nifi.controller.ReportingTaskNode;
import org.apache.nifi.controller.ValidationContextFactory;
-import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.ComponentVariableRegistry;
import org.apache.nifi.reporting.ReportingContext;
import org.apache.nifi.reporting.ReportingTask;
@@ -38,14 +38,14 @@ public class StandardReportingTaskNode extends AbstractReportingTaskNode impleme
public StandardReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final FlowController controller,
final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory,
- final VariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
+ final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
super(reportingTask, id, controller, processScheduler, validationContextFactory, variableRegistry, reloadComponent);
this.flowController = controller;
}
public StandardReportingTaskNode(final LoggableComponent<ReportingTask> reportingTask, final String id, final FlowController controller,
final ProcessScheduler processScheduler, final ValidationContextFactory validationContextFactory,
- final String componentType, final String canonicalClassName, final VariableRegistry variableRegistry,
+ final String componentType, final String canonicalClassName, final ComponentVariableRegistry variableRegistry,
final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
super(reportingTask, id, controller, processScheduler, validationContextFactory, componentType, canonicalClassName,
variableRegistry, reloadComponent, isExtensionMissing);
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java
index 0af8657..58d25bb 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/EventDrivenSchedulingAgent.java
@@ -45,7 +45,6 @@ import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.processor.StandardProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.util.Connectables;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.ReflectionUtils;
@@ -62,7 +61,6 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
private final ProcessContextFactory contextFactory;
private final AtomicInteger maxThreadCount;
private final StringEncryptor encryptor;
- private final VariableRegistry variableRegistry;
private volatile String adminYieldDuration = "1 sec";
@@ -70,8 +68,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
private final ConcurrentMap<Connectable, ScheduleState> scheduleStates = new ConcurrentHashMap<>();
public EventDrivenSchedulingAgent(final FlowEngine flowEngine, final ControllerServiceProvider serviceProvider, final StateManagerProvider stateManagerProvider,
- final EventDrivenWorkerQueue workerQueue, final ProcessContextFactory contextFactory, final int maxThreadCount, final StringEncryptor encryptor,
- final VariableRegistry variableRegistry) {
+ final EventDrivenWorkerQueue workerQueue, final ProcessContextFactory contextFactory, final int maxThreadCount, final StringEncryptor encryptor) {
super(flowEngine);
this.serviceProvider = serviceProvider;
this.stateManagerProvider = stateManagerProvider;
@@ -79,7 +76,6 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
this.contextFactory = contextFactory;
this.maxThreadCount = new AtomicInteger(maxThreadCount);
this.encryptor = encryptor;
- this.variableRegistry = variableRegistry;
for (int i = 0; i < maxThreadCount; i++) {
final Runnable eventDrivenTask = new EventDrivenTask(workerQueue);
@@ -188,8 +184,7 @@ public class EventDrivenSchedulingAgent extends AbstractSchedulingAgent {
if (connectable instanceof ProcessorNode) {
final ProcessorNode procNode = (ProcessorNode) connectable;
- final StandardProcessContext standardProcessContext = new StandardProcessContext(procNode, serviceProvider,
- encryptor, getStateManager(connectable.getIdentifier()), variableRegistry);
+ final StandardProcessContext standardProcessContext = new StandardProcessContext(procNode, serviceProvider, encryptor, getStateManager(connectable.getIdentifier()));
final long runNanos = procNode.getRunDuration(TimeUnit.NANOSECONDS);
final ProcessSessionFactory sessionFactory;
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
index bd3c2bb..eb855d4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/QuartzSchedulingAgent.java
@@ -38,7 +38,6 @@ import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.engine.FlowEngine;
import org.apache.nifi.processor.StandardProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.util.FormatUtils;
import org.quartz.CronExpression;
import org.slf4j.Logger;
@@ -51,18 +50,15 @@ public class QuartzSchedulingAgent extends AbstractSchedulingAgent {
private final FlowController flowController;
private final ProcessContextFactory contextFactory;
private final StringEncryptor encryptor;
- private final VariableRegistry variableRegistry;
private volatile String adminYieldDuration = "1 sec";
private final Map<Object, List<AtomicBoolean>> canceledTriggers = new HashMap<>();
- public QuartzSchedulingAgent(final FlowController flowController, final FlowEngine flowEngine, final ProcessContextFactory contextFactory, final StringEncryptor enryptor,
- final VariableRegistry variableRegistry) {
+ public QuartzSchedulingAgent(final FlowController flowController, final FlowEngine flowEngine, final ProcessContextFactory contextFactory, final StringEncryptor enryptor) {
super(flowEngine);
this.flowController = flowController;
this.contextFactory = contextFactory;
this.encryptor = enryptor;
- this.variableRegistry = variableRegistry;
}
private StateManager getStateManager(final String componentId) {
@@ -145,7 +141,7 @@ public class QuartzSchedulingAgent extends AbstractSchedulingAgent {
if (connectable.getConnectableType() == ConnectableType.PROCESSOR) {
final ProcessorNode procNode = (ProcessorNode) connectable;
- final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier()), variableRegistry);
+ final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier()));
ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController, contextFactory, scheduleState, standardProcContext);
continuallyRunTask = runnableTask;
} else {
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
index 5368d37..122e50c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
@@ -53,7 +53,6 @@ import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.SimpleProcessLogger;
import org.apache.nifi.processor.StandardProcessContext;
-import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.reporting.ReportingTask;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.util.FormatUtils;
@@ -84,19 +83,16 @@ public final class StandardProcessScheduler implements ProcessScheduler {
private final ScheduledExecutorService componentMonitoringThreadPool = new FlowEngine(8, "StandardProcessScheduler", true);
private final StringEncryptor encryptor;
- private final VariableRegistry variableRegistry;
public StandardProcessScheduler(
final ControllerServiceProvider controllerServiceProvider,
final StringEncryptor encryptor,
final StateManagerProvider stateManagerProvider,
- final VariableRegistry variableRegistry,
final NiFiProperties nifiProperties
) {
this.controllerServiceProvider = controllerServiceProvider;
this.encryptor = encryptor;
this.stateManagerProvider = stateManagerProvider;
- this.variableRegistry = variableRegistry;
administrativeYieldDuration = nifiProperties.getAdministrativeYieldDuration();
administrativeYieldMillis = FormatUtils.getTimeDuration(administrativeYieldDuration, TimeUnit.MILLISECONDS);
@@ -301,15 +297,17 @@ public final class StandardProcessScheduler implements ProcessScheduler {
* @see StandardProcessorNode#start(ScheduledExecutorService, long, org.apache.nifi.processor.ProcessContext, Runnable).
*/
@Override
- public synchronized void startProcessor(final ProcessorNode procNode) {
+ public synchronized CompletableFuture<Void> startProcessor(final ProcessorNode procNode) {
StandardProcessContext processContext = new StandardProcessContext(procNode, this.controllerServiceProvider,
- this.encryptor, getStateManager(procNode.getIdentifier()), variableRegistry);
+ this.encryptor, getStateManager(procNode.getIdentifier()));
final ScheduleState scheduleState = getScheduleState(requireNonNull(procNode));
+ final CompletableFuture<Void> future = new CompletableFuture<>();
SchedulingAgentCallback callback = new SchedulingAgentCallback() {
@Override
public void trigger() {
getSchedulingAgent(procNode).schedule(procNode, scheduleState);
+ future.complete(null);
}
@Override
@@ -324,7 +322,9 @@ public final class StandardProcessScheduler implements ProcessScheduler {
}
};
+ LOG.info("Starting {}", procNode);
procNode.start(this.componentLifeCycleThreadPool, this.administrativeYieldMillis, processContext, callback);
+ return future;
}
/**
@@ -335,12 +335,13 @@ public final class StandardProcessScheduler implements ProcessScheduler {
* @see StandardProcessorNode#stop(ScheduledExecutorService, org.apache.nifi.processor.ProcessContext, SchedulingAgent, ScheduleState)
*/
@Override
- public synchronized void stopProcessor(final ProcessorNode procNode) {
+ public synchronized CompletableFuture<Void> stopProcessor(final ProcessorNode procNode) {
StandardProcessContext processContext = new StandardProcessContext(procNode, this.controllerServiceProvider,
- this.encryptor, getStateManager(procNode.getIdentifier()), variableRegistry);
+ this.encryptor, getStateManager(procNode.getIdentifier()));
final ScheduleState state = getScheduleState(procNode);
- procNode.stop(this.componentLifeCycleThreadPool, processContext, getSchedulingAgent(procNode), state);
+ LOG.info("Stopping {}", procNode);
+ return procNode.stop(this.componentLifeCycleThreadPool, processContext, getSchedulingAgent(procNode), state);
}
@Override
@@ -537,20 +538,35 @@ public final class StandardProcessScheduler implements ProcessScheduler {
@Override
public CompletableFuture<Void> enableControllerService(final ControllerServiceNode service) {
+ LOG.info("Enabling " + service);
return service.enable(this.componentLifeCycleThreadPool, this.administrativeYieldMillis);
}
@Override
- public void disableControllerService(final ControllerServiceNode service) {
- service.disable(this.componentLifeCycleThreadPool);
+ public CompletableFuture<Void> disableControllerService(final ControllerServiceNode service) {
+ LOG.info("Disabling {}", service);
+ return service.disable(this.componentLifeCycleThreadPool);
}
@Override
- public void disableControllerServices(final List<ControllerServiceNode> services) {
+ public CompletableFuture<Void> disableControllerServices(final List<ControllerServiceNode> services) {
+ if (services == null || services.isEmpty()) {
+ return CompletableFuture.completedFuture(null);
+ }
+
+ CompletableFuture<Void> future = null;
if (!requireNonNull(services).isEmpty()) {
for (ControllerServiceNode controllerServiceNode : services) {
- this.disableControllerService(controllerServiceNode);
+ final CompletableFuture<Void> serviceFuture = this.disableControllerService(controllerServiceNode);
+
+ if (future == null) {
+ future = serviceFuture;
+ } else {
+ future = CompletableFuture.allOf(future, serviceFuture);
+ }
}
}
+
+ return future;
}
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
index a82fde4..9dc329a 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/TimerDrivenSchedulingAgent.java
@@ -37,7 +37,6 @@ import org.apache.nifi.engine.FlowEngine;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.StandardProcessContext;
import org.apache.nifi.processor.exception.ProcessException;
-import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.util.FormatUtils;
import org.apache.nifi.util.NiFiProperties;
import org.slf4j.Logger;
@@ -51,7 +50,6 @@ public class TimerDrivenSchedulingAgent extends AbstractSchedulingAgent {
private final FlowController flowController;
private final ProcessContextFactory contextFactory;
private final StringEncryptor encryptor;
- private final VariableRegistry variableRegistry;
private volatile String adminYieldDuration = "1 sec";
@@ -60,13 +58,11 @@ public class TimerDrivenSchedulingAgent extends AbstractSchedulingAgent {
final FlowEngine flowEngine,
final ProcessContextFactory contextFactory,
final StringEncryptor encryptor,
- final VariableRegistry variableRegistry,
final NiFiProperties nifiProperties) {
super(flowEngine);
this.flowController = flowController;
this.contextFactory = contextFactory;
this.encryptor = encryptor;
- this.variableRegistry = variableRegistry;
final String boredYieldDuration = nifiProperties.getBoredYieldDuration();
try {
@@ -109,7 +105,7 @@ public class TimerDrivenSchedulingAgent extends AbstractSchedulingAgent {
// Determine the task to run and create it.
if (connectable.getConnectableType() == ConnectableType.PROCESSOR) {
final ProcessorNode procNode = (ProcessorNode) connectable;
- final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier()), variableRegistry);
+ final StandardProcessContext standardProcContext = new StandardProcessContext(procNode, flowController, encryptor, getStateManager(connectable.getIdentifier()));
final ContinuallyRunProcessorTask runnableTask = new ContinuallyRunProcessorTask(this, procNode, flowController,
contextFactory, scheduleState, standardProcContext);
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
index 61d9d29..f30a71e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/FlowFromDOMFactory.java
@@ -144,6 +144,16 @@ public class FlowFromDOMFactory {
dto.setPosition(getPosition(DomUtils.getChild(element, "position")));
dto.setComments(getString(element, "comment"));
+ final Map<String, String> variables = new HashMap<>();
+ final NodeList variableList = DomUtils.getChildNodesByTagName(element, "variable");
+ for (int i = 0; i < variableList.getLength(); i++) {
+ final Element variableElement = (Element) variableList.item(i);
+ final String name = variableElement.getAttribute("name");
+ final String value = variableElement.getAttribute("value");
+ variables.put(name, value);
+ }
+ dto.setVariables(variables);
+
final Set<ProcessorDTO> processors = new HashSet<>();
final Set<ConnectionDTO> connections = new HashSet<>();
final Set<FunnelDTO> funnels = new HashSet<>();
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
index 4a9a0f7..2a8df96 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/StandardFlowSerializer.java
@@ -37,6 +37,8 @@ import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.RemoteProcessGroup;
import org.apache.nifi.persistence.TemplateSerializer;
import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.util.CharacterFilterUtils;
@@ -70,7 +72,7 @@ import java.util.concurrent.TimeUnit;
*/
public class StandardFlowSerializer implements FlowSerializer {
- private static final String MAX_ENCODING_VERSION = "1.1";
+ private static final String MAX_ENCODING_VERSION = "1.2";
private final StringEncryptor encryptor;
@@ -202,6 +204,18 @@ public class StandardFlowSerializer implements FlowSerializer {
for (final Template template : group.getTemplates()) {
addTemplate(element, template);
}
+
+ final VariableRegistry variableRegistry = group.getVariableRegistry();
+ for (final Map.Entry<VariableDescriptor, String> entry : variableRegistry.getVariableMap().entrySet()) {
+ addVariable(element, entry.getKey().getName(), entry.getValue());
+ }
+ }
+
+ private static void addVariable(final Element parentElement, final String variableName, final String variableValue) {
+ final Element variableElement = parentElement.getOwnerDocument().createElement("variable");
+ variableElement.setAttribute("name", variableName);
+ variableElement.setAttribute("value", variableValue);
+ parentElement.appendChild(variableElement);
}
private static void addBundle(final Element parentElement, final BundleCoordinate coordinate) {
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.java
new file mode 100644
index 0000000..148b847
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ServiceStateTransition.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.controller.service;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+public class ServiceStateTransition {
+ private ControllerServiceState state = ControllerServiceState.DISABLED;
+ private final List<CompletableFuture<?>> enabledFutures = new ArrayList<>();
+ private final List<CompletableFuture<?>> disabledFutures = new ArrayList<>();
+
+
+ public synchronized boolean transitionToEnabling(final ControllerServiceState expectedState, final CompletableFuture<?> enabledFuture) {
+ if (expectedState != state) {
+ return false;
+ }
+
+ state = ControllerServiceState.ENABLING;
+ enabledFutures.add(enabledFuture);
+ return true;
+ }
+
+ public synchronized boolean enable() {
+ if (state != ControllerServiceState.ENABLING) {
+ return false;
+ }
+
+ state = ControllerServiceState.ENABLED;
+ enabledFutures.stream().forEach(future -> future.complete(null));
+ return true;
+ }
+
+ public synchronized boolean transitionToDisabling(final ControllerServiceState expectedState, final CompletableFuture<?> disabledFuture) {
+ if (expectedState != state) {
+ return false;
+ }
+
+ state = ControllerServiceState.DISABLING;
+ disabledFutures.add(disabledFuture);
+ return true;
+ }
+
+ public synchronized void disable() {
+ state = ControllerServiceState.DISABLED;
+ disabledFutures.stream().forEach(future -> future.complete(null));
+ }
+
+ public synchronized ControllerServiceState getState() {
+ return state;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
index fa4ab84..216a996 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java
@@ -16,6 +16,24 @@
*/
package org.apache.nifi.controller.service;
+import java.lang.reflect.InvocationTargetException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.Restricted;
import org.apache.nifi.annotation.documentation.DeprecationNotice;
@@ -30,48 +48,30 @@ import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.controller.AbstractConfiguredComponent;
-import org.apache.nifi.controller.ReloadComponent;
import org.apache.nifi.controller.ConfigurationContext;
import org.apache.nifi.controller.ConfiguredComponent;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.LoggableComponent;
+import org.apache.nifi.controller.ReloadComponent;
import org.apache.nifi.controller.ValidationContextFactory;
import org.apache.nifi.controller.exception.ControllerServiceInstantiationException;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.logging.ComponentLog;
import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.processor.SimpleProcessLogger;
-import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.ComponentVariableRegistry;
import org.apache.nifi.util.CharacterFilterUtils;
import org.apache.nifi.util.ReflectionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.lang.reflect.InvocationTargetException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
public class StandardControllerServiceNode extends AbstractConfiguredComponent implements ControllerServiceNode {
private static final Logger LOG = LoggerFactory.getLogger(StandardControllerServiceNode.class);
private final AtomicReference<ControllerServiceDetails> controllerServiceHolder = new AtomicReference<>(null);
private final ControllerServiceProvider serviceProvider;
- private final AtomicReference<ControllerServiceState> stateRef = new AtomicReference<>(ControllerServiceState.DISABLED);
+ private final ServiceStateTransition stateTransition = new ServiceStateTransition();
private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
private final Lock readLock = rwLock.readLock();
@@ -85,7 +85,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
public StandardControllerServiceNode(final LoggableComponent<ControllerService> implementation, final LoggableComponent<ControllerService> proxiedControllerService,
final ControllerServiceInvocationHandler invocationHandler, final String id, final ValidationContextFactory validationContextFactory,
- final ControllerServiceProvider serviceProvider, final VariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
+ final ControllerServiceProvider serviceProvider, final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
this(implementation, proxiedControllerService, invocationHandler, id, validationContextFactory, serviceProvider,
implementation.getComponent().getClass().getSimpleName(), implementation.getComponent().getClass().getCanonicalName(), variableRegistry, reloadComponent, false);
@@ -94,7 +94,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
public StandardControllerServiceNode(final LoggableComponent<ControllerService> implementation, final LoggableComponent<ControllerService> proxiedControllerService,
final ControllerServiceInvocationHandler invocationHandler, final String id, final ValidationContextFactory validationContextFactory,
final ControllerServiceProvider serviceProvider, final String componentType, final String componentCanonicalClass,
- final VariableRegistry variableRegistry, final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
+ final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
this.serviceProvider = serviceProvider;
@@ -363,7 +363,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
@Override
public ControllerServiceState getState() {
- return stateRef.get();
+ return stateTransition.getState();
}
@Override
@@ -394,7 +394,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
public CompletableFuture<Void> enable(final ScheduledExecutorService scheduler, final long administrativeYieldMillis) {
final CompletableFuture<Void> future = new CompletableFuture<>();
- if (this.stateRef.compareAndSet(ControllerServiceState.DISABLED, ControllerServiceState.ENABLING)) {
+ if (this.stateTransition.transitionToEnabling(ControllerServiceState.DISABLED, future)) {
synchronized (active) {
this.active.set(true);
}
@@ -410,17 +410,15 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
boolean shouldEnable = false;
synchronized (active) {
- shouldEnable = active.get() && stateRef.compareAndSet(ControllerServiceState.ENABLING, ControllerServiceState.ENABLED);
+ shouldEnable = active.get() && stateTransition.enable();
}
- future.complete(null);
-
if (!shouldEnable) {
LOG.debug("Disabling service " + this + " after it has been enabled due to disable action being initiated.");
// Can only happen if user initiated DISABLE operation before service finished enabling. It's state will be
// set to DISABLING (see disable() operation)
invokeDisable(configContext);
- stateRef.set(ControllerServiceState.DISABLED);
+ stateTransition.disable();
}
} catch (Exception e) {
future.completeExceptionally(e);
@@ -437,7 +435,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
try (final NarCloseable nc = NarCloseable.withComponentNarLoader(getControllerServiceImplementation().getClass(), getIdentifier())) {
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnDisabled.class, getControllerServiceImplementation(), configContext);
}
- stateRef.set(ControllerServiceState.DISABLED);
+ stateTransition.disable();
}
}
}
@@ -464,7 +462,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
* DISABLED state.
*/
@Override
- public void disable(ScheduledExecutorService scheduler) {
+ public CompletableFuture<Void> disable(ScheduledExecutorService scheduler) {
/*
* The reason for synchronization is to ensure consistency of the
* service state when another thread is in the middle of enabling this
@@ -475,7 +473,8 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
this.active.set(false);
}
- if (this.stateRef.compareAndSet(ControllerServiceState.ENABLED, ControllerServiceState.DISABLING)) {
+ final CompletableFuture<Void> future = new CompletableFuture<>();
+ if (this.stateTransition.transitionToDisabling(ControllerServiceState.ENABLED, future)) {
final ConfigurationContext configContext = new StandardConfigurationContext(this, this.serviceProvider, null, getVariableRegistry());
scheduler.execute(new Runnable() {
@Override
@@ -483,13 +482,15 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
try {
invokeDisable(configContext);
} finally {
- stateRef.set(ControllerServiceState.DISABLED);
+ stateTransition.disable();
}
}
});
} else {
- this.stateRef.compareAndSet(ControllerServiceState.ENABLING, ControllerServiceState.DISABLING);
+ this.stateTransition.transitionToDisabling(ControllerServiceState.ENABLING, future);
}
+
+ return future;
}
/**
@@ -515,7 +516,7 @@ public class StandardControllerServiceNode extends AbstractConfiguredComponent i
@Override
public Collection<ValidationResult> getValidationErrors(Set<String> serviceIdentifiersNotToValidate) {
Collection<ValidationResult> results = null;
- if (stateRef.get() == ControllerServiceState.DISABLED) {
+ if (getState() == ControllerServiceState.DISABLED) {
results = super.getValidationErrors(serviceIdentifiersNotToValidate);
}
return results != null ? results : Collections.emptySet();
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
index 5c4e394..0745ed0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
@@ -30,6 +30,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.Future;
@@ -62,7 +63,9 @@ 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.registry.ComponentVariableRegistry;
import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.StandardComponentVariableRegistry;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.reporting.Severity;
import org.apache.nifi.util.NiFiProperties;
@@ -148,8 +151,9 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
final LoggableComponent<ControllerService> originalLoggableComponent = new LoggableComponent<>(originalService, bundleCoordinate, serviceLogger);
final LoggableComponent<ControllerService> proxiedLoggableComponent = new LoggableComponent<>(proxiedService, bundleCoordinate, serviceLogger);
+ final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
final ControllerServiceNode serviceNode = new StandardControllerServiceNode(originalLoggableComponent, proxiedLoggableComponent, invocationHandler,
- id, validationContextFactory, this, variableRegistry, flowController);
+ id, validationContextFactory, this, componentVarRegistry, flowController);
serviceNode.setName(rawClass.getSimpleName());
invocationHandler.setServiceNode(serviceNode);
@@ -226,8 +230,9 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
final LoggableComponent<ControllerService> proxiedLoggableComponent = new LoggableComponent<>(proxiedService, bundleCoordinate, null);
+ final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
final ControllerServiceNode serviceNode = new StandardControllerServiceNode(proxiedLoggableComponent, proxiedLoggableComponent, invocationHandler, id,
- new StandardValidationContextFactory(this, variableRegistry), this, componentType, type, variableRegistry, flowController, true);
+ new StandardValidationContextFactory(this, variableRegistry), this, componentType, type, componentVarRegistry, flowController, true);
serviceCache.putIfAbsent(id, serviceNode);
return serviceNode;
@@ -235,9 +240,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
@Override
public Set<ConfiguredComponent> disableReferencingServices(final ControllerServiceNode serviceNode) {
- // Get a list of all Controller Services that need to be disabled, in the order that they need to be
- // disabled.
- final List<ControllerServiceNode> toDisable = findRecursiveReferences(serviceNode, ControllerServiceNode.class);
+ // Get a list of all Controller Services that need to be disabled, in the order that they need to be disabled.
+ final List<ControllerServiceNode> toDisable = serviceNode.getReferences().findRecursiveReferences(ControllerServiceNode.class);
final Set<ControllerServiceNode> serviceSet = new HashSet<>(toDisable);
@@ -258,8 +262,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
public Set<ConfiguredComponent> scheduleReferencingComponents(final ControllerServiceNode serviceNode) {
// find all of the schedulable components (processors, reporting tasks) that refer to this Controller Service,
// or a service that references this controller service, etc.
- final List<ProcessorNode> processors = findRecursiveReferences(serviceNode, ProcessorNode.class);
- final List<ReportingTaskNode> reportingTasks = findRecursiveReferences(serviceNode, ReportingTaskNode.class);
+ final List<ProcessorNode> processors = serviceNode.getReferences().findRecursiveReferences(ProcessorNode.class);
+ final List<ReportingTaskNode> reportingTasks = serviceNode.getReferences().findRecursiveReferences(ReportingTaskNode.class);
final Set<ConfiguredComponent> updated = new HashSet<>();
@@ -298,8 +302,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
public Set<ConfiguredComponent> unscheduleReferencingComponents(final ControllerServiceNode serviceNode) {
// find all of the schedulable components (processors, reporting tasks) that refer to this Controller Service,
// or a service that references this controller service, etc.
- final List<ProcessorNode> processors = findRecursiveReferences(serviceNode, ProcessorNode.class);
- final List<ReportingTaskNode> reportingTasks = findRecursiveReferences(serviceNode, ReportingTaskNode.class);
+ final List<ProcessorNode> processors = serviceNode.getReferences().findRecursiveReferences(ProcessorNode.class);
+ final List<ReportingTaskNode> reportingTasks = serviceNode.getReferences().findRecursiveReferences(ReportingTaskNode.class);
final Set<ConfiguredComponent> updated = new HashSet<>();
@@ -333,7 +337,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
}
@Override
- public Future<Void> enableControllerService(final ControllerServiceNode serviceNode) {
+ public CompletableFuture<Void> enableControllerService(final ControllerServiceNode serviceNode) {
serviceNode.verifyCanEnable();
return processScheduler.enableControllerService(serviceNode);
}
@@ -450,9 +454,9 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
}
@Override
- public void disableControllerService(final ControllerServiceNode serviceNode) {
+ public CompletableFuture<Void> disableControllerService(final ControllerServiceNode serviceNode) {
serviceNode.verifyCanDisable();
- processScheduler.disableControllerService(serviceNode);
+ return processScheduler.disableControllerService(serviceNode);
}
@Override
@@ -589,43 +593,10 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
return allServices;
}
- /**
- * Returns a List of all components that reference the given referencedNode
- * (either directly or indirectly through another service) that are also of
- * the given componentType. The list that is returned is in the order in
- * which they will need to be 'activated' (enabled/started).
- *
- * @param referencedNode node
- * @param componentType type
- * @return list of components
- */
- private <T> List<T> findRecursiveReferences(final ControllerServiceNode referencedNode, final Class<T> componentType) {
- final List<T> references = new ArrayList<>();
-
- for (final ConfiguredComponent referencingComponent : referencedNode.getReferences().getReferencingComponents()) {
- if (componentType.isAssignableFrom(referencingComponent.getClass())) {
- references.add(componentType.cast(referencingComponent));
- }
-
- if (referencingComponent instanceof ControllerServiceNode) {
- final ControllerServiceNode referencingNode = (ControllerServiceNode) referencingComponent;
-
- // find components recursively that depend on referencingNode.
- final List<T> recursive = findRecursiveReferences(referencingNode, componentType);
-
- // For anything that depends on referencing node, we want to add it to the list, but we know
- // that it must come after the referencing node, so we first remove any existing occurrence.
- references.removeAll(recursive);
- references.addAll(recursive);
- }
- }
-
- return references;
- }
@Override
public Set<ConfiguredComponent> enableReferencingServices(final ControllerServiceNode serviceNode) {
- final List<ControllerServiceNode> recursiveReferences = findRecursiveReferences(serviceNode, ControllerServiceNode.class);
+ final List<ControllerServiceNode> recursiveReferences = serviceNode.getReferences().findRecursiveReferences(ControllerServiceNode.class);
logger.debug("Enabling the following Referencing Services for {}: {}", serviceNode, recursiveReferences);
return enableReferencingServices(serviceNode, recursiveReferences);
}
@@ -658,7 +629,7 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
@Override
public void verifyCanEnableReferencingServices(final ControllerServiceNode serviceNode) {
- final List<ControllerServiceNode> referencingServices = findRecursiveReferences(serviceNode, ControllerServiceNode.class);
+ final List<ControllerServiceNode> referencingServices = serviceNode.getReferences().findRecursiveReferences(ControllerServiceNode.class);
final Set<ControllerServiceNode> referencingServiceSet = new HashSet<>(referencingServices);
for (final ControllerServiceNode referencingService : referencingServices) {
@@ -668,9 +639,9 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
@Override
public void verifyCanScheduleReferencingComponents(final ControllerServiceNode serviceNode) {
- final List<ControllerServiceNode> referencingServices = findRecursiveReferences(serviceNode, ControllerServiceNode.class);
- final List<ReportingTaskNode> referencingReportingTasks = findRecursiveReferences(serviceNode, ReportingTaskNode.class);
- final List<ProcessorNode> referencingProcessors = findRecursiveReferences(serviceNode, ProcessorNode.class);
+ final List<ControllerServiceNode> referencingServices = serviceNode.getReferences().findRecursiveReferences(ControllerServiceNode.class);
+ final List<ReportingTaskNode> referencingReportingTasks = serviceNode.getReferences().findRecursiveReferences(ReportingTaskNode.class);
+ final List<ProcessorNode> referencingProcessors = serviceNode.getReferences().findRecursiveReferences(ProcessorNode.class);
final Set<ControllerServiceNode> referencingServiceSet = new HashSet<>(referencingServices);
@@ -689,9 +660,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
@Override
public void verifyCanDisableReferencingServices(final ControllerServiceNode serviceNode) {
- // Get a list of all Controller Services that need to be disabled, in the order that they need to be
- // disabled.
- final List<ControllerServiceNode> toDisable = findRecursiveReferences(serviceNode, ControllerServiceNode.class);
+ // Get a list of all Controller Services that need to be disabled, in the order that they need to be disabled.
+ final List<ControllerServiceNode> toDisable = serviceNode.getReferences().findRecursiveReferences(ControllerServiceNode.class);
final Set<ControllerServiceNode> serviceSet = new HashSet<>(toDisable);
for (final ControllerServiceNode nodeToDisable : toDisable) {
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java
index d2f3833..285b8dc 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java
@@ -16,8 +16,10 @@
*/
package org.apache.nifi.controller.service;
+import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
+import java.util.List;
import java.util.Set;
import org.apache.nifi.controller.ConfiguredComponent;
@@ -101,4 +103,35 @@ public class StandardControllerServiceReference implements ControllerServiceRefe
return references;
}
+
+
+ @Override
+ public <T> List<T> findRecursiveReferences(final Class<T> componentType) {
+ return findRecursiveReferences(referenced, componentType);
+ }
+
+ private <T> List<T> findRecursiveReferences(final ControllerServiceNode referencedNode, final Class<T> componentType) {
+ final List<T> references = new ArrayList<>();
+
+ for (final ConfiguredComponent referencingComponent : referencedNode.getReferences().getReferencingComponents()) {
+ if (componentType.isAssignableFrom(referencingComponent.getClass())) {
+ references.add(componentType.cast(referencingComponent));
+ }
+
+ if (referencingComponent instanceof ControllerServiceNode) {
+ final ControllerServiceNode referencingNode = (ControllerServiceNode) referencingComponent;
+
+ // find components recursively that depend on referencingNode.
+ final List<T> recursive = findRecursiveReferences(referencingNode, componentType);
+
+ // For anything that depends on referencing node, we want to add it to the list, but we know
+ // that it must come after the referencing node, so we first remove any existing occurrence.
+ references.removeAll(recursive);
+ references.addAll(recursive);
+ }
+ }
+
+ return references;
+ }
+
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
index 1ef3e8b..3957955 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
@@ -16,6 +16,24 @@
*/
package org.apache.nifi.fingerprint;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.ConfigurableComponent;
@@ -38,23 +56,6 @@ import org.w3c.dom.Node;
import org.w3c.dom.NodeList;
import org.xml.sax.SAXException;
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.security.NoSuchAlgorithmException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
/**
* <p>Creates a fingerprint of a flow.xml. The order of elements or attributes in the flow.xml does not influence the fingerprint generation.
*
@@ -324,9 +325,22 @@ public class FingerprintFactory {
addFunnelFingerprint(builder, funnelElem);
}
+ // add variables
+ final NodeList variableElems = DomUtils.getChildNodesByTagName(processGroupElem, "variable");
+ final List<Element> sortedVarList = sortElements(variableElems, getVariableNameComparator());
+ for (final Element varElem : sortedVarList) {
+ addVariableFingerprint(builder, varElem);
+ }
+
return builder;
}
+ private void addVariableFingerprint(final StringBuilder builder, final Element variableElement) {
+ final String variableName = variableElement.getAttribute("name");
+ final String variableValue = variableElement.getAttribute("value");
+ builder.append(variableName).append("=").append(variableValue);
+ }
+
private StringBuilder addFlowFileProcessorFingerprint(final StringBuilder builder, final Element processorElem) throws FingerprintException {
// id
appendFirstValue(builder, DomUtils.getChildNodesByTagName(processorElem, "id"));
@@ -662,6 +676,27 @@ public class FingerprintFactory {
};
}
+ private Comparator<Element> getVariableNameComparator() {
+ return new Comparator<Element>() {
+ @Override
+ public int compare(final Element e1, final Element e2) {
+ if (e1 == null && e2 == null) {
+ return 0;
+ }
+ if (e1 == null) {
+ return 1;
+ }
+ if (e2 == null) {
+ return -1;
+ }
+
+ final String varName1 = e1.getAttribute("name");
+ final String varName2 = e2.getAttribute("name");
+ return varName1.compareTo(varName2);
+ }
+ };
+ }
+
private Comparator<Element> getProcessorPropertiesComparator() {
return new Comparator<Element>() {
@Override
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
index 2907704..2b7b51d 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
@@ -16,13 +16,31 @@
*/
package org.apache.nifi.groups;
-import com.google.common.collect.Sets;
+import static java.util.Objects.requireNonNull;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
+
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.commons.lang3.builder.ToStringBuilder;
import org.apache.commons.lang3.builder.ToStringStyle;
import org.apache.nifi.annotation.lifecycle.OnRemoved;
import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.attribute.expression.language.Query;
+import org.apache.nifi.attribute.expression.language.VariableImpact;
import org.apache.nifi.authorization.Resource;
import org.apache.nifi.authorization.resource.Authorizable;
import org.apache.nifi.authorization.resource.ResourceFactory;
@@ -39,6 +57,7 @@ import org.apache.nifi.connectable.Port;
import org.apache.nifi.connectable.Position;
import org.apache.nifi.connectable.Positionable;
import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.ConfiguredComponent;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.ProcessorNode;
@@ -50,13 +69,15 @@ import org.apache.nifi.controller.label.Label;
import org.apache.nifi.controller.scheduling.StandardProcessScheduler;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.controller.service.ControllerServiceReference;
import org.apache.nifi.controller.service.StandardConfigurationContext;
import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.logging.LogRepositoryFactory;
import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.NarCloseable;
import org.apache.nifi.processor.StandardProcessContext;
-import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.registry.variable.MutableVariableRegistry;
import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.remote.RootGroupPort;
import org.apache.nifi.util.NiFiProperties;
@@ -66,20 +87,7 @@ import org.apache.nifi.web.api.dto.TemplateDTO;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import static java.util.Objects.requireNonNull;
+import com.google.common.collect.Sets;
public final class StandardProcessGroup implements ProcessGroup {
@@ -104,7 +112,7 @@ public final class StandardProcessGroup implements ProcessGroup {
private final Map<String, ControllerServiceNode> controllerServices = new HashMap<>();
private final Map<String, Template> templates = new HashMap<>();
private final StringEncryptor encryptor;
- private final VariableRegistry variableRegistry;
+ private final MutableVariableRegistry variableRegistry;
private final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
private final Lock readLock = rwLock.readLock();
@@ -114,7 +122,7 @@ public final class StandardProcessGroup implements ProcessGroup {
public StandardProcessGroup(final String id, final ControllerServiceProvider serviceProvider, final StandardProcessScheduler scheduler,
final NiFiProperties nifiProps, final StringEncryptor encryptor, final FlowController flowController,
- final VariableRegistry variableRegistry) {
+ final MutableVariableRegistry variableRegistry) {
this.id = id;
this.controllerServiceProvider = serviceProvider;
this.parent = new AtomicReference<>();
@@ -361,7 +369,7 @@ public final class StandardProcessGroup implements ProcessGroup {
private void shutdown(final ProcessGroup procGroup) {
for (final ProcessorNode node : procGroup.getProcessors()) {
try (final NarCloseable x = NarCloseable.withComponentNarLoader(node.getProcessor().getClass(), node.getIdentifier())) {
- final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor, getStateManager(node.getIdentifier()), variableRegistry);
+ final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor, getStateManager(node.getIdentifier()));
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, node.getProcessor(), processContext);
}
}
@@ -548,6 +556,8 @@ public final class StandardProcessGroup implements ProcessGroup {
writeLock.lock();
try {
group.setParent(this);
+ group.getVariableRegistry().setParent(getVariableRegistry());
+
processGroups.put(Objects.requireNonNull(group).getIdentifier(), group);
flowController.onProcessGroupAdded(group);
} finally {
@@ -709,6 +719,7 @@ public final class StandardProcessGroup implements ProcessGroup {
}
processor.setProcessGroup(this);
+ processor.getVariableRegistry().setParent(getVariableRegistry());
processors.put(processorId, processor);
flowController.onProcessorAdded(processor);
} finally {
@@ -732,7 +743,7 @@ public final class StandardProcessGroup implements ProcessGroup {
}
try (final NarCloseable x = NarCloseable.withComponentNarLoader(processor.getProcessor().getClass(), processor.getIdentifier())) {
- final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor, getStateManager(processor.getIdentifier()), variableRegistry);
+ final StandardProcessContext processContext = new StandardProcessContext(processor, controllerServiceProvider, encryptor, getStateManager(processor.getIdentifier()));
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, processor.getProcessor(), processContext);
} catch (final Exception e) {
throw new ComponentLifeCycleException("Failed to invoke 'OnRemoved' methods of processor with id " + processor.getIdentifier(), e);
@@ -1081,7 +1092,7 @@ public final class StandardProcessGroup implements ProcessGroup {
}
@Override
- public void startProcessor(final ProcessorNode processor) {
+ public CompletableFuture<Void> startProcessor(final ProcessorNode processor) {
readLock.lock();
try {
if (getProcessor(processor.getIdentifier()) == null) {
@@ -1092,10 +1103,10 @@ public final class StandardProcessGroup implements ProcessGroup {
if (state == ScheduledState.DISABLED) {
throw new IllegalStateException("Processor is disabled");
} else if (state == ScheduledState.RUNNING) {
- return;
+ return CompletableFuture.completedFuture(null);
}
- scheduler.startProcessor(processor);
+ return scheduler.startProcessor(processor);
} finally {
readLock.unlock();
}
@@ -1162,7 +1173,7 @@ public final class StandardProcessGroup implements ProcessGroup {
}
@Override
- public void stopProcessor(final ProcessorNode processor) {
+ public CompletableFuture<Void> stopProcessor(final ProcessorNode processor) {
readLock.lock();
try {
if (!processors.containsKey(processor.getIdentifier())) {
@@ -1173,10 +1184,10 @@ public final class StandardProcessGroup implements ProcessGroup {
if (state == ScheduledState.DISABLED) {
throw new IllegalStateException("Processor is disabled");
} else if (state == ScheduledState.STOPPED) {
- return;
+ return CompletableFuture.completedFuture(null);
}
- scheduler.stopProcessor(processor);
+ return scheduler.stopProcessor(processor);
} finally {
readLock.unlock();
}
@@ -1854,6 +1865,7 @@ public final class StandardProcessGroup implements ProcessGroup {
}
service.setProcessGroup(this);
+ service.getVariableRegistry().setParent(getVariableRegistry());
this.controllerServices.put(service.getIdentifier(), service);
LOG.info("{} added to {}", service, this);
} finally {
@@ -2583,4 +2595,129 @@ public final class StandardProcessGroup implements ProcessGroup {
readLock.unlock();
}
}
+
+ @Override
+ public MutableVariableRegistry getVariableRegistry() {
+ return variableRegistry;
+ }
+
+ @Override
+ public void verifyCanUpdateVariables(final Map<String, String> updatedVariables) {
+ if (updatedVariables == null || updatedVariables.isEmpty()) {
+ return;
+ }
+
+ readLock.lock();
+ try {
+ final Set<String> updatedVariableNames = getUpdatedVariables(updatedVariables);
+ if (updatedVariableNames.isEmpty()) {
+ return;
+ }
+
+ for (final ProcessorNode processor : findAllProcessors()) {
+ if (!processor.isRunning()) {
+ continue;
+ }
+
+ for (final String variableName : updatedVariableNames) {
+ for (final VariableImpact impact : getVariableImpact(processor)) {
+ if (impact.isImpacted(variableName)) {
+ throw new IllegalStateException("Cannot update variable '" + variableName + "' because it is referenced by " + processor + ", which is currently running");
+ }
+ }
+ }
+ }
+
+ for (final ControllerServiceNode service : findAllControllerServices()) {
+ if (!service.isActive()) {
+ continue;
+ }
+
+ for (final String variableName : updatedVariableNames) {
+ for (final VariableImpact impact : getVariableImpact(service)) {
+ if (impact.isImpacted(variableName)) {
+ throw new IllegalStateException("Cannot update variable '" + variableName + "' because it is referenced by " + service + ", which is currently running");
+ }
+ }
+ }
+ }
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ @Override
+ public Set<ConfiguredComponent> getComponentsAffectedByVariable(final String variableName) {
+ final Set<ConfiguredComponent> affected = new HashSet<>();
+
+ // Determine any Processors that references the variable
+ for (final ProcessorNode processor : findAllProcessors()) {
+ for (final VariableImpact impact : getVariableImpact(processor)) {
+ if (impact.isImpacted(variableName)) {
+ affected.add(processor);
+ }
+ }
+ }
+
+ // Determine any Controller Service that references the variable. If Service A references a variable,
+ // then that means that any other component that references that service is also affected, so recursively
+ // find any references to that service and add it.
+ for (final ControllerServiceNode service : findAllControllerServices()) {
+ for (final VariableImpact impact : getVariableImpact(service)) {
+ if (impact.isImpacted(variableName)) {
+ affected.add(service);
+
+ final ControllerServiceReference reference = service.getReferences();
+ affected.addAll(reference.findRecursiveReferences(ConfiguredComponent.class));
+ }
+ }
+ }
+
+ return affected;
+ }
+
+
+ private Set<String> getUpdatedVariables(final Map<String, String> newVariableValues) {
+ final Set<String> updatedVariableNames = new HashSet<>();
+
+ final MutableVariableRegistry registry = getVariableRegistry();
+ for (final Map.Entry<String, String> entry : newVariableValues.entrySet()) {
+ final String varName = entry.getKey();
+ final String newValue = entry.getValue();
+
+ final String curValue = registry.getVariableValue(varName);
+ if (!Objects.equals(newValue, curValue)) {
+ updatedVariableNames.add(varName);
+ }
+ }
+
+ return updatedVariableNames;
+ }
+
+ private List<VariableImpact> getVariableImpact(final ConfiguredComponent component) {
+ return component.getProperties().values().stream()
+ .map(propVal -> Query.prepare(propVal).getVariableImpact())
+ .collect(Collectors.toList());
+ }
+
+ @Override
+ public void setVariables(final Map<String, String> variables) {
+ writeLock.lock();
+ try {
+ verifyCanUpdateVariables(variables);
+
+ if (variables == null) {
+ return;
+ }
+
+ final Map<VariableDescriptor, String> variableMap = new HashMap<>();
+ variables.entrySet().stream() // cannot use Collectors.toMap because value may be null
+ .forEach(entry -> variableMap.put(new VariableDescriptor(entry.getKey()), entry.getValue()));
+
+ variableRegistry.setVariables(variableMap);
+ } finally {
+ writeLock.unlock();
+ }
+ }
+
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
index 2714392..dfba330 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
@@ -26,8 +26,8 @@ import java.util.Set;
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.attribute.expression.language.Query.Range;
+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;
@@ -37,7 +37,6 @@ import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.service.ControllerServiceProvider;
import org.apache.nifi.encrypt.StringEncryptor;
-import org.apache.nifi.registry.VariableRegistry;
import org.apache.nifi.util.Connectables;
public class StandardProcessContext implements ProcessContext, ControllerServiceLookup {
@@ -47,15 +46,12 @@ public class StandardProcessContext implements ProcessContext, ControllerService
private final Map<PropertyDescriptor, PreparedQuery> preparedQueries;
private final StringEncryptor encryptor;
private final StateManager stateManager;
- private final VariableRegistry variableRegistry;
- public StandardProcessContext(final ProcessorNode processorNode, final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor, final StateManager stateManager,
- final VariableRegistry variableRegistry) {
+ public StandardProcessContext(final ProcessorNode processorNode, final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor, final StateManager stateManager) {
this.procNode = processorNode;
this.controllerServiceProvider = controllerServiceProvider;
this.encryptor = encryptor;
this.stateManager = stateManager;
- this.variableRegistry = variableRegistry;
preparedQueries = new HashMap<>();
for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getProperties().entrySet()) {
@@ -93,12 +89,12 @@ public class StandardProcessContext implements ProcessContext, ControllerService
final String setPropertyValue = procNode.getProperty(descriptor);
final String propValue = (setPropertyValue == null) ? descriptor.getDefaultValue() : setPropertyValue;
- return new StandardPropertyValue(propValue, this, preparedQueries.get(descriptor), variableRegistry);
+ return new StandardPropertyValue(propValue, this, preparedQueries.get(descriptor), procNode.getVariableRegistry());
}
@Override
public PropertyValue newPropertyValue(final String rawValue) {
- return new StandardPropertyValue(rawValue, this, Query.prepare(rawValue), variableRegistry);
+ return new StandardPropertyValue(rawValue, this, Query.prepare(rawValue), procNode.getVariableRegistry());
}
@Override
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
index 662169c..2f38aee 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
@@ -53,8 +53,8 @@ public class StandardValidationContext implements ValidationContext {
private final String componentId;
public StandardValidationContext(final ControllerServiceProvider controllerServiceProvider, final Map<PropertyDescriptor, String> properties,
- final String annotationData, final String groupId, final String componentId, VariableRegistry variableRegistry) {
- this(controllerServiceProvider, Collections.<String> emptySet(), properties, annotationData, groupId, componentId,variableRegistry);
+ final String annotationData, final String groupId, final String componentId, final VariableRegistry variableRegistry) {
+ this(controllerServiceProvider, Collections.<String> emptySet(), properties, annotationData, groupId, componentId, variableRegistry);
}
public StandardValidationContext(
@@ -63,7 +63,8 @@ public class StandardValidationContext implements ValidationContext {
final Map<PropertyDescriptor, String> properties,
final String annotationData,
final String groupId,
- final String componentId, VariableRegistry variableRegistry) {
+ final String componentId,
+ final VariableRegistry variableRegistry) {
this.controllerServiceProvider = controllerServiceProvider;
this.properties = new HashMap<>(properties);
this.annotationData = annotationData;
[4/9] nifi git commit: NIFI-4224: - Initial implementation of Process
Group level Variable Registry - Updated to incorporate PR Feedback - Changed
log message because slf4j-simple apparently has a memory leak;
passing a String instead of passing in the C
Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/FileBasedVariableRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/FileBasedVariableRegistry.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/FileBasedVariableRegistry.java
new file mode 100644
index 0000000..fb7a19d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/FileBasedVariableRegistry.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.variable;
+
+import java.io.BufferedInputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.registry.VariableRegistry;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A file based variable registry that loads all properties from files specified
+ * during construction and is backed by system properties and environment
+ * variables accessible to the JVM.
+ */
+public class FileBasedVariableRegistry implements VariableRegistry {
+
+ private final static Logger LOG = LoggerFactory.getLogger(FileBasedVariableRegistry.class);
+ final Map<VariableDescriptor, String> map;
+
+ public FileBasedVariableRegistry(final Path[] propertiesPaths) {
+ final Map<VariableDescriptor, String> newMap = new HashMap<>(VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY.getVariableMap());
+ final int systemEnvPropCount = newMap.size();
+ int totalPropertiesLoaded = systemEnvPropCount;
+ LOG.info("Loaded {} properties from system properties and environment variables",systemEnvPropCount);
+ try {
+ for (final Path path : propertiesPaths) {
+ if (Files.exists(path)) {
+ final AtomicInteger propsLoaded = new AtomicInteger(0);
+ try (final InputStream inStream = new BufferedInputStream(new FileInputStream(path.toFile()))) {
+ Properties properties = new Properties();
+ properties.load(inStream);
+ properties.entrySet().stream().forEach((entry) -> {
+ final VariableDescriptor desc = new VariableDescriptor.Builder(entry.getKey().toString())
+ .description(path.toString())
+ .sensitive(false)
+ .build();
+ newMap.put(desc, entry.getValue().toString());
+ propsLoaded.incrementAndGet();
+ });
+ }
+ totalPropertiesLoaded += propsLoaded.get();
+ if(propsLoaded.get() > 0){
+ LOG.info("Loaded {} properties from '{}'", propsLoaded.get(), path);
+ }else{
+ LOG.warn("No properties loaded from '{}'", path);
+ }
+ } else {
+ LOG.warn("Skipping property file {} as it does not appear to exist", path);
+ }
+ }
+ } catch (final IOException ioe) {
+ LOG.error("Unable to complete variable registry loading from files due to ", ioe);
+ }
+
+ LOG.info("Loaded a total of {} properties. Including precedence overrides effective accessible registry key size is {}", totalPropertiesLoaded, newMap.size());
+ map = Collections.unmodifiableMap(newMap);
+ }
+
+ @Override
+ public Map<VariableDescriptor, String> getVariableMap() {
+ return map;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/MutableVariableRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/MutableVariableRegistry.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/MutableVariableRegistry.java
new file mode 100644
index 0000000..e37b402
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/MutableVariableRegistry.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.registry.variable;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.registry.VariableRegistry;
+
+public class MutableVariableRegistry extends StandardComponentVariableRegistry implements VariableRegistry {
+ private volatile Map<VariableDescriptor, String> variableMap = new HashMap<>();
+
+ public MutableVariableRegistry(final VariableRegistry parent) {
+ super(parent);
+ }
+
+ @Override
+ public Map<VariableDescriptor, String> getVariableMap() {
+ return variableMap;
+ }
+
+ public void setVariables(final Map<VariableDescriptor, String> variables) {
+ final Map<VariableDescriptor, String> curVariableMap = this.variableMap;
+ final Map<VariableDescriptor, String> updatedVariableMap = new HashMap<>(curVariableMap);
+ for (final Map.Entry<VariableDescriptor, String> entry : variables.entrySet()) {
+ if (entry.getValue() == null) {
+ updatedVariableMap.remove(entry.getKey());
+ } else {
+ updatedVariableMap.put(entry.getKey(), entry.getValue());
+ }
+ }
+
+ this.variableMap = Collections.unmodifiableMap(updatedVariableMap);
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/StandardComponentVariableRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/StandardComponentVariableRegistry.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/StandardComponentVariableRegistry.java
new file mode 100644
index 0000000..b3c31f3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/registry/variable/StandardComponentVariableRegistry.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.registry.variable;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.registry.VariableRegistry;
+
+public class StandardComponentVariableRegistry implements ComponentVariableRegistry {
+ private volatile VariableRegistry parent;
+
+ public StandardComponentVariableRegistry(final VariableRegistry parent) {
+ this.parent = Objects.requireNonNull(parent);
+ }
+
+ @Override
+ public Map<VariableDescriptor, String> getVariableMap() {
+ return Collections.emptyMap();
+ }
+
+ @Override
+ public VariableRegistry getParent() {
+ return parent;
+ }
+
+ @Override
+ public void setParent(final VariableRegistry parentRegistry) {
+ this.parent = parentRegistry;
+ }
+
+ @Override
+ public VariableDescriptor getVariableKey(final String name) {
+ if (name == null) {
+ return null;
+ }
+
+ final VariableDescriptor spec = new VariableDescriptor(name);
+ for (final Map.Entry<VariableDescriptor, String> entry : getVariableMap().entrySet()) {
+ if (entry.getKey().equals(spec)) {
+ return entry.getKey();
+ }
+ }
+
+ return null;
+ }
+
+ @Override
+ public String getVariableValue(final String name) {
+ if (name == null) {
+ return null;
+ }
+
+ final VariableDescriptor descriptor = new VariableDescriptor(name);
+ final String value = getVariableMap().get(descriptor);
+ if (value != null) {
+ return value;
+ }
+
+ return parent.getVariableValue(descriptor);
+ }
+
+ @Override
+ public String getVariableValue(final VariableDescriptor descriptor) {
+ if (descriptor == null) {
+ return null;
+ }
+
+ final String value = getVariableMap().get(descriptor);
+ if (value != null) {
+ return value;
+ }
+
+ return parent.getVariableValue(descriptor);
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/FileBasedVariableRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/FileBasedVariableRegistry.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/FileBasedVariableRegistry.java
deleted file mode 100644
index c4079a7..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/FileBasedVariableRegistry.java
+++ /dev/null
@@ -1,88 +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.util;
-
-import java.io.BufferedInputStream;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A file based variable registry that loads all properties from files specified
- * during construction and is backed by system properties and environment
- * variables accessible to the JVM.
- */
-public class FileBasedVariableRegistry implements VariableRegistry {
-
- private final static Logger LOG = LoggerFactory.getLogger(FileBasedVariableRegistry.class);
- final Map<VariableDescriptor, String> map;
-
- public FileBasedVariableRegistry(final Path[] propertiesPaths) {
- final Map<VariableDescriptor, String> newMap = new HashMap<>(VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY.getVariableMap());
- final int systemEnvPropCount = newMap.size();
- int totalPropertiesLoaded = systemEnvPropCount;
- LOG.info("Loaded {} properties from system properties and environment variables",systemEnvPropCount);
- try {
- for (final Path path : propertiesPaths) {
- if (Files.exists(path)) {
- final AtomicInteger propsLoaded = new AtomicInteger(0);
- try (final InputStream inStream = new BufferedInputStream(new FileInputStream(path.toFile()))) {
- Properties properties = new Properties();
- properties.load(inStream);
- properties.entrySet().stream().forEach((entry) -> {
- final VariableDescriptor desc = new VariableDescriptor.Builder(entry.getKey().toString())
- .description(path.toString())
- .sensitive(false)
- .build();
- newMap.put(desc, entry.getValue().toString());
- propsLoaded.incrementAndGet();
- });
- }
- totalPropertiesLoaded += propsLoaded.get();
- if(propsLoaded.get() > 0){
- LOG.info("Loaded {} properties from '{}'", propsLoaded.get(), path);
- }else{
- LOG.warn("No properties loaded from '{}'", path);
- }
- } else {
- LOG.warn("Skipping property file {} as it does not appear to exist", path);
- }
- }
- } catch (final IOException ioe) {
- LOG.error("Unable to complete variable registry loading from files due to ", ioe);
- }
- LOG.info("Loaded a total of {} properties. Including precedence overrides effective accessible registry key size is {}", totalPropertiesLoaded, newMap.size());
- map = newMap;
- }
-
- @Override
- public Map<VariableDescriptor, String> getVariableMap() {
- return Collections.unmodifiableMap(map);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
index 30fff1f..247a790 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
@@ -147,9 +147,15 @@
<xs:element name="connection" type="ConnectionType" minOccurs="0" maxOccurs="unbounded" />
<xs:element name="controllerService" type="ControllerServiceType" minOccurs="0" maxOccurs="unbounded" />
<xs:element name="template" type="TemplateType" minOccurs="0" maxOccurs="unbounded" />
+ <xs:element name="variable" type="VariableType" minOccurs="0" maxOccurs="unbounded" />
</xs:sequence>
</xs:complexType>
+ <xs:complexType name="VariableType">
+ <xs:attribute name="name" />
+ <xs:attribute name="value" />
+ </xs:complexType>
+
<!-- Same as ProcessGroupType except that instead of input ports & output ports being of type PortType,
they are of type RootGroupPortType -->
<xs:complexType name="RootProcessGroupType">
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
index bf3fd2a..b3c426c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/nifi-context.xml
@@ -23,7 +23,7 @@
</bean>
<!-- variable registry -->
- <bean id="variableRegistry" class="org.apache.nifi.util.FileBasedVariableRegistry">
+ <bean id="variableRegistry" class="org.apache.nifi.registry.variable.FileBasedVariableRegistry">
<constructor-arg type="java.nio.file.Path[]" value="#{nifiProperties.getVariableRegistryPropertiesPaths()}" />
</bean>
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
index 1957cf4..6973d12 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/StandardFlowServiceTest.java
@@ -28,7 +28,7 @@ import org.apache.nifi.controller.serialization.StandardFlowSerializer;
import org.apache.nifi.encrypt.StringEncryptor;
import org.apache.nifi.events.VolatileBulletinRepository;
import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.util.FileBasedVariableRegistry;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.api.dto.ConnectableDTO;
import org.apache.nifi.web.api.dto.ConnectionDTO;
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
index 691af10..596c00f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestFlowController.java
@@ -49,9 +49,9 @@ import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.provenance.MockProvenanceRepository;
import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
import org.apache.nifi.reporting.BulletinRepository;
import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.util.FileBasedVariableRegistry;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.api.dto.BundleDTO;
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
index 32b6f53..33c33c9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/TestStandardProcessorNode.java
@@ -17,6 +17,26 @@
package org.apache.nifi.controller;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
@@ -46,6 +66,7 @@ import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.registry.VariableDescriptor;
import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.StandardComponentVariableRegistry;
import org.apache.nifi.test.processors.ModifiesClasspathNoAnnotationProcessor;
import org.apache.nifi.test.processors.ModifiesClasspathProcessor;
import org.apache.nifi.util.MockPropertyValue;
@@ -56,26 +77,6 @@ import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
-import java.io.File;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledExecutorService;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
public class TestStandardProcessorNode {
private MockVariableRegistry variableRegistry;
@@ -98,10 +99,10 @@ public class TestStandardProcessorNode {
final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(processor, coordinate, null);
final StandardProcessorNode procNode = new StandardProcessorNode(loggableComponent, uuid, createValidationContextFactory(), null, null,
- NiFiProperties.createBasicNiFiProperties(null, null), VariableRegistry.EMPTY_REGISTRY, reloadComponent);
+ NiFiProperties.createBasicNiFiProperties(null, null), new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent);
final ScheduledExecutorService taskScheduler = new FlowEngine(2, "TestClasspathResources", true);
- final StandardProcessContext processContext = new StandardProcessContext(procNode, null, null, null, null);
+ final StandardProcessContext processContext = new StandardProcessContext(procNode, null, null, null);
final SchedulingAgentCallback schedulingAgentCallback = new SchedulingAgentCallback() {
@Override
public void postMonitor() {
@@ -414,7 +415,8 @@ public class TestStandardProcessorNode {
processor.initialize(initContext);
final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(processor, systemBundle.getBundleDetails().getCoordinate(), componentLog);
- return new StandardProcessorNode(loggableComponent, uuid, validationContextFactory, processScheduler, null, niFiProperties, variableRegistry, reloadComponent);
+ return new StandardProcessorNode(loggableComponent, uuid, validationContextFactory, processScheduler,
+ null, niFiProperties, new StandardComponentVariableRegistry(variableRegistry), reloadComponent);
}
private static class MockReloadComponent implements ReloadComponent {
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
index 1903179..1dc74ab 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/reporting/TestStandardReportingContext.java
@@ -16,15 +16,6 @@
*/
package org.apache.nifi.controller.reporting;
-import static org.junit.Assert.assertEquals;
-
-import java.io.File;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Set;
-
import org.apache.commons.io.FileUtils;
import org.apache.nifi.admin.service.AuditService;
import org.apache.nifi.authorization.AbstractPolicyBasedAuthorizer;
@@ -45,14 +36,23 @@ import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.provenance.MockProvenanceRepository;
import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
import org.apache.nifi.reporting.BulletinRepository;
-import org.apache.nifi.util.FileBasedVariableRegistry;
import org.apache.nifi.util.NiFiProperties;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
+import java.io.File;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+
public class TestStandardReportingContext {
private FlowController controller;
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
index 7f36a65..c544ef4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestProcessorLifecycle.java
@@ -46,7 +46,7 @@ import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.provenance.MockProvenanceRepository;
-import org.apache.nifi.util.FileBasedVariableRegistry;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
import org.apache.nifi.util.NiFiProperties;
import org.junit.After;
import org.junit.Before;
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
index dfd627f..2c59964 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/scheduling/TestStandardProcessScheduler.java
@@ -74,6 +74,7 @@ import org.apache.nifi.processor.StandardProcessorInitializationContext;
import org.apache.nifi.processor.StandardValidationContextFactory;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.StandardComponentVariableRegistry;
import org.apache.nifi.reporting.AbstractReportingTask;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.reporting.ReportingContext;
@@ -110,7 +111,7 @@ public class TestStandardProcessScheduler {
systemBundle = SystemBundle.create(nifiProperties);
ExtensionManager.discoverExtensions(systemBundle, Collections.emptySet());
- scheduler = new StandardProcessScheduler(Mockito.mock(ControllerServiceProvider.class), null, stateMgrProvider, variableRegistry, nifiProperties);
+ scheduler = new StandardProcessScheduler(Mockito.mock(ControllerServiceProvider.class), null, stateMgrProvider, nifiProperties);
scheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, Mockito.mock(SchedulingAgent.class));
reportingTask = new TestReportingTask();
@@ -122,7 +123,8 @@ public class TestStandardProcessScheduler {
final ComponentLog logger = Mockito.mock(ComponentLog.class);
final ReloadComponent reloadComponent = Mockito.mock(ReloadComponent.class);
final LoggableComponent<ReportingTask> loggableComponent = new LoggableComponent<>(reportingTask, systemBundle.getBundleDetails().getCoordinate(), logger);
- taskNode = new StandardReportingTaskNode(loggableComponent, UUID.randomUUID().toString(), null, scheduler, validationContextFactory, variableRegistry, reloadComponent);
+ taskNode = new StandardReportingTaskNode(loggableComponent, UUID.randomUUID().toString(), null, scheduler, validationContextFactory,
+ new StandardComponentVariableRegistry(variableRegistry), reloadComponent);
controller = Mockito.mock(FlowController.class);
@@ -196,7 +198,7 @@ public class TestStandardProcessScheduler {
final LoggableComponent<Processor> loggableComponent = new LoggableComponent<>(proc, systemBundle.getBundleDetails().getCoordinate(), null);
final ProcessorNode procNode = new StandardProcessorNode(loggableComponent, uuid,
new StandardValidationContextFactory(serviceProvider, variableRegistry),
- scheduler, serviceProvider, nifiProperties, VariableRegistry.EMPTY_REGISTRY, reloadComponent);
+ scheduler, serviceProvider, nifiProperties, new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent);
rootGroup.addProcessor(procNode);
Map<String, String> procProps = new HashMap<>();
@@ -580,6 +582,6 @@ public class TestStandardProcessScheduler {
}
private ProcessScheduler createScheduler() {
- return new StandardProcessScheduler(null, null, stateMgrProvider, variableRegistry, nifiProperties);
+ return new StandardProcessScheduler(null, null, stateMgrProvider, nifiProperties);
}
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java
index c235a57..7a49103 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/serialization/StandardFlowSerializerTest.java
@@ -29,8 +29,8 @@ import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.provenance.MockProvenanceRepository;
import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
import org.apache.nifi.reporting.BulletinRepository;
-import org.apache.nifi.util.FileBasedVariableRegistry;
import org.apache.nifi.util.NiFiProperties;
import org.junit.Before;
import org.junit.Test;
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
index 4a97b8a..55263f8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
@@ -24,8 +24,8 @@ import org.apache.nifi.nar.ExtensionManager;
import org.apache.nifi.nar.NarClassLoaders;
import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
import org.apache.nifi.reporting.InitializationException;
-import org.apache.nifi.util.FileBasedVariableRegistry;
import org.apache.nifi.util.NiFiProperties;
import org.junit.Before;
import org.junit.BeforeClass;
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
index e82085e..0d15143 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
@@ -17,6 +17,22 @@
*/
package org.apache.nifi.controller.service;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.beans.PropertyDescriptor;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+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 org.apache.nifi.bundle.Bundle;
import org.apache.nifi.components.state.StateManager;
import org.apache.nifi.components.state.StateManagerProvider;
@@ -40,6 +56,8 @@ import org.apache.nifi.nar.SystemBundle;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.StandardValidationContextFactory;
import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.MutableVariableRegistry;
+import org.apache.nifi.registry.variable.StandardComponentVariableRegistry;
import org.apache.nifi.util.NiFiProperties;
import org.junit.Assert;
import org.junit.Before;
@@ -49,22 +67,6 @@ import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
-import java.beans.PropertyDescriptor;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
public class TestStandardControllerServiceProvider {
private static StateManagerProvider stateManagerProvider = new StateManagerProvider() {
@@ -129,7 +131,7 @@ public class TestStandardControllerServiceProvider {
}
private StandardProcessScheduler createScheduler() {
- return new StandardProcessScheduler(null, null, stateManagerProvider, variableRegistry, niFiProperties);
+ return new StandardProcessScheduler(null, null, stateManagerProvider, niFiProperties);
}
private void setProperty(ControllerServiceNode serviceNode, String propName, String propValue) {
@@ -432,9 +434,10 @@ public class TestStandardControllerServiceProvider {
final LoggableComponent<Processor> dummyProcessor = new LoggableComponent<>(new DummyProcessor(), systemBundle.getBundleDetails().getCoordinate(), null);
final ProcessorNode procNode = new StandardProcessorNode(dummyProcessor, UUID.randomUUID().toString(),
new StandardValidationContextFactory(serviceProvider, null), scheduler, serviceProvider, niFiProperties,
- VariableRegistry.EMPTY_REGISTRY, reloadComponent);
+ new StandardComponentVariableRegistry(VariableRegistry.EMPTY_REGISTRY), reloadComponent);
- final ProcessGroup group = new StandardProcessGroup(UUID.randomUUID().toString(), serviceProvider, scheduler, null, null, Mockito.mock(FlowController.class), variableRegistry);
+ final ProcessGroup group = new StandardProcessGroup(UUID.randomUUID().toString(), serviceProvider, scheduler, null, null, Mockito.mock(FlowController.class),
+ new MutableVariableRegistry(variableRegistry));
group.addProcessor(procNode);
procNode.setProcessGroup(group);
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
index b6f70a1..9725ed8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/MockProcessGroup.java
@@ -18,11 +18,13 @@
package org.apache.nifi.controller.service.mock;
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.concurrent.CompletableFuture;
import org.apache.nifi.authorization.Resource;
import org.apache.nifi.authorization.resource.Authorizable;
@@ -32,6 +34,7 @@ import org.apache.nifi.connectable.Funnel;
import org.apache.nifi.connectable.Port;
import org.apache.nifi.connectable.Position;
import org.apache.nifi.connectable.Positionable;
+import org.apache.nifi.controller.ConfiguredComponent;
import org.apache.nifi.controller.FlowController;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.Snippet;
@@ -41,12 +44,15 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.groups.ProcessGroupCounts;
import org.apache.nifi.groups.RemoteProcessGroup;
+import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.MutableVariableRegistry;
import org.apache.nifi.remote.RemoteGroupPort;
public class MockProcessGroup implements ProcessGroup {
private final Map<String, ControllerServiceNode> serviceMap = new HashMap<>();
private final Map<String, ProcessorNode> processorMap = new HashMap<>();
private final FlowController flowController;
+ private final MutableVariableRegistry variableRegistry = new MutableVariableRegistry(VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY);
public MockProcessGroup(final FlowController flowController) {
this.flowController = flowController;
@@ -143,8 +149,8 @@ public class MockProcessGroup implements ProcessGroup {
}
@Override
- public void startProcessor(final ProcessorNode processor) {
-
+ public CompletableFuture<Void> startProcessor(final ProcessorNode processor) {
+ return CompletableFuture.completedFuture(null);
}
@Override
@@ -163,8 +169,8 @@ public class MockProcessGroup implements ProcessGroup {
}
@Override
- public void stopProcessor(final ProcessorNode processor) {
-
+ public CompletableFuture<Void> stopProcessor(final ProcessorNode processor) {
+ return CompletableFuture.completedFuture(null);
}
@Override
@@ -601,4 +607,22 @@ public class MockProcessGroup implements ProcessGroup {
@Override
public void verifyCanStop(final Connectable connectable) {
}
+
+ @Override
+ public MutableVariableRegistry getVariableRegistry() {
+ return variableRegistry;
+ }
+
+ @Override
+ public void verifyCanUpdateVariables(Map<String, String> updatedVariables) {
+ }
+
+ @Override
+ public void setVariables(Map<String, String> variables) {
+ }
+
+ @Override
+ public Set<ConfiguredComponent> getComponentsAffectedByVariable(String variableName) {
+ return Collections.emptySet();
+ }
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/util/TestFileBasedVariableRegistry.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/util/TestFileBasedVariableRegistry.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/util/TestFileBasedVariableRegistry.java
index cf44435..daed0e0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/util/TestFileBasedVariableRegistry.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/util/TestFileBasedVariableRegistry.java
@@ -21,6 +21,7 @@ import java.nio.file.Paths;
import java.util.Map;
import org.apache.nifi.registry.VariableDescriptor;
import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.FileBasedVariableRegistry;
import org.junit.Test;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertEquals;
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java
index 391f9e2..22ad122 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ProcessGroupAuditor.java
@@ -26,6 +26,7 @@ import org.apache.nifi.action.details.FlowChangeMoveDetails;
import org.apache.nifi.authorization.user.NiFiUser;
import org.apache.nifi.authorization.user.NiFiUserUtils;
import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.web.api.dto.ProcessGroupDTO;
import org.apache.nifi.web.dao.ProcessGroupDAO;
@@ -175,6 +176,60 @@ public class ProcessGroupAuditor extends NiFiAuditor {
+ "execution(void scheduleComponents(java.lang.String, org.apache.nifi.controller.ScheduledState, java.util.Set)) && "
+ "args(groupId, state)")
public void scheduleComponentsAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ScheduledState state) throws Throwable {
+ final Operation operation;
+ // determine the running state
+ if (ScheduledState.RUNNING.equals(state)) {
+ operation = Operation.Start;
+ } else {
+ operation = Operation.Stop;
+ }
+
+ saveUpdateAction(proceedingJoinPoint, groupId, operation);
+ }
+
+
+ /**
+ * Audits the update of controller serivce state
+ *
+ * @param proceedingJoinPoint join point
+ * @param groupId group id
+ * @param state controller serivce state state
+ * @throws Throwable ex
+ */
+ @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && "
+ + "execution(java.util.concurrent.Future activateControllerServices(java.lang.String, org.apache.nifi.controller.service.ControllerServiceState, java.util.Set)) && "
+ + "args(groupId, state)")
+ public void activateControllerServicesAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId, ControllerServiceState state) throws Throwable {
+
+ // determine the service state
+ final Operation operation;
+ if (ControllerServiceState.ENABLED.equals(state)) {
+ operation = Operation.Enable;
+ } else {
+ operation = Operation.Disable;
+ }
+
+ saveUpdateAction(proceedingJoinPoint, groupId, operation);
+ }
+
+ /**
+ * Audits the update of process group variable registry.
+ *
+ * @param proceedingJoinPoint join point
+ * @param groupId group id
+ * @throws Throwable ex
+ */
+ @Around("within(org.apache.nifi.web.dao.ProcessGroupDAO+) && "
+ + "execution(org.apache.nifi.groups.ProcessGroup updateVariableRegistry(org.apache.nifi.web.api.dto.VariableRegistryDTO)) && "
+ + "args(groupId)")
+ public void updateVariableRegistryAdvice(ProceedingJoinPoint proceedingJoinPoint, String groupId) throws Throwable {
+ final Operation operation = Operation.Configure;
+ saveUpdateAction(proceedingJoinPoint, groupId, operation);
+ }
+
+
+
+ private void saveUpdateAction(final ProceedingJoinPoint proceedingJoinPoint, final String groupId, final Operation operation) throws Throwable {
ProcessGroupDAO processGroupDAO = getProcessGroupDAO();
ProcessGroup processGroup = processGroupDAO.getProcessGroup(groupId);
@@ -191,13 +246,7 @@ public class ProcessGroupAuditor extends NiFiAuditor {
action.setSourceName(processGroup.getName());
action.setSourceType(Component.ProcessGroup);
action.setTimestamp(new Date());
-
- // determine the running state
- if (ScheduledState.RUNNING.equals(state)) {
- action.setOperation(Operation.Start);
- } else {
- action.setOperation(Operation.Stop);
- }
+ action.setOperation(operation);
// add this action
saveAction(action, logger);
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateRequest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateRequest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateRequest.java
new file mode 100644
index 0000000..82d4683
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateRequest.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.registry.variable;
+
+import java.util.Date;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class VariableRegistryUpdateRequest {
+ private final String requestId;
+ private final String processGroupId;
+ private volatile Date submissionTime = new Date();
+ private volatile Date lastUpdated = new Date();
+ private volatile boolean complete = false;
+
+ private final AtomicReference<String> failureReason = new AtomicReference<>();
+
+ private final VariableRegistryUpdateStep identifyComponentsStep = new VariableRegistryUpdateStep("Identifying components affected");
+ private final VariableRegistryUpdateStep stopProcessors = new VariableRegistryUpdateStep("Stopping affected Processors");
+ private final VariableRegistryUpdateStep disableServices = new VariableRegistryUpdateStep("Disabling affected Controller Services");
+ private final VariableRegistryUpdateStep applyUpdates = new VariableRegistryUpdateStep("Applying Updates");
+ private final VariableRegistryUpdateStep enableServices = new VariableRegistryUpdateStep("Re-Enabling affected Controller Services");
+ private final VariableRegistryUpdateStep startProcessors = new VariableRegistryUpdateStep("Restarting affected Processors");
+
+ public VariableRegistryUpdateRequest(final String requestId, final String processGroupId) {
+ this.requestId = requestId;
+ this.processGroupId = processGroupId;
+ }
+
+ public String getProcessGroupId() {
+ return processGroupId;
+ }
+
+
+ public String getRequestId() {
+ return requestId;
+ }
+
+ public Date getSubmissionTime() {
+ return submissionTime;
+ }
+
+ public Date getLastUpdated() {
+ return lastUpdated;
+ }
+
+ public void setLastUpdated(Date lastUpdated) {
+ this.lastUpdated = lastUpdated;
+ }
+
+ public VariableRegistryUpdateStep getIdentifyRelevantComponentsStep() {
+ return identifyComponentsStep;
+ }
+
+ public VariableRegistryUpdateStep getStopProcessorsStep() {
+ return stopProcessors;
+ }
+
+ public VariableRegistryUpdateStep getDisableServicesStep() {
+ return disableServices;
+ }
+
+ public VariableRegistryUpdateStep getApplyUpdatesStep() {
+ return applyUpdates;
+ }
+
+ public VariableRegistryUpdateStep getEnableServicesStep() {
+ return enableServices;
+ }
+
+ public VariableRegistryUpdateStep getStartProcessorsStep() {
+ return startProcessors;
+ }
+
+ public boolean isComplete() {
+ return complete;
+ }
+
+ public void setComplete(boolean complete) {
+ this.complete = complete;
+ }
+
+ public String getFailureReason() {
+ return failureReason.get();
+ }
+
+ public void setFailureReason(String reason) {
+ this.failureReason.set(reason);
+ }
+
+ public void cancel() {
+ this.failureReason.compareAndSet(null, "Update was cancelled");
+ this.complete = true;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateStep.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateStep.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateStep.java
new file mode 100644
index 0000000..6013795
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/registry/variable/VariableRegistryUpdateStep.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.registry.variable;
+
+public class VariableRegistryUpdateStep {
+ private final String description;
+ private volatile boolean complete;
+ private volatile String failureReason;
+
+ public VariableRegistryUpdateStep(final String description) {
+ this.description = description;
+ }
+
+ public boolean isComplete() {
+ return complete;
+ }
+
+ public void setComplete(boolean complete) {
+ this.complete = complete;
+ }
+
+ public String getFailureReason() {
+ return failureReason;
+ }
+
+ public void setFailureReason(String failureReason) {
+ this.failureReason = failureReason;
+ }
+
+ public String getDescription() {
+ return description;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
index 6f9ea98..6fed58e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
@@ -16,6 +16,13 @@
*/
package org.apache.nifi.web;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Function;
+
import org.apache.nifi.authorization.AuthorizeAccess;
import org.apache.nifi.authorization.RequestAction;
import org.apache.nifi.authorization.user.NiFiUser;
@@ -24,6 +31,7 @@ import org.apache.nifi.controller.repository.claim.ContentDirection;
import org.apache.nifi.controller.service.ControllerServiceState;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.web.api.dto.AccessPolicyDTO;
+import org.apache.nifi.web.api.dto.AffectedComponentDTO;
import org.apache.nifi.web.api.dto.BulletinBoardDTO;
import org.apache.nifi.web.api.dto.BulletinDTO;
import org.apache.nifi.web.api.dto.BulletinQueryDTO;
@@ -57,6 +65,7 @@ import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO;
import org.apache.nifi.web.api.dto.TemplateDTO;
import org.apache.nifi.web.api.dto.UserDTO;
import org.apache.nifi.web.api.dto.UserGroupDTO;
+import org.apache.nifi.web.api.dto.VariableRegistryDTO;
import org.apache.nifi.web.api.dto.action.HistoryDTO;
import org.apache.nifi.web.api.dto.action.HistoryQueryDTO;
import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO;
@@ -67,6 +76,7 @@ import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
import org.apache.nifi.web.api.dto.status.ControllerStatusDTO;
import org.apache.nifi.web.api.entity.AccessPolicyEntity;
import org.apache.nifi.web.api.entity.ActionEntity;
+import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
import org.apache.nifi.web.api.entity.BulletinEntity;
import org.apache.nifi.web.api.entity.ConnectionEntity;
import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
@@ -96,13 +106,7 @@ import org.apache.nifi.web.api.entity.StatusHistoryEntity;
import org.apache.nifi.web.api.entity.TemplateEntity;
import org.apache.nifi.web.api.entity.UserEntity;
import org.apache.nifi.web.api.entity.UserGroupEntity;
-
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.function.Function;
+import org.apache.nifi.web.api.entity.VariableRegistryEntity;
/**
* Defines the NiFiServiceFacade interface.
@@ -887,6 +891,48 @@ public interface NiFiServiceFacade {
ProcessGroupEntity getProcessGroup(String groupId);
/**
+ * Returns the Variable Registry for the Process Group with the given ID
+ *
+ * @param groupId the ID of the Process Group
+ * @param includeAncestorGroups whether or not to include the variables that are defined in the the process group's parent group & its parent group, etc.
+ * @return the Variable Registry transfer object
+ */
+ VariableRegistryEntity getVariableRegistry(String groupId, boolean includeAncestorGroups);
+
+ /**
+ * Returns a Variable Registry that includes the variables in the given DTO but has the affected components populated
+ *
+ * @param variableRegistryDto the Variable Registry that contains the variables of interest
+ * @return a Variable Registry that has the affected components populated
+ */
+ VariableRegistryEntity populateAffectedComponents(VariableRegistryDTO variableRegistryDto);
+
+ /**
+ * Updates the variable registry on behalf of the user currently logged in
+ *
+ * @param revision Revision to compare with current base revision
+ * @param variableRegistryDto the Variable Registry
+ */
+ VariableRegistryEntity updateVariableRegistry(Revision revision, VariableRegistryDTO variableRegistryDto);
+
+ /**
+ * Updates the variable registry on behalf of the given user
+ *
+ * @param user the user who performed the action
+ * @param revision Revision to compare with current base revision
+ * @param variableRegistryDto the Variable Registry
+ */
+ VariableRegistryEntity updateVariableRegistry(NiFiUser user, Revision revision, VariableRegistryDTO variableRegistryDto);
+
+ /**
+ * Determines which components will be affected by updating the given Variable Registry
+ *
+ * @param variableRegistryDto the variable registry
+ * @return the components that will be affected
+ */
+ Set<AffectedComponentDTO> getComponentsAffectedByVariableRegistryUpdate(VariableRegistryDTO variableRegistryDto);
+
+ /**
* Gets all process groups in the specified parent group.
*
* @param parentGroupId The id of the parent group
@@ -904,7 +950,37 @@ public interface NiFiServiceFacade {
void verifyScheduleComponents(String processGroupId, ScheduledState state, Set<String> componentIds);
/**
- * Schedules all applicable components under the specified ProcessGroup.
+ * Verifies the controller services with the given ID's can be enabled or disabled
+ *
+ * @param processGroupId the ID of the process group
+ * @param state the state
+ * @param serviceIds the id's of the services
+ */
+ void verifyActivateControllerServices(String processGroupId, ControllerServiceState state, Set<String> serviceIds);
+
+ /**
+ * Enables or disables the controller services with the given IDs & Revisions on behalf of the currently logged in user
+ *
+ * @param processGroupId the ID of the process group
+ * @param state the desired state of the services
+ * @param serviceRevisions a mapping of Controller Service ID to current Revision
+ * @return snapshot
+ */
+ ActivateControllerServicesEntity activateControllerServices(String processGroupId, ControllerServiceState state, Map<String, Revision> serviceRevisions);
+
+ /**
+ * Enables or disables the controller services with the given IDs & Revisions on behalf of the given user
+ *
+ * @param user the user performing the action
+ * @param processGroupId the ID of the process group
+ * @param state the desired state of the services
+ * @param serviceRevisions a mapping of Controller Service ID to current Revision
+ * @return snapshot
+ */
+ ActivateControllerServicesEntity activateControllerServices(NiFiUser user, String processGroupId, ControllerServiceState state, Map<String, Revision> serviceRevisions);
+
+ /**
+ * Schedules all applicable components under the specified ProcessGroup on behalf of the currently logged in user.
*
* @param processGroupId The ProcessGroup id
* @param state schedule state
@@ -914,6 +990,17 @@ public interface NiFiServiceFacade {
ScheduleComponentsEntity scheduleComponents(String processGroupId, ScheduledState state, Map<String, Revision> componentRevisions);
/**
+ * Schedules all applicable components under the specified ProcessGroup on behalf of the given user.
+ *
+ * @param user the user performing the action
+ * @param processGroupId The ProcessGroup id
+ * @param state schedule state
+ * @param componentRevisions components and their revision
+ * @return snapshot
+ */
+ ScheduleComponentsEntity scheduleComponents(NiFiUser user, String processGroupId, ScheduledState state, Map<String, Revision> componentRevisions);
+
+ /**
* Updates the specified process group.
*
* @param revision Revision to compare with current base revision
@@ -923,6 +1010,14 @@ public interface NiFiServiceFacade {
ProcessGroupEntity updateProcessGroup(Revision revision, ProcessGroupDTO processGroupDTO);
/**
+ * Verifies that the Process Group identified by the given DTO can be updated in the manner appropriate according
+ * to the DTO
+ *
+ * @param processGroupDTO the DTO that indicates the updates to occur
+ */
+ void verifyUpdateProcessGroup(ProcessGroupDTO processGroupDTO);
+
+ /**
* Verifies the specified process group can be removed.
*
* @param groupId The id of the process group
@@ -1378,9 +1473,22 @@ public interface NiFiServiceFacade {
* Gets all controller services that belong to the given group and its parent/ancestor groups
*
* @param groupId the id of the process group of interest
+ * @param includeAncestorGroups if true, parent and ancestor groups' services will be returned as well
+ * @param includeDescendantGroups if true, child and descendant groups' services will be returned as well
+ * @return services
+ */
+ Set<ControllerServiceEntity> getControllerServices(String groupId, boolean includeAncestorGroups, boolean includeDescendantGroups);
+
+ /**
+ * Gets all controller services that belong to the given group and its parent/ancestor groups
+ *
+ * @param groupId the id of the process group of interest
+ * @param includeAncestorGroups if true, parent and ancestor groups' services will be returned as well
+ * @param includeDescendantGroups if true, child and descendant groups' services will be returned as well
+ * @param user the user that is retrieving the Controller Services
* @return services
*/
- Set<ControllerServiceEntity> getControllerServices(String groupId);
+ Set<ControllerServiceEntity> getControllerServices(String groupId, boolean includeAncestorGroups, boolean includeDescendantGroups, NiFiUser user);
/**
* Gets the specified controller service.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java
index 11573a5..d0230db 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacadeLock.java
@@ -103,6 +103,18 @@ public class NiFiServiceFacadeLock {
}
@Around("within(org.apache.nifi.web.NiFiServiceFacade+) && "
+ + "execution(* activate*(..))")
+ public Object activateLock(ProceedingJoinPoint proceedingJoinPoint) throws Throwable {
+ return proceedWithWriteLock(proceedingJoinPoint);
+ }
+
+ @Around("within(org.apache.nifi.web.NiFiServiceFacade+) && "
+ + "execution(* populate*(..))")
+ public Object populateLock(ProceedingJoinPoint proceedingJoinPoint) throws Throwable {
+ return proceedWithWriteLock(proceedingJoinPoint);
+ }
+
+ @Around("within(org.apache.nifi.web.NiFiServiceFacade+) && "
+ "execution(* get*(..))")
public Object getLock(ProceedingJoinPoint proceedingJoinPoint) throws Throwable {
return proceedWithReadLock(proceedingJoinPoint);
[7/9] nifi git commit: NIFI-4224: - Initial implementation of Process
Group level Variable Registry - Updated to incorporate PR Feedback - Changed
log message because slf4j-simple apparently has a memory leak;
passing a String instead of passing in the C
Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java
new file mode 100644
index 0000000..53a27fe
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java
@@ -0,0 +1,1122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.attribute.expression.language.compile;
+
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_ATTRIBUTES;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_DELINEATED_VALUES;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_MATCHING_ATTRIBUTES;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.AND;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_ATTRIBUTE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_DELINEATED_VALUE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_MATCHING_ATTRIBUTE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.APPEND;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ATTRIBUTE_REFERENCE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ATTR_NAME;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.BASE64_DECODE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.BASE64_ENCODE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.CONTAINS;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.COUNT;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.DECIMAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.DIVIDE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ENDS_WITH;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EQUALS;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EQUALS_IGNORE_CASE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_CSV;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_HTML3;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_HTML4;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_JSON;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ESCAPE_XML;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EXPRESSION;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FALSE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FIND;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FORMAT;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FROM_RADIX;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GET_DELIMITED_FIELD;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GET_STATE_VALUE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GREATER_THAN;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GREATER_THAN_OR_EQUAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.HOSTNAME;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IF_ELSE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IN;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.INDEX_OF;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IP;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IS_EMPTY;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IS_NULL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.JOIN;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.JSON_PATH;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LAST_INDEX_OF;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LENGTH;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LESS_THAN;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LESS_THAN_OR_EQUAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MATCHES;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MATH;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MINUS;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MOD;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MULTIPLY;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MULTI_ATTRIBUTE_REFERENCE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NEXT_INT;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOT;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOT_NULL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOW;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.OR;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PLUS;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PREPEND;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.RANDOM;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_ALL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_EMPTY;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_FIRST;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_NULL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.STARTS_WITH;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.STRING_LITERAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_AFTER;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_AFTER_LAST;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE_LAST;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_DATE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_DECIMAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_LITERAL;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_LOWER;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_NUMBER;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_RADIX;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_STRING;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_UPPER;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TRIM;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TRUE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_CSV;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_HTML3;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_HTML4;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_JSON;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UNESCAPE_XML;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.URL_DECODE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.URL_ENCODE;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UUID;
+import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.WHOLE_NUMBER;
+
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.antlr.runtime.ANTLRStringStream;
+import org.antlr.runtime.CharStream;
+import org.antlr.runtime.CommonTokenStream;
+import org.antlr.runtime.tree.Tree;
+import org.apache.nifi.attribute.expression.language.Query;
+import org.apache.nifi.attribute.expression.language.Query.Range;
+import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionLexer;
+import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser;
+import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.cast.BooleanCastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.cast.DateCastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.cast.DecimalCastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.cast.NumberCastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.cast.StringCastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.cast.WholeNumberCastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.AndEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.AppendEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.Base64DecodeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.Base64EncodeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.CharSequenceTranslatorEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ContainsEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.DivideEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.EndsWithEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.EqualsEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.EqualsIgnoreCaseEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.FindEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.FormatEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.FromRadixEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.GetDelimitedFieldEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.GetStateVariableEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.GreaterThanEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.GreaterThanOrEqualEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.HostnameEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.IPEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.IfElseEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.InEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.IndexOfEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.IsEmptyEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.IsNullEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.JsonPathEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.LastIndexOfEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.LengthEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.LessThanEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.LessThanOrEqualEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.MatchesEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.MathEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.MinusEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ModEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.MultiplyEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.NotEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.NotNullEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.NowEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.NumberToDateEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.OneUpSequenceEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.OrEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.PlusEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.PrependEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.RandomNumberGeneratorEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceAllEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceEmptyEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceFirstEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceNullEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.StartsWithEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.StringToDateEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringAfterEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringAfterLastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringBeforeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringBeforeLastEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ToLowerEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ToRadixEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ToStringEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.ToUpperEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.TrimEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.UrlDecodeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.UrlEncodeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.functions.UuidEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.literals.BooleanLiteralEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.literals.DecimalLiteralEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.literals.ToLiteralEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.literals.WholeNumberLiteralEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.reduce.CountEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.reduce.JoinEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.reduce.ReduceEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.AllAttributesEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.AnyAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.AttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.DelineatedAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.IteratingEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MappingEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiMatchAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiNamedAttributeEvaluator;
+import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
+import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
+import org.apache.nifi.expression.AttributeExpression.ResultType;
+import org.apache.nifi.flowfile.FlowFile;
+
+public class ExpressionCompiler {
+ private final Set<Evaluator<?>> evaluators = new HashSet<>();
+
+ public CompiledExpression compile(final String expression) {
+ try {
+ final CharStream input = new ANTLRStringStream(expression);
+ final AttributeExpressionLexer lexer = new AttributeExpressionLexer(input);
+ final CommonTokenStream lexerTokenStream = new CommonTokenStream(lexer);
+
+ final AttributeExpressionParser parser = new AttributeExpressionParser(lexerTokenStream);
+ final Tree ast = (Tree) parser.query().getTree();
+ final Tree tree = ast.getChild(0);
+
+ final Evaluator<?> evaluator = buildEvaluator(tree);
+ verifyMappingEvaluatorReduced(evaluator);
+
+ final Set<Evaluator<?>> allEvaluators = new HashSet<>(evaluators);
+ this.evaluators.clear();
+
+ return new CompiledExpression(expression, evaluator, tree, allEvaluators);
+ } catch (final AttributeExpressionLanguageParsingException e) {
+ throw e;
+ } catch (final Exception e) {
+ throw new AttributeExpressionLanguageParsingException(e);
+ }
+ }
+
+ private Tree compileTree(final String expression) throws AttributeExpressionLanguageParsingException {
+ try {
+ final CharStream input = new ANTLRStringStream(expression);
+ final AttributeExpressionLexer lexer = new AttributeExpressionLexer(input);
+ final CommonTokenStream lexerTokenStream = new CommonTokenStream(lexer);
+
+ final AttributeExpressionParser parser = new AttributeExpressionParser(lexerTokenStream);
+ final Tree ast = (Tree) parser.query().getTree();
+ final Tree tree = ast.getChild(0);
+
+ // ensure that we are able to build the evaluators, so that we validate syntax
+ final Evaluator<?> evaluator = buildEvaluator(tree);
+ verifyMappingEvaluatorReduced(evaluator);
+
+ return tree;
+ } catch (final AttributeExpressionLanguageParsingException e) {
+ throw e;
+ } catch (final Exception e) {
+ throw new AttributeExpressionLanguageParsingException(e);
+ }
+ }
+
+ private void verifyMappingEvaluatorReduced(final Evaluator<?> evaluator) {
+ final Evaluator<?> rightMostEvaluator;
+ if (evaluator instanceof IteratingEvaluator) {
+ rightMostEvaluator = ((IteratingEvaluator<?>) evaluator).getLogicEvaluator();
+ } else {
+ rightMostEvaluator = evaluator;
+ }
+
+ Evaluator<?> eval = rightMostEvaluator.getSubjectEvaluator();
+ Evaluator<?> lastEval = rightMostEvaluator;
+ while (eval != null) {
+ if (eval instanceof ReduceEvaluator) {
+ throw new AttributeExpressionLanguageParsingException("Expression attempts to call function '" + lastEval.getToken() + "' on the result of '" + eval.getToken() +
+ "'. This is not allowed. Instead, use \"${literal( ${<embedded expression>} ):" + lastEval.getToken() + "(...)}\"");
+ }
+
+ lastEval = eval;
+ eval = eval.getSubjectEvaluator();
+ }
+
+ // if the result type of the evaluator is BOOLEAN, then it will always
+ // be reduced when evaluator.
+ final ResultType resultType = evaluator.getResultType();
+ if (resultType == ResultType.BOOLEAN) {
+ return;
+ }
+
+ final Evaluator<?> rootEvaluator = getRootSubjectEvaluator(evaluator);
+ if (rootEvaluator != null && rootEvaluator instanceof MultiAttributeEvaluator) {
+ final MultiAttributeEvaluator multiAttrEval = (MultiAttributeEvaluator) rootEvaluator;
+ switch (multiAttrEval.getEvaluationType()) {
+ case ALL_ATTRIBUTES:
+ case ALL_MATCHING_ATTRIBUTES:
+ case ALL_DELINEATED_VALUES: {
+ if (!(evaluator instanceof ReduceEvaluator)) {
+ throw new AttributeExpressionLanguageParsingException("Cannot evaluate expression because it attempts to reference multiple attributes but does not use a reducing function");
+ }
+ break;
+ }
+ default:
+ throw new AttributeExpressionLanguageParsingException("Cannot evaluate expression because it attempts to reference multiple attributes but does not use a reducing function");
+ }
+ }
+ }
+
+ private Evaluator<?> getRootSubjectEvaluator(final Evaluator<?> evaluator) {
+ if (evaluator == null) {
+ return null;
+ }
+
+ final Evaluator<?> subject = evaluator.getSubjectEvaluator();
+ if (subject == null) {
+ return evaluator;
+ }
+
+ return getRootSubjectEvaluator(subject);
+ }
+
+ @SuppressWarnings({"rawtypes", "unchecked"})
+ private Evaluator<?> buildExpressionEvaluator(final Tree tree) {
+ if (tree.getChildCount() == 0) {
+ throw new AttributeExpressionLanguageParsingException("EXPRESSION tree node has no children");
+ }
+
+ final Evaluator<?> evaluator;
+ if (tree.getChildCount() == 1) {
+ evaluator = buildEvaluator(tree.getChild(0));
+ } else {
+ // we can chain together functions in the form of:
+ // ${x:trim():substring(1,2):trim()}
+ // in this case, the subject of the right-most function is the function to its left; its
+ // subject is the function to its left (the first trim()), and its subject is the value of
+ // the 'x' attribute. We accomplish this logic by iterating over all of the children of the
+ // tree from the right-most child going left-ward.
+ evaluator = buildFunctionExpressionEvaluator(tree, 0);
+ }
+
+ Evaluator<?> chosenEvaluator = evaluator;
+ final Evaluator<?> rootEvaluator = getRootSubjectEvaluator(evaluator);
+ if (rootEvaluator != null) {
+ if (rootEvaluator instanceof MultiAttributeEvaluator) {
+ final MultiAttributeEvaluator multiAttrEval = (MultiAttributeEvaluator) rootEvaluator;
+
+ switch (multiAttrEval.getEvaluationType()) {
+ case ANY_ATTRIBUTE:
+ case ANY_MATCHING_ATTRIBUTE:
+ case ANY_DELINEATED_VALUE:
+ chosenEvaluator = new AnyAttributeEvaluator((BooleanEvaluator) evaluator, multiAttrEval);
+ break;
+ case ALL_ATTRIBUTES:
+ case ALL_MATCHING_ATTRIBUTES:
+ case ALL_DELINEATED_VALUES: {
+ final ResultType resultType = evaluator.getResultType();
+ if (resultType == ResultType.BOOLEAN) {
+ chosenEvaluator = new AllAttributesEvaluator((BooleanEvaluator) evaluator, multiAttrEval);
+ } else if (evaluator instanceof ReduceEvaluator) {
+ chosenEvaluator = new MappingEvaluator((ReduceEvaluator) evaluator, multiAttrEval);
+ } else {
+ throw new AttributeExpressionLanguageException("Cannot evaluate Expression because it attempts to reference multiple attributes but does not use a reducing function");
+ }
+ break;
+ }
+ }
+
+ evaluators.add(chosenEvaluator);
+ switch (multiAttrEval.getEvaluationType()) {
+ case ANY_ATTRIBUTE:
+ chosenEvaluator.setToken("anyAttribute");
+ break;
+ case ANY_MATCHING_ATTRIBUTE:
+ chosenEvaluator.setToken("anyMatchingAttribute");
+ break;
+ case ANY_DELINEATED_VALUE:
+ chosenEvaluator.setToken("anyDelineatedValue");
+ break;
+ case ALL_ATTRIBUTES:
+ chosenEvaluator.setToken("allAttributes");
+ break;
+ case ALL_MATCHING_ATTRIBUTES:
+ chosenEvaluator.setToken("allMatchingAttributes");
+ break;
+ case ALL_DELINEATED_VALUES:
+ chosenEvaluator.setToken("allDelineatedValues");
+ break;
+ }
+ }
+ }
+
+ return chosenEvaluator;
+ }
+
+ private Evaluator<?> buildFunctionExpressionEvaluator(final Tree tree, final int offset) {
+ if (tree.getChildCount() == 0) {
+ throw new AttributeExpressionLanguageParsingException("EXPRESSION tree node has no children");
+ }
+ final int firstChildIndex = tree.getChildCount() - offset - 1;
+ if (firstChildIndex == 0) {
+ return buildEvaluator(tree.getChild(0));
+ }
+
+ final Tree functionTree = tree.getChild(firstChildIndex);
+ final Evaluator<?> subjectEvaluator = buildFunctionExpressionEvaluator(tree, offset + 1);
+
+ final Tree functionNameTree = functionTree.getChild(0);
+ final List<Evaluator<?>> argEvaluators = new ArrayList<>();
+ for (int i = 1; i < functionTree.getChildCount(); i++) {
+ argEvaluators.add(buildEvaluator(functionTree.getChild(i)));
+ }
+ return buildFunctionEvaluator(functionNameTree, subjectEvaluator, argEvaluators);
+ }
+
+ private List<Evaluator<?>> verifyArgCount(final List<Evaluator<?>> args, final int count, final String functionName) {
+ if (args.size() != count) {
+ throw new AttributeExpressionLanguageParsingException(functionName + "() function takes " + count + " arguments");
+ }
+ return args;
+ }
+
+ private Evaluator<String> toStringEvaluator(final Evaluator<?> evaluator) {
+ return toStringEvaluator(evaluator, null);
+ }
+
+ private Evaluator<String> toStringEvaluator(final Evaluator<?> evaluator, final String location) {
+ if (evaluator.getResultType() == ResultType.STRING) {
+ return (StringEvaluator) evaluator;
+ }
+
+ return addToken(new StringCastEvaluator(evaluator), evaluator.getToken());
+ }
+
+ @SuppressWarnings("unchecked")
+ private Evaluator<Boolean> toBooleanEvaluator(final Evaluator<?> evaluator, final String location) {
+ switch (evaluator.getResultType()) {
+ case BOOLEAN:
+ return (Evaluator<Boolean>) evaluator;
+ case STRING:
+ return addToken(new BooleanCastEvaluator((StringEvaluator) evaluator), evaluator.getToken());
+ default:
+ throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.BOOLEAN
+ + (location == null ? "" : " at location [" + location + "]"));
+ }
+
+ }
+
+ private Evaluator<Boolean> toBooleanEvaluator(final Evaluator<?> evaluator) {
+ return toBooleanEvaluator(evaluator, null);
+ }
+
+ private Evaluator<Long> toWholeNumberEvaluator(final Evaluator<?> evaluator) {
+ return toWholeNumberEvaluator(evaluator, null);
+ }
+
+ @SuppressWarnings("unchecked")
+ private Evaluator<Long> toWholeNumberEvaluator(final Evaluator<?> evaluator, final String location) {
+ switch (evaluator.getResultType()) {
+ case WHOLE_NUMBER:
+ return (Evaluator<Long>) evaluator;
+ case STRING:
+ case DATE:
+ case DECIMAL:
+ case NUMBER:
+ return addToken(new WholeNumberCastEvaluator(evaluator), evaluator.getToken());
+ default:
+ throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.WHOLE_NUMBER
+ + (location == null ? "" : " at location [" + location + "]"));
+ }
+ }
+
+ private Evaluator<Double> toDecimalEvaluator(final Evaluator<?> evaluator) {
+ return toDecimalEvaluator(evaluator, null);
+ }
+
+ @SuppressWarnings("unchecked")
+ private Evaluator<Double> toDecimalEvaluator(final Evaluator<?> evaluator, final String location) {
+ switch (evaluator.getResultType()) {
+ case DECIMAL:
+ return (Evaluator<Double>) evaluator;
+ case WHOLE_NUMBER:
+ case STRING:
+ case DATE:
+ case NUMBER:
+ return addToken(new DecimalCastEvaluator(evaluator), evaluator.getToken());
+ default:
+ throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.DECIMAL
+ + (location == null ? "" : " at location [" + location + "]"));
+ }
+ }
+
+ private Evaluator<Number> toNumberEvaluator(final Evaluator<?> evaluator) {
+ return toNumberEvaluator(evaluator, null);
+ }
+
+ @SuppressWarnings("unchecked")
+ private Evaluator<Number> toNumberEvaluator(final Evaluator<?> evaluator, final String location) {
+ switch (evaluator.getResultType()) {
+ case NUMBER:
+ return (Evaluator<Number>) evaluator;
+ case STRING:
+ case DATE:
+ case DECIMAL:
+ case WHOLE_NUMBER:
+ return addToken(new NumberCastEvaluator(evaluator), evaluator.getToken());
+ default:
+ throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.WHOLE_NUMBER
+ + (location == null ? "" : " at location [" + location + "]"));
+ }
+ }
+
+ private DateEvaluator toDateEvaluator(final Evaluator<?> evaluator) {
+ return toDateEvaluator(evaluator, null);
+ }
+
+ private DateEvaluator toDateEvaluator(final Evaluator<?> evaluator, final String location) {
+ if (evaluator.getResultType() == ResultType.DATE) {
+ return (DateEvaluator) evaluator;
+ }
+
+ return new DateCastEvaluator(evaluator);
+ }
+
+ private Evaluator<?> buildFunctionEvaluator(final Tree tree, final Evaluator<?> subjectEvaluator, final List<Evaluator<?>> argEvaluators) {
+ switch (tree.getType()) {
+ case TRIM: {
+ verifyArgCount(argEvaluators, 0, "trim");
+ return addToken(new TrimEvaluator(toStringEvaluator(subjectEvaluator)), "trim");
+ }
+ case TO_STRING: {
+ verifyArgCount(argEvaluators, 0, "toString");
+ return addToken(new ToStringEvaluator(subjectEvaluator), "toString");
+ }
+ case TO_LOWER: {
+ verifyArgCount(argEvaluators, 0, "toLower");
+ return addToken(new ToLowerEvaluator(toStringEvaluator(subjectEvaluator)), "toLower");
+ }
+ case TO_UPPER: {
+ verifyArgCount(argEvaluators, 0, "toUpper");
+ return addToken(new ToUpperEvaluator(toStringEvaluator(subjectEvaluator)), "toUpper");
+ }
+ case URL_ENCODE: {
+ verifyArgCount(argEvaluators, 0, "urlEncode");
+ return addToken(new UrlEncodeEvaluator(toStringEvaluator(subjectEvaluator)), "urlEncode");
+ }
+ case URL_DECODE: {
+ verifyArgCount(argEvaluators, 0, "urlDecode");
+ return addToken(new UrlDecodeEvaluator(toStringEvaluator(subjectEvaluator)), "urlDecode");
+ }
+ case BASE64_ENCODE: {
+ verifyArgCount(argEvaluators, 0, "base64Encode");
+ return addToken(new Base64EncodeEvaluator(toStringEvaluator(subjectEvaluator)), "base64Encode");
+ }
+ case BASE64_DECODE: {
+ verifyArgCount(argEvaluators, 0, "base64Decode");
+ return addToken(new Base64DecodeEvaluator(toStringEvaluator(subjectEvaluator)), "base64Decode");
+ }
+ case ESCAPE_CSV: {
+ verifyArgCount(argEvaluators, 0, "escapeCsv");
+ return addToken(CharSequenceTranslatorEvaluator.csvEscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+ }
+ case ESCAPE_HTML3: {
+ verifyArgCount(argEvaluators, 0, "escapeHtml3");
+ return addToken(CharSequenceTranslatorEvaluator.html3EscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+ }
+ case ESCAPE_HTML4: {
+ verifyArgCount(argEvaluators, 0, "escapeHtml4");
+ return addToken(CharSequenceTranslatorEvaluator.html4EscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+ }
+ case ESCAPE_JSON: {
+ verifyArgCount(argEvaluators, 0, "escapeJson");
+ return addToken(CharSequenceTranslatorEvaluator.jsonEscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+ }
+ case ESCAPE_XML: {
+ verifyArgCount(argEvaluators, 0, "escapeXml");
+ return addToken(CharSequenceTranslatorEvaluator.xmlEscapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+ }
+ case UNESCAPE_CSV: {
+ verifyArgCount(argEvaluators, 0, "unescapeCsv");
+ return addToken(CharSequenceTranslatorEvaluator.csvUnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+ }
+ case UNESCAPE_HTML3: {
+ verifyArgCount(argEvaluators, 0, "unescapeHtml3");
+ return addToken(CharSequenceTranslatorEvaluator.html3UnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+ }
+ case UNESCAPE_HTML4: {
+ verifyArgCount(argEvaluators, 0, "unescapeHtml4");
+ return addToken(CharSequenceTranslatorEvaluator.html4UnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+ }
+ case UNESCAPE_JSON: {
+ verifyArgCount(argEvaluators, 0, "unescapeJson");
+ return addToken(CharSequenceTranslatorEvaluator.jsonUnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+ }
+ case UNESCAPE_XML: {
+ verifyArgCount(argEvaluators, 0, "unescapeXml");
+ return addToken(CharSequenceTranslatorEvaluator.xmlUnescapeEvaluator(toStringEvaluator(subjectEvaluator)), "escapeJson");
+ }
+ case SUBSTRING_BEFORE: {
+ verifyArgCount(argEvaluators, 1, "substringBefore");
+ return addToken(new SubstringBeforeEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to substringBefore")), "substringBefore");
+ }
+ case SUBSTRING_BEFORE_LAST: {
+ verifyArgCount(argEvaluators, 1, "substringBeforeLast");
+ return addToken(new SubstringBeforeLastEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to substringBeforeLast")), "substringBeforeLast");
+ }
+ case SUBSTRING_AFTER: {
+ verifyArgCount(argEvaluators, 1, "substringAfter");
+ return addToken(new SubstringAfterEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to substringAfter")), "substringAfter");
+ }
+ case SUBSTRING_AFTER_LAST: {
+ verifyArgCount(argEvaluators, 1, "substringAfterLast");
+ return addToken(new SubstringAfterLastEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to substringAfterLast")), "substringAfterLast");
+ }
+ case REPLACE_NULL: {
+ verifyArgCount(argEvaluators, 1, "replaceNull");
+ return addToken(new ReplaceNullEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to replaceNull")), "replaceNull");
+ }
+ case REPLACE_EMPTY: {
+ verifyArgCount(argEvaluators, 1, "replaceEmtpy");
+ return addToken(new ReplaceEmptyEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument to replaceEmpty")), "replaceEmpty");
+ }
+ case REPLACE: {
+ verifyArgCount(argEvaluators, 2, "replace");
+ return addToken(new ReplaceEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to replace"),
+ toStringEvaluator(argEvaluators.get(1), "second argument to replace")), "replace");
+ }
+ case REPLACE_FIRST: {
+ verifyArgCount(argEvaluators, 2, "replaceFirst");
+ return addToken(new ReplaceFirstEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to replaceFirst"),
+ toStringEvaluator(argEvaluators.get(1), "second argument to replaceFirst")), "replaceFirst");
+ }
+ case REPLACE_ALL: {
+ verifyArgCount(argEvaluators, 2, "replaceAll");
+ return addToken(new ReplaceAllEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to replaceAll"),
+ toStringEvaluator(argEvaluators.get(1), "second argument to replaceAll")), "replaceAll");
+ }
+ case APPEND: {
+ verifyArgCount(argEvaluators, 1, "append");
+ return addToken(new AppendEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to append")), "append");
+ }
+ case PREPEND: {
+ verifyArgCount(argEvaluators, 1, "prepend");
+ return addToken(new PrependEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to prepend")), "prepend");
+ }
+ case SUBSTRING: {
+ final int numArgs = argEvaluators.size();
+ if (numArgs == 1) {
+ return addToken(new SubstringEvaluator(toStringEvaluator(subjectEvaluator),
+ toWholeNumberEvaluator(argEvaluators.get(0), "first argument to substring")), "substring");
+ } else if (numArgs == 2) {
+ return addToken(new SubstringEvaluator(toStringEvaluator(subjectEvaluator),
+ toWholeNumberEvaluator(argEvaluators.get(0), "first argument to substring"),
+ toWholeNumberEvaluator(argEvaluators.get(1), "second argument to substring")), "substring");
+ } else {
+ throw new AttributeExpressionLanguageParsingException("substring() function can take either 1 or 2 arguments but cannot take " + numArgs + " arguments");
+ }
+ }
+ case JOIN: {
+ verifyArgCount(argEvaluators, 1, "join");
+ return addToken(new JoinEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0))), "join");
+ }
+ case COUNT: {
+ verifyArgCount(argEvaluators, 0, "count");
+ return addToken(new CountEvaluator(subjectEvaluator), "count");
+ }
+ case IS_NULL: {
+ verifyArgCount(argEvaluators, 0, "isNull");
+ return addToken(new IsNullEvaluator(toStringEvaluator(subjectEvaluator)), "isNull");
+ }
+ case IS_EMPTY: {
+ verifyArgCount(argEvaluators, 0, "isEmpty");
+ return addToken(new IsEmptyEvaluator(toStringEvaluator(subjectEvaluator)), "isEmpty");
+ }
+ case NOT_NULL: {
+ verifyArgCount(argEvaluators, 0, "notNull");
+ return addToken(new NotNullEvaluator(toStringEvaluator(subjectEvaluator)), "notNull");
+ }
+ case STARTS_WITH: {
+ verifyArgCount(argEvaluators, 1, "startsWith");
+ return addToken(new StartsWithEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to startsWith")), "startsWith");
+ }
+ case ENDS_WITH: {
+ verifyArgCount(argEvaluators, 1, "endsWith");
+ return addToken(new EndsWithEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to endsWith")), "endsWith");
+ }
+ case CONTAINS: {
+ verifyArgCount(argEvaluators, 1, "contains");
+ return addToken(new ContainsEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to contains")), "contains");
+ }
+ case IN: {
+ List<Evaluator<String>> list = new ArrayList<Evaluator<String>>();
+ for (int i = 0; i < argEvaluators.size(); i++) {
+ list.add(toStringEvaluator(argEvaluators.get(i), i + "th argument to in"));
+ }
+ return addToken(new InEvaluator(toStringEvaluator(subjectEvaluator), list), "in");
+ }
+ case FIND: {
+ verifyArgCount(argEvaluators, 1, "find");
+ return addToken(new FindEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to find")), "find");
+ }
+ case MATCHES: {
+ verifyArgCount(argEvaluators, 1, "matches");
+ return addToken(new MatchesEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to matches")), "matches");
+ }
+ case EQUALS: {
+ verifyArgCount(argEvaluators, 1, "equals");
+ return addToken(new EqualsEvaluator(subjectEvaluator, argEvaluators.get(0)), "equals");
+ }
+ case EQUALS_IGNORE_CASE: {
+ verifyArgCount(argEvaluators, 1, "equalsIgnoreCase");
+ return addToken(new EqualsIgnoreCaseEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to equalsIgnoreCase")), "equalsIgnoreCase");
+ }
+ case GREATER_THAN: {
+ verifyArgCount(argEvaluators, 1, "gt");
+ return addToken(new GreaterThanEvaluator(toNumberEvaluator(subjectEvaluator),
+ toNumberEvaluator(argEvaluators.get(0), "first argument to gt")), "gt");
+ }
+ case GREATER_THAN_OR_EQUAL: {
+ verifyArgCount(argEvaluators, 1, "ge");
+ return addToken(new GreaterThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator),
+ toNumberEvaluator(argEvaluators.get(0), "first argument to ge")), "ge");
+ }
+ case LESS_THAN: {
+ verifyArgCount(argEvaluators, 1, "lt");
+ return addToken(new LessThanEvaluator(toNumberEvaluator(subjectEvaluator),
+ toNumberEvaluator(argEvaluators.get(0), "first argument to lt")), "lt");
+ }
+ case LESS_THAN_OR_EQUAL: {
+ verifyArgCount(argEvaluators, 1, "le");
+ return addToken(new LessThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator),
+ toNumberEvaluator(argEvaluators.get(0), "first argument to le")), "le");
+ }
+ case LENGTH: {
+ verifyArgCount(argEvaluators, 0, "length");
+ return addToken(new LengthEvaluator(toStringEvaluator(subjectEvaluator)), "length");
+ }
+ case TO_DATE: {
+ if (argEvaluators.isEmpty()) {
+ return addToken(new NumberToDateEvaluator(toWholeNumberEvaluator(subjectEvaluator)), "toDate");
+ } else if (subjectEvaluator.getResultType() == ResultType.STRING && argEvaluators.size() == 1) {
+ return addToken(new StringToDateEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), null), "toDate");
+ } else if (subjectEvaluator.getResultType() == ResultType.STRING && argEvaluators.size() == 2) {
+ return addToken(new StringToDateEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), toStringEvaluator(argEvaluators.get(1))), "toDate");
+ } else {
+ return addToken(new NumberToDateEvaluator(toWholeNumberEvaluator(subjectEvaluator)), "toDate");
+ }
+ }
+ case TO_NUMBER: {
+ verifyArgCount(argEvaluators, 0, "toNumber");
+ switch (subjectEvaluator.getResultType()) {
+ case STRING:
+ case WHOLE_NUMBER:
+ case DECIMAL:
+ case NUMBER:
+ case DATE:
+ return addToken(toWholeNumberEvaluator(subjectEvaluator), "toNumber");
+ default:
+ throw new AttributeExpressionLanguageParsingException(subjectEvaluator + " returns type " + subjectEvaluator.getResultType() + " but expected to get " + ResultType.STRING +
+ ", " + ResultType.DECIMAL + ", or " + ResultType.DATE);
+ }
+ }
+ case TO_DECIMAL: {
+ verifyArgCount(argEvaluators, 0, "toDecimal");
+ switch (subjectEvaluator.getResultType()) {
+ case WHOLE_NUMBER:
+ case DECIMAL:
+ case STRING:
+ case NUMBER:
+ case DATE:
+ return addToken(toDecimalEvaluator(subjectEvaluator), "toDecimal");
+ default:
+ throw new AttributeExpressionLanguageParsingException(subjectEvaluator + " returns type " + subjectEvaluator.getResultType() + " but expected to get " + ResultType.STRING +
+ ", " + ResultType.WHOLE_NUMBER + ", or " + ResultType.DATE);
+ }
+ }
+ case TO_RADIX: {
+ if (argEvaluators.size() == 1) {
+ return addToken(new ToRadixEvaluator(toWholeNumberEvaluator(subjectEvaluator),
+ toWholeNumberEvaluator(argEvaluators.get(0))), "toRadix");
+ } else {
+ return addToken(new ToRadixEvaluator(toWholeNumberEvaluator(subjectEvaluator),
+ toWholeNumberEvaluator(argEvaluators.get(0)), toWholeNumberEvaluator(argEvaluators.get(1))), "toRadix");
+ }
+ }
+ case FROM_RADIX: {
+ return addToken(new FromRadixEvaluator(toStringEvaluator(subjectEvaluator),
+ toWholeNumberEvaluator(argEvaluators.get(0))), "fromRadix");
+ }
+ case MOD: {
+ return addToken(new ModEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "mod");
+ }
+ case PLUS: {
+ return addToken(new PlusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "plus");
+ }
+ case MINUS: {
+ return addToken(new MinusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "minus");
+ }
+ case MULTIPLY: {
+ return addToken(new MultiplyEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "multiply");
+ }
+ case DIVIDE: {
+ return addToken(new DivideEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))), "divide");
+ }
+ case MATH: {
+ if (argEvaluators.size() == 1) {
+ return addToken(new MathEvaluator(toNumberEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), null), "math");
+ } else if (argEvaluators.size() == 2) {
+ return addToken(new MathEvaluator(toNumberEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), toNumberEvaluator(argEvaluators.get(1))), "math");
+ } else {
+ throw new AttributeExpressionLanguageParsingException("math() function takes 1 or 2 arguments");
+ }
+ }
+ case RANDOM: {
+ return addToken(new RandomNumberGeneratorEvaluator(), "random");
+ }
+ case INDEX_OF: {
+ verifyArgCount(argEvaluators, 1, "indexOf");
+ return addToken(new IndexOfEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to indexOf")), "indexOf");
+ }
+ case LAST_INDEX_OF: {
+ verifyArgCount(argEvaluators, 1, "lastIndexOf");
+ return addToken(new LastIndexOfEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to lastIndexOf")), "lastIndexOf");
+ }
+ case FORMAT: {
+ if (argEvaluators.size() == 1) {
+ return addToken(new FormatEvaluator(toDateEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument of format"), null), "format");
+ } else if (argEvaluators.size() == 2) {
+ return addToken(new FormatEvaluator(toDateEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0)), toStringEvaluator(argEvaluators.get(1))), "format");
+ } else {
+ throw new AttributeExpressionLanguageParsingException("format() function takes 1 or 2 arguments");
+ }
+ }
+ case OR: {
+ return addToken(new OrEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0))), "or");
+ }
+ case AND: {
+ return addToken(new AndEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0))), "and");
+ }
+ case NOT: {
+ return addToken(new NotEvaluator(toBooleanEvaluator(subjectEvaluator)), "not");
+ }
+ case GET_DELIMITED_FIELD: {
+ if (argEvaluators.size() == 1) {
+ // Only a single argument - the index to return.
+ return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
+ toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField")), "getDelimitedField");
+ } else if (argEvaluators.size() == 2) {
+ // two arguments - index and delimiter.
+ return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
+ toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
+ toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField")),
+ "getDelimitedField");
+ } else if (argEvaluators.size() == 3) {
+ // 3 arguments - index, delimiter, quote char.
+ return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
+ toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
+ toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField"),
+ toStringEvaluator(argEvaluators.get(2), "third argument of getDelimitedField")),
+ "getDelimitedField");
+ } else if (argEvaluators.size() == 4) {
+ // 4 arguments - index, delimiter, quote char, escape char
+ return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
+ toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
+ toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField"),
+ toStringEvaluator(argEvaluators.get(2), "third argument of getDelimitedField"),
+ toStringEvaluator(argEvaluators.get(3), "fourth argument of getDelimitedField")),
+ "getDelimitedField");
+ } else {
+ // 5 arguments - index, delimiter, quote char, escape char, strip escape/quote chars flag
+ return addToken(new GetDelimitedFieldEvaluator(toStringEvaluator(subjectEvaluator),
+ toWholeNumberEvaluator(argEvaluators.get(0), "first argument of getDelimitedField"),
+ toStringEvaluator(argEvaluators.get(1), "second argument of getDelimitedField"),
+ toStringEvaluator(argEvaluators.get(2), "third argument of getDelimitedField"),
+ toStringEvaluator(argEvaluators.get(3), "fourth argument of getDelimitedField"),
+ toBooleanEvaluator(argEvaluators.get(4), "fifth argument of getDelimitedField")),
+ "getDelimitedField");
+ }
+ }
+ case JSON_PATH: {
+ verifyArgCount(argEvaluators, 1, "jsonPath");
+ return addToken(new JsonPathEvaluator(toStringEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "first argument to jsonPath")), "jsonPath");
+ }
+ case IF_ELSE: {
+ verifyArgCount(argEvaluators, 2, "ifElse");
+ return addToken(new IfElseEvaluator(toBooleanEvaluator(subjectEvaluator),
+ toStringEvaluator(argEvaluators.get(0), "argument to return if true"),
+ toStringEvaluator(argEvaluators.get(1), "argument to return if false")), "ifElse");
+ }
+ default:
+ throw new AttributeExpressionLanguageParsingException("Expected a Function-type expression but got " + tree.toString());
+ }
+ }
+
+ public Evaluator<?> buildEvaluator(final Tree tree) {
+ switch (tree.getType()) {
+ case EXPRESSION: {
+ return buildExpressionEvaluator(tree);
+ }
+ case ATTRIBUTE_REFERENCE: {
+ final Evaluator<?> childEvaluator = buildEvaluator(tree.getChild(0));
+ if (childEvaluator instanceof MultiAttributeEvaluator) {
+ return childEvaluator;
+ }
+ final AttributeEvaluator eval = new AttributeEvaluator(toStringEvaluator(childEvaluator));
+ evaluators.add(eval);
+ return eval;
+ }
+ case MULTI_ATTRIBUTE_REFERENCE: {
+
+ final Tree functionTypeTree = tree.getChild(0);
+ final int multiAttrType = functionTypeTree.getType();
+ if (multiAttrType == ANY_DELINEATED_VALUE || multiAttrType == ALL_DELINEATED_VALUES) {
+ final Evaluator<String> delineatedValueEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(1)));
+ final Evaluator<String> delimiterEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(2)));
+
+ final String token = (multiAttrType == ANY_DELINEATED_VALUE) ? "anyDelineatedValue" : "allDelineatedValues";
+ return addToken(new DelineatedAttributeEvaluator(delineatedValueEvaluator, delimiterEvaluator, multiAttrType), token);
+ }
+
+ final List<String> attributeNames = new ArrayList<>();
+ for (int i = 1; i < tree.getChildCount(); i++) { // skip the first child because that's the name of the multi-attribute function
+ attributeNames.add(newStringLiteralEvaluator(tree.getChild(i).getText()).evaluate(null).getValue());
+ }
+
+ switch (multiAttrType) {
+ case ALL_ATTRIBUTES:
+ for (final String attributeName : attributeNames) {
+ try {
+ FlowFile.KeyValidator.validateKey(attributeName);
+ } catch (final IllegalArgumentException iae) {
+ throw new AttributeExpressionLanguageParsingException("Invalid Attribute Name: " + attributeName + ". " + iae.getMessage());
+ }
+ }
+
+ return addToken(new MultiNamedAttributeEvaluator(attributeNames, ALL_ATTRIBUTES), "allAttributes");
+ case ALL_MATCHING_ATTRIBUTES:
+ return addToken(new MultiMatchAttributeEvaluator(attributeNames, ALL_MATCHING_ATTRIBUTES), "allMatchingAttributes");
+ case ANY_ATTRIBUTE:
+ for (final String attributeName : attributeNames) {
+ try {
+ FlowFile.KeyValidator.validateKey(attributeName);
+ } catch (final IllegalArgumentException iae) {
+ throw new AttributeExpressionLanguageParsingException("Invalid Attribute Name: " + attributeName + ". " + iae.getMessage());
+ }
+ }
+
+ return addToken(new MultiNamedAttributeEvaluator(attributeNames, ANY_ATTRIBUTE), "anyAttribute");
+ case ANY_MATCHING_ATTRIBUTE:
+ return addToken(new MultiMatchAttributeEvaluator(attributeNames, ANY_MATCHING_ATTRIBUTE), "anyMatchingAttribute");
+ default:
+ throw new AssertionError("Illegal Multi-Attribute Reference: " + functionTypeTree.toString());
+ }
+ }
+ case ATTR_NAME: {
+ return newStringLiteralEvaluator(tree.getChild(0).getText());
+ }
+ case WHOLE_NUMBER: {
+ return addToken(new WholeNumberLiteralEvaluator(tree.getText()), "wholeNumber");
+ }
+ case STRING_LITERAL: {
+ return newStringLiteralEvaluator(tree.getText());
+ }
+ case DECIMAL: {
+ return addToken(new DecimalLiteralEvaluator(tree.getText()), "decimal");
+ }
+ case TRUE:
+ case FALSE:
+ return buildBooleanEvaluator(tree);
+ case UUID: {
+ return addToken(new UuidEvaluator(), "uuid");
+ }
+ case NOW: {
+ return addToken(new NowEvaluator(), "now");
+ }
+ case TO_LITERAL: {
+ final Evaluator<?> argEvaluator = buildEvaluator(tree.getChild(0));
+ return addToken(new ToLiteralEvaluator(argEvaluator), "toLiteral");
+ }
+ case IP: {
+ try {
+ return addToken(new IPEvaluator(), "ip");
+ } catch (final UnknownHostException e) {
+ throw new AttributeExpressionLanguageException(e);
+ }
+ }
+ case HOSTNAME: {
+ if (tree.getChildCount() == 0) {
+ try {
+ return addToken(new HostnameEvaluator(false), "hostname");
+ } catch (final UnknownHostException e) {
+ throw new AttributeExpressionLanguageException(e);
+ }
+ } else if (tree.getChildCount() == 1) {
+ final Tree childTree = tree.getChild(0);
+ try {
+ switch (childTree.getType()) {
+ case TRUE:
+ return addToken(new HostnameEvaluator(true), "hostname");
+ case FALSE:
+ return addToken(new HostnameEvaluator(false), "hostname");
+ default:
+ throw new AttributeExpressionLanguageParsingException("Call to hostname() must take 0 or 1 (boolean) parameter");
+ }
+ } catch (final UnknownHostException e) {
+ throw new AttributeExpressionLanguageException(e);
+ }
+ } else {
+ throw new AttributeExpressionLanguageParsingException("Call to hostname() must take 0 or 1 (boolean) parameter");
+ }
+ }
+ case NEXT_INT: {
+ return addToken(new OneUpSequenceEvaluator(), "nextInt");
+ }
+ case RANDOM: {
+ return addToken(new RandomNumberGeneratorEvaluator(), "random");
+ }
+ case MATH: {
+ if (tree.getChildCount() == 1) {
+ return addToken(new MathEvaluator(null, toStringEvaluator(buildEvaluator(tree.getChild(0))), null), "math");
+ } else {
+ throw new AttributeExpressionLanguageParsingException("Call to math() as the subject must take exactly 1 parameter");
+ }
+ }
+ case GET_STATE_VALUE: {
+ final Tree childTree = tree.getChild(0);
+ final Evaluator<?> argEvaluator = buildEvaluator(childTree);
+ final Evaluator<String> stringEvaluator = toStringEvaluator(argEvaluator);
+ final GetStateVariableEvaluator eval = new GetStateVariableEvaluator(stringEvaluator);
+ evaluators.add(eval);
+ return eval;
+ }
+ default:
+ throw new AttributeExpressionLanguageParsingException("Unexpected token: " + tree.toString());
+ }
+ }
+
+ private <T> Evaluator<T> addToken(final Evaluator<T> evaluator, final String token) {
+ evaluator.setToken(token);
+ evaluators.add(evaluator);
+ return evaluator;
+ }
+
+
+ private Evaluator<String> newStringLiteralEvaluator(final String literalValue) {
+ if (literalValue == null || literalValue.length() < 2) {
+ return addToken(new StringLiteralEvaluator(literalValue), literalValue);
+ }
+
+ final List<Range> ranges = Query.extractExpressionRanges(literalValue);
+ if (ranges.isEmpty()) {
+ return addToken(new StringLiteralEvaluator(literalValue), literalValue);
+ }
+
+ final List<Evaluator<?>> evaluators = new ArrayList<>();
+
+ int lastIndex = 0;
+ for (final Range range : ranges) {
+ if (range.getStart() > lastIndex) {
+ evaluators.add(newStringLiteralEvaluator(literalValue.substring(lastIndex, range.getStart())));
+ }
+
+ final String treeText = literalValue.substring(range.getStart(), range.getEnd() + 1);
+ evaluators.add(buildEvaluator(compileTree(treeText)));
+ lastIndex = range.getEnd() + 1;
+ }
+
+ final Range lastRange = ranges.get(ranges.size() - 1);
+ if (lastRange.getEnd() + 1 < literalValue.length()) {
+ final String treeText = literalValue.substring(lastRange.getEnd() + 1);
+ evaluators.add(newStringLiteralEvaluator(treeText));
+ }
+
+ if (evaluators.size() == 1) {
+ return toStringEvaluator(evaluators.get(0));
+ }
+
+ Evaluator<String> lastEvaluator = toStringEvaluator(evaluators.get(0));
+ for (int i = 1; i < evaluators.size(); i++) {
+ lastEvaluator = new AppendEvaluator(lastEvaluator, toStringEvaluator(evaluators.get(i)));
+ }
+
+ this.evaluators.addAll(evaluators);
+ return lastEvaluator;
+ }
+
+
+ private Evaluator<Boolean> buildBooleanEvaluator(final Tree tree) {
+ switch (tree.getType()) {
+ case TRUE:
+ return addToken(new BooleanLiteralEvaluator(true), "true");
+ case FALSE:
+ return addToken(new BooleanLiteralEvaluator(false), "true");
+ }
+ throw new AttributeExpressionLanguageParsingException("Cannot build Boolean evaluator from tree " + tree.toString());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java
deleted file mode 100644
index a0695a9..0000000
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java
+++ /dev/null
@@ -1,45 +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.attribute.expression.language.evaluation.functions;
-
-import java.util.Map;
-
-import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
-import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
-import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
-
-public class AttributeEvaluator extends StringEvaluator {
-
- private final Evaluator<String> nameEvaluator;
-
- public AttributeEvaluator(final Evaluator<String> nameEvaluator) {
- this.nameEvaluator = nameEvaluator;
- }
-
- @Override
- public QueryResult<String> evaluate(final Map<String, String> attributes) {
- final String nameValue = nameEvaluator.evaluate(attributes).getValue();
- final String attributeValue = attributes.get(nameValue);
- return new StringQueryResult(attributeValue);
- }
-
- @Override
- public Evaluator<?> getSubjectEvaluator() {
- return null;
- }
-}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
index ccf9d19..8c4213b 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java
@@ -70,4 +70,8 @@ public class AllAttributesEvaluator extends BooleanEvaluator implements Iteratin
public Evaluator<?> getLogicEvaluator() {
return booleanEvaluator;
}
+
+ public MultiAttributeEvaluator getVariableIteratingEvaluator() {
+ return multiAttributeEvaluator;
+ }
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
index 6e8d485..eac571b 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java
@@ -70,4 +70,8 @@ public class AnyAttributeEvaluator extends BooleanEvaluator implements Iterating
public Evaluator<Boolean> getLogicEvaluator() {
return booleanEvaluator;
}
+
+ public MultiAttributeEvaluator getVariableIteratingEvaluator() {
+ return multiAttributeEvaluator;
+ }
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AttributeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AttributeEvaluator.java
new file mode 100644
index 0000000..0da063e
--- /dev/null
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AttributeEvaluator.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.attribute.expression.language.evaluation.selection;
+
+import java.util.Map;
+
+import org.apache.nifi.attribute.expression.language.evaluation.Evaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
+import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator;
+import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult;
+
+public class AttributeEvaluator extends StringEvaluator {
+
+ private final Evaluator<String> nameEvaluator;
+
+ public AttributeEvaluator(final Evaluator<String> nameEvaluator) {
+ this.nameEvaluator = nameEvaluator;
+ }
+
+ @Override
+ public QueryResult<String> evaluate(final Map<String, String> attributes) {
+ final String nameValue = nameEvaluator.evaluate(attributes).getValue();
+ final String attributeValue = attributes.get(nameValue);
+ return new StringQueryResult(attributeValue);
+ }
+
+ @Override
+ public Evaluator<?> getSubjectEvaluator() {
+ return null;
+ }
+
+ public Evaluator<String> getNameEvaluator() {
+ return nameEvaluator;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java
index e007a56..5f182c7 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MappingEvaluator.java
@@ -68,4 +68,8 @@ public class MappingEvaluator<T> implements Evaluator<T> {
public void setToken(final String token) {
this.token = token;
}
+
+ public MultiAttributeEvaluator getVariableIteratingEvaluator() {
+ return multiAttributeEvaluator;
+ }
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
index 509d7dd..cff9185 100644
--- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
+++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java
@@ -66,4 +66,8 @@ public class MultiNamedAttributeEvaluator extends MultiAttributeEvaluator {
public Evaluator<?> getLogicEvaluator() {
return this;
}
+
+ public List<String> getAttributeNames() {
+ return attributeNames;
+ }
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
index 5946aa5..c4113c0 100644
--- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
+++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java
@@ -19,11 +19,11 @@ package org.apache.nifi.attribute.expression.language;
import static java.lang.Double.NEGATIVE_INFINITY;
import static java.lang.Double.NaN;
import static java.lang.Double.POSITIVE_INFINITY;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThan;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.Assert.fail;
import java.io.BufferedInputStream;
@@ -40,6 +40,7 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
+import org.antlr.runtime.tree.Tree;
import org.apache.nifi.attribute.expression.language.Query.Range;
import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult;
import org.apache.nifi.attribute.expression.language.evaluation.QueryResult;
@@ -47,14 +48,10 @@ import org.apache.nifi.attribute.expression.language.exception.AttributeExpressi
import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException;
import org.apache.nifi.expression.AttributeExpression.ResultType;
import org.apache.nifi.flowfile.FlowFile;
-import org.antlr.runtime.tree.Tree;
-
import org.apache.nifi.registry.VariableRegistry;
import org.junit.Assert;
-
import org.junit.Ignore;
import org.junit.Test;
-
import org.mockito.Mockito;
public class TestQuery {
@@ -304,7 +301,7 @@ public class TestQuery {
@Test(expected = AttributeExpressionLanguageException.class)
public void testCannotCombineWithNonReducingFunction() {
- Query.compileTree("${allAttributes( 'a.1' ):plus(1)}");
+ Query.compile("${allAttributes( 'a.1' ):plus(1)}");
}
@Test
@@ -357,6 +354,7 @@ public class TestQuery {
assertEquals("Val", evaluateQueryForEscape("${attr:replaceAll(\"My (Val)ue{1,2}\", '$1')}", attributes));
}
+ @SuppressWarnings("unchecked")
private String evaluateQueryForEscape(final String queryString, final Map<String, String> attributes) {
final FlowFile mockFlowFile = Mockito.mock(FlowFile.class);
Mockito.when(mockFlowFile.getAttributes()).thenReturn(attributes);
@@ -639,7 +637,7 @@ public class TestQuery {
final String query = "${ abc:equals('abc'):or( \n\t${xx:isNull()}\n) }";
assertEquals(ResultType.BOOLEAN, Query.getResultType(query));
Query.validateExpression(query, false);
- assertEquals("true", Query.evaluateExpressions(query, Collections.EMPTY_MAP));
+ assertEquals("true", Query.evaluateExpressions(query, Collections.emptyMap()));
}
@Test
@@ -1675,25 +1673,25 @@ public class TestQuery {
verifyEquals("${string:escapeHtml4()}", attributes, "special ♣");
}
- @Test
- public void testUnescapeFunctions() {
- final Map<String, String> attributes = new HashMap<>();
+ @Test
+ public void testUnescapeFunctions() {
+ final Map<String, String> attributes = new HashMap<>();
- attributes.put("string", "making air \\\"QUOTES\\\".");
- verifyEquals("${string:unescapeJson()}", attributes, "making air \"QUOTES\".");
+ attributes.put("string", "making air \\\"QUOTES\\\".");
+ verifyEquals("${string:unescapeJson()}", attributes, "making air \"QUOTES\".");
- attributes.put("string", "M & M");
- verifyEquals("${string:unescapeXml()}", attributes, "M & M");
+ attributes.put("string", "M & M");
+ verifyEquals("${string:unescapeXml()}", attributes, "M & M");
- attributes.put("string", "\"making air \"\"QUOTES\"\".\"");
- verifyEquals("${string:unescapeCsv()}", attributes, "making air \"QUOTES\".");
+ attributes.put("string", "\"making air \"\"QUOTES\"\".\"");
+ verifyEquals("${string:unescapeCsv()}", attributes, "making air \"QUOTES\".");
- attributes.put("string", "special ¡");
- verifyEquals("${string:unescapeHtml3()}", attributes, "special ¡");
+ attributes.put("string", "special ¡");
+ verifyEquals("${string:unescapeHtml3()}", attributes, "special ¡");
- attributes.put("string", "special ♣");
- verifyEquals("${string:unescapeHtml4()}", attributes, "special ♣");
- }
+ attributes.put("string", "special ♣");
+ verifyEquals("${string:unescapeHtml4()}", attributes, "special ♣");
+ }
@Test
public void testIfElse() {
@@ -1709,9 +1707,9 @@ public class TestQuery {
verifyEquals("${attr2:isNull():ifElse('a', 'b')}", attributes, "a");
verifyEquals("${literal(true):ifElse('a', 'b')}", attributes, "a");
verifyEquals("${literal(true):ifElse(false, 'b')}", attributes, "false");
-
}
+
private void verifyEquals(final String expression, final Map<String, String> attributes, final Object expectedResult) {
verifyEquals(expression,attributes, null, expectedResult);
}
[6/9] nifi git commit: NIFI-4224: - Initial implementation of Process
Group level Variable Registry - Updated to incorporate PR Feedback - Changed
log message because slf4j-simple apparently has a memory leak;
passing a String instead of passing in the C
Posted by mc...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
index 5acba8d..ea7cb96 100644
--- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
+++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java
@@ -17,9 +17,13 @@
package org.apache.nifi.attribute.expression.language;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.Map;
+import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.junit.Ignore;
@@ -46,6 +50,7 @@ public class TestStandardPreparedQuery {
}
@Test
+ @Ignore("Intended for manual performance testing; should not be run in an automated environment")
public void test10MIterations() {
final Map<String, String> attrs = new HashMap<>();
attrs.put("xx", "world");
@@ -84,6 +89,42 @@ public class TestStandardPreparedQuery {
}
+ @Test
+ public void testVariableImpacted() {
+ final Set<String> attr = new HashSet<>();
+ attr.add("attr");
+
+ final Set<String> attr2 = new HashSet<>();
+ attr2.add("attr");
+ attr2.add("attr2");
+
+ final Set<String> abc = new HashSet<>();
+ abc.add("a");
+ abc.add("b");
+ abc.add("c");
+
+ assertTrue(Query.prepare("${attr}").getVariableImpact().isImpacted("attr"));
+ assertFalse(Query.prepare("${attr}").getVariableImpact().isImpacted("attr2"));
+ assertTrue(Query.prepare("${attr:trim():toUpper():equals('abc')}").getVariableImpact().isImpacted("attr"));
+
+ assertFalse(Query.prepare("${anyAttribute('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("attr"));
+ assertTrue(Query.prepare("${anyAttribute('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("a"));
+ assertTrue(Query.prepare("${anyAttribute('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("b"));
+ assertTrue(Query.prepare("${anyAttribute('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("c"));
+
+ assertFalse(Query.prepare("${allAttributes('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("attr"));
+ assertTrue(Query.prepare("${allAttributes('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("a"));
+ assertTrue(Query.prepare("${allAttributes('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("b"));
+ assertTrue(Query.prepare("${allAttributes('a', 'b', 'c'):equals('hello')}").getVariableImpact().isImpacted("c"));
+
+ assertTrue(Query.prepare("${attr:equals('${attr2}')}").getVariableImpact().isImpacted("attr"));
+ assertTrue(Query.prepare("${attr:equals('${attr2}')}").getVariableImpact().isImpacted("attr2"));
+ assertFalse(Query.prepare("${attr:equals('${attr2}')}").getVariableImpact().isImpacted("attr3"));
+
+ assertTrue(Query.prepare("${allMatchingAttributes('a.*'):equals('hello')}").getVariableImpact().isImpacted("attr"));
+ assertTrue(Query.prepare("${anyMatchingAttribute('a.*'):equals('hello')}").getVariableImpact().isImpacted("attr"));
+ }
+
private String evaluate(final String query, final Map<String, String> attrs) {
final String evaluated = ((StandardPreparedQuery) Query.prepare(query)).evaluateExpressions(attrs, null);
return evaluated;
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestValueLookup.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestValueLookup.java b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestValueLookup.java
index ec8ba66..1010ac7 100644
--- a/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestValueLookup.java
+++ b/nifi-commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestValueLookup.java
@@ -25,12 +25,10 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertEquals;
-/**
- *
- */
public class TestValueLookup {
@Test
+ @SuppressWarnings("unchecked")
public void testCreateCustomVariableRegistry() {
final VariableRegistry variableRegistry = VariableRegistry.ENVIRONMENT_SYSTEM_REGISTRY;
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AffectedComponentDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AffectedComponentDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AffectedComponentDTO.java
new file mode 100644
index 0000000..5d631ed
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/AffectedComponentDTO.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.dto;
+
+import javax.xml.bind.annotation.XmlType;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "affectedComponent")
+public class AffectedComponentDTO {
+ public static final String COMPONENT_TYPE_PROCESSOR = "PROCESSOR";
+ public static final String COMPONENT_TYPE_CONTROLLER_SERVICE = "CONTROLLER_SERVICE";
+
+ private String parentGroupId;
+ private String componentId;
+ private String componentType;
+
+ @ApiModelProperty("The UUID of the Process Group that this component is in")
+ public String getParentGroupId() {
+ return parentGroupId;
+ }
+
+ public void setParentGroupId(final String parentGroupId) {
+ this.parentGroupId = parentGroupId;
+ }
+
+ @ApiModelProperty("The UUID of this component")
+ public String getComponentId() {
+ return componentId;
+ }
+
+ public void setComponentId(final String componentId) {
+ this.componentId = componentId;
+ }
+
+ @ApiModelProperty(value = "The type of this component", allowableValues = COMPONENT_TYPE_PROCESSOR + "," + COMPONENT_TYPE_CONTROLLER_SERVICE)
+ public String getComponentType() {
+ return componentType;
+ }
+
+ public void setComponentType(final String componentType) {
+ this.componentType = componentType;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
index 42d605d..f4b6f31 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java
@@ -17,6 +17,9 @@
package org.apache.nifi.web.api.dto;
import com.wordnik.swagger.annotations.ApiModelProperty;
+
+import java.util.Map;
+
import javax.xml.bind.annotation.XmlType;
/**
@@ -27,6 +30,7 @@ public class ProcessGroupDTO extends ComponentDTO {
private String name;
private String comments;
+ private Map<String, String> variables;
private Integer runningCount;
private Integer stoppedCount;
@@ -200,4 +204,16 @@ public class ProcessGroupDTO extends ComponentDTO {
this.inactiveRemotePortCount = inactiveRemotePortCount;
}
+
+ @ApiModelProperty(value = "The variables that are configured for the Process Group. Note that this map contains only "
+ + "those variables that are defined on this Process Group and not any variables that are defined in the parent "
+ + "Process Group, etc. I.e., this Map will not contain all variables that are accessible by components in this "
+ + "Process Group by rather only the variables that are defined for this Process Group itself.", readOnly = true)
+ public Map<String, String> getVariables() {
+ return variables;
+ }
+
+ public void setVariables(final Map<String, String> variables) {
+ this.variables = variables;
+ }
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableDTO.java
new file mode 100644
index 0000000..c686316
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableDTO.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.dto;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import javax.xml.bind.annotation.XmlType;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "variable")
+public class VariableDTO {
+ private String name;
+ private String value;
+ private String processGroupId;
+ private Set<AffectedComponentDTO> affectedComponents = new HashSet<>();
+
+ @ApiModelProperty("The name of the variable")
+ public String getName() {
+ return name;
+ }
+
+ public void setName(String name) {
+ this.name = name;
+ }
+
+ @ApiModelProperty("The value of the variable")
+ public String getValue() {
+ return value;
+ }
+
+ public void setValue(String value) {
+ this.value = value;
+ }
+
+ @ApiModelProperty(value = "The ID of the Process Group where this Variable is defined", readOnly = true)
+ public String getProcessGroupId() {
+ return processGroupId;
+ }
+
+ public void setProcessGroupId(String groupId) {
+ this.processGroupId = groupId;
+ }
+
+ @ApiModelProperty(value = "A set of all components that will be affected if the value of this variable is changed", readOnly = true)
+ public Set<AffectedComponentDTO> getAffectedComponents() {
+ return affectedComponents;
+ }
+
+ public void setAffectedComponents(Set<AffectedComponentDTO> affectedComponents) {
+ this.affectedComponents = affectedComponents;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryDTO.java
new file mode 100644
index 0000000..c106a9a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryDTO.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.dto;
+
+import java.util.Set;
+
+import javax.xml.bind.annotation.XmlType;
+
+import org.apache.nifi.web.api.entity.VariableEntity;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "variableRegistry")
+public class VariableRegistryDTO {
+ private Set<VariableEntity> variables;
+ private String groupId;
+
+ public void setVariables(final Set<VariableEntity> variables) {
+ this.variables = variables;
+ }
+
+ @ApiModelProperty("The variables that are available in this Variable Registry")
+ public Set<VariableEntity> getVariables() {
+ return variables;
+ }
+
+ public void setProcessGroupId(final String groupId) {
+ this.groupId = groupId;
+ }
+
+ @ApiModelProperty("The UUID of the Process Group that this Variable Registry belongs to")
+ public String getProcessGroupId() {
+ return groupId;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateRequestDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateRequestDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateRequestDTO.java
new file mode 100644
index 0000000..06a0dc2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateRequestDTO.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.web.api.dto;
+
+import java.util.Date;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlType;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+
+import org.apache.nifi.web.api.dto.util.TimestampAdapter;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "variableRegistryUpdateRequest")
+public class VariableRegistryUpdateRequestDTO {
+ private String requestId;
+ private String processGroupId;
+ private String uri;
+ private Date submissionTime = new Date();
+ private Date lastUpdated = new Date();
+ private boolean complete = false;
+ private String failureReason;
+ private List<VariableRegistryUpdateStepDTO> updateSteps;
+
+
+ @ApiModelProperty("The unique ID of the Process Group that the variable registry belongs to")
+ public String getProcessGroupId() {
+ return processGroupId;
+ }
+
+ public void setProcessGroupId(String processGroupId) {
+ this.processGroupId = processGroupId;
+ }
+
+ @ApiModelProperty(value = "The unique ID of this request.", readOnly = true)
+ public String getRequestId() {
+ return requestId;
+ }
+
+ public void setRequestId(String requestId) {
+ this.requestId = requestId;
+ }
+
+ @ApiModelProperty(value = "The URI for future requests to this drop request.", readOnly = true)
+ public String getUri() {
+ return uri;
+ }
+
+ public void setUri(String uri) {
+ this.uri = uri;
+ }
+
+ @XmlJavaTypeAdapter(TimestampAdapter.class)
+ @ApiModelProperty(value = "The time at which this request was submitted.", dataType = "string", readOnly = true)
+ public Date getSubmissionTime() {
+ return submissionTime;
+ }
+
+ public void setSubmissionTime(Date submissionTime) {
+ this.submissionTime = submissionTime;
+ }
+
+ @XmlJavaTypeAdapter(TimestampAdapter.class)
+ @ApiModelProperty(value = "The last time this request was updated.", dataType = "string", readOnly = true)
+ public Date getLastUpdated() {
+ return lastUpdated;
+ }
+
+ public void setLastUpdated(Date lastUpdated) {
+ this.lastUpdated = lastUpdated;
+ }
+
+ @ApiModelProperty(value = "The steps that are required in order to complete the request, along with the status of each", readOnly = true)
+ public List<VariableRegistryUpdateStepDTO> getUpdateSteps() {
+ return updateSteps;
+ }
+
+ public void setUpdateSteps(List<VariableRegistryUpdateStepDTO> updateSteps) {
+ this.updateSteps = updateSteps;
+ }
+
+ @ApiModelProperty(value = "Whether or not this request has completed", readOnly = true)
+ public boolean isComplete() {
+ return complete;
+ }
+
+ public void setComplete(boolean complete) {
+ this.complete = complete;
+ }
+
+ @ApiModelProperty(value = "An explanation of why this request failed, or null if this request has not failed", readOnly = true)
+ public String getFailureReason() {
+ return failureReason;
+ }
+
+ public void setFailureReason(String reason) {
+ this.failureReason = reason;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java
new file mode 100644
index 0000000..e1c8cee
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/VariableRegistryUpdateStepDTO.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.dto;
+
+import javax.xml.bind.annotation.XmlType;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "varaibleRegistryUpdateStep")
+public class VariableRegistryUpdateStepDTO {
+ private String description;
+ private boolean complete;
+ private String failureReason;
+
+ public VariableRegistryUpdateStepDTO() {
+ }
+
+ @ApiModelProperty(value = "Explanation of what happens in this step", readOnly = true)
+ public String getDescription() {
+ return description;
+ }
+
+ public void setDescription(String description) {
+ this.description = description;
+ }
+
+ @ApiModelProperty(value = "Whether or not this step has completed", readOnly = true)
+ public boolean isComplete() {
+ return complete;
+ }
+
+ public void setComplete(boolean complete) {
+ this.complete = complete;
+ }
+
+ @ApiModelProperty(value = "An explanation of why this step failed, or null if this step did not fail", readOnly = true)
+ public String getFailureReason() {
+ return failureReason;
+ }
+
+ public void setFailureReason(String reason) {
+ this.failureReason = reason;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ActivateControllerServicesEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ActivateControllerServicesEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ActivateControllerServicesEntity.java
new file mode 100644
index 0000000..a58f821
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ActivateControllerServicesEntity.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.web.api.entity;
+
+import java.util.Map;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.nifi.web.api.dto.RevisionDTO;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlRootElement(name = "activateControllerServicesEntity")
+public class ActivateControllerServicesEntity extends Entity {
+ public static final String STATE_ENABLED = "ENABLED";
+ public static final String STATE_DISABLED = "DISABLED";
+
+ private String id;
+ private String state;
+ private Map<String, RevisionDTO> components;
+
+ @ApiModelProperty("The id of the ProcessGroup")
+ public String getId() {
+ return id;
+ }
+
+ public void setId(String id) {
+ this.id = id;
+ }
+
+ /**
+ * @return The desired state of the descendant components. Possible states are 'RUNNING' and 'STOPPED'
+ */
+ @ApiModelProperty(value = "The desired state of the descendant components",
+ allowableValues = STATE_ENABLED + ", " + STATE_DISABLED)
+ public String getState() {
+ return state;
+ }
+
+ public void setState(String state) {
+ this.state = state;
+ }
+
+ @ApiModelProperty("Optional services to schedule. If not specified, all authorized descendant controller services will be used.")
+ public Map<String, RevisionDTO> getComponents() {
+ return components;
+ }
+
+ public void setComponents(Map<String, RevisionDTO> components) {
+ this.components = components;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ScheduleComponentsEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ScheduleComponentsEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ScheduleComponentsEntity.java
index 9aeef40..3e4c9a4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ScheduleComponentsEntity.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ScheduleComponentsEntity.java
@@ -16,17 +16,21 @@
*/
package org.apache.nifi.web.api.entity;
-import com.wordnik.swagger.annotations.ApiModelProperty;
-import org.apache.nifi.web.api.dto.RevisionDTO;
+import java.util.Map;
import javax.xml.bind.annotation.XmlRootElement;
-import java.util.Map;
+
+import org.apache.nifi.web.api.dto.RevisionDTO;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
/**
* A serialized representation of this class can be placed in the entity body of a request to the API.
*/
@XmlRootElement(name = "scheduleComponentEntity")
public class ScheduleComponentsEntity extends Entity {
+ public static final String STATE_RUNNING = "RUNNING";
+ public static final String STATE_STOPPED = "STOPPED";
private String id;
private String state;
@@ -51,7 +55,7 @@ public class ScheduleComponentsEntity extends Entity {
*/
@ApiModelProperty(
value = "The desired state of the descendant components",
- allowableValues = "RUNNING, STOPPED"
+ allowableValues = STATE_RUNNING + ", " + STATE_STOPPED
)
public String getState() {
return state;
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableEntity.java
new file mode 100644
index 0000000..06f6fcf
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableEntity.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.entity;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.nifi.web.api.dto.VariableDTO;
+import org.apache.nifi.web.api.dto.WritablePermission;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlRootElement(name = "variableEntity")
+public class VariableEntity extends Entity implements WritablePermission {
+ private VariableDTO variable;
+ private Boolean canWrite;
+
+ @Override
+ @ApiModelProperty(value = "Indicates whether the user can write a given resource.", readOnly = true)
+ public Boolean getCanWrite() {
+ return canWrite;
+ }
+
+ @Override
+ public void setCanWrite(Boolean canWrite) {
+ this.canWrite = canWrite;
+ }
+
+ @ApiModelProperty("The variable information")
+ public VariableDTO getVariable() {
+ return variable;
+ }
+
+ public void setVariable(VariableDTO variable) {
+ this.variable = variable;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryEntity.java
new file mode 100644
index 0000000..d876453
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryEntity.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.entity;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.dto.VariableRegistryDTO;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlRootElement(name = "variableRegistryEntity")
+public class VariableRegistryEntity extends Entity {
+ private RevisionDTO groupRevision;
+ private VariableRegistryDTO variableRegistry;
+
+
+ @ApiModelProperty("The Variable Registry.")
+ public VariableRegistryDTO getVariableRegistry() {
+ return variableRegistry;
+ }
+
+ public void setVariableRegistry(VariableRegistryDTO variableRegistry) {
+ this.variableRegistry = variableRegistry;
+ }
+
+ @ApiModelProperty("The revision of the Process Group that the Variable Registry belongs to")
+ public RevisionDTO getProcessGroupRevision() {
+ return groupRevision;
+ }
+
+ public void setProcessGroupRevision(RevisionDTO revision) {
+ this.groupRevision = revision;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryUpdateRequestEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryUpdateRequestEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryUpdateRequestEntity.java
new file mode 100644
index 0000000..77257af
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/VariableRegistryUpdateRequestEntity.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.web.api.entity;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.nifi.web.api.dto.RevisionDTO;
+import org.apache.nifi.web.api.dto.VariableRegistryUpdateRequestDTO;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlRootElement(name = "variableRegistryUpdateRequestEntity")
+public class VariableRegistryUpdateRequestEntity extends Entity {
+ private VariableRegistryUpdateRequestDTO requestDto;
+ private RevisionDTO processGroupRevision;
+
+ @ApiModelProperty("The revision for the Process Group that owns this variable registry.")
+ public RevisionDTO getProcessGroupRevision() {
+ return processGroupRevision;
+ }
+
+ public void setProcessGroupRevision(RevisionDTO revision) {
+ this.processGroupRevision = revision;
+ }
+
+ @ApiModelProperty("The Variable Registry Update Request")
+ public VariableRegistryUpdateRequestDTO getRequestDto() {
+ return requestDto;
+ }
+
+ public void setRequestDto(VariableRegistryUpdateRequestDTO requestDto) {
+ this.requestDto = requestDto;
+ }
+}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java
index 2fc55a4..c102746 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/http/StandardHttpResponseMapper.java
@@ -16,6 +16,16 @@
*/
package org.apache.nifi.cluster.coordination.http;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import javax.ws.rs.core.StreamingOutput;
+
import org.apache.nifi.cluster.coordination.http.endpoints.AccessPolicyEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.BulletinBoardEndpointMerger;
import org.apache.nifi.cluster.coordination.http.endpoints.ComponentStateEndpointMerger;
@@ -77,15 +87,6 @@ import org.apache.nifi.util.NiFiProperties;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import javax.ws.rs.core.StreamingOutput;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-
public class StandardHttpResponseMapper implements HttpResponseMapper {
private Logger logger = LoggerFactory.getLogger(StandardHttpResponseMapper.class);
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
index 45c6ead..ff5a8af 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java
@@ -16,24 +16,6 @@
*/
package org.apache.nifi.controller;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
-import org.apache.nifi.bundle.Bundle;
-import org.apache.nifi.bundle.BundleCoordinate;
-import org.apache.nifi.components.ConfigurableComponent;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.controller.service.ControllerServiceNode;
-import org.apache.nifi.controller.service.ControllerServiceProvider;
-import org.apache.nifi.nar.ExtensionManager;
-import org.apache.nifi.nar.NarCloseable;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.util.CharacterFilterUtils;
-import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
import java.net.MalformedURLException;
import java.net.URL;
import java.util.ArrayList;
@@ -53,6 +35,24 @@ import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Consumer;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.attribute.expression.language.StandardPropertyValue;
+import org.apache.nifi.bundle.Bundle;
+import org.apache.nifi.bundle.BundleCoordinate;
+import org.apache.nifi.components.ConfigurableComponent;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.service.ControllerServiceNode;
+import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.nar.ExtensionManager;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.util.CharacterFilterUtils;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
public abstract class AbstractConfiguredComponent implements ConfigurableComponent, ConfiguredComponent {
private static final Logger logger = LoggerFactory.getLogger(AbstractConfiguredComponent.class);
@@ -64,7 +64,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
private final AtomicReference<ValidationContext> validationContext = new AtomicReference<>();
private final String componentType;
private final String componentCanonicalClass;
- private final VariableRegistry variableRegistry;
+ private final ComponentVariableRegistry variableRegistry;
private final ReloadComponent reloadComponent;
private final AtomicBoolean isExtensionMissing;
@@ -74,7 +74,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
public AbstractConfiguredComponent(final String id,
final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
- final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry,
+ final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
this.id = id;
this.validationContextFactory = validationContextFactory;
@@ -541,7 +541,7 @@ public abstract class AbstractConfiguredComponent implements ConfigurableCompone
}
}
- protected VariableRegistry getVariableRegistry() {
+ public ComponentVariableRegistry getVariableRegistry() {
return this.variableRegistry;
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
index 069e6ce..940ac21 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java
@@ -29,6 +29,7 @@ import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.registry.ComponentVariableRegistry;
import java.net.URL;
import java.util.Collection;
@@ -38,6 +39,7 @@ import java.util.Set;
public interface ConfiguredComponent extends ComponentAuthorizable {
+ @Override
public String getIdentifier();
public String getName();
@@ -99,6 +101,11 @@ public interface ConfiguredComponent extends ComponentAuthorizable {
*/
boolean isDeprecated();
+ /**
+ * @return the variable registry for this component
+ */
+ ComponentVariableRegistry getVariableRegistry();
+
@Override
default AuthorizationResult checkAuthorization(Authorizer authorizer, RequestAction action, NiFiUser user, Map<String, String> resourceContext) {
// if this is a modification request and the reporting task is restricted ensure the user has elevated privileges. if this
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java
index 5bb8981..c6f30b5 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java
@@ -18,6 +18,7 @@ package org.apache.nifi.controller;
import java.util.List;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Funnel;
@@ -41,7 +42,7 @@ public interface ProcessScheduler {
* @param procNode to start
* @throws IllegalStateException if the Processor is disabled
*/
- void startProcessor(ProcessorNode procNode);
+ Future<Void> startProcessor(ProcessorNode procNode);
/**
* Stops scheduling the given processor to run and invokes all methods on
@@ -52,7 +53,7 @@ public interface ProcessScheduler {
*
* @param procNode to stop
*/
- void stopProcessor(ProcessorNode procNode);
+ Future<Void> stopProcessor(ProcessorNode procNode);
/**
* Starts scheduling the given Port to run. If the Port is already scheduled
@@ -169,12 +170,12 @@ public interface ProcessScheduler {
* Disables all of the given Controller Services in the order provided by the List
* @param services the controller services to disable
*/
- void disableControllerServices(List<ControllerServiceNode> services);
+ CompletableFuture<Void> disableControllerServices(List<ControllerServiceNode> services);
/**
* Disables the Controller Service so that it can be updated
*
* @param service to disable
*/
- void disableControllerService(ControllerServiceNode service);
+ CompletableFuture<Void> disableControllerService(ControllerServiceNode service);
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
index aac5e52..ba2e59b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java
@@ -16,6 +16,13 @@
*/
package org.apache.nifi.controller;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.controller.scheduling.ScheduleState;
@@ -26,18 +33,12 @@ import org.apache.nifi.logging.LogLevel;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.ComponentVariableRegistry;
import org.apache.nifi.scheduling.ExecutionNode;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
public abstract class ProcessorNode extends AbstractConfiguredComponent implements Connectable {
private static final Logger logger = LoggerFactory.getLogger(ProcessorNode.class);
@@ -46,7 +47,7 @@ public abstract class ProcessorNode extends AbstractConfiguredComponent implemen
public ProcessorNode(final String id,
final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider,
- final String componentType, final String componentCanonicalClass, final VariableRegistry variableRegistry,
+ final String componentType, final String componentCanonicalClass, final ComponentVariableRegistry variableRegistry,
final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
super(id, validationContextFactory, serviceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
this.scheduledState = new AtomicReference<>(ScheduledState.STOPPED);
@@ -185,7 +186,7 @@ public abstract class ProcessorNode extends AbstractConfiguredComponent implemen
* the ScheduleState that can be used to ensure that the running state (STOPPED, RUNNING, etc.)
* as well as the active thread counts are kept in sync
*/
- public abstract <T extends ProcessContext & ControllerServiceLookup> void stop(ScheduledExecutorService scheduler,
+ public abstract <T extends ProcessContext & ControllerServiceLookup> CompletableFuture<Void> stop(ScheduledExecutorService scheduler,
T processContext, SchedulingAgent schedulingAgent, ScheduleState scheduleState);
/**
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
index faf530f..3dd1076 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java
@@ -108,7 +108,7 @@ public interface ControllerServiceNode extends ConfiguredComponent {
* implementation of {@link ScheduledExecutorService} used to
* initiate service disabling task
*/
- void disable(ScheduledExecutorService scheduler);
+ CompletableFuture<Void> disable(ScheduledExecutorService scheduler);
/**
* @return the ControllerServiceReference that describes which components are referencing this Controller Service
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
index f7ba5e5..010ecdf 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
@@ -19,7 +19,7 @@ package org.apache.nifi.controller.service;
import java.net.URL;
import java.util.Collection;
import java.util.Set;
-import java.util.concurrent.Future;
+import java.util.concurrent.CompletableFuture;
import org.apache.nifi.annotation.lifecycle.OnAdded;
import org.apache.nifi.bundle.BundleCoordinate;
@@ -72,7 +72,7 @@ public interface ControllerServiceProvider extends ControllerServiceLookup {
* @param serviceNode the service node
* @return a Future that can be used to wait for the service to finish being enabled.
*/
- Future<Void> enableControllerService(ControllerServiceNode serviceNode);
+ CompletableFuture<Void> enableControllerService(ControllerServiceNode serviceNode);
/**
* Enables the collection of services. If a service in this collection
@@ -90,7 +90,7 @@ public interface ControllerServiceProvider extends ControllerServiceLookup {
*
* @param serviceNode the node
*/
- void disableControllerService(ControllerServiceNode serviceNode);
+ CompletableFuture<Void> disableControllerService(ControllerServiceNode serviceNode);
/**
* @return a Set of all Controller Services that exist for this service
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java
index df18c62..13c5844 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java
@@ -16,6 +16,7 @@
*/
package org.apache.nifi.controller.service;
+import java.util.List;
import java.util.Set;
import org.apache.nifi.controller.ConfiguredComponent;
@@ -43,4 +44,13 @@ public interface ControllerServiceReference {
* Controller Services)
*/
Set<ConfiguredComponent> getActiveReferences();
+
+ /**
+ * Returns a List of all components that reference this Controller Service (recursively) that
+ * are of the given type
+ *
+ * @param componentType the type of component that is desirable
+ * @return a List of all components that reference this Controller Service that are of the given type
+ */
+ <T> List<T> findRecursiveReferences(Class<T> componentType);
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
index 122e454..bf789f7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java
@@ -16,12 +16,20 @@
*/
package org.apache.nifi.groups;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+
import org.apache.nifi.authorization.resource.ComponentAuthorizable;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.connectable.Funnel;
import org.apache.nifi.connectable.Port;
import org.apache.nifi.connectable.Positionable;
+import org.apache.nifi.controller.ConfiguredComponent;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.ScheduledState;
import org.apache.nifi.controller.Snippet;
@@ -30,13 +38,9 @@ import org.apache.nifi.controller.label.Label;
import org.apache.nifi.controller.service.ControllerServiceNode;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.Processor;
+import org.apache.nifi.registry.ComponentVariableRegistry;
import org.apache.nifi.remote.RemoteGroupPort;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-import java.util.function.Predicate;
-
/**
* <p>
* ProcessGroup objects are containers for processing entities, such as
@@ -84,6 +88,7 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable {
/**
* @return the ID of the ProcessGroup
*/
+ @Override
String getIdentifier();
/**
@@ -159,7 +164,7 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable {
* @throws IllegalStateException if the processor is not valid, or is
* already running
*/
- void startProcessor(ProcessorNode processor);
+ CompletableFuture<Void> startProcessor(ProcessorNode processor);
/**
* Starts the given Input Port
@@ -187,7 +192,7 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable {
*
* @param processor to stop
*/
- void stopProcessor(ProcessorNode processor);
+ CompletableFuture<Void> stopProcessor(ProcessorNode processor);
/**
* Stops the given Port
@@ -814,6 +819,15 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable {
void verifyCanMove(Snippet snippet, ProcessGroup newProcessGroup);
/**
+ * Ensures that the given variables can be updated
+ *
+ * @param updatedVariables the new set of variable names and values
+ *
+ * @throws IllegalStateException if one or more variables that are listed cannot be updated at this time
+ */
+ void verifyCanUpdateVariables(Map<String, String> updatedVariables);
+
+ /**
* Adds the given template to this Process Group
*
* @param template the template to add
@@ -853,4 +867,27 @@ public interface ProcessGroup extends ComponentAuthorizable, Positionable {
* @return a Set of all Templates that belong to this Process Group and any descendant Process Groups
*/
Set<Template> findAllTemplates();
+
+ /**
+ * Updates the variables that are provided by this Process Group
+ *
+ * @param variables the variables to provide
+ * @throws IllegalStateException if the Process Group is not in a state that allows the variables to be updated
+ */
+ void setVariables(Map<String, String> variables);
+
+ /**
+ * Returns the Variable Registry for this Process Group
+ *
+ * @return the Variable Registry for this Process Group
+ */
+ ComponentVariableRegistry getVariableRegistry();
+
+ /**
+ * Returns a set of all components that are affected by the variable with the given name
+ *
+ * @param variableName the name of the variable
+ * @return a set of all components that are affected by the variable with the given name
+ */
+ Set<ConfiguredComponent> getComponentsAffectedByVariable(String variableName);
}
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.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 2e351a3..94645cf 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
@@ -34,9 +34,9 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
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;
@@ -195,7 +195,10 @@ import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventType;
import org.apache.nifi.provenance.ProvenanceRepository;
import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+import org.apache.nifi.registry.ComponentVariableRegistry;
import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.variable.MutableVariableRegistry;
+import org.apache.nifi.registry.variable.StandardComponentVariableRegistry;
import org.apache.nifi.remote.HttpRemoteSiteListener;
import org.apache.nifi.remote.RemoteGroupPort;
import org.apache.nifi.remote.RemoteResourceManager;
@@ -495,15 +498,15 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
throw new RuntimeException(e);
}
- processScheduler = new StandardProcessScheduler(this, encryptor, stateManagerProvider, this.variableRegistry, this.nifiProperties);
+ processScheduler = new StandardProcessScheduler(this, encryptor, stateManagerProvider, this.nifiProperties);
eventDrivenWorkerQueue = new EventDrivenWorkerQueue(false, false, processScheduler);
final ProcessContextFactory contextFactory = new ProcessContextFactory(contentRepository, flowFileRepository, flowFileEventRepository, counterRepositoryRef.get(), provenanceRepository);
processScheduler.setSchedulingAgent(SchedulingStrategy.EVENT_DRIVEN, new EventDrivenSchedulingAgent(
- eventDrivenEngineRef.get(), this, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor, this.variableRegistry));
+ eventDrivenEngineRef.get(), this, stateManagerProvider, eventDrivenWorkerQueue, contextFactory, maxEventDrivenThreads.get(), encryptor));
- final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.variableRegistry);
- final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.variableRegistry, this.nifiProperties);
+ final QuartzSchedulingAgent quartzSchedulingAgent = new QuartzSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor);
+ final TimerDrivenSchedulingAgent timerDrivenAgent = new TimerDrivenSchedulingAgent(this, timerDrivenEngineRef.get(), contextFactory, encryptor, this.nifiProperties);
processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, timerDrivenAgent);
// PRIMARY_NODE_ONLY is deprecated, but still exists to handle processors that are still defined with it (they haven't been re-configured with executeNode = PRIMARY).
processScheduler.setSchedulingAgent(SchedulingStrategy.PRIMARY_NODE_ONLY, timerDrivenAgent);
@@ -540,7 +543,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
this.snippetManager = new SnippetManager();
final ProcessGroup rootGroup = new StandardProcessGroup(ComponentIdGenerator.generateId().toString(), this, processScheduler,
- nifiProperties, encryptor, this, this.variableRegistry);
+ nifiProperties, encryptor, this, new MutableVariableRegistry(this.variableRegistry));
rootGroup.setName(DEFAULT_ROOT_GROUP_NAME);
setRootGroup(rootGroup);
instanceId = ComponentIdGenerator.generateId().toString();
@@ -1022,7 +1025,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
* @throws NullPointerException if the argument is null
*/
public ProcessGroup createProcessGroup(final String id) {
- return new StandardProcessGroup(requireNonNull(id).intern(), this, processScheduler, nifiProperties, encryptor, this, variableRegistry);
+ return new StandardProcessGroup(requireNonNull(id).intern(), this, processScheduler, nifiProperties, encryptor, this, new MutableVariableRegistry(variableRegistry));
}
/**
@@ -1099,15 +1102,17 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
creationSuccessful = false;
}
- final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, variableRegistry);
+ final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
+ final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, componentVarRegistry);
final ProcessorNode procNode;
if (creationSuccessful) {
- procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider, nifiProperties, variableRegistry, this);
+ procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider,
+ nifiProperties, componentVarRegistry, this);
} else {
final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type;
final String componentType = "(Missing) " + simpleClassName;
- procNode = new StandardProcessorNode(
- processor, id, validationContextFactory, processScheduler, controllerServiceProvider, componentType, type, nifiProperties, variableRegistry, this, true);
+ procNode = new StandardProcessorNode(processor, id, validationContextFactory, processScheduler, controllerServiceProvider,
+ componentType, type, nifiProperties, componentVarRegistry, this, true);
}
final LogRepository logRepository = LogRepositoryFactory.getRepository(id);
@@ -1223,7 +1228,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
// call OnRemoved for the existing processor using the previous instance class loader
try (final NarCloseable x = NarCloseable.withComponentNarLoader(existingInstanceClassLoader)) {
final StandardProcessContext processContext = new StandardProcessContext(
- existingNode, controllerServiceProvider, encryptor, getStateManagerProvider().getStateManager(id), variableRegistry);
+ existingNode, controllerServiceProvider, encryptor, getStateManagerProvider().getStateManager(id));
ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, existingNode.getProcessor(), processContext);
} finally {
ExtensionManager.closeURLClassLoader(id, existingInstanceClassLoader);
@@ -1943,6 +1948,10 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
childGroup.setPosition(toPosition(groupDTO.getPosition()));
childGroup.setComments(groupDTO.getComments());
childGroup.setName(groupDTO.getName());
+ if (groupDTO.getVariables() != null) {
+ childGroup.setVariables(groupDTO.getVariables());
+ }
+
group.addProcessGroup(childGroup);
final FlowSnippetDTO contents = groupDTO.getContents();
@@ -3124,15 +3133,16 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
creationSuccessful = false;
}
- final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, variableRegistry);
+ final ComponentVariableRegistry componentVarRegistry = new StandardComponentVariableRegistry(this.variableRegistry);
+ final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(controllerServiceProvider, componentVarRegistry);
final ReportingTaskNode taskNode;
if (creationSuccessful) {
- taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, variableRegistry, this);
+ taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentVarRegistry, this);
} else {
final String simpleClassName = type.contains(".") ? StringUtils.substringAfterLast(type, ".") : type;
final String componentType = "(Missing) " + simpleClassName;
- taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentType, type, variableRegistry, this, true);
+ taskNode = new StandardReportingTaskNode(task, id, this, processScheduler, validationContextFactory, componentType, type, componentVarRegistry, this, true);
}
taskNode.setName(taskNode.getReportingTask().getClass().getSimpleName());
@@ -3395,7 +3405,7 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
}
@Override
- public Future<Void> enableControllerService(final ControllerServiceNode serviceNode) {
+ public CompletableFuture<Void> enableControllerService(final ControllerServiceNode serviceNode) {
return controllerServiceProvider.enableControllerService(serviceNode);
}
@@ -3405,9 +3415,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
}
@Override
- public void disableControllerService(final ControllerServiceNode serviceNode) {
+ public CompletableFuture<Void> disableControllerService(final ControllerServiceNode serviceNode) {
serviceNode.verifyCanDisable();
- controllerServiceProvider.disableControllerService(serviceNode);
+ return controllerServiceProvider.disableControllerService(serviceNode);
}
@Override
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
index 01dd35e..58bb90f 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
@@ -16,6 +16,34 @@
*/
package org.apache.nifi.controller;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.zip.GZIPInputStream;
+
+import javax.xml.XMLConstants;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.authorization.Authorizer;
@@ -92,33 +120,6 @@ import org.w3c.dom.Node;
import org.w3c.dom.NodeList;
import org.xml.sax.SAXException;
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URL;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.StandardOpenOption;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-import java.util.zip.GZIPInputStream;
-
/**
*/
public class StandardFlowSynchronizer implements FlowSynchronizer {
@@ -1037,6 +1038,21 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
parentGroup.addProcessGroup(processGroup);
}
+ // Set the variables for the variable registry
+ final Map<String, String> variables = new HashMap<>();
+ final List<Element> variableElements = getChildrenByTagName(processGroupElement, "variable");
+ for (final Element variableElement : variableElements) {
+ final String variableName = variableElement.getAttribute("name");
+ final String variableValue = variableElement.getAttribute("value");
+ if (variableName == null || variableValue == null) {
+ continue;
+ }
+
+ variables.put(variableName, variableValue);
+ }
+
+ processGroup.setVariables(variables);
+
// Add Controller Services
final List<Element> serviceNodeList = getChildrenByTagName(processGroupElement, "controllerService");
if (!serviceNodeList.isEmpty()) {
http://git-wip-us.apache.org/repos/asf/nifi/blob/5cd8e93b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
index 1ff09d7..36cb62e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
@@ -16,6 +16,31 @@
*/
package org.apache.nifi.controller;
+import static java.util.Objects.requireNonNull;
+
+import java.lang.reflect.InvocationTargetException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
@@ -56,7 +81,7 @@ import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processor.Processor;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.SimpleProcessLogger;
-import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.registry.ComponentVariableRegistry;
import org.apache.nifi.scheduling.ExecutionNode;
import org.apache.nifi.scheduling.SchedulingStrategy;
import org.apache.nifi.util.CharacterFilterUtils;
@@ -68,30 +93,6 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.springframework.util.Assert;
-import java.lang.reflect.InvocationTargetException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static java.util.Objects.requireNonNull;
-
/**
* ProcessorNode provides thread-safe access to a FlowFileProcessor as it exists
* within a controlled flow. This node keeps track of the processor, its
@@ -137,7 +138,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
public StandardProcessorNode(final LoggableComponent<Processor> processor, final String uuid,
final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler,
final ControllerServiceProvider controllerServiceProvider, final NiFiProperties nifiProperties,
- final VariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
+ final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent) {
this(processor, uuid, validationContextFactory, scheduler, controllerServiceProvider,
processor.getComponent().getClass().getSimpleName(), processor.getComponent().getClass().getCanonicalName(), nifiProperties, variableRegistry, reloadComponent, false);
@@ -147,7 +148,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
final ValidationContextFactory validationContextFactory, final ProcessScheduler scheduler,
final ControllerServiceProvider controllerServiceProvider,
final String componentType, final String componentCanonicalClass, final NiFiProperties nifiProperties,
- final VariableRegistry variableRegistry, final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
+ final ComponentVariableRegistry variableRegistry, final ReloadComponent reloadComponent, final boolean isExtensionMissing) {
super(uuid, validationContextFactory, controllerServiceProvider, componentType, componentCanonicalClass, variableRegistry, reloadComponent, isExtensionMissing);
@@ -1372,13 +1373,13 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
* </p>
*/
@Override
- public <T extends ProcessContext & ControllerServiceLookup> void stop(final ScheduledExecutorService scheduler,
+ public <T extends ProcessContext & ControllerServiceLookup> CompletableFuture<Void> stop(final ScheduledExecutorService scheduler,
final T processContext, final SchedulingAgent schedulingAgent, final ScheduleState scheduleState) {
final Processor processor = processorRef.get().getProcessor();
LOG.info("Stopping processor: " + processor.getClass());
-
+ final CompletableFuture<Void> future = new CompletableFuture<>();
if (this.scheduledState.compareAndSet(ScheduledState.RUNNING, ScheduledState.STOPPING)) { // will ensure that the Processor represented by this node can only be stopped once
scheduleState.incrementActiveThreadCount();
@@ -1405,6 +1406,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
scheduleState.decrementActiveThreadCount();
scheduledState.set(ScheduledState.STOPPED);
+ future.complete(null);
} else {
// Not all of the active threads have finished. Try again in 100 milliseconds.
scheduler.schedule(this, 100, TimeUnit.MILLISECONDS);
@@ -1415,16 +1417,17 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
}
});
} else {
- /*
- * We do compareAndSet() instead of set() to ensure that Processor
- * stoppage is handled consistently including a condition where
- * Processor never got a chance to transition to RUNNING state
- * before stop() was called. If that happens the stop processor
- * routine will be initiated in start() method, otherwise the IF
- * part will handle the stop processor routine.
- */
+ // We do compareAndSet() instead of set() to ensure that Processor
+ // stoppage is handled consistently including a condition where
+ // Processor never got a chance to transition to RUNNING state
+ // before stop() was called. If that happens the stop processor
+ // routine will be initiated in start() method, otherwise the IF
+ // part will handle the stop processor routine.
this.scheduledState.compareAndSet(ScheduledState.STARTING, ScheduledState.STOPPING);
+ future.complete(null);
}
+
+ return future;
}
/**