mirror of https://github.com/apache/nifi.git
NIFI-1554:
- Introducing new REST endpoints to align with the authorizable resources. - Additionally changes to support the new endpoints. - Addressing comments in PR. - This closes #374.
This commit is contained in:
parent
32398693d5
commit
add298168d
|
@ -136,7 +136,7 @@ public class AuthorizerFactoryBean implements FactoryBean, DisposableBean, Autho
|
|||
final JAXBElement<Authorizers> element = unmarshaller.unmarshal(new StreamSource(authorizersConfigurationFile), Authorizers.class);
|
||||
return element.getValue();
|
||||
} catch (SAXException | JAXBException e) {
|
||||
throw new Exception("Unable to load the authorizer configuration file at: " + authorizersConfigurationFile.getAbsolutePath());
|
||||
throw new Exception("Unable to load the authorizer configuration file at: " + authorizersConfigurationFile.getAbsolutePath(), e);
|
||||
}
|
||||
} else {
|
||||
throw new Exception("Unable to find the authorizer configuration file at " + authorizersConfigurationFile.getAbsolutePath());
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.nifi.web.api.dto;
|
||||
|
||||
import com.wordnik.swagger.annotations.ApiModelProperty;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
|
||||
/**
|
||||
|
@ -28,6 +29,9 @@ public class AboutDTO {
|
|||
private String title;
|
||||
private String version;
|
||||
|
||||
private String uri;
|
||||
private String contentViewerUrl;
|
||||
|
||||
/* getters / setters */
|
||||
/**
|
||||
* The title to be used on the page and in the About dialog.
|
||||
|
@ -61,4 +65,31 @@ public class AboutDTO {
|
|||
this.version = version;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return URI for this NiFi controller
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The URI for the NiFi."
|
||||
)
|
||||
public String getUri() {
|
||||
return uri;
|
||||
}
|
||||
|
||||
public void setUri(String uri) {
|
||||
this.uri = uri;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the URL for the content viewer if configured
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The URL for the content viewer if configured."
|
||||
)
|
||||
public String getContentViewerUrl() {
|
||||
return contentViewerUrl;
|
||||
}
|
||||
|
||||
public void setContentViewerUrl(String contentViewerUrl) {
|
||||
this.contentViewerUrl = contentViewerUrl;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,9 +39,6 @@ public class ControllerConfigurationDTO {
|
|||
private Date currentTime;
|
||||
private Integer timeOffset;
|
||||
|
||||
private String contentViewerUrl;
|
||||
private String uri;
|
||||
|
||||
/**
|
||||
* @return maximum number of timer driven threads this NiFi has available
|
||||
*/
|
||||
|
@ -156,32 +153,4 @@ public class ControllerConfigurationDTO {
|
|||
public void setTimeOffset(Integer timeOffset) {
|
||||
this.timeOffset = timeOffset;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the URL for the content viewer if configured
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The URL for the content viewer if configured."
|
||||
)
|
||||
public String getContentViewerUrl() {
|
||||
return contentViewerUrl;
|
||||
}
|
||||
|
||||
public void setContentViewerUrl(String contentViewerUrl) {
|
||||
this.contentViewerUrl = contentViewerUrl;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return URI for this NiFi controller
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The URI for the NiFi."
|
||||
)
|
||||
public String getUri() {
|
||||
return uri;
|
||||
}
|
||||
|
||||
public void setUri(String uri) {
|
||||
this.uri = uri;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,65 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.dto;
|
||||
|
||||
import com.wordnik.swagger.annotations.ApiModelProperty;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
|
||||
/**
|
||||
* Resource that supports access/authorization policies.
|
||||
*/
|
||||
@XmlType(name = "resource")
|
||||
public class ResourceDTO {
|
||||
|
||||
private String identifier;
|
||||
private String name;
|
||||
|
||||
/* getters / setters */
|
||||
/**
|
||||
* The name of the resource.
|
||||
*
|
||||
* @return The name of the resource
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The name of the resource."
|
||||
)
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public void setName(String name) {
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
/**
|
||||
* The identifier of the resource.
|
||||
*
|
||||
* @return The identifier of the resource
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The identifier of the resource."
|
||||
)
|
||||
public String getIdentifier() {
|
||||
return identifier;
|
||||
}
|
||||
|
||||
public void setIdentifier(String identifier) {
|
||||
this.identifier = identifier;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,65 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.entity;
|
||||
|
||||
import com.wordnik.swagger.annotations.ApiModelProperty;
|
||||
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
|
||||
/**
|
||||
* A serialized representation of this class can be placed in the entity body of a request to the API.
|
||||
*/
|
||||
@XmlRootElement(name = "copySnippetRequestEntity")
|
||||
public class CopySnippetRequestEntity extends Entity {
|
||||
|
||||
private String snippetId;
|
||||
private Double originX;
|
||||
private Double originY;
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The identifier of the snippet."
|
||||
)
|
||||
public String getSnippetId() {
|
||||
return snippetId;
|
||||
}
|
||||
|
||||
public void setSnippetId(String snippetId) {
|
||||
this.snippetId = snippetId;
|
||||
}
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The x coordinate of the origin of the bounding box where the new components will be placed."
|
||||
)
|
||||
public Double getOriginX() {
|
||||
return originX;
|
||||
}
|
||||
|
||||
public void setOriginX(Double originX) {
|
||||
this.originX = originX;
|
||||
}
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The y coordinate of the origin of the bounding box where the new components will be placed."
|
||||
)
|
||||
public Double getOriginY() {
|
||||
return originY;
|
||||
}
|
||||
|
||||
public void setOriginY(Double originY) {
|
||||
this.originY = originY;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,65 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.entity;
|
||||
|
||||
import com.wordnik.swagger.annotations.ApiModelProperty;
|
||||
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
|
||||
/**
|
||||
* A serialized representation of this class can be placed in the entity body of a request to the API.
|
||||
*/
|
||||
@XmlRootElement(name = "copySnippetRequestEntity")
|
||||
public class CreateTemplateRequestEntity extends Entity {
|
||||
|
||||
private String name;
|
||||
private String description;
|
||||
private String snippetId;
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The name of the template."
|
||||
)
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public void setName(String name) {
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The description of the template."
|
||||
)
|
||||
public String getDescription() {
|
||||
return description;
|
||||
}
|
||||
|
||||
public void setDescription(String description) {
|
||||
this.description = description;
|
||||
}
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The identifier of the snippet."
|
||||
)
|
||||
public String getSnippetId() {
|
||||
return snippetId;
|
||||
}
|
||||
|
||||
public void setSnippetId(String snippetId) {
|
||||
this.snippetId = snippetId;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,65 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.entity;
|
||||
|
||||
import com.wordnik.swagger.annotations.ApiModelProperty;
|
||||
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
|
||||
/**
|
||||
* A serialized representation of this class can be placed in the entity body of a request to the API.
|
||||
*/
|
||||
@XmlRootElement(name = "instantiateTemplateRequestEntity")
|
||||
public class InstantiateTemplateRequestEntity extends Entity {
|
||||
|
||||
private String templateId;
|
||||
private Double originX;
|
||||
private Double originY;
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The identifier of the template."
|
||||
)
|
||||
public String getTemplateId() {
|
||||
return templateId;
|
||||
}
|
||||
|
||||
public void setTemplateId(String templateId) {
|
||||
this.templateId = templateId;
|
||||
}
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The x coordinate of the origin of the bounding box where the new components will be placed."
|
||||
)
|
||||
public Double getOriginX() {
|
||||
return originX;
|
||||
}
|
||||
|
||||
public void setOriginX(Double originX) {
|
||||
this.originX = originX;
|
||||
}
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The y coordinate of the origin of the bounding box where the new components will be placed."
|
||||
)
|
||||
public Double getOriginY() {
|
||||
return originY;
|
||||
}
|
||||
|
||||
public void setOriginY(Double originY) {
|
||||
this.originY = originY;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,45 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api.entity;
|
||||
|
||||
import org.apache.nifi.web.api.dto.ResourceDTO;
|
||||
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A serialized representation of this class can be placed in the entity body of a request or response to or from the API. This particular entity holds references to ResourceDTOs.
|
||||
*/
|
||||
@XmlRootElement(name = "resourcesEntity")
|
||||
public class ResourcesEntity extends Entity {
|
||||
|
||||
private List<ResourceDTO> resources;
|
||||
|
||||
/**
|
||||
* The AboutDTO that is being serialized.
|
||||
*
|
||||
* @return The AboutDTO object
|
||||
*/
|
||||
public List<ResourceDTO> getResources() {
|
||||
return resources;
|
||||
}
|
||||
|
||||
public void setResources(List<ResourceDTO> resources) {
|
||||
this.resources = resources;
|
||||
}
|
||||
|
||||
}
|
|
@ -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.web.api.entity;
|
||||
|
||||
import com.wordnik.swagger.annotations.ApiModelProperty;
|
||||
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
|
||||
/**
|
||||
* A serialized representation of this class can be placed in the entity body of a request to the API.
|
||||
*/
|
||||
@XmlRootElement(name = "copySnippetRequestEntity")
|
||||
public class SubmitReplayRequestEntity extends Entity {
|
||||
|
||||
private Long eventId;
|
||||
private String clusterNodeId;
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The event identifier"
|
||||
)
|
||||
public Long getEventId() {
|
||||
return eventId;
|
||||
}
|
||||
|
||||
public void setEventId(Long eventId) {
|
||||
this.eventId = eventId;
|
||||
}
|
||||
|
||||
@ApiModelProperty(
|
||||
value = "The identifier of the node where to submit the replay request."
|
||||
)
|
||||
public String getClusterNodeId() {
|
||||
return clusterNodeId;
|
||||
}
|
||||
|
||||
public void setClusterNodeId(String clusterNodeId) {
|
||||
this.clusterNodeId = clusterNodeId;
|
||||
}
|
||||
}
|
|
@ -25,6 +25,7 @@ public enum ResourceType {
|
|||
RemoteProcessGroup("/remote-process-groups"),
|
||||
Label("/labels"),
|
||||
ControllerService("/controller-services"),
|
||||
ReportingTask("/reporting-tasks"),
|
||||
Template("/templates");
|
||||
|
||||
final String value;
|
||||
|
|
|
@ -331,7 +331,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
|
|||
|
||||
public static final Pattern PROCESSORS_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors");
|
||||
public static final Pattern PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}");
|
||||
public static final Pattern PROCESSOR_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}/status");
|
||||
public static final Pattern PROCESSOR_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/processors/[a-f0-9\\-]{36}/status");
|
||||
public static final Pattern PROCESSOR_STATE_URI_PATTERN = Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}/state");
|
||||
public static final Pattern CLUSTER_PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/cluster/processors/[a-f0-9\\-]{36}");
|
||||
|
||||
|
@ -339,20 +339,20 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
|
|||
public static final Pattern REMOTE_PROCESS_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/remote-process-groups/[a-f0-9\\-]{36}");
|
||||
|
||||
public static final Pattern PROCESS_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))");
|
||||
public static final Pattern GROUP_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/status");
|
||||
public static final Pattern CONTROLLER_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/controller/status");
|
||||
public static final Pattern GROUP_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/status");
|
||||
public static final Pattern CONTROLLER_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/status");
|
||||
public static final Pattern TEMPLATE_INSTANCE_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/template-instance");
|
||||
public static final Pattern FLOW_SNIPPET_INSTANCE_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/snippet-instance");
|
||||
|
||||
public static final String PROVENANCE_URI = "/nifi-api/controller/provenance";
|
||||
public static final Pattern PROVENANCE_QUERY_URI = Pattern.compile("/nifi-api/controller/provenance/[a-f0-9\\-]{36}");
|
||||
public static final Pattern PROVENANCE_EVENT_URI = Pattern.compile("/nifi-api/controller/provenance/events/[0-9]+");
|
||||
public static final String PROVENANCE_URI = "/nifi-api/provenance";
|
||||
public static final Pattern PROVENANCE_QUERY_URI = Pattern.compile("/nifi-api/provenance/[a-f0-9\\-]{36}");
|
||||
public static final Pattern PROVENANCE_EVENT_URI = Pattern.compile("/nifi-api/provenance/events/[0-9]+");
|
||||
|
||||
public static final String CONTROLLER_SERVICES_URI = "/nifi-api/controller-services/node";
|
||||
public static final Pattern CONTROLLER_SERVICES_URI = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/controller-services/node");
|
||||
public static final Pattern CONTROLLER_SERVICE_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/node/[a-f0-9\\-]{36}");
|
||||
public static final Pattern CONTROLLER_SERVICE_STATE_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/node/[a-f0-9\\-]{36}/state");
|
||||
public static final Pattern CONTROLLER_SERVICE_REFERENCES_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/node/[a-f0-9\\-]{36}/references");
|
||||
public static final String REPORTING_TASKS_URI = "/nifi-api/reporting-tasks/node";
|
||||
public static final String REPORTING_TASKS_URI = "/nifi-api/controller/reporting-tasks/node";
|
||||
public static final Pattern REPORTING_TASK_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/node/[a-f0-9\\-]{36}");
|
||||
public static final Pattern REPORTING_TASK_STATE_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/node/[a-f0-9\\-]{36}/state");
|
||||
public static final Pattern BULLETIN_BOARD_URI_PATTERN = Pattern.compile("/nifi-api/bulletin-board");
|
||||
|
@ -361,17 +361,17 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
|
|||
public static final Pattern COUNTER_URI_PATTERN = Pattern.compile("/nifi-api/controller/counters/[a-f0-9\\-]{36}");
|
||||
|
||||
public static final Pattern PROCESSOR_STATUS_HISTORY_URI_PATTERN =
|
||||
Pattern.compile("/nifi-api/processors/[a-f0-9\\-]{36}/status/history");
|
||||
public static final Pattern PROCESS_GROUP_STATUS_HISTORY_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/status/history");
|
||||
Pattern.compile("/nifi-api/flow/processors/[a-f0-9\\-]{36}/status/history");
|
||||
public static final Pattern PROCESS_GROUP_STATUS_HISTORY_URI_PATTERN = Pattern.compile("/nifi-api/flow/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/status/history");
|
||||
public static final Pattern REMOTE_PROCESS_GROUP_STATUS_HISTORY_URI_PATTERN = Pattern
|
||||
.compile("/nifi-api/remote-process-groups/[a-f0-9\\-]{36}/status/history");
|
||||
.compile("/nifi-api/flow/remote-process-groups/[a-f0-9\\-]{36}/status/history");
|
||||
public static final Pattern CONNECTION_STATUS_HISTORY_URI_PATTERN = Pattern
|
||||
.compile("/nifi-api/connections/[a-f0-9\\-]{36}/status/history");
|
||||
.compile("/nifi-api/flow/connections/[a-f0-9\\-]{36}/status/history");
|
||||
|
||||
public static final Pattern CONNECTION_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/connections/[a-f0-9\\-]{36}/status");
|
||||
public static final Pattern INPUT_PORT_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/input-ports/[a-f0-9\\-]{36}/status");
|
||||
public static final Pattern OUTPUT_PORT_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/output-ports/[a-f0-9\\-]{36}/status");
|
||||
public static final Pattern REMOTE_PROCESS_GROUP_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/remote-process-groups/[a-f0-9\\-]{36}/status");
|
||||
public static final Pattern CONNECTION_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/connections/[a-f0-9\\-]{36}/status");
|
||||
public static final Pattern INPUT_PORT_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/input-ports/[a-f0-9\\-]{36}/status");
|
||||
public static final Pattern OUTPUT_PORT_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/output-ports/[a-f0-9\\-]{36}/status");
|
||||
public static final Pattern REMOTE_PROCESS_GROUP_STATUS_URI_PATTERN = Pattern.compile("/nifi-api/flow/remote-process-groups/[a-f0-9\\-]{36}/status");
|
||||
|
||||
public static final Pattern DROP_REQUESTS_URI = Pattern.compile("/nifi-api/connections/[a-f0-9\\-]{36}/drop-requests");
|
||||
public static final Pattern DROP_REQUEST_URI = Pattern.compile("/nifi-api/connections/[a-f0-9\\-]{36}/drop-requests/[a-f0-9\\-]{36}");
|
||||
|
@ -2412,13 +2412,13 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
|
|||
}
|
||||
|
||||
private static boolean isControllerServicesEndpoint(final URI uri, final String method) {
|
||||
return "GET".equalsIgnoreCase(method) && CONTROLLER_SERVICES_URI.equals(uri.getPath());
|
||||
return "GET".equalsIgnoreCase(method) && CONTROLLER_SERVICES_URI.matcher(uri.getPath()).matches();
|
||||
}
|
||||
|
||||
private static boolean isControllerServiceEndpoint(final URI uri, final String method) {
|
||||
if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && CONTROLLER_SERVICE_URI_PATTERN.matcher(uri.getPath()).matches()) {
|
||||
return true;
|
||||
} else if ("POST".equalsIgnoreCase(method) && CONTROLLER_SERVICES_URI.equals(uri.getPath())) {
|
||||
} else if ("POST".equalsIgnoreCase(method) && CONTROLLER_SERVICES_URI.matcher(uri.getPath()).matches()) {
|
||||
return true;
|
||||
}
|
||||
|
||||
|
|
|
@ -506,6 +506,13 @@ public interface ProcessGroup {
|
|||
*/
|
||||
ProcessGroup findProcessGroup(String id);
|
||||
|
||||
/**
|
||||
* @return a List of all ProcessGroups that are children or descendants of this
|
||||
* ProcessGroup. This performs a recursive search of all descendant
|
||||
* ProcessGroups
|
||||
*/
|
||||
List<ProcessGroup> findAllProcessGroups();
|
||||
|
||||
/**
|
||||
* @param id of the group
|
||||
* @return the RemoteProcessGroup with the given ID, if it exists as a child
|
||||
|
@ -559,6 +566,13 @@ public interface ProcessGroup {
|
|||
*/
|
||||
Port findInputPort(String id);
|
||||
|
||||
/**
|
||||
* @return a List of all InputPorts that are children or descendants of this
|
||||
* ProcessGroup. This performs a recursive search of all descendant
|
||||
* ProcessGroups
|
||||
*/
|
||||
List<Port> findAllInputPorts();
|
||||
|
||||
/**
|
||||
* @param name of port
|
||||
* @return the input port with the given name, if it exists; otherwise
|
||||
|
@ -574,6 +588,13 @@ public interface ProcessGroup {
|
|||
*/
|
||||
Port findOutputPort(String id);
|
||||
|
||||
/**
|
||||
* @return a List of all OutputPorts that are children or descendants of this
|
||||
* ProcessGroup. This performs a recursive search of all descendant
|
||||
* ProcessGroups
|
||||
*/
|
||||
List<Port> findAllOutputPorts();
|
||||
|
||||
/**
|
||||
* @param name of the port
|
||||
* @return the output port with the given name, if it exists; otherwise
|
||||
|
|
|
@ -71,6 +71,10 @@
|
|||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-site-to-site-client</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-framework-authorization</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.quartz-scheduler</groupId>
|
||||
<artifactId>quartz</artifactId>
|
||||
|
|
|
@ -1380,6 +1380,19 @@ public final class StandardProcessGroup implements ProcessGroup {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ProcessGroup> findAllProcessGroups() {
|
||||
return findAllProcessGroups(this);
|
||||
}
|
||||
|
||||
private List<ProcessGroup> findAllProcessGroups(final ProcessGroup start) {
|
||||
final List<ProcessGroup> allProcessGroups = new ArrayList<>(start.getProcessGroups());
|
||||
for (final ProcessGroup childGroup : start.getProcessGroups()) {
|
||||
allProcessGroups.addAll(findAllProcessGroups(childGroup));
|
||||
}
|
||||
return allProcessGroups;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<RemoteProcessGroup> findAllRemoteProcessGroups() {
|
||||
return findAllRemoteProcessGroups(this);
|
||||
|
@ -1535,11 +1548,37 @@ public final class StandardProcessGroup implements ProcessGroup {
|
|||
return findPort(id, this, new InputPortRetriever());
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Port> findAllInputPorts() {
|
||||
return findAllInputPorts(this);
|
||||
}
|
||||
|
||||
private List<Port> findAllInputPorts(final ProcessGroup start) {
|
||||
final List<Port> allOutputPorts = new ArrayList<>(start.getInputPorts());
|
||||
for (final ProcessGroup group : start.getProcessGroups()) {
|
||||
allOutputPorts.addAll(findAllInputPorts(group));
|
||||
}
|
||||
return allOutputPorts;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Port findOutputPort(final String id) {
|
||||
return findPort(id, this, new OutputPortRetriever());
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Port> findAllOutputPorts() {
|
||||
return findAllOutputPorts(this);
|
||||
}
|
||||
|
||||
private List<Port> findAllOutputPorts(final ProcessGroup start) {
|
||||
final List<Port> allOutputPorts = new ArrayList<>(start.getOutputPorts());
|
||||
for (final ProcessGroup group : start.getProcessGroups()) {
|
||||
allOutputPorts.addAll(findAllOutputPorts(group));
|
||||
}
|
||||
return allOutputPorts;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Port getInputPortByName(final String name) {
|
||||
return getPortByName(name, this, new InputPortRetriever());
|
||||
|
|
|
@ -80,8 +80,8 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
|
||||
private static final Logger logger = LoggerFactory.getLogger(StandardRemoteProcessGroup.class);
|
||||
|
||||
public static final String CONTROLLER_URI_PATH = "/controller";
|
||||
public static final String ROOT_GROUP_STATUS_URI_PATH = "/controller/process-groups/root/status";
|
||||
public static final String SITE_TO_SITE_URI_PATH = "/site-to-site";
|
||||
public static final String ROOT_GROUP_STATUS_URI_PATH = "/flow/process-groups/root/status";
|
||||
|
||||
// status codes
|
||||
public static final int OK_STATUS_CODE = Status.OK.getStatusCode();
|
||||
|
@ -749,7 +749,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
}
|
||||
|
||||
final RemoteNiFiUtils utils = new RemoteNiFiUtils(isWebApiSecure() ? sslContext : null);
|
||||
final String uriVal = apiUri.toString() + CONTROLLER_URI_PATH;
|
||||
final String uriVal = apiUri.toString() + SITE_TO_SITE_URI_PATH;
|
||||
URI uri;
|
||||
try {
|
||||
uri = new URI(uriVal);
|
||||
|
@ -1080,7 +1080,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
|
|||
public void run() {
|
||||
try {
|
||||
final RemoteNiFiUtils utils = new RemoteNiFiUtils(isWebApiSecure() ? sslContext : null);
|
||||
final ClientResponse response = utils.get(new URI(apiUri + CONTROLLER_URI_PATH), getCommunicationsTimeout(TimeUnit.MILLISECONDS));
|
||||
final ClientResponse response = utils.get(new URI(apiUri + SITE_TO_SITE_URI_PATH), getCommunicationsTimeout(TimeUnit.MILLISECONDS));
|
||||
|
||||
final int statusCode = response.getStatus();
|
||||
|
||||
|
|
|
@ -197,6 +197,11 @@
|
|||
<artifactId>nifi-utils</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-framework-authorization</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.servlet</groupId>
|
||||
<artifactId>javax.servlet-api</artifactId>
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
|
|||
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
|
||||
import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
|
||||
import org.apache.nifi.web.api.dto.ReportingTaskDTO;
|
||||
import org.apache.nifi.web.api.dto.ResourceDTO;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.dto.SnippetDTO;
|
||||
import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO;
|
||||
|
@ -67,6 +68,7 @@ import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO;
|
|||
import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
|
||||
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
|
@ -1400,4 +1402,12 @@ public interface NiFiServiceFacade {
|
|||
* @return the system diagnostics
|
||||
*/
|
||||
SystemDiagnosticsDTO getSystemDiagnostics();
|
||||
|
||||
// ----------------------------------------
|
||||
// Resources
|
||||
// ----------------------------------------
|
||||
/**
|
||||
* @return the resources
|
||||
*/
|
||||
List<ResourceDTO> getResources();
|
||||
}
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.web;
|
||||
|
||||
import org.apache.nifi.admin.service.KeyService;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.web.security.anonymous.NiFiAnonymousUserFilter;
|
||||
import org.apache.nifi.web.security.jwt.JwtAuthenticationFilter;
|
||||
|
@ -54,7 +53,6 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
|
|||
private static final Logger logger = LoggerFactory.getLogger(NiFiWebApiSecurityConfiguration.class);
|
||||
|
||||
private NiFiProperties properties;
|
||||
private KeyService keyService;
|
||||
|
||||
private NodeAuthorizedUserFilter nodeAuthorizedUserFilter;
|
||||
|
||||
|
@ -174,16 +172,10 @@ public class NiFiWebApiSecurityConfiguration extends WebSecurityConfigurerAdapte
|
|||
public NiFiAnonymousUserFilter anonymousFilterBean() throws Exception {
|
||||
if (anonymousAuthenticationFilter == null) {
|
||||
anonymousAuthenticationFilter = new NiFiAnonymousUserFilter();
|
||||
anonymousAuthenticationFilter.setKeyService(keyService);
|
||||
}
|
||||
return anonymousAuthenticationFilter;
|
||||
}
|
||||
|
||||
@Autowired
|
||||
public void setKeyService(KeyService keyService) {
|
||||
this.keyService = keyService;
|
||||
}
|
||||
|
||||
@Autowired
|
||||
public void setProperties(NiFiProperties properties) {
|
||||
this.properties = properties;
|
||||
|
|
|
@ -56,10 +56,10 @@ public class StandardNiFiContentAccess implements ContentAccess {
|
|||
public static final String CLIENT_ID_PARAM = "clientId";
|
||||
|
||||
private static final Pattern FLOWFILE_CONTENT_URI_PATTERN = Pattern
|
||||
.compile("/controller/process-groups/((?:root)|(?:[a-f0-9\\-]{36}))/connections/([a-f0-9\\-]{36})/flowfiles/([a-f0-9\\-]{36})/content.*");
|
||||
.compile("/flowfile-queues/([a-f0-9\\-]{36})/flowfiles/([a-f0-9\\-]{36})/content.*");
|
||||
|
||||
private static final Pattern PROVENANCE_CONTENT_URI_PATTERN = Pattern
|
||||
.compile("/controller/provenance/events/([0-9]+)/content/((?:input)|(?:output)).*");
|
||||
.compile("/provenance/events/([0-9]+)/content/((?:input)|(?:output)).*");
|
||||
|
||||
private NiFiProperties properties;
|
||||
private NiFiServiceFacade serviceFacade;
|
||||
|
@ -140,8 +140,8 @@ public class StandardNiFiContentAccess implements ContentAccess {
|
|||
return new DownloadableContent(filename, contentType, clientResponse.getEntityInputStream());
|
||||
} else {
|
||||
// example URIs:
|
||||
// http://localhost:8080/nifi-api/controller/provenance/events/{id}/content/{input|output}
|
||||
// http://localhost:8080/nifi-api/controller/process-groups/{root|uuid}/connections/{uuid}/flowfiles/{uuid}/content
|
||||
// http://localhost:8080/nifi-api/provenance/events/{id}/content/{input|output}
|
||||
// http://localhost:8080/nifi-api/flowfile-queues/{uuid}/flowfiles/{uuid}/content
|
||||
|
||||
// get just the context path for comparison
|
||||
final String dataUri = StringUtils.substringAfter(request.getDataUri(), "/nifi-api");
|
||||
|
@ -152,11 +152,10 @@ public class StandardNiFiContentAccess implements ContentAccess {
|
|||
// flowfile listing content
|
||||
final Matcher flowFileMatcher = FLOWFILE_CONTENT_URI_PATTERN.matcher(dataUri);
|
||||
if (flowFileMatcher.matches()) {
|
||||
final String groupId = flowFileMatcher.group(1);
|
||||
final String connectionId = flowFileMatcher.group(2);
|
||||
final String flowfileId = flowFileMatcher.group(3);
|
||||
final String connectionId = flowFileMatcher.group(1);
|
||||
final String flowfileId = flowFileMatcher.group(2);
|
||||
|
||||
return getFlowFileContent(groupId, connectionId, flowfileId, dataUri);
|
||||
return getFlowFileContent(connectionId, flowfileId, dataUri);
|
||||
}
|
||||
|
||||
// provenance event content
|
||||
|
@ -177,7 +176,7 @@ public class StandardNiFiContentAccess implements ContentAccess {
|
|||
}
|
||||
}
|
||||
|
||||
private DownloadableContent getFlowFileContent(final String groupId, final String connectionId, final String flowfileId, final String dataUri) {
|
||||
private DownloadableContent getFlowFileContent(final String connectionId, final String flowfileId, final String dataUri) {
|
||||
// TODO - ensure the user is authorized - not checking with @PreAuthorized annotation as aspect not trigger on call within a class
|
||||
// if (!NiFiUserUtils.getAuthorities().contains(Authority.ROLE_DFM.toString())) {
|
||||
// throw new AccessDeniedException("Access is denied.");
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.nifi.action.Operation;
|
|||
import org.apache.nifi.action.details.FlowChangePurgeDetails;
|
||||
import org.apache.nifi.admin.service.AuditService;
|
||||
import org.apache.nifi.admin.service.KeyService;
|
||||
import org.apache.nifi.authorization.Resource;
|
||||
import org.apache.nifi.cluster.context.ClusterContext;
|
||||
import org.apache.nifi.cluster.context.ClusterContextThreadLocal;
|
||||
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
|
||||
|
@ -97,6 +98,7 @@ import org.apache.nifi.web.api.dto.PropertyHistoryDTO;
|
|||
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
|
||||
import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
|
||||
import org.apache.nifi.web.api.dto.ReportingTaskDTO;
|
||||
import org.apache.nifi.web.api.dto.ResourceDTO;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.dto.SnippetDTO;
|
||||
import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO;
|
||||
|
@ -1529,6 +1531,16 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
return dtoFactory.createSystemDiagnosticsDto(sysDiagnostics);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ResourceDTO> getResources() {
|
||||
final List<Resource> resources = controllerFacade.getResources();
|
||||
final List<ResourceDTO> resourceDtos = new ArrayList<>(resources.size());
|
||||
for (final Resource resource : resources) {
|
||||
resourceDtos.add(dtoFactory.createResourceDto(resource));
|
||||
}
|
||||
return resourceDtos;
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensures the specified user has permission to access the specified port.
|
||||
*/
|
||||
|
@ -1624,9 +1636,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final long refreshInterval = FormatUtils.getTimeDuration(properties.getAutoRefreshInterval(), TimeUnit.SECONDS);
|
||||
controllerConfig.setAutoRefreshIntervalSeconds(refreshInterval);
|
||||
|
||||
// get the content viewer url
|
||||
controllerConfig.setContentViewerUrl(properties.getProperty(NiFiProperties.CONTENT_VIEWER_URL));
|
||||
|
||||
final Date now = new Date();
|
||||
controllerConfig.setTimeOffset(TimeZone.getDefault().getOffset(now.getTime()));
|
||||
controllerConfig.setCurrentTime(now);
|
||||
|
|
|
@ -415,7 +415,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
|
|||
// create the request URL
|
||||
URI requestUrl;
|
||||
try {
|
||||
String path = "/nifi-api/controller/controller-services/node/" + URLEncoder.encode(id, "UTF-8");
|
||||
String path = "/nifi-api/controller-services/node/" + URLEncoder.encode(id, "UTF-8");
|
||||
requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null);
|
||||
} catch (final URISyntaxException | UnsupportedEncodingException use) {
|
||||
throw new ClusterRequestException(use);
|
||||
|
@ -471,7 +471,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
|
|||
// create the request URL
|
||||
URI requestUrl;
|
||||
try {
|
||||
String path = "/nifi-api/controller/controller-services/node/" + URLEncoder.encode(id, "UTF-8");
|
||||
String path = "/nifi-api/controller-services/node/" + URLEncoder.encode(id, "UTF-8");
|
||||
requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null);
|
||||
} catch (final URISyntaxException | UnsupportedEncodingException use) {
|
||||
throw new ClusterRequestException(use);
|
||||
|
@ -550,7 +550,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
|
|||
// create the request URL
|
||||
URI requestUrl;
|
||||
try {
|
||||
String path = "/nifi-api/controller/reporting-tasks/node/" + URLEncoder.encode(id, "UTF-8");
|
||||
String path = "/nifi-api/reporting-tasks/node/" + URLEncoder.encode(id, "UTF-8");
|
||||
requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null);
|
||||
} catch (final URISyntaxException | UnsupportedEncodingException use) {
|
||||
throw new ClusterRequestException(use);
|
||||
|
@ -606,7 +606,7 @@ public class StandardNiFiWebConfigurationContext implements NiFiWebConfiguration
|
|||
// create the request URL
|
||||
URI requestUrl;
|
||||
try {
|
||||
String path = "/nifi-api/controller/reporting-tasks/node/" + URLEncoder.encode(id, "UTF-8");
|
||||
String path = "/nifi-api/reporting-tasks/node/" + URLEncoder.encode(id, "UTF-8");
|
||||
requestUrl = new URI(requestContext.getScheme(), null, "localhost", 0, path, null, null);
|
||||
} catch (final URISyntaxException | UnsupportedEncodingException use) {
|
||||
throw new ClusterRequestException(use);
|
||||
|
|
|
@ -79,12 +79,12 @@ import java.util.List;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing a cluster.
|
||||
* RESTful endpoint for managing access.
|
||||
*/
|
||||
@Path("/access")
|
||||
@Api(
|
||||
value = "/access",
|
||||
description = "Endpoints for obtaining an access token or checking access status"
|
||||
value = "/access",
|
||||
description = "Endpoints for obtaining an access token or checking access status."
|
||||
)
|
||||
public class AccessResource extends ApplicationResource {
|
||||
|
||||
|
|
|
@ -337,6 +337,12 @@ public abstract class ApplicationResource {
|
|||
return params;
|
||||
}
|
||||
|
||||
/**
|
||||
* Used when replicating requests to ensure the client id is the same on each node.
|
||||
*
|
||||
* @param entity entity
|
||||
* @return entity with the client id set
|
||||
*/
|
||||
protected Entity updateClientId(final Entity entity) {
|
||||
if (entity != null && entity.getRevision() != null && StringUtils.isBlank(entity.getRevision().getClientId())) {
|
||||
entity.getRevision().setClientId(new ClientIdParameter().getClientId());
|
||||
|
|
|
@ -1,187 +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.web.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 org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.api.dto.BulletinBoardDTO;
|
||||
import org.apache.nifi.web.api.dto.BulletinQueryDTO;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.entity.BulletinBoardEntity;
|
||||
import org.apache.nifi.web.api.request.BulletinBoardPatternParameter;
|
||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||
import org.apache.nifi.web.api.request.IntegerParameter;
|
||||
import org.apache.nifi.web.api.request.LongParameter;
|
||||
|
||||
import javax.ws.rs.Consumes;
|
||||
import javax.ws.rs.DefaultValue;
|
||||
import javax.ws.rs.GET;
|
||||
import javax.ws.rs.HttpMethod;
|
||||
import javax.ws.rs.Path;
|
||||
import javax.ws.rs.Produces;
|
||||
import javax.ws.rs.QueryParam;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.Response;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing a Template.
|
||||
*/
|
||||
@Path("bulletin-board")
|
||||
public class BulletinBoardResource extends ApplicationResource {
|
||||
|
||||
private NiFiProperties properties;
|
||||
private WebClusterManager clusterManager;
|
||||
|
||||
private NiFiServiceFacade serviceFacade;
|
||||
|
||||
/**
|
||||
* Retrieves all the of templates in this NiFi.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a
|
||||
* new one will be generated. This value (whether specified or generated) is
|
||||
* included in the response.
|
||||
* @param after Supporting querying for bulletins after a particular
|
||||
* bulletin id.
|
||||
* @param limit The max number of bulletins to return.
|
||||
* @param sourceName Source name filter. Supports a regular expression.
|
||||
* @param message Message filter. Supports a regular expression.
|
||||
* @param sourceId Source id filter. Supports a regular expression.
|
||||
* @param groupId Group id filter. Supports a regular expression.
|
||||
* @return A bulletinBoardEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("") // necessary due to bug in swagger
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets current bulletins",
|
||||
response = BulletinBoardEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@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 = 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 getBulletinBoard(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "Includes bulletins with an id after this value.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("after") LongParameter after,
|
||||
@ApiParam(
|
||||
value = "Includes bulletins originating from this sources whose name match this regular expression.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("sourceName") BulletinBoardPatternParameter sourceName,
|
||||
@ApiParam(
|
||||
value = "Includes bulletins whose message that match this regular expression.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("message") BulletinBoardPatternParameter message,
|
||||
@ApiParam(
|
||||
value = "Includes bulletins originating from this sources whose id match this regular expression.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("sourceId") BulletinBoardPatternParameter sourceId,
|
||||
@ApiParam(
|
||||
value = "Includes bulletins originating from this sources whose group id match this regular expression.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("groupId") BulletinBoardPatternParameter groupId,
|
||||
@ApiParam(
|
||||
value = "The number of bulletins to limit the response to.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("limit") IntegerParameter limit) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// build the bulletin query
|
||||
final BulletinQueryDTO query = new BulletinQueryDTO();
|
||||
|
||||
if (sourceId != null) {
|
||||
query.setSourceId(sourceId.getRawPattern());
|
||||
}
|
||||
if (groupId != null) {
|
||||
query.setGroupId(groupId.getRawPattern());
|
||||
}
|
||||
if (sourceName != null) {
|
||||
query.setName(sourceName.getRawPattern());
|
||||
}
|
||||
if (message != null) {
|
||||
query.setMessage(message.getRawPattern());
|
||||
}
|
||||
if (after != null) {
|
||||
query.setAfter(after.getLong());
|
||||
}
|
||||
if (limit != null) {
|
||||
query.setLimit(limit.getInteger());
|
||||
}
|
||||
|
||||
// get the bulletin board
|
||||
final BulletinBoardDTO bulletinBoard = serviceFacade.getBulletinBoard(query);
|
||||
|
||||
// create the revision
|
||||
RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
BulletinBoardEntity entity = new BulletinBoardEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setBulletinBoard(bulletinBoard);
|
||||
|
||||
// generate the response
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
// setters
|
||||
public void setServiceFacade(NiFiServiceFacade serviceFacade) {
|
||||
this.serviceFacade = serviceFacade;
|
||||
}
|
||||
|
||||
public void setClusterManager(WebClusterManager clusterManager) {
|
||||
this.clusterManager = clusterManager;
|
||||
}
|
||||
|
||||
public void setProperties(NiFiProperties properties) {
|
||||
this.properties = properties;
|
||||
}
|
||||
}
|
|
@ -66,7 +66,7 @@ import java.util.List;
|
|||
@Path("/cluster")
|
||||
@Api(
|
||||
value = "/cluster",
|
||||
description = "Provides access to the cluster of Nodes that comprise this NiFi"
|
||||
description = "Endpoint for managing the cluster."
|
||||
)
|
||||
public class ClusterResource extends ApplicationResource {
|
||||
|
||||
|
|
|
@ -16,39 +16,23 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
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 org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.cluster.context.ClusterContext;
|
||||
import org.apache.nifi.cluster.context.ClusterContextThreadLocal;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
|
||||
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
|
||||
import org.apache.nifi.cluster.node.Node;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.web.ConfigurationSnapshot;
|
||||
import org.apache.nifi.web.DownloadableContent;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.Revision;
|
||||
import org.apache.nifi.web.api.dto.ConnectionDTO;
|
||||
import org.apache.nifi.web.api.dto.DropRequestDTO;
|
||||
import org.apache.nifi.web.api.dto.FlowFileDTO;
|
||||
import org.apache.nifi.web.api.dto.FlowFileSummaryDTO;
|
||||
import org.apache.nifi.web.api.dto.ListingRequestDTO;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO;
|
||||
import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
|
||||
import org.apache.nifi.web.api.entity.ConnectionEntity;
|
||||
import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
|
||||
import org.apache.nifi.web.api.entity.DropRequestEntity;
|
||||
import org.apache.nifi.web.api.entity.FlowFileEntity;
|
||||
import org.apache.nifi.web.api.entity.ListingRequestEntity;
|
||||
import org.apache.nifi.web.api.entity.StatusHistoryEntity;
|
||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||
import org.apache.nifi.web.api.request.LongParameter;
|
||||
|
||||
|
@ -58,33 +42,27 @@ 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.WebApplicationException;
|
||||
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.StreamingOutput;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.net.URI;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing a Connection.
|
||||
*/
|
||||
@Path("connections")
|
||||
@Path("/connections")
|
||||
@Api(
|
||||
value = "/connections",
|
||||
description = "Endpoint for managing a Connection."
|
||||
)
|
||||
public class ConnectionResource extends ApplicationResource {
|
||||
|
||||
private NiFiServiceFacade serviceFacade;
|
||||
|
@ -211,169 +189,6 @@ public class ConnectionResource extends ApplicationResource {
|
|||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves the specified connection status.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param id The id of the connection history to retrieve.
|
||||
* @return A connectionStatusEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{id}/status")
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets status for a connection",
|
||||
response = ConnectionStatusEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@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 getConnectionStatus(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "Whether or not to include the breakdown per node. Optional, defaults to false",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("nodewise") @DefaultValue(NODEWISE) Boolean nodewise,
|
||||
@ApiParam(
|
||||
value = "The id of the node where to get the status.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("clusterNodeId") String clusterNodeId,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("id") String id) {
|
||||
|
||||
// ensure a valid request
|
||||
if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {
|
||||
throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node.");
|
||||
}
|
||||
|
||||
if (properties.isClusterManager()) {
|
||||
// determine where this request should be sent
|
||||
if (clusterNodeId == null) {
|
||||
final NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders());
|
||||
final ConnectionStatusEntity entity = (ConnectionStatusEntity) nodeResponse.getUpdatedEntity();
|
||||
|
||||
// ensure there is an updated entity (result of merging) and prune the response as necessary
|
||||
if (entity != null && !nodewise) {
|
||||
entity.getConnectionStatus().setNodeSnapshots(null);
|
||||
}
|
||||
|
||||
return nodeResponse.getResponse();
|
||||
} else {
|
||||
// get the target node and ensure it exists
|
||||
final Node targetNode = clusterManager.getNode(clusterNodeId);
|
||||
if (targetNode == null) {
|
||||
throw new UnknownNodeException("The specified cluster node does not exist.");
|
||||
}
|
||||
|
||||
final Set<NodeIdentifier> targetNodes = new HashSet<>();
|
||||
targetNodes.add(targetNode.getNodeId());
|
||||
|
||||
// replicate the request to the specific node
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
|
||||
}
|
||||
}
|
||||
|
||||
// get the specified connection status
|
||||
final ConnectionStatusDTO connectionStatus = serviceFacade.getConnectionStatus(id);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// generate the response entity
|
||||
final ConnectionStatusEntity entity = new ConnectionStatusEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setConnectionStatus(connectionStatus);
|
||||
|
||||
// generate the response
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves the specified connection status history.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param id The id of the connection to retrieve.
|
||||
* @return A statusHistoryEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{id}/status/history")
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets the status history for a connection",
|
||||
response = StatusHistoryEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@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 getConnectionStatusHistory(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("id") String id) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// get the specified processor status history
|
||||
final StatusHistoryDTO connectionStatusHistory = serviceFacade.getConnectionStatusHistory(id);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// generate the response entity
|
||||
final StatusHistoryEntity entity = new StatusHistoryEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setStatusHistory(connectionStatusHistory);
|
||||
|
||||
// generate the response
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the specified connection.
|
||||
*
|
||||
|
@ -557,631 +372,6 @@ public class ConnectionResource extends ApplicationResource {
|
|||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the specified flowfile from the specified connection.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param connectionId The connection id
|
||||
* @param flowFileUuid The flowfile uuid
|
||||
* @param clusterNodeId The cluster node id where the flowfile resides
|
||||
* @return a flowFileDTO
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{connection-id}/flowfiles/{flowfile-uuid}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Gets a FlowFile from a Connection.",
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 getFlowFile(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String connectionId,
|
||||
@ApiParam(
|
||||
value = "The flowfile uuid.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("flowfile-uuid") String flowFileUuid,
|
||||
@ApiParam(
|
||||
value = "The id of the node where the content exists if clustered.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("clusterNodeId") String clusterNodeId) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
// determine where this request should be sent
|
||||
if (clusterNodeId == null) {
|
||||
throw new IllegalArgumentException("The id of the node in the cluster is required.");
|
||||
} else {
|
||||
// get the target node and ensure it exists
|
||||
final Node targetNode = clusterManager.getNode(clusterNodeId);
|
||||
if (targetNode == null) {
|
||||
throw new UnknownNodeException("The specified cluster node does not exist.");
|
||||
}
|
||||
|
||||
final Set<NodeIdentifier> targetNodes = new HashSet<>();
|
||||
targetNodes.add(targetNode.getNodeId());
|
||||
|
||||
// replicate the request to the specific node
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
|
||||
}
|
||||
}
|
||||
|
||||
// get the flowfile
|
||||
final FlowFileDTO flowfileDto = serviceFacade.getFlowFile(connectionId, flowFileUuid);
|
||||
populateRemainingFlowFileContent(connectionId, flowfileDto);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
final FlowFileEntity entity = new FlowFileEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setFlowFile(flowfileDto);
|
||||
|
||||
return generateOkResponse(entity).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the content for the specified flowfile in the specified connection.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param connectionId The connection id
|
||||
* @param flowFileUuid The flowfile uuid
|
||||
* @param clusterNodeId The cluster node id
|
||||
* @return The content stream
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.WILDCARD)
|
||||
@Path("/{connection-id}/flowfiles/{flowfile-uuid}/content")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Gets the content for a FlowFile in a Connection.",
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 downloadFlowFileContent(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String connectionId,
|
||||
@ApiParam(
|
||||
value = "The flowfile uuid.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("flowfile-uuid") String flowFileUuid,
|
||||
@ApiParam(
|
||||
value = "The id of the node where the content exists if clustered.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("clusterNodeId") String clusterNodeId) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
// determine where this request should be sent
|
||||
if (clusterNodeId == null) {
|
||||
throw new IllegalArgumentException("The id of the node in the cluster is required.");
|
||||
} else {
|
||||
// get the target node and ensure it exists
|
||||
final Node targetNode = clusterManager.getNode(clusterNodeId);
|
||||
if (targetNode == null) {
|
||||
throw new UnknownNodeException("The specified cluster node does not exist.");
|
||||
}
|
||||
|
||||
final Set<NodeIdentifier> targetNodes = new HashSet<>();
|
||||
targetNodes.add(targetNode.getNodeId());
|
||||
|
||||
// replicate the request to the specific node
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
|
||||
}
|
||||
}
|
||||
|
||||
// get the uri of the request
|
||||
final String uri = generateResourceUri("connections", connectionId, "flowfiles", flowFileUuid, "content");
|
||||
|
||||
// get an input stream to the content
|
||||
final DownloadableContent content = serviceFacade.getContent(connectionId, flowFileUuid, uri);
|
||||
|
||||
// generate a streaming response
|
||||
final StreamingOutput response = new StreamingOutput() {
|
||||
@Override
|
||||
public void write(OutputStream output) throws IOException, WebApplicationException {
|
||||
try (InputStream is = content.getContent()) {
|
||||
// stream the content to the response
|
||||
StreamUtils.copy(is, output);
|
||||
|
||||
// flush the response
|
||||
output.flush();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// use the appropriate content type
|
||||
String contentType = content.getType();
|
||||
if (contentType == null) {
|
||||
contentType = MediaType.APPLICATION_OCTET_STREAM;
|
||||
}
|
||||
|
||||
return generateOkResponse(response).type(contentType).header("Content-Disposition", String.format("attachment; filename=\"%s\"", content.getFilename())).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a request to list the flowfiles in the queue of the specified connection.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param id The id of the connection
|
||||
* @return A listRequestEntity
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{connection-id}/listing-requests")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Lists the contents of the queue in this connection.",
|
||||
response = ListingRequestEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@ApiResponses(
|
||||
value = {
|
||||
@ApiResponse(code = 202, message = "The request has been accepted. A HTTP response header will contain the URI where the response can be polled."),
|
||||
@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 createFlowFileListing(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String id) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
serviceFacade.verifyListQueue(id);
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
// ensure the id is the same across the cluster
|
||||
final String listingRequestId;
|
||||
final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
|
||||
if (clusterContext != null) {
|
||||
listingRequestId = UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString();
|
||||
} else {
|
||||
listingRequestId = UUID.randomUUID().toString();
|
||||
}
|
||||
|
||||
// submit the listing request
|
||||
final ListingRequestDTO listingRequest = serviceFacade.createFlowFileListingRequest(id, listingRequestId);
|
||||
populateRemainingFlowFileListingContent(id, listingRequest);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
|
||||
// create the response entity
|
||||
final ListingRequestEntity entity = new ListingRequestEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setListingRequest(listingRequest);
|
||||
|
||||
// generate the URI where the response will be
|
||||
final URI location = URI.create(listingRequest.getUri());
|
||||
return Response.status(Status.ACCEPTED).location(location).entity(entity).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the status of an outstanding listing request.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param connectionId The id of the connection
|
||||
* @param listingRequestId The id of the drop request
|
||||
* @return A dropRequestEntity
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{connection-id}/listing-requests/{listing-request-id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Gets the current status of a listing request for the specified connection.",
|
||||
response = ListingRequestEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 getListingRequest(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String connectionId,
|
||||
@ApiParam(
|
||||
value = "The listing request id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("listing-request-id") String listingRequestId) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// get the listing request
|
||||
final ListingRequestDTO listingRequest = serviceFacade.getFlowFileListingRequest(connectionId, listingRequestId);
|
||||
populateRemainingFlowFileListingContent(connectionId, listingRequest);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
final ListingRequestEntity entity = new ListingRequestEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setListingRequest(listingRequest);
|
||||
|
||||
return generateOkResponse(entity).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes the specified listing request.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param connectionId The connection id
|
||||
* @param listingRequestId The drop request id
|
||||
* @return A dropRequestEntity
|
||||
*/
|
||||
@DELETE
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{connection-id}/listing-requests/{listing-request-id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Cancels and/or removes a request to list the contents of this connection.",
|
||||
response = DropRequestEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 deleteListingRequest(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String connectionId,
|
||||
@ApiParam(
|
||||
value = "The listing request id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("listing-request-id") String listingRequestId) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
// delete the listing request
|
||||
final ListingRequestDTO listingRequest = serviceFacade.deleteFlowFileListingRequest(connectionId, listingRequestId);
|
||||
|
||||
// prune the results as they were already received when the listing completed
|
||||
listingRequest.setFlowFileSummaries(null);
|
||||
|
||||
// populate remaining content
|
||||
populateRemainingFlowFileListingContent(connectionId, listingRequest);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
final ListingRequestEntity entity = new ListingRequestEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setListingRequest(listingRequest);
|
||||
|
||||
return generateOkResponse(entity).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a request to delete the flowfiles in the queue of the specified connection.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param id The id of the connection
|
||||
* @return A dropRequestEntity
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{connection-id}/drop-requests")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Creates a request to drop the contents of the queue in this connection.",
|
||||
response = DropRequestEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@ApiResponses(
|
||||
value = {
|
||||
@ApiResponse(code = 202, message = "The request has been accepted. A HTTP response header will contain the URI where the response can be polled."),
|
||||
@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 createDropRequest(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String id) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
// ensure the id is the same across the cluster
|
||||
final String dropRequestId;
|
||||
final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
|
||||
if (clusterContext != null) {
|
||||
dropRequestId = UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString();
|
||||
} else {
|
||||
dropRequestId = UUID.randomUUID().toString();
|
||||
}
|
||||
|
||||
// submit the drop request
|
||||
final DropRequestDTO dropRequest = serviceFacade.createFlowFileDropRequest(id, dropRequestId);
|
||||
dropRequest.setUri(generateResourceUri("connections", id, "drop-requests", dropRequest.getId()));
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
|
||||
// create the response entity
|
||||
final DropRequestEntity entity = new DropRequestEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setDropRequest(dropRequest);
|
||||
|
||||
// generate the URI where the response will be
|
||||
final URI location = URI.create(dropRequest.getUri());
|
||||
return Response.status(Status.ACCEPTED).location(location).entity(entity).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the status of an outstanding drop request.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param connectionId The id of the connection
|
||||
* @param dropRequestId The id of the drop request
|
||||
* @return A dropRequestEntity
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{connection-id}/drop-requests/{drop-request-id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Gets the current status of a drop request for the specified connection.",
|
||||
response = DropRequestEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 getDropRequest(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String connectionId,
|
||||
@ApiParam(
|
||||
value = "The drop request id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("drop-request-id") String dropRequestId) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// get the drop request
|
||||
final DropRequestDTO dropRequest = serviceFacade.getFlowFileDropRequest(connectionId, dropRequestId);
|
||||
dropRequest.setUri(generateResourceUri("connections", connectionId, "drop-requests", dropRequestId));
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
final DropRequestEntity entity = new DropRequestEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setDropRequest(dropRequest);
|
||||
|
||||
return generateOkResponse(entity).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes the specified drop request.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param connectionId The connection id
|
||||
* @param dropRequestId The drop request id
|
||||
* @return A dropRequestEntity
|
||||
*/
|
||||
@DELETE
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{connection-id}/drop-requests/{drop-request-id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Cancels and/or removes a request to drop the contents of this connection.",
|
||||
response = DropRequestEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 removeDropRequest(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String connectionId,
|
||||
@ApiParam(
|
||||
value = "The drop request id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("drop-request-id") String dropRequestId) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
// delete the drop request
|
||||
final DropRequestDTO dropRequest = serviceFacade.deleteFlowFileDropRequest(connectionId, dropRequestId);
|
||||
dropRequest.setUri(generateResourceUri("connections", connectionId, "drop-requests", dropRequestId));
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
final DropRequestEntity entity = new DropRequestEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setDropRequest(dropRequest);
|
||||
|
||||
return generateOkResponse(entity).build();
|
||||
}
|
||||
|
||||
// setters
|
||||
public void setServiceFacade(NiFiServiceFacade serviceFacade) {
|
||||
this.serviceFacade = serviceFacade;
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -16,14 +16,13 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
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 org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.cluster.context.ClusterContext;
|
||||
import org.apache.nifi.cluster.context.ClusterContextThreadLocal;
|
||||
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
|
||||
import org.apache.nifi.controller.ScheduledState;
|
||||
import org.apache.nifi.controller.service.ControllerServiceState;
|
||||
|
@ -42,7 +41,6 @@ import org.apache.nifi.web.api.dto.RevisionDTO;
|
|||
import org.apache.nifi.web.api.entity.ComponentStateEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServicesEntity;
|
||||
import org.apache.nifi.web.api.entity.Entity;
|
||||
import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
|
||||
import org.apache.nifi.web.api.entity.UpdateControllerServiceReferenceRequestEntity;
|
||||
|
@ -69,17 +67,19 @@ import javax.ws.rs.core.Context;
|
|||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.net.URI;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing a Controller Service.
|
||||
*/
|
||||
@Path("controller-services")
|
||||
@Path("/controller-services")
|
||||
@Api(
|
||||
value = "/controller-services",
|
||||
description = "Endpoint for managing a Controller Service."
|
||||
)
|
||||
public class ControllerServiceResource extends ApplicationResource {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(ControllerServiceResource.class);
|
||||
|
@ -133,7 +133,7 @@ public class ControllerServiceResource extends ApplicationResource {
|
|||
* @param availability avail
|
||||
* @return avail
|
||||
*/
|
||||
private Availability parseAvailability(final String availability) {
|
||||
public Availability parseAvailability(final String availability) {
|
||||
final Availability avail;
|
||||
try {
|
||||
avail = Availability.valueOf(availability.toUpperCase());
|
||||
|
@ -149,176 +149,6 @@ public class ControllerServiceResource extends ApplicationResource {
|
|||
return avail;
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves all the of controller services in this NiFi.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a
|
||||
* new one will be generated. This value (whether specified or generated) is
|
||||
* included in the response.
|
||||
* @param availability Whether the controller service is available on the
|
||||
* NCM only (ncm) or on the nodes only (node). If this instance is not
|
||||
* clustered all services should use the node availability.
|
||||
* @return A controllerServicesEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("{availability}")
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets all controller services",
|
||||
response = ControllerServicesEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@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 = 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 getControllerServices(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "Whether the controller service is available on the NCM or nodes. If the NiFi is standalone the availability should be NODE.",
|
||||
allowableValues = "NCM, NODE",
|
||||
required = true
|
||||
)
|
||||
@PathParam("availability") String availability) {
|
||||
|
||||
final Availability avail = parseAvailability(availability);
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager() && Availability.NODE.equals(avail)) {
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// get all the controller services
|
||||
final Set<ControllerServiceDTO> controllerServices = populateRemainingControllerServicesContent(availability, serviceFacade.getControllerServices());
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
final ControllerServicesEntity entity = new ControllerServicesEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setControllerServices(controllerServices);
|
||||
|
||||
// generate the response
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new Controller Service.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param availability Whether the controller service is available on the
|
||||
* NCM only (ncm) or on the nodes only (node). If this instance is not
|
||||
* clustered all services should use the node availability.
|
||||
* @param controllerServiceEntity A controllerServiceEntity.
|
||||
* @return A controllerServiceEntity.
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("{availability}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Creates a new controller service",
|
||||
response = ControllerServiceEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 = 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 createControllerService(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "Whether the controller service is available on the NCM or nodes. If the NiFi is standalone the availability should be NODE.",
|
||||
allowableValues = "NCM, NODE",
|
||||
required = true
|
||||
)
|
||||
@PathParam("availability") String availability,
|
||||
@ApiParam(
|
||||
value = "The controller service configuration details.",
|
||||
required = true
|
||||
) ControllerServiceEntity controllerServiceEntity) {
|
||||
|
||||
final Availability avail = parseAvailability(availability);
|
||||
|
||||
if (controllerServiceEntity == null || controllerServiceEntity.getControllerService() == null) {
|
||||
throw new IllegalArgumentException("Controller service details must be specified.");
|
||||
}
|
||||
|
||||
if (controllerServiceEntity.getRevision() == null) {
|
||||
throw new IllegalArgumentException("Revision must be specified.");
|
||||
}
|
||||
|
||||
if (controllerServiceEntity.getControllerService().getId() != null) {
|
||||
throw new IllegalArgumentException("Controller service ID cannot be specified.");
|
||||
}
|
||||
|
||||
if (StringUtils.isBlank(controllerServiceEntity.getControllerService().getType())) {
|
||||
throw new IllegalArgumentException("The type of controller service to create must be specified.");
|
||||
}
|
||||
|
||||
// get the revision
|
||||
final RevisionDTO revision = controllerServiceEntity.getRevision();
|
||||
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(controllerServiceEntity), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
// set the processor id as appropriate
|
||||
final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
|
||||
if (clusterContext != null) {
|
||||
controllerServiceEntity.getControllerService().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
|
||||
} else {
|
||||
controllerServiceEntity.getControllerService().setId(UUID.randomUUID().toString());
|
||||
}
|
||||
|
||||
// create the controller service and generate the json
|
||||
final ConfigurationSnapshot<ControllerServiceDTO> controllerResponse = serviceFacade.createControllerService(
|
||||
new Revision(revision.getVersion(), revision.getClientId()), controllerServiceEntity.getControllerService());
|
||||
final ControllerServiceDTO controllerService = controllerResponse.getConfiguration();
|
||||
|
||||
// get the updated revision
|
||||
final RevisionDTO updatedRevision = new RevisionDTO();
|
||||
updatedRevision.setClientId(revision.getClientId());
|
||||
updatedRevision.setVersion(controllerResponse.getVersion());
|
||||
|
||||
// build the response entity
|
||||
final ControllerServiceEntity entity = new ControllerServiceEntity();
|
||||
entity.setRevision(updatedRevision);
|
||||
entity.setControllerService(populateRemainingControllerServiceContent(availability, controllerService));
|
||||
|
||||
// build the response
|
||||
return clusterContext(generateCreatedResponse(URI.create(controllerService.getUri()), entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves the specified controller service.
|
||||
*
|
||||
|
|
|
@ -0,0 +1,755 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
import 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 org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.cluster.context.ClusterContext;
|
||||
import org.apache.nifi.cluster.context.ClusterContextThreadLocal;
|
||||
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
|
||||
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
|
||||
import org.apache.nifi.cluster.node.Node;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.web.DownloadableContent;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.api.dto.DropRequestDTO;
|
||||
import org.apache.nifi.web.api.dto.FlowFileDTO;
|
||||
import org.apache.nifi.web.api.dto.FlowFileSummaryDTO;
|
||||
import org.apache.nifi.web.api.dto.ListingRequestDTO;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.entity.DropRequestEntity;
|
||||
import org.apache.nifi.web.api.entity.FlowFileEntity;
|
||||
import org.apache.nifi.web.api.entity.ListingRequestEntity;
|
||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||
|
||||
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.Path;
|
||||
import javax.ws.rs.PathParam;
|
||||
import javax.ws.rs.Produces;
|
||||
import javax.ws.rs.QueryParam;
|
||||
import javax.ws.rs.WebApplicationException;
|
||||
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.StreamingOutput;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.net.URI;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing a flowfile queue.
|
||||
*/
|
||||
@Path("/flowfile-queues")
|
||||
@Api(
|
||||
value = "/flowfile-queues",
|
||||
description = "Endpoint for managing a FlowFile Queue."
|
||||
)
|
||||
public class FlowFileQueueResource extends ApplicationResource {
|
||||
|
||||
private NiFiServiceFacade serviceFacade;
|
||||
private WebClusterManager clusterManager;
|
||||
private NiFiProperties properties;
|
||||
|
||||
/**
|
||||
* Populate the URIs for the specified flowfile listing.
|
||||
*
|
||||
* @param connectionId connection
|
||||
* @param flowFileListing flowfile listing
|
||||
* @return dto
|
||||
*/
|
||||
public ListingRequestDTO populateRemainingFlowFileListingContent(final String connectionId, final ListingRequestDTO flowFileListing) {
|
||||
// uri of the listing
|
||||
flowFileListing.setUri(generateResourceUri("flowfile-queues", connectionId, "listing-requests", flowFileListing.getId()));
|
||||
|
||||
// uri of each flowfile
|
||||
if (flowFileListing.getFlowFileSummaries() != null) {
|
||||
for (FlowFileSummaryDTO flowFile : flowFileListing.getFlowFileSummaries()) {
|
||||
populateRemainingFlowFileContent(connectionId, flowFile);
|
||||
}
|
||||
}
|
||||
return flowFileListing;
|
||||
}
|
||||
|
||||
/**
|
||||
* Populate the URIs for the specified flowfile.
|
||||
*
|
||||
* @param connectionId the connection id
|
||||
* @param flowFile the flowfile
|
||||
* @return the dto
|
||||
*/
|
||||
public FlowFileSummaryDTO populateRemainingFlowFileContent(final String connectionId, final FlowFileSummaryDTO flowFile) {
|
||||
flowFile.setUri(generateResourceUri("flowfile-queues", connectionId, "flowfiles", flowFile.getUuid()));
|
||||
return flowFile;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the specified flowfile from the specified connection.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param connectionId The connection id
|
||||
* @param flowFileUuid The flowfile uuid
|
||||
* @param clusterNodeId The cluster node id where the flowfile resides
|
||||
* @return a flowFileDTO
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("{connection-id}/flowfiles/{flowfile-uuid}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Gets a FlowFile from a Connection.",
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 getFlowFile(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String connectionId,
|
||||
@ApiParam(
|
||||
value = "The flowfile uuid.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("flowfile-uuid") String flowFileUuid,
|
||||
@ApiParam(
|
||||
value = "The id of the node where the content exists if clustered.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("clusterNodeId") String clusterNodeId) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
// determine where this request should be sent
|
||||
if (clusterNodeId == null) {
|
||||
throw new IllegalArgumentException("The id of the node in the cluster is required.");
|
||||
} else {
|
||||
// get the target node and ensure it exists
|
||||
final Node targetNode = clusterManager.getNode(clusterNodeId);
|
||||
if (targetNode == null) {
|
||||
throw new UnknownNodeException("The specified cluster node does not exist.");
|
||||
}
|
||||
|
||||
final Set<NodeIdentifier> targetNodes = new HashSet<>();
|
||||
targetNodes.add(targetNode.getNodeId());
|
||||
|
||||
// replicate the request to the specific node
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
|
||||
}
|
||||
}
|
||||
|
||||
// get the flowfile
|
||||
final FlowFileDTO flowfileDto = serviceFacade.getFlowFile(connectionId, flowFileUuid);
|
||||
populateRemainingFlowFileContent(connectionId, flowfileDto);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
final FlowFileEntity entity = new FlowFileEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setFlowFile(flowfileDto);
|
||||
|
||||
return generateOkResponse(entity).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the content for the specified flowfile in the specified connection.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param connectionId The connection id
|
||||
* @param flowFileUuid The flowfile uuid
|
||||
* @param clusterNodeId The cluster node id
|
||||
* @return The content stream
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.WILDCARD)
|
||||
@Path("{connection-id}/flowfiles/{flowfile-uuid}/content")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Gets the content for a FlowFile in a Connection.",
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 downloadFlowFileContent(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String connectionId,
|
||||
@ApiParam(
|
||||
value = "The flowfile uuid.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("flowfile-uuid") String flowFileUuid,
|
||||
@ApiParam(
|
||||
value = "The id of the node where the content exists if clustered.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("clusterNodeId") String clusterNodeId) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
// determine where this request should be sent
|
||||
if (clusterNodeId == null) {
|
||||
throw new IllegalArgumentException("The id of the node in the cluster is required.");
|
||||
} else {
|
||||
// get the target node and ensure it exists
|
||||
final Node targetNode = clusterManager.getNode(clusterNodeId);
|
||||
if (targetNode == null) {
|
||||
throw new UnknownNodeException("The specified cluster node does not exist.");
|
||||
}
|
||||
|
||||
final Set<NodeIdentifier> targetNodes = new HashSet<>();
|
||||
targetNodes.add(targetNode.getNodeId());
|
||||
|
||||
// replicate the request to the specific node
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
|
||||
}
|
||||
}
|
||||
|
||||
// get the uri of the request
|
||||
final String uri = generateResourceUri("flowfile-queues", connectionId, "flowfiles", flowFileUuid, "content");
|
||||
|
||||
// get an input stream to the content
|
||||
final DownloadableContent content = serviceFacade.getContent(connectionId, flowFileUuid, uri);
|
||||
|
||||
// generate a streaming response
|
||||
final StreamingOutput response = new StreamingOutput() {
|
||||
@Override
|
||||
public void write(OutputStream output) throws IOException, WebApplicationException {
|
||||
try (InputStream is = content.getContent()) {
|
||||
// stream the content to the response
|
||||
StreamUtils.copy(is, output);
|
||||
|
||||
// flush the response
|
||||
output.flush();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// use the appropriate content type
|
||||
String contentType = content.getType();
|
||||
if (contentType == null) {
|
||||
contentType = MediaType.APPLICATION_OCTET_STREAM;
|
||||
}
|
||||
|
||||
return generateOkResponse(response).type(contentType).header("Content-Disposition", String.format("attachment; filename=\"%s\"", content.getFilename())).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a request to list the flowfiles in the queue of the specified connection.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param id The id of the connection
|
||||
* @return A listRequestEntity
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("{connection-id}/listing-requests")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Lists the contents of the queue in this connection.",
|
||||
response = ListingRequestEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@ApiResponses(
|
||||
value = {
|
||||
@ApiResponse(code = 202, message = "The request has been accepted. A HTTP response header will contain the URI where the response can be polled."),
|
||||
@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 createFlowFileListing(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String id) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
serviceFacade.verifyListQueue(id);
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
// ensure the id is the same across the cluster
|
||||
final String listingRequestId;
|
||||
final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
|
||||
if (clusterContext != null) {
|
||||
listingRequestId = UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString();
|
||||
} else {
|
||||
listingRequestId = UUID.randomUUID().toString();
|
||||
}
|
||||
|
||||
// submit the listing request
|
||||
final ListingRequestDTO listingRequest = serviceFacade.createFlowFileListingRequest(id, listingRequestId);
|
||||
populateRemainingFlowFileListingContent(id, listingRequest);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
|
||||
// create the response entity
|
||||
final ListingRequestEntity entity = new ListingRequestEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setListingRequest(listingRequest);
|
||||
|
||||
// generate the URI where the response will be
|
||||
final URI location = URI.create(listingRequest.getUri());
|
||||
return Response.status(Status.ACCEPTED).location(location).entity(entity).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the status of an outstanding listing request.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param connectionId The id of the connection
|
||||
* @param listingRequestId The id of the drop request
|
||||
* @return A dropRequestEntity
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("{connection-id}/listing-requests/{listing-request-id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Gets the current status of a listing request for the specified connection.",
|
||||
response = ListingRequestEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 getListingRequest(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String connectionId,
|
||||
@ApiParam(
|
||||
value = "The listing request id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("listing-request-id") String listingRequestId) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// get the listing request
|
||||
final ListingRequestDTO listingRequest = serviceFacade.getFlowFileListingRequest(connectionId, listingRequestId);
|
||||
populateRemainingFlowFileListingContent(connectionId, listingRequest);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
final ListingRequestEntity entity = new ListingRequestEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setListingRequest(listingRequest);
|
||||
|
||||
return generateOkResponse(entity).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes the specified listing request.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param connectionId The connection id
|
||||
* @param listingRequestId The drop request id
|
||||
* @return A dropRequestEntity
|
||||
*/
|
||||
@DELETE
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("{connection-id}/listing-requests/{listing-request-id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Cancels and/or removes a request to list the contents of this connection.",
|
||||
response = DropRequestEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 deleteListingRequest(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String connectionId,
|
||||
@ApiParam(
|
||||
value = "The listing request id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("listing-request-id") String listingRequestId) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
// delete the listing request
|
||||
final ListingRequestDTO listingRequest = serviceFacade.deleteFlowFileListingRequest(connectionId, listingRequestId);
|
||||
|
||||
// prune the results as they were already received when the listing completed
|
||||
listingRequest.setFlowFileSummaries(null);
|
||||
|
||||
// populate remaining content
|
||||
populateRemainingFlowFileListingContent(connectionId, listingRequest);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
final ListingRequestEntity entity = new ListingRequestEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setListingRequest(listingRequest);
|
||||
|
||||
return generateOkResponse(entity).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a request to delete the flowfiles in the queue of the specified connection.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param id The id of the connection
|
||||
* @return A dropRequestEntity
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("{connection-id}/drop-requests")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Creates a request to drop the contents of the queue in this connection.",
|
||||
response = DropRequestEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@ApiResponses(
|
||||
value = {
|
||||
@ApiResponse(code = 202, message = "The request has been accepted. A HTTP response header will contain the URI where the response can be polled."),
|
||||
@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 createDropRequest(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String id) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
// ensure the id is the same across the cluster
|
||||
final String dropRequestId;
|
||||
final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
|
||||
if (clusterContext != null) {
|
||||
dropRequestId = UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString();
|
||||
} else {
|
||||
dropRequestId = UUID.randomUUID().toString();
|
||||
}
|
||||
|
||||
// submit the drop request
|
||||
final DropRequestDTO dropRequest = serviceFacade.createFlowFileDropRequest(id, dropRequestId);
|
||||
dropRequest.setUri(generateResourceUri("flowfile-queues", id, "drop-requests", dropRequest.getId()));
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
|
||||
// create the response entity
|
||||
final DropRequestEntity entity = new DropRequestEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setDropRequest(dropRequest);
|
||||
|
||||
// generate the URI where the response will be
|
||||
final URI location = URI.create(dropRequest.getUri());
|
||||
return Response.status(Status.ACCEPTED).location(location).entity(entity).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the status of an outstanding drop request.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param connectionId The id of the connection
|
||||
* @param dropRequestId The id of the drop request
|
||||
* @return A dropRequestEntity
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("{connection-id}/drop-requests/{drop-request-id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Gets the current status of a drop request for the specified connection.",
|
||||
response = DropRequestEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 getDropRequest(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String connectionId,
|
||||
@ApiParam(
|
||||
value = "The drop request id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("drop-request-id") String dropRequestId) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// get the drop request
|
||||
final DropRequestDTO dropRequest = serviceFacade.getFlowFileDropRequest(connectionId, dropRequestId);
|
||||
dropRequest.setUri(generateResourceUri("flowfile-queues", connectionId, "drop-requests", dropRequestId));
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
final DropRequestEntity entity = new DropRequestEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setDropRequest(dropRequest);
|
||||
|
||||
return generateOkResponse(entity).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes the specified drop request.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param connectionId The connection id
|
||||
* @param dropRequestId The drop request id
|
||||
* @return A dropRequestEntity
|
||||
*/
|
||||
@DELETE
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("{connection-id}/drop-requests/{drop-request-id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Cancels and/or removes a request to drop the contents of this connection.",
|
||||
response = DropRequestEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 removeDropRequest(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The connection id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("connection-id") String connectionId,
|
||||
@ApiParam(
|
||||
value = "The drop request id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("drop-request-id") String dropRequestId) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
// delete the drop request
|
||||
final DropRequestDTO dropRequest = serviceFacade.deleteFlowFileDropRequest(connectionId, dropRequestId);
|
||||
dropRequest.setUri(generateResourceUri("flowfile-queues", connectionId, "drop-requests", dropRequestId));
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
final DropRequestEntity entity = new DropRequestEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setDropRequest(dropRequest);
|
||||
|
||||
return generateOkResponse(entity).build();
|
||||
}
|
||||
|
||||
// setters
|
||||
public void setServiceFacade(NiFiServiceFacade serviceFacade) {
|
||||
this.serviceFacade = serviceFacade;
|
||||
}
|
||||
|
||||
public void setClusterManager(WebClusterManager clusterManager) {
|
||||
this.clusterManager = clusterManager;
|
||||
}
|
||||
|
||||
public void setProperties(NiFiProperties properties) {
|
||||
this.properties = properties;
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
import com.wordnik.swagger.annotations.Api;
|
||||
import com.wordnik.swagger.annotations.ApiOperation;
|
||||
import com.wordnik.swagger.annotations.ApiParam;
|
||||
import com.wordnik.swagger.annotations.ApiResponse;
|
||||
|
@ -57,7 +58,11 @@ import java.util.Set;
|
|||
/**
|
||||
* RESTful endpoint for managing a Funnel.
|
||||
*/
|
||||
@Path("funnels")
|
||||
@Path("/funnels")
|
||||
@Api(
|
||||
value = "/funnel",
|
||||
description = "Endpoint for managing a Funnel."
|
||||
)
|
||||
public class FunnelResource extends ApplicationResource {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(FunnelResource.class);
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
import com.wordnik.swagger.annotations.Api;
|
||||
import com.wordnik.swagger.annotations.ApiOperation;
|
||||
import com.wordnik.swagger.annotations.ApiParam;
|
||||
import com.wordnik.swagger.annotations.ApiResponse;
|
||||
|
@ -48,7 +49,11 @@ import javax.ws.rs.core.Response;
|
|||
/**
|
||||
* RESTful endpoint for querying the history of this Controller.
|
||||
*/
|
||||
@Path("history")
|
||||
@Path("/history")
|
||||
@Api(
|
||||
value = "/history",
|
||||
description = "Endpoint for accessing flow history."
|
||||
)
|
||||
public class HistoryResource extends ApplicationResource {
|
||||
|
||||
private NiFiServiceFacade serviceFacade;
|
||||
|
|
|
@ -16,26 +16,21 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
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 org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
|
||||
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
|
||||
import org.apache.nifi.cluster.node.Node;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.web.ConfigurationSnapshot;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.Revision;
|
||||
import org.apache.nifi.web.api.dto.PortDTO;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.dto.status.PortStatusDTO;
|
||||
import org.apache.nifi.web.api.entity.InputPortEntity;
|
||||
import org.apache.nifi.web.api.entity.PortStatusEntity;
|
||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||
import org.apache.nifi.web.api.request.LongParameter;
|
||||
|
||||
|
@ -55,14 +50,17 @@ import javax.ws.rs.core.MediaType;
|
|||
import javax.ws.rs.core.Response;
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing an Input Port.
|
||||
*/
|
||||
@Path("input-ports")
|
||||
@Path("/input-ports")
|
||||
@Api(
|
||||
value = "/input-ports",
|
||||
description = "Endpoint for managing an Input Port."
|
||||
)
|
||||
public class InputPortResource extends ApplicationResource {
|
||||
|
||||
private NiFiServiceFacade serviceFacade;
|
||||
|
@ -153,106 +151,6 @@ public class InputPortResource extends ApplicationResource {
|
|||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves the specified input port status.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param id The id of the processor history to retrieve.
|
||||
* @return A portStatusEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{id}/status")
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets status for an input port",
|
||||
response = PortStatusEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@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 getInputPortStatus(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "Whether or not to include the breakdown per node. Optional, defaults to false",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("nodewise") @DefaultValue(NODEWISE) Boolean nodewise,
|
||||
@ApiParam(
|
||||
value = "The id of the node where to get the status.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("clusterNodeId") String clusterNodeId,
|
||||
@ApiParam(
|
||||
value = "The input port id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("id") String id) {
|
||||
|
||||
// ensure a valid request
|
||||
if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {
|
||||
throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node.");
|
||||
}
|
||||
|
||||
if (properties.isClusterManager()) {
|
||||
// determine where this request should be sent
|
||||
if (clusterNodeId == null) {
|
||||
final NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders());
|
||||
final PortStatusEntity entity = (PortStatusEntity) nodeResponse.getUpdatedEntity();
|
||||
|
||||
// ensure there is an updated entity (result of merging) and prune the response as necessary
|
||||
if (entity != null && !nodewise) {
|
||||
entity.getPortStatus().setNodeSnapshots(null);
|
||||
}
|
||||
|
||||
return nodeResponse.getResponse();
|
||||
} else {
|
||||
// get the target node and ensure it exists
|
||||
final Node targetNode = clusterManager.getNode(clusterNodeId);
|
||||
if (targetNode == null) {
|
||||
throw new UnknownNodeException("The specified cluster node does not exist.");
|
||||
}
|
||||
|
||||
final Set<NodeIdentifier> targetNodes = new HashSet<>();
|
||||
targetNodes.add(targetNode.getNodeId());
|
||||
|
||||
// replicate the request to the specific node
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
|
||||
}
|
||||
}
|
||||
|
||||
// get the specified input port status
|
||||
final PortStatusDTO portStatus = serviceFacade.getInputPortStatus(id);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// generate the response entity
|
||||
final PortStatusEntity entity = new PortStatusEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setPortStatus(portStatus);
|
||||
|
||||
// generate the response
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the specified input port.
|
||||
*
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
import com.wordnik.swagger.annotations.Api;
|
||||
import com.wordnik.swagger.annotations.ApiOperation;
|
||||
import com.wordnik.swagger.annotations.ApiParam;
|
||||
import com.wordnik.swagger.annotations.ApiResponse;
|
||||
|
@ -57,7 +58,11 @@ import java.util.Set;
|
|||
/**
|
||||
* RESTful endpoint for managing a Label.
|
||||
*/
|
||||
@Path("labels")
|
||||
@Path("/labels")
|
||||
@Api(
|
||||
value = "/labels",
|
||||
description = "Endpoint for managing a Label."
|
||||
)
|
||||
public class LabelResource extends ApplicationResource {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(LabelResource.class);
|
||||
|
|
|
@ -16,26 +16,21 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
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 org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
|
||||
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
|
||||
import org.apache.nifi.cluster.node.Node;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.web.ConfigurationSnapshot;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.Revision;
|
||||
import org.apache.nifi.web.api.dto.PortDTO;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.dto.status.PortStatusDTO;
|
||||
import org.apache.nifi.web.api.entity.OutputPortEntity;
|
||||
import org.apache.nifi.web.api.entity.PortStatusEntity;
|
||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||
import org.apache.nifi.web.api.request.LongParameter;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -57,14 +52,17 @@ import javax.ws.rs.core.MediaType;
|
|||
import javax.ws.rs.core.Response;
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing an Output Port.
|
||||
*/
|
||||
@Path("output-ports")
|
||||
@Path("/output-ports")
|
||||
@Api(
|
||||
value = "/output-ports",
|
||||
description = "Endpoint for managing an Output Port."
|
||||
)
|
||||
public class OutputPortResource extends ApplicationResource {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(OutputPortResource.class);
|
||||
|
@ -157,106 +155,6 @@ public class OutputPortResource extends ApplicationResource {
|
|||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves the specified output port status.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param id The id of the processor history to retrieve.
|
||||
* @return A portStatusEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{id}/status")
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets status for an output port",
|
||||
response = PortStatusEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@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 getOutputPortStatus(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "Whether or not to include the breakdown per node. Optional, defaults to false",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("nodewise") @DefaultValue(NODEWISE) Boolean nodewise,
|
||||
@ApiParam(
|
||||
value = "The id of the node where to get the status.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("clusterNodeId") String clusterNodeId,
|
||||
@ApiParam(
|
||||
value = "The output port id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("id") String id) {
|
||||
|
||||
// ensure a valid request
|
||||
if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {
|
||||
throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node.");
|
||||
}
|
||||
|
||||
if (properties.isClusterManager()) {
|
||||
// determine where this request should be sent
|
||||
if (clusterNodeId == null) {
|
||||
final NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders());
|
||||
final PortStatusEntity entity = (PortStatusEntity) nodeResponse.getUpdatedEntity();
|
||||
|
||||
// ensure there is an updated entity (result of merging) and prune the response as necessary
|
||||
if (entity != null && !nodewise) {
|
||||
entity.getPortStatus().setNodeSnapshots(null);
|
||||
}
|
||||
|
||||
return nodeResponse.getResponse();
|
||||
} else {
|
||||
// get the target node and ensure it exists
|
||||
final Node targetNode = clusterManager.getNode(clusterNodeId);
|
||||
if (targetNode == null) {
|
||||
throw new UnknownNodeException("The specified cluster node does not exist.");
|
||||
}
|
||||
|
||||
final Set<NodeIdentifier> targetNodes = new HashSet<>();
|
||||
targetNodes.add(targetNode.getNodeId());
|
||||
|
||||
// replicate the request to the specific node
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
|
||||
}
|
||||
}
|
||||
|
||||
// get the specified output port status
|
||||
final PortStatusDTO portStatus = serviceFacade.getOutputPortStatus(id);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// generate the response entity
|
||||
final PortStatusEntity entity = new PortStatusEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setPortStatus(portStatus);
|
||||
|
||||
// generate the response
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the specified output port.
|
||||
*
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -16,18 +16,16 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
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 org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException;
|
||||
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
|
||||
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
|
||||
import org.apache.nifi.cluster.node.Node;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.scheduling.SchedulingStrategy;
|
||||
import org.apache.nifi.ui.extension.UiExtension;
|
||||
import org.apache.nifi.ui.extension.UiExtensionMapping;
|
||||
|
@ -41,14 +39,10 @@ import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
|
|||
import org.apache.nifi.web.api.dto.ProcessorDTO;
|
||||
import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO;
|
||||
import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
|
||||
import org.apache.nifi.web.api.entity.ComponentStateEntity;
|
||||
import org.apache.nifi.web.api.entity.Entity;
|
||||
import org.apache.nifi.web.api.entity.ProcessorEntity;
|
||||
import org.apache.nifi.web.api.entity.ProcessorStatusEntity;
|
||||
import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
|
||||
import org.apache.nifi.web.api.entity.StatusHistoryEntity;
|
||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||
import org.apache.nifi.web.api.request.LongParameter;
|
||||
|
||||
|
@ -70,14 +64,17 @@ import javax.ws.rs.core.MediaType;
|
|||
import javax.ws.rs.core.Response;
|
||||
import java.net.URI;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing a Processor.
|
||||
*/
|
||||
@Path("processors")
|
||||
@Path("/processors")
|
||||
@Api(
|
||||
value = "/processors",
|
||||
description = "Endpoint for managing a Processor."
|
||||
)
|
||||
public class ProcessorResource extends ApplicationResource {
|
||||
|
||||
private static final List<Long> POSSIBLE_RUN_DURATIONS = Arrays.asList(0L, 25L, 50L, 100L, 250L, 500L, 1000L, 2000L);
|
||||
|
@ -196,169 +193,6 @@ public class ProcessorResource extends ApplicationResource {
|
|||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves the specified processor status.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param id The id of the processor history to retrieve.
|
||||
* @return A processorStatusEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{id}/status")
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets status for a processor",
|
||||
response = ProcessorStatusEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@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 getProcessorStatus(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "Whether or not to include the breakdown per node. Optional, defaults to false",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("nodewise") @DefaultValue(NODEWISE) Boolean nodewise,
|
||||
@ApiParam(
|
||||
value = "The id of the node where to get the status.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("clusterNodeId") String clusterNodeId,
|
||||
@ApiParam(
|
||||
value = "The processor id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("id") String id) {
|
||||
|
||||
// ensure a valid request
|
||||
if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {
|
||||
throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node.");
|
||||
}
|
||||
|
||||
if (properties.isClusterManager()) {
|
||||
// determine where this request should be sent
|
||||
if (clusterNodeId == null) {
|
||||
final NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders());
|
||||
final ProcessorStatusEntity entity = (ProcessorStatusEntity) nodeResponse.getUpdatedEntity();
|
||||
|
||||
// ensure there is an updated entity (result of merging) and prune the response as necessary
|
||||
if (entity != null && !nodewise) {
|
||||
entity.getProcessorStatus().setNodeSnapshots(null);
|
||||
}
|
||||
|
||||
return nodeResponse.getResponse();
|
||||
} else {
|
||||
// get the target node and ensure it exists
|
||||
final Node targetNode = clusterManager.getNode(clusterNodeId);
|
||||
if (targetNode == null) {
|
||||
throw new UnknownNodeException("The specified cluster node does not exist.");
|
||||
}
|
||||
|
||||
final Set<NodeIdentifier> targetNodes = new HashSet<>();
|
||||
targetNodes.add(targetNode.getNodeId());
|
||||
|
||||
// replicate the request to the specific node
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
|
||||
}
|
||||
}
|
||||
|
||||
// get the specified processor status
|
||||
final ProcessorStatusDTO processorStatus = serviceFacade.getProcessorStatus(id);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// generate the response entity
|
||||
final ProcessorStatusEntity entity = new ProcessorStatusEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setProcessorStatus(processorStatus);
|
||||
|
||||
// generate the response
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves the specified processor status history.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param id The id of the processor history to retrieve.
|
||||
* @return A statusHistoryEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{id}/status/history")
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets status history for a processor",
|
||||
response = StatusHistoryEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@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 getProcessorStatusHistory(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The processor id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("id") String id) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// get the specified processor status history
|
||||
final StatusHistoryDTO processorStatusHistory = serviceFacade.getProcessorStatusHistory(id);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// generate the response entity
|
||||
final StatusHistoryEntity entity = new StatusHistoryEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setStatusHistory(processorStatusHistory);
|
||||
|
||||
// generate the response
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the descriptor for the specified property.
|
||||
*
|
||||
|
|
|
@ -38,17 +38,14 @@ import org.apache.nifi.web.api.dto.RevisionDTO;
|
|||
import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO;
|
||||
import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO;
|
||||
import org.apache.nifi.web.api.dto.provenance.ProvenanceOptionsDTO;
|
||||
import org.apache.nifi.web.api.dto.provenance.ProvenanceRequestDTO;
|
||||
import org.apache.nifi.web.api.dto.provenance.lineage.LineageDTO;
|
||||
import org.apache.nifi.web.api.dto.provenance.lineage.LineageRequestDTO;
|
||||
import org.apache.nifi.web.api.dto.provenance.lineage.LineageRequestDTO.LineageRequestType;
|
||||
import org.apache.nifi.web.api.entity.LineageEntity;
|
||||
import org.apache.nifi.web.api.entity.ProvenanceEntity;
|
||||
import org.apache.nifi.web.api.entity.ProvenanceEventEntity;
|
||||
import org.apache.nifi.web.api.entity.ProvenanceOptionsEntity;
|
||||
import org.apache.nifi.web.api.entity.SubmitReplayRequestEntity;
|
||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||
import org.apache.nifi.web.api.request.DateTimeParameter;
|
||||
import org.apache.nifi.web.api.request.IntegerParameter;
|
||||
import org.apache.nifi.web.api.request.LongParameter;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -57,7 +54,6 @@ import javax.servlet.http.HttpServletRequest;
|
|||
import javax.ws.rs.Consumes;
|
||||
import javax.ws.rs.DELETE;
|
||||
import javax.ws.rs.DefaultValue;
|
||||
import javax.ws.rs.FormParam;
|
||||
import javax.ws.rs.GET;
|
||||
import javax.ws.rs.HttpMethod;
|
||||
import javax.ws.rs.POST;
|
||||
|
@ -68,7 +64,6 @@ import javax.ws.rs.QueryParam;
|
|||
import javax.ws.rs.WebApplicationException;
|
||||
import javax.ws.rs.core.Context;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.MultivaluedMap;
|
||||
import javax.ws.rs.core.Response;
|
||||
import javax.ws.rs.core.StreamingOutput;
|
||||
import java.io.IOException;
|
||||
|
@ -78,7 +73,6 @@ import java.net.URI;
|
|||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
|
@ -86,7 +80,11 @@ import java.util.UUID;
|
|||
/**
|
||||
* RESTful endpoint for querying data provenance.
|
||||
*/
|
||||
@Api(hidden = true)
|
||||
@Path("/provenance")
|
||||
@Api(
|
||||
value = "/provenance",
|
||||
description = "Endpoint for accessing data flow provenance."
|
||||
)
|
||||
public class ProvenanceResource extends ApplicationResource {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(ProvenanceResource.class);
|
||||
|
@ -100,7 +98,7 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
* Populates the uri for the specified provenance.
|
||||
*/
|
||||
private ProvenanceDTO populateRemainingProvenanceContent(ProvenanceDTO provenance) {
|
||||
provenance.setUri(generateResourceUri("controller", "provenance", provenance.getId()));
|
||||
provenance.setUri(generateResourceUri("provenance", provenance.getId()));
|
||||
return provenance;
|
||||
}
|
||||
|
||||
|
@ -108,7 +106,7 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
* Populates the uri for the specified lineage.
|
||||
*/
|
||||
private LineageDTO populateRemainingLineageContent(LineageDTO lineage) {
|
||||
lineage.setUri(generateResourceUri("controller", "provenance", "lineage", lineage.getId()));
|
||||
lineage.setUri(generateResourceUri("provenance", "lineage", lineage.getId()));
|
||||
return lineage;
|
||||
}
|
||||
|
||||
|
@ -120,8 +118,8 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("/search-options")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("search-options")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
|
||||
@ApiOperation(
|
||||
value = "Gets the searchable attributes for provenance events",
|
||||
|
@ -170,15 +168,13 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
* Creates a new replay request for the content associated with the specified provenance event id.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param clusterNodeId The id of the node in the cluster that has the specified event. Required if clustered.
|
||||
* @param eventId The provenance event id.
|
||||
* @param replayRequestEntity The replay request
|
||||
* @return A provenanceEventEntity
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("/replays")
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("replays")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE') and hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Replays content from a provenance event",
|
||||
|
@ -197,36 +193,25 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
}
|
||||
)
|
||||
public Response submitReplay(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The id of the node where the content exists if clustered.",
|
||||
required = false
|
||||
)
|
||||
@FormParam("clusterNodeId") String clusterNodeId,
|
||||
@ApiParam(
|
||||
value = "The provenance event id.",
|
||||
required = true
|
||||
)
|
||||
@FormParam("eventId") LongParameter eventId) {
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "The replay request.",
|
||||
required = true
|
||||
) SubmitReplayRequestEntity replayRequestEntity) {
|
||||
|
||||
// ensure the event id is specified
|
||||
if (eventId == null) {
|
||||
if (replayRequestEntity == null || replayRequestEntity.getEventId() == null) {
|
||||
throw new IllegalArgumentException("The id of the event must be specified.");
|
||||
}
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
// determine where this request should be sent
|
||||
if (clusterNodeId == null) {
|
||||
if (replayRequestEntity.getClusterNodeId() == null) {
|
||||
throw new IllegalArgumentException("The id of the node in the cluster is required.");
|
||||
} else {
|
||||
// get the target node and ensure it exists
|
||||
final Node targetNode = clusterManager.getNode(clusterNodeId);
|
||||
final Node targetNode = clusterManager.getNode(replayRequestEntity.getClusterNodeId());
|
||||
if (targetNode == null) {
|
||||
throw new UnknownNodeException("The specified cluster node does not exist.");
|
||||
}
|
||||
|
@ -235,7 +220,7 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
targetNodes.add(targetNode.getNodeId());
|
||||
|
||||
// replicate the request to the specific node
|
||||
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
|
||||
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), replayRequestEntity, getHeaders(), targetNodes).getResponse();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -246,11 +231,12 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
}
|
||||
|
||||
// submit the provenance replay request
|
||||
final ProvenanceEventDTO event = serviceFacade.submitReplay(eventId.getLong());
|
||||
final RevisionDTO requestRevision = replayRequestEntity.getRevision();
|
||||
final ProvenanceEventDTO event = serviceFacade.submitReplay(replayRequestEntity.getEventId());
|
||||
|
||||
// create a revision to return
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
revision.setClientId(requestRevision.getClientId());
|
||||
|
||||
// create a response entity
|
||||
final ProvenanceEventEntity entity = new ProvenanceEventEntity();
|
||||
|
@ -258,7 +244,7 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
entity.setRevision(revision);
|
||||
|
||||
// generate the response
|
||||
URI uri = URI.create(generateResourceUri("controller", "provenance", "events", event.getId()));
|
||||
URI uri = URI.create(generateResourceUri("provenance", "events", event.getId()));
|
||||
return clusterContext(generateCreatedResponse(uri, entity)).build();
|
||||
}
|
||||
|
||||
|
@ -273,7 +259,7 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.WILDCARD)
|
||||
@Path("/events/{id}/content/input")
|
||||
@Path("events/{id}/content/input")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
|
||||
@ApiOperation(
|
||||
value = "Gets the input content for a provenance event",
|
||||
|
@ -333,7 +319,7 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
}
|
||||
|
||||
// get the uri of the request
|
||||
final String uri = generateResourceUri("controller", "provenance", "events", String.valueOf(id.getLong()), "content", "input");
|
||||
final String uri = generateResourceUri("provenance", "events", String.valueOf(id.getLong()), "content", "input");
|
||||
|
||||
// get an input stream to the content
|
||||
final DownloadableContent content = serviceFacade.getContent(id.getLong(), uri, ContentDirection.INPUT);
|
||||
|
@ -372,7 +358,7 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.WILDCARD)
|
||||
@Path("/events/{id}/content/output")
|
||||
@Path("events/{id}/content/output")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
|
||||
@ApiOperation(
|
||||
value = "Gets the output content for a provenance event",
|
||||
|
@ -432,7 +418,7 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
}
|
||||
|
||||
// get the uri of the request
|
||||
final String uri = generateResourceUri("controller", "provenance", "events", String.valueOf(id.getLong()), "content", "output");
|
||||
final String uri = generateResourceUri("provenance", "events", String.valueOf(id.getLong()), "content", "output");
|
||||
|
||||
// get an input stream to the content
|
||||
final DownloadableContent content = serviceFacade.getContent(id.getLong(), uri, ContentDirection.OUTPUT);
|
||||
|
@ -460,100 +446,6 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
return generateOkResponse(response).type(contentType).header("Content-Disposition", String.format("attachment; filename=\"%s\"", content.getFilename())).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates provenance using the specified query criteria.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param startDate The start date.
|
||||
* @param endDate The end date.
|
||||
* @param minimumFileSize The minimum size of the content after the event.
|
||||
* @param maximumFileSize The maximum size of the content after the event.
|
||||
* @param maxResults The maximum number of results to return.
|
||||
* @param clusterNodeId The id of node in the cluster to search. This is optional and only relevant when clustered. If clustered and it is not specified the entire cluster is searched.
|
||||
* @param formParams Additionally, the search parameters are specified in the form parameters. Because the search parameters differ based on configuration they are specified in a map-like fashion:
|
||||
* <br>
|
||||
* <ul>
|
||||
* <li>search[filename]=myFile.txt</li>
|
||||
* <li>search[eventType]=RECEIVED</li>
|
||||
* </ul>
|
||||
*
|
||||
* @return A provenanceEntity
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.APPLICATION_FORM_URLENCODED)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("") // necessary due to bug in swagger
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
|
||||
public Response submitProvenanceRequest(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@FormParam("startDate") DateTimeParameter startDate,
|
||||
@FormParam("endDate") DateTimeParameter endDate,
|
||||
@FormParam("minimumFileSize") String minimumFileSize,
|
||||
@FormParam("maximumFileSize") String maximumFileSize,
|
||||
@FormParam("maxResults") IntegerParameter maxResults,
|
||||
@FormParam("clusterNodeId") String clusterNodeId,
|
||||
MultivaluedMap<String, String> formParams) {
|
||||
|
||||
// ensure the max results has been specified
|
||||
if (maxResults == null) {
|
||||
throw new IllegalArgumentException("Max results must be specified.");
|
||||
} else if (maxResults.getInteger() > MAX_MAX_RESULTS) {
|
||||
throw new IllegalArgumentException("The maximum number of results cannot be greater than " + MAX_MAX_RESULTS);
|
||||
}
|
||||
|
||||
// create collections for holding the search terms
|
||||
final Map<String, String> searchTerms = new LinkedHashMap<>();
|
||||
|
||||
// go through each parameter and look for processor properties
|
||||
for (String parameterName : formParams.keySet()) {
|
||||
if (StringUtils.isNotBlank(parameterName)) {
|
||||
// see if the parameter name starts with an expected parameter type...
|
||||
// if so, store the parameter name and value in the corresponding collection
|
||||
if (parameterName.startsWith("search[")) {
|
||||
final int startIndex = StringUtils.indexOf(parameterName, "[");
|
||||
final int endIndex = StringUtils.lastIndexOf(parameterName, "]");
|
||||
if (startIndex != -1 && endIndex != -1) {
|
||||
final String searchTerm = StringUtils.substring(parameterName, startIndex + 1, endIndex);
|
||||
searchTerms.put(searchTerm, formParams.getFirst(parameterName));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Build request object from all params
|
||||
final ProvenanceRequestDTO request = new ProvenanceRequestDTO();
|
||||
request.setSearchTerms(searchTerms);
|
||||
request.setMinimumFileSize(minimumFileSize);
|
||||
request.setMaximumFileSize(maximumFileSize);
|
||||
request.setMaxResults(maxResults.getInteger());
|
||||
|
||||
// add date range
|
||||
if (startDate != null) {
|
||||
request.setStartDate(startDate.getDateTime());
|
||||
}
|
||||
if (endDate != null) {
|
||||
request.setEndDate(endDate.getDateTime());
|
||||
}
|
||||
|
||||
// build the provenance object
|
||||
final ProvenanceDTO provenanceDto = new ProvenanceDTO();
|
||||
provenanceDto.setClusterNodeId(clusterNodeId);
|
||||
provenanceDto.setRequest(request);
|
||||
|
||||
// create a revision to return
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the request entity
|
||||
final ProvenanceEntity entity = new ProvenanceEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setProvenance(provenanceDto);
|
||||
|
||||
return submitProvenanceRequest(httpServletRequest, entity);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates provenance using the specified query criteria.
|
||||
*
|
||||
|
@ -562,8 +454,8 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
* @return A provenanceEntity
|
||||
*/
|
||||
@POST
|
||||
@Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("") // necessary due to bug in swagger
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
|
||||
@ApiOperation(
|
||||
|
@ -679,8 +571,8 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("/{id}")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("{id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
|
||||
@ApiOperation(
|
||||
value = "Gets a provenance query",
|
||||
|
@ -765,8 +657,8 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
*/
|
||||
@DELETE
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("/{id}")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("{id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
|
||||
@ApiOperation(
|
||||
value = "Deletes a provenance query",
|
||||
|
@ -854,8 +746,8 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("/events/{id}")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("events/{id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
|
||||
@ApiOperation(
|
||||
value = "Gets a provenance event",
|
||||
|
@ -934,72 +826,6 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Submits a lineage request based on an event or a flowfile uuid.
|
||||
*
|
||||
* When querying for the lineage of an event you must specify the eventId and the eventDirection. The eventDirection must be 'parents' or 'children' and specifies whether we are going up or down
|
||||
* the flowfile ancestry. The uuid cannot be specified in these cases.
|
||||
*
|
||||
* When querying for the lineage of a flowfile you must specify the uuid. The eventId and eventDirection cannot be specified in this case.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param eventId The id of an event to get the lineage for. Must also specify the eventDirection and not the uuid.
|
||||
* @param lineageRequest Either 'PARENTS', 'CHILDREN', or 'FLOWFILE'. PARENTS will return the lineage for the flowfiles that are parents of the specified event. CHILDREN will return the lineage of
|
||||
* for the flowfiles that are children of the specified event. FLOWFILE will return the lineage for the specified flowfile.
|
||||
* @param uuid The uuid of the flowfile to get the lineage for. Must not specify the eventId or eventDirection.
|
||||
* @param clusterNodeId The id of node in the cluster that the event/flowfile originated from. This is only required when clustered.
|
||||
* @return A lineageEntity
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.APPLICATION_FORM_URLENCODED)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("/lineage")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
|
||||
public Response submitLineageRequest(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@FormParam("lineageRequestType") String lineageRequest,
|
||||
@FormParam("eventId") LongParameter eventId,
|
||||
@FormParam("uuid") String uuid,
|
||||
@FormParam("clusterNodeId") String clusterNodeId) {
|
||||
|
||||
// create the lineage request
|
||||
final LineageRequestDTO request = new LineageRequestDTO();
|
||||
|
||||
// ensure the lineage request type is specified
|
||||
try {
|
||||
final LineageRequestType direction = LineageRequestType.valueOf(lineageRequest);
|
||||
request.setLineageRequestType(direction);
|
||||
} catch (final IllegalArgumentException iae) {
|
||||
throw new IllegalArgumentException(String.format("The event direction must be one of %s", StringUtils.join(LineageRequestType.values())));
|
||||
}
|
||||
|
||||
// set the uuid (may be null if based on event)
|
||||
request.setUuid(uuid);
|
||||
|
||||
// set the event id (may be null is based on flowfile)
|
||||
if (eventId != null) {
|
||||
request.setEventId(eventId.getLong());
|
||||
}
|
||||
|
||||
// create the lineage
|
||||
final LineageDTO lineage = new LineageDTO();
|
||||
lineage.setClusterNodeId(clusterNodeId);
|
||||
lineage.setRequest(request);
|
||||
|
||||
// create a revision to return
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create a response entity
|
||||
final LineageEntity entity = new LineageEntity();
|
||||
entity.setLineage(lineage);
|
||||
entity.setRevision(revision);
|
||||
|
||||
return submitLineageRequest(httpServletRequest, entity);
|
||||
}
|
||||
|
||||
/**
|
||||
* Submits a lineage request based on an event or a flowfile uuid.
|
||||
*
|
||||
|
@ -1013,9 +839,9 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
* @return A lineageEntity
|
||||
*/
|
||||
@POST
|
||||
@Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("/lineage")
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("lineage")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
|
||||
@ApiOperation(
|
||||
value = "Submits a lineage query",
|
||||
|
@ -1136,8 +962,8 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("/lineage/{id}")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("lineage/{id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
|
||||
@ApiOperation(
|
||||
value = "Gets a lineage query",
|
||||
|
@ -1220,8 +1046,8 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
*/
|
||||
@DELETE
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("/lineage/{id}")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("lineage/{id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_PROVENANCE')")
|
||||
@ApiOperation(
|
||||
value = "Deletes a lineage query",
|
||||
|
|
|
@ -16,17 +16,14 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
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 org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
|
||||
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
|
||||
import org.apache.nifi.cluster.node.Node;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.web.ConfigurationSnapshot;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
|
@ -34,22 +31,15 @@ import org.apache.nifi.web.Revision;
|
|||
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
|
||||
import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO;
|
||||
import org.apache.nifi.web.api.dto.status.StatusHistoryDTO;
|
||||
import org.apache.nifi.web.api.entity.ProcessorStatusEntity;
|
||||
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
|
||||
import org.apache.nifi.web.api.entity.RemoteProcessGroupPortEntity;
|
||||
import org.apache.nifi.web.api.entity.RemoteProcessGroupStatusEntity;
|
||||
import org.apache.nifi.web.api.entity.StatusHistoryEntity;
|
||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||
import org.apache.nifi.web.api.request.IntegerParameter;
|
||||
import org.apache.nifi.web.api.request.LongParameter;
|
||||
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.ws.rs.Consumes;
|
||||
import javax.ws.rs.DELETE;
|
||||
import javax.ws.rs.DefaultValue;
|
||||
import javax.ws.rs.FormParam;
|
||||
import javax.ws.rs.GET;
|
||||
import javax.ws.rs.HttpMethod;
|
||||
import javax.ws.rs.PUT;
|
||||
|
@ -62,14 +52,17 @@ import javax.ws.rs.core.MediaType;
|
|||
import javax.ws.rs.core.Response;
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing a Remote group.
|
||||
*/
|
||||
@Path("remote-process-groups")
|
||||
@Path("/remote-process-groups")
|
||||
@Api(
|
||||
value = "/remote-process-groups",
|
||||
description = "Endpoint for managing a Remote Process Group."
|
||||
)
|
||||
public class RemoteProcessGroupResource extends ApplicationResource {
|
||||
|
||||
private static final String VERBOSE_DEFAULT_VALUE = "false";
|
||||
|
@ -177,169 +170,6 @@ public class RemoteProcessGroupResource extends ApplicationResource {
|
|||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves the specified remote process group status.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param id The id of the processor history to retrieve.
|
||||
* @return A remoteProcessGroupStatusEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{id}/status")
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets status for a remote process group",
|
||||
response = ProcessorStatusEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@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 getRemoteProcessGroupStatus(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "Whether or not to include the breakdown per node. Optional, defaults to false",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("nodewise") @DefaultValue(NODEWISE) Boolean nodewise,
|
||||
@ApiParam(
|
||||
value = "The id of the node where to get the status.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("clusterNodeId") String clusterNodeId,
|
||||
@ApiParam(
|
||||
value = "The remote process group id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("id") String id) {
|
||||
|
||||
// ensure a valid request
|
||||
if (Boolean.TRUE.equals(nodewise) && clusterNodeId != null) {
|
||||
throw new IllegalArgumentException("Nodewise requests cannot be directed at a specific node.");
|
||||
}
|
||||
|
||||
if (properties.isClusterManager()) {
|
||||
// determine where this request should be sent
|
||||
if (clusterNodeId == null) {
|
||||
final NodeResponse nodeResponse = clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders());
|
||||
final RemoteProcessGroupStatusEntity entity = (RemoteProcessGroupStatusEntity) nodeResponse.getUpdatedEntity();
|
||||
|
||||
// ensure there is an updated entity (result of merging) and prune the response as necessary
|
||||
if (entity != null && !nodewise) {
|
||||
entity.getRemoteProcessGroupStatus().setNodeSnapshots(null);
|
||||
}
|
||||
|
||||
return nodeResponse.getResponse();
|
||||
} else {
|
||||
// get the target node and ensure it exists
|
||||
final Node targetNode = clusterManager.getNode(clusterNodeId);
|
||||
if (targetNode == null) {
|
||||
throw new UnknownNodeException("The specified cluster node does not exist.");
|
||||
}
|
||||
|
||||
final Set<NodeIdentifier> targetNodes = new HashSet<>();
|
||||
targetNodes.add(targetNode.getNodeId());
|
||||
|
||||
// replicate the request to the specific node
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders(), targetNodes).getResponse();
|
||||
}
|
||||
}
|
||||
|
||||
// get the specified remote process group status
|
||||
final RemoteProcessGroupStatusDTO remoteProcessGroupStatus = serviceFacade.getRemoteProcessGroupStatus(id);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// generate the response entity
|
||||
final RemoteProcessGroupStatusEntity entity = new RemoteProcessGroupStatusEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setRemoteProcessGroupStatus(remoteProcessGroupStatus);
|
||||
|
||||
// generate the response
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves the specified remote process groups status history.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param id The id of the remote process group to retrieve the status fow.
|
||||
* @return A statusHistoryEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{id}/status/history")
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets the status history",
|
||||
response = StatusHistoryEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@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 getRemoteProcessGroupStatusHistory(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The remote process group id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("id") String id) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// get the specified processor status history
|
||||
final StatusHistoryDTO remoteProcessGroupStatusHistory = serviceFacade.getRemoteProcessGroupStatusHistory(id);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// generate the response entity
|
||||
final StatusHistoryEntity entity = new StatusHistoryEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setStatusHistory(remoteProcessGroupStatusHistory);
|
||||
|
||||
// generate the response
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the specified remote process group.
|
||||
*
|
||||
|
@ -352,7 +182,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
|
|||
@DELETE
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{id}")
|
||||
@Path("{id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Deletes a remote process group",
|
||||
|
@ -421,61 +251,6 @@ public class RemoteProcessGroupResource extends ApplicationResource {
|
|||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the specified remote process group input port.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param version The revision is used to verify the client is working with the latest version of the flow.
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @param id The id of the remote process group to update.
|
||||
* @param portId The id of the input port to update.
|
||||
* @param isTransmitting Whether or not this port is transmitting.
|
||||
* @param isCompressed Whether or not this port should compress.
|
||||
* @param concurrentlySchedulableTaskCount The number of concurrent tasks that should be supported
|
||||
*
|
||||
* @return A remoteProcessGroupPortEntity
|
||||
*/
|
||||
@PUT
|
||||
@Consumes(MediaType.APPLICATION_FORM_URLENCODED)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{id}/input-ports/{port-id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
public Response updateRemoteProcessGroupInputPort(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@FormParam(VERSION) LongParameter version,
|
||||
@FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@PathParam("id") String id,
|
||||
@PathParam("port-id") String portId,
|
||||
@FormParam("transmitting") Boolean isTransmitting,
|
||||
@FormParam("compressed") Boolean isCompressed,
|
||||
@FormParam("concurrentlySchedulableTaskCount") IntegerParameter concurrentlySchedulableTaskCount) {
|
||||
|
||||
// create the remote group port dto
|
||||
final RemoteProcessGroupPortDTO remotePort = new RemoteProcessGroupPortDTO();
|
||||
remotePort.setId(portId);
|
||||
remotePort.setUseCompression(isCompressed);
|
||||
remotePort.setTransmitting(isTransmitting);
|
||||
|
||||
if (concurrentlySchedulableTaskCount != null) {
|
||||
remotePort.setConcurrentlySchedulableTaskCount(concurrentlySchedulableTaskCount.getInteger());
|
||||
}
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
if (version != null) {
|
||||
revision.setVersion(version.getLong());
|
||||
}
|
||||
|
||||
// create the remote group port entity
|
||||
final RemoteProcessGroupPortEntity entity = new RemoteProcessGroupPortEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setRemoteProcessGroupPort(remotePort);
|
||||
|
||||
return updateRemoteProcessGroupInputPort(httpServletRequest, id, portId, entity);
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the specified remote process group input port.
|
||||
*
|
||||
|
@ -489,7 +264,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
|
|||
@PUT
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{id}/input-ports/{port-id}")
|
||||
@Path("{id}/input-ports/{port-id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Updates a remote port",
|
||||
|
@ -578,7 +353,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
|
|||
@PUT
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{id}/output-ports/{port-id}")
|
||||
@Path("{id}/output-ports/{port-id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Updates a remote port",
|
||||
|
@ -665,7 +440,7 @@ public class RemoteProcessGroupResource extends ApplicationResource {
|
|||
@PUT
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("/{id}")
|
||||
@Path("{id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Updates a remote process group",
|
||||
|
|
|
@ -16,14 +16,13 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
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 org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.cluster.context.ClusterContext;
|
||||
import org.apache.nifi.cluster.context.ClusterContextThreadLocal;
|
||||
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
|
||||
import org.apache.nifi.ui.extension.UiExtension;
|
||||
import org.apache.nifi.ui.extension.UiExtensionMapping;
|
||||
|
@ -40,7 +39,6 @@ import org.apache.nifi.web.api.entity.ComponentStateEntity;
|
|||
import org.apache.nifi.web.api.entity.Entity;
|
||||
import org.apache.nifi.web.api.entity.PropertyDescriptorEntity;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
import org.apache.nifi.web.api.entity.ReportingTasksEntity;
|
||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||
import org.apache.nifi.web.api.request.LongParameter;
|
||||
import org.apache.nifi.web.util.Availability;
|
||||
|
@ -62,15 +60,17 @@ import javax.ws.rs.core.Context;
|
|||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.net.URI;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing a Reporting Task.
|
||||
*/
|
||||
@Path("reporting-tasks")
|
||||
@Path("/reporting-tasks")
|
||||
@Api(
|
||||
value = "/reporting-tasks",
|
||||
description = "Endpoint for managing a Reporting Task."
|
||||
)
|
||||
public class ReportingTaskResource extends ApplicationResource {
|
||||
|
||||
private NiFiServiceFacade serviceFacade;
|
||||
|
@ -86,7 +86,7 @@ public class ReportingTaskResource extends ApplicationResource {
|
|||
* @param reportingTasks tasks
|
||||
* @return tasks
|
||||
*/
|
||||
private Set<ReportingTaskDTO> populateRemainingReportingTasksContent(final String availability, final Set<ReportingTaskDTO> reportingTasks) {
|
||||
public Set<ReportingTaskDTO> populateRemainingReportingTasksContent(final String availability, final Set<ReportingTaskDTO> reportingTasks) {
|
||||
for (ReportingTaskDTO reportingTask : reportingTasks) {
|
||||
populateRemainingReportingTaskContent(availability, reportingTask);
|
||||
}
|
||||
|
@ -96,7 +96,7 @@ public class ReportingTaskResource extends ApplicationResource {
|
|||
/**
|
||||
* Populates the uri for the specified reporting task.
|
||||
*/
|
||||
private ReportingTaskDTO populateRemainingReportingTaskContent(final String availability, final ReportingTaskDTO reportingTask) {
|
||||
public ReportingTaskDTO populateRemainingReportingTaskContent(final String availability, final ReportingTaskDTO reportingTask) {
|
||||
// populate the reporting task href
|
||||
reportingTask.setUri(generateResourceUri("reporting-tasks", availability, reportingTask.getId()));
|
||||
reportingTask.setAvailability(availability);
|
||||
|
@ -119,7 +119,7 @@ public class ReportingTaskResource extends ApplicationResource {
|
|||
* Parses the availability and ensure that the specified availability makes
|
||||
* sense for the given NiFi instance.
|
||||
*/
|
||||
private Availability parseAvailability(final String availability) {
|
||||
public Availability parseAvailability(final String availability) {
|
||||
final Availability avail;
|
||||
try {
|
||||
avail = Availability.valueOf(availability.toUpperCase());
|
||||
|
@ -135,176 +135,6 @@ public class ReportingTaskResource extends ApplicationResource {
|
|||
return avail;
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves all the of reporting tasks in this NiFi.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a
|
||||
* new one will be generated. This value (whether specified or generated) is
|
||||
* included in the response.
|
||||
* @param availability Whether the reporting task is available on the NCM
|
||||
* only (ncm) or on the nodes only (node). If this instance is not clustered
|
||||
* all tasks should use the node availability.
|
||||
* @return A reportingTasksEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("{availability}")
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets all reporting tasks",
|
||||
response = ReportingTasksEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@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 = 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 getReportingTasks(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "Whether the reporting task is available on the NCM or nodes. If the NiFi is standalone the availability should be NODE.",
|
||||
allowableValues = "NCM, NODE",
|
||||
required = true
|
||||
)
|
||||
@PathParam("availability") String availability) {
|
||||
|
||||
final Availability avail = parseAvailability(availability);
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager() && Availability.NODE.equals(avail)) {
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// get all the reporting tasks
|
||||
final Set<ReportingTaskDTO> reportingTasks = populateRemainingReportingTasksContent(availability, serviceFacade.getReportingTasks());
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
final ReportingTasksEntity entity = new ReportingTasksEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setReportingTasks(reportingTasks);
|
||||
|
||||
// generate the response
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new Reporting Task.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param availability Whether the reporting task is available on the NCM
|
||||
* only (ncm) or on the nodes only (node). If this instance is not clustered
|
||||
* all tasks should use the node availability.
|
||||
* @param reportingTaskEntity A reportingTaskEntity.
|
||||
* @return A reportingTaskEntity.
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("{availability}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Creates a new reporting task",
|
||||
response = ReportingTaskEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 = 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 createReportingTask(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "Whether the reporting task is available on the NCM or nodes. If the NiFi is standalone the availability should be NODE.",
|
||||
allowableValues = "NCM, NODE",
|
||||
required = true
|
||||
)
|
||||
@PathParam("availability") String availability,
|
||||
@ApiParam(
|
||||
value = "The reporting task configuration details.",
|
||||
required = true
|
||||
) ReportingTaskEntity reportingTaskEntity) {
|
||||
|
||||
final Availability avail = parseAvailability(availability);
|
||||
|
||||
if (reportingTaskEntity == null || reportingTaskEntity.getReportingTask() == null) {
|
||||
throw new IllegalArgumentException("Reporting task details must be specified.");
|
||||
}
|
||||
|
||||
if (reportingTaskEntity.getRevision() == null) {
|
||||
throw new IllegalArgumentException("Revision must be specified.");
|
||||
}
|
||||
|
||||
if (reportingTaskEntity.getReportingTask().getId() != null) {
|
||||
throw new IllegalArgumentException("Reporting task ID cannot be specified.");
|
||||
}
|
||||
|
||||
if (StringUtils.isBlank(reportingTaskEntity.getReportingTask().getType())) {
|
||||
throw new IllegalArgumentException("The type of reporting task to create must be specified.");
|
||||
}
|
||||
|
||||
// get the revision
|
||||
final RevisionDTO revision = reportingTaskEntity.getRevision();
|
||||
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(reportingTaskEntity), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
// set the processor id as appropriate
|
||||
final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
|
||||
if (clusterContext != null) {
|
||||
reportingTaskEntity.getReportingTask().setId(UUID.nameUUIDFromBytes(clusterContext.getIdGenerationSeed().getBytes(StandardCharsets.UTF_8)).toString());
|
||||
} else {
|
||||
reportingTaskEntity.getReportingTask().setId(UUID.randomUUID().toString());
|
||||
}
|
||||
|
||||
// create the reporting task and generate the json
|
||||
final ConfigurationSnapshot<ReportingTaskDTO> controllerResponse = serviceFacade.createReportingTask(
|
||||
new Revision(revision.getVersion(), revision.getClientId()), reportingTaskEntity.getReportingTask());
|
||||
final ReportingTaskDTO reportingTask = controllerResponse.getConfiguration();
|
||||
|
||||
// get the updated revision
|
||||
final RevisionDTO updatedRevision = new RevisionDTO();
|
||||
updatedRevision.setClientId(revision.getClientId());
|
||||
updatedRevision.setVersion(controllerResponse.getVersion());
|
||||
|
||||
// build the response entity
|
||||
final ReportingTaskEntity entity = new ReportingTaskEntity();
|
||||
entity.setRevision(updatedRevision);
|
||||
entity.setReportingTask(populateRemainingReportingTaskContent(availability, reportingTask));
|
||||
|
||||
// build the response
|
||||
return clusterContext(generateCreatedResponse(URI.create(reportingTask.getUri()), entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves the specified reporting task.
|
||||
*
|
||||
|
|
|
@ -0,0 +1,125 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
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 org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.api.dto.ResourceDTO;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.entity.ResourcesEntity;
|
||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||
|
||||
import javax.ws.rs.Consumes;
|
||||
import javax.ws.rs.DefaultValue;
|
||||
import javax.ws.rs.GET;
|
||||
import javax.ws.rs.HttpMethod;
|
||||
import javax.ws.rs.Path;
|
||||
import javax.ws.rs.Produces;
|
||||
import javax.ws.rs.QueryParam;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for retrieving system diagnostics.
|
||||
*/
|
||||
@Path("/resources")
|
||||
@Api(
|
||||
value = "/resources",
|
||||
description = "Provides the resources in this NiFi that can have access/authorization policies."
|
||||
)
|
||||
public class ResourceResource extends ApplicationResource {
|
||||
|
||||
private NiFiServiceFacade serviceFacade;
|
||||
private WebClusterManager clusterManager;
|
||||
private NiFiProperties properties;
|
||||
|
||||
/**
|
||||
* Gets the available resources that support access/authorization policies.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a
|
||||
* new one will be generated. This value (whether specified or generated) is
|
||||
* included in the response.
|
||||
* @return A resourcesEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets the available resources that support access/authorization policies",
|
||||
response = ResourcesEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@ApiResponses(
|
||||
value = {
|
||||
@ApiResponse(code = 401, message = "Client could not be authenticated."),
|
||||
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),}
|
||||
)
|
||||
public Response getResources(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) {
|
||||
|
||||
// replicate if the cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// TODO - if unsecure, return no resources?
|
||||
final List<ResourceDTO> resources = serviceFacade.getResources();
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response
|
||||
final ResourcesEntity entity = new ResourcesEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setResources(resources);
|
||||
|
||||
// generate the response
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
// setters
|
||||
public void setServiceFacade(NiFiServiceFacade serviceFacade) {
|
||||
this.serviceFacade = serviceFacade;
|
||||
}
|
||||
|
||||
public void setClusterManager(WebClusterManager clusterManager) {
|
||||
this.clusterManager = clusterManager;
|
||||
}
|
||||
|
||||
public void setProperties(NiFiProperties properties) {
|
||||
this.properties = properties;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,126 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
import com.sun.jersey.api.core.ResourceContext;
|
||||
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 org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.api.dto.ControllerDTO;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.entity.ControllerEntity;
|
||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||
|
||||
import javax.ws.rs.Consumes;
|
||||
import javax.ws.rs.DefaultValue;
|
||||
import javax.ws.rs.GET;
|
||||
import javax.ws.rs.HttpMethod;
|
||||
import javax.ws.rs.Path;
|
||||
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;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing a Flow Controller.
|
||||
*/
|
||||
@Path("/site-to-site")
|
||||
@Api(
|
||||
value = "/site-to-site",
|
||||
description = "Provide access to site to site with this NiFi"
|
||||
)
|
||||
public class SiteToSiteResource extends ApplicationResource {
|
||||
|
||||
private NiFiServiceFacade serviceFacade;
|
||||
private WebClusterManager clusterManager;
|
||||
private NiFiProperties properties;
|
||||
|
||||
@Context
|
||||
private ResourceContext resourceContext;
|
||||
|
||||
/**
|
||||
* Returns the details of this NiFi.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
|
||||
* @return A controllerEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_NIFI')")
|
||||
@ApiOperation(
|
||||
value = "Returns the details about this NiFi necessary to communicate via site to site",
|
||||
response = ControllerEntity.class,
|
||||
authorizations = @Authorization(value = "NiFi", type = "ROLE_NIFI")
|
||||
)
|
||||
@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 = 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 getController(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) {
|
||||
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// get the controller dto
|
||||
final ControllerDTO controller = serviceFacade.getController();
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// build the response entity
|
||||
final ControllerEntity entity = new ControllerEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setController(controller);
|
||||
|
||||
// generate the response
|
||||
return clusterContext(noCache(Response.ok(entity))).build();
|
||||
}
|
||||
|
||||
// setters
|
||||
public void setServiceFacade(NiFiServiceFacade serviceFacade) {
|
||||
this.serviceFacade = serviceFacade;
|
||||
}
|
||||
|
||||
public void setClusterManager(WebClusterManager clusterManager) {
|
||||
this.clusterManager = clusterManager;
|
||||
}
|
||||
|
||||
public void setProperties(NiFiProperties properties) {
|
||||
this.properties = properties;
|
||||
}
|
||||
|
||||
}
|
|
@ -1,664 +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.web.api;
|
||||
|
||||
import com.sun.jersey.api.core.ResourceContext;
|
||||
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 org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
import org.apache.nifi.web.ConfigurationSnapshot;
|
||||
import org.apache.nifi.web.NiFiServiceFacade;
|
||||
import org.apache.nifi.web.Revision;
|
||||
import org.apache.nifi.web.api.dto.FlowSnippetDTO;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.dto.SnippetDTO;
|
||||
import org.apache.nifi.web.api.entity.SnippetEntity;
|
||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||
import org.apache.nifi.web.api.request.LongParameter;
|
||||
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.FormParam;
|
||||
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.WebApplicationException;
|
||||
import javax.ws.rs.core.Context;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing a Snippet.
|
||||
*/
|
||||
@Api(hidden = true)
|
||||
public class SnippetResource extends ApplicationResource {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(SnippetResource.class);
|
||||
private static final String LINKED = "false";
|
||||
private static final String VERBOSE = "false";
|
||||
|
||||
@Context
|
||||
private ResourceContext resourceContext;
|
||||
|
||||
private NiFiServiceFacade serviceFacade;
|
||||
private WebClusterManager clusterManager;
|
||||
private NiFiProperties properties;
|
||||
|
||||
private ProcessorResource processorResource;
|
||||
private InputPortResource inputPortResource;
|
||||
private OutputPortResource outputPortResource;
|
||||
private FunnelResource funnelResource;
|
||||
private LabelResource labelResource;
|
||||
private RemoteProcessGroupResource remoteProcessGroupResource;
|
||||
private ConnectionResource connectionResource;
|
||||
private ProcessGroupResource processGroupResource;
|
||||
|
||||
/**
|
||||
* Populates the uri for the specified snippet.
|
||||
*/
|
||||
private SnippetDTO populateRemainingSnippetContent(SnippetDTO snippet) {
|
||||
// populate the snippet href
|
||||
snippet.setUri(generateResourceUri("controller", "snippets", snippet.getId()));
|
||||
|
||||
String snippetGroupId = snippet.getParentGroupId();
|
||||
FlowSnippetDTO snippetContents = snippet.getContents();
|
||||
|
||||
// populate the snippet content uris
|
||||
if (snippet.getContents() != null) {
|
||||
processorResource.populateRemainingProcessorsContent(snippetContents.getProcessors());
|
||||
connectionResource.populateRemainingConnectionsContent(snippetContents.getConnections());
|
||||
inputPortResource.populateRemainingInputPortsContent(snippetContents.getInputPorts());
|
||||
outputPortResource.populateRemainingOutputPortsContent(snippetContents.getOutputPorts());
|
||||
remoteProcessGroupResource.populateRemainingRemoteProcessGroupsContent(snippetContents.getRemoteProcessGroups());
|
||||
funnelResource.populateRemainingFunnelsContent(snippetContents.getFunnels());
|
||||
labelResource.populateRemainingLabelsContent(snippetContents.getLabels());
|
||||
processGroupResource.populateRemainingProcessGroupsContent(snippetContents.getProcessGroups());
|
||||
}
|
||||
|
||||
return snippet;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new snippet based on the specified contents.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param version The revision is used to verify the client is working with
|
||||
* the latest version of the flow.
|
||||
* @param clientId Optional client id. If the client id is not specified, a
|
||||
* new one will be generated. This value (whether specified or generated) is
|
||||
* included in the response.
|
||||
* @param parentGroupId The id of the process group the components in this
|
||||
* snippet belong to.
|
||||
* @param linked Whether or not this snippet is linked to the underlying
|
||||
* data flow. If a linked snippet is deleted, the components that comprise
|
||||
* the snippet are also deleted.
|
||||
* @param processorIds The ids of any processors in this snippet.
|
||||
* @param processGroupIds The ids of any process groups in this snippet.
|
||||
* @param remoteProcessGroupIds The ids of any remote process groups in this
|
||||
* snippet.
|
||||
* @param inputPortIds The ids of any input ports in this snippet.
|
||||
* @param outputPortIds The ids of any output ports in this snippet.
|
||||
* @param connectionIds The ids of any connections in this snippet.
|
||||
* @param labelIds The ids of any labels in this snippet.
|
||||
* @param funnelIds The ids of any funnels in this snippet.
|
||||
* @return A snippetEntity
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.APPLICATION_FORM_URLENCODED)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
public Response createSnippet(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@FormParam(VERSION) LongParameter version,
|
||||
@FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@FormParam("parentGroupId") String parentGroupId,
|
||||
@FormParam("linked") @DefaultValue(LINKED) Boolean linked,
|
||||
@FormParam("processorIds[]") List<String> processorIds,
|
||||
@FormParam("processGroupIds[]") List<String> processGroupIds,
|
||||
@FormParam("remoteProcessGroupIds[]") List<String> remoteProcessGroupIds,
|
||||
@FormParam("inputPortIds[]") List<String> inputPortIds,
|
||||
@FormParam("outputPortIds[]") List<String> outputPortIds,
|
||||
@FormParam("connectionIds[]") List<String> connectionIds,
|
||||
@FormParam("labelIds[]") List<String> labelIds,
|
||||
@FormParam("funnelIds[]") List<String> funnelIds) {
|
||||
|
||||
// create the snippet dto
|
||||
final SnippetDTO snippetDTO = new SnippetDTO();
|
||||
snippetDTO.setParentGroupId(parentGroupId);
|
||||
snippetDTO.setLinked(linked);
|
||||
snippetDTO.setProcessors(new HashSet<>(processorIds));
|
||||
snippetDTO.setProcessGroups(new HashSet<>(processGroupIds));
|
||||
snippetDTO.setRemoteProcessGroups(new HashSet<>(remoteProcessGroupIds));
|
||||
snippetDTO.setInputPorts(new HashSet<>(inputPortIds));
|
||||
snippetDTO.setOutputPorts(new HashSet<>(outputPortIds));
|
||||
snippetDTO.setConnections(new HashSet<>(connectionIds));
|
||||
snippetDTO.setLabels(new HashSet<>(labelIds));
|
||||
snippetDTO.setFunnels(new HashSet<>(funnelIds));
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
if (version != null) {
|
||||
revision.setVersion(version.getLong());
|
||||
}
|
||||
|
||||
// create the snippet entity
|
||||
final SnippetEntity snippetEntity = new SnippetEntity();
|
||||
snippetEntity.setRevision(revision);
|
||||
snippetEntity.setSnippet(snippetDTO);
|
||||
|
||||
// build the response
|
||||
return createSnippet(httpServletRequest, snippetEntity);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a snippet based off the specified configuration.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param snippetEntity A snippetEntity
|
||||
* @return A snippetEntity
|
||||
*/
|
||||
@POST
|
||||
@Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("") // necessary due to bug in swagger
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Creates a snippet",
|
||||
response = SnippetEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@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 createSnippet(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "The snippet configuration details.",
|
||||
required = true
|
||||
)
|
||||
final SnippetEntity snippetEntity) {
|
||||
|
||||
if (snippetEntity == null || snippetEntity.getSnippet() == null) {
|
||||
throw new IllegalArgumentException("Snippet details must be specified.");
|
||||
}
|
||||
|
||||
if (snippetEntity.getRevision() == null) {
|
||||
throw new IllegalArgumentException("Revision must be specified.");
|
||||
}
|
||||
|
||||
if (snippetEntity.getSnippet().getId() != null) {
|
||||
throw new IllegalArgumentException("Snippet ID cannot be specified.");
|
||||
}
|
||||
|
||||
// ensure the group id has been specified
|
||||
if (snippetEntity.getSnippet().getParentGroupId() == null) {
|
||||
throw new IllegalArgumentException("The group id must be specified when creating a snippet.");
|
||||
}
|
||||
|
||||
// if cluster manager, convert POST to PUT (to maintain same ID across nodes) and replicate
|
||||
if (properties.isClusterManager()) {
|
||||
|
||||
// create ID for resource
|
||||
final String id = UUID.randomUUID().toString();
|
||||
|
||||
// set ID for resource
|
||||
snippetEntity.getSnippet().setId(id);
|
||||
|
||||
// convert POST request to PUT request to force entity ID to be the same across nodes
|
||||
URI putUri = null;
|
||||
try {
|
||||
putUri = new URI(getAbsolutePath().toString() + "/" + id);
|
||||
} catch (final URISyntaxException e) {
|
||||
throw new WebApplicationException(e);
|
||||
}
|
||||
|
||||
// change content type to JSON for serializing entity
|
||||
final Map<String, String> headersToOverride = new HashMap<>();
|
||||
headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
|
||||
|
||||
// replicate put request
|
||||
return (Response) clusterManager.applyRequest(HttpMethod.PUT, putUri, updateClientId(snippetEntity), getHeaders(headersToOverride)).getResponse();
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
// create the snippet
|
||||
final RevisionDTO revision = snippetEntity.getRevision();
|
||||
final ConfigurationSnapshot<SnippetDTO> response = serviceFacade.createSnippet(new Revision(revision.getVersion(), revision.getClientId()), snippetEntity.getSnippet());
|
||||
|
||||
// get the snippet
|
||||
final SnippetDTO snippet = response.getConfiguration();
|
||||
|
||||
// always prune the response when creating
|
||||
snippet.setContents(null);
|
||||
|
||||
// get the updated revision
|
||||
final RevisionDTO updatedRevision = new RevisionDTO();
|
||||
updatedRevision.setClientId(revision.getClientId());
|
||||
updatedRevision.setVersion(response.getVersion());
|
||||
|
||||
// build the response entity
|
||||
SnippetEntity entity = new SnippetEntity();
|
||||
entity.setRevision(updatedRevision);
|
||||
entity.setSnippet(populateRemainingSnippetContent(snippet));
|
||||
|
||||
// build the response
|
||||
return clusterContext(generateCreatedResponse(URI.create(snippet.getUri()), entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves the specified snippet.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a
|
||||
* new one will be generated. This value (whether specified or generated) is
|
||||
* included in the response.
|
||||
* @param verbose Whether or not to include the contents of the snippet in
|
||||
* the response.
|
||||
* @param id The id of the snippet to retrieve.
|
||||
* @return A snippetEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("{id}")
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets a snippet",
|
||||
response = SnippetEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@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 getSnippet(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "Whether to include configuration details for the components specified in the snippet.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam("verbose") @DefaultValue(VERBOSE) Boolean verbose,
|
||||
@ApiParam(
|
||||
value = "The snippet id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("id") String id) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// get the snippet
|
||||
final SnippetDTO snippet = serviceFacade.getSnippet(id);
|
||||
|
||||
// prune the response if necessary
|
||||
if (!verbose) {
|
||||
snippet.setContents(null);
|
||||
}
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
final SnippetEntity entity = new SnippetEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setSnippet(populateRemainingSnippetContent(snippet));
|
||||
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the specified snippet.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param version The revision is used to verify the client is working with
|
||||
* the latest version of the flow.
|
||||
* @param clientId Optional client id. If the client id is not specified, a
|
||||
* new one will be generated. This value (whether specified or generated) is
|
||||
* included in the response.
|
||||
* @param verbose Whether or not to include the contents of the snippet in
|
||||
* the response.
|
||||
* @param id The id of the snippet to update.
|
||||
* @param parentGroupId The id of the process group to move the contents of
|
||||
* this snippet to.
|
||||
* @param linked Whether or not this snippet is linked to the underlying
|
||||
* data flow. If a linked snippet is deleted, the components that comprise
|
||||
* the snippet are also deleted.
|
||||
* @return A snippetEntity.
|
||||
*/
|
||||
@PUT
|
||||
@Consumes(MediaType.APPLICATION_FORM_URLENCODED)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("{id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
public Response updateSnippet(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@FormParam(VERSION) LongParameter version,
|
||||
@FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@FormParam("verbose") @DefaultValue(VERBOSE) Boolean verbose,
|
||||
@PathParam("id") String id,
|
||||
@FormParam("parentGroupId") String parentGroupId,
|
||||
@FormParam("linked") Boolean linked) {
|
||||
|
||||
// create the snippet dto
|
||||
final SnippetDTO snippetDTO = new SnippetDTO();
|
||||
snippetDTO.setId(id);
|
||||
snippetDTO.setParentGroupId(parentGroupId);
|
||||
snippetDTO.setLinked(linked);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
if (version != null) {
|
||||
revision.setVersion(version.getLong());
|
||||
}
|
||||
|
||||
// create the snippet entity
|
||||
final SnippetEntity entity = new SnippetEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setSnippet(snippetDTO);
|
||||
|
||||
// build the response
|
||||
return updateSnippet(httpServletRequest, id, entity);
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the specified snippet. The contents of the snippet (component
|
||||
* ids) cannot be updated once the snippet is created.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param id The id of the snippet.
|
||||
* @param snippetEntity A snippetEntity
|
||||
* @return A snippetEntity
|
||||
*/
|
||||
@PUT
|
||||
@Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("{id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Updates a snippet",
|
||||
response = SnippetEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 updateSnippet(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "The snippet id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("id") String id,
|
||||
@ApiParam(
|
||||
value = "The snippet configuration details.",
|
||||
required = true
|
||||
)
|
||||
final SnippetEntity snippetEntity) {
|
||||
|
||||
if (snippetEntity == null || snippetEntity.getSnippet() == null) {
|
||||
throw new IllegalArgumentException("Snippet details must be specified.");
|
||||
}
|
||||
|
||||
if (snippetEntity.getRevision() == null) {
|
||||
throw new IllegalArgumentException("Revision must be specified.");
|
||||
}
|
||||
|
||||
// ensure the ids are the same
|
||||
final SnippetDTO requestSnippetDTO = snippetEntity.getSnippet();
|
||||
if (!id.equals(requestSnippetDTO.getId())) {
|
||||
throw new IllegalArgumentException(String.format("The snippet id (%s) in the request body does not equal the "
|
||||
+ "snippet id of the requested resource (%s).", requestSnippetDTO.getId(), id));
|
||||
}
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
// change content type to JSON for serializing entity
|
||||
final Map<String, String> headersToOverride = new HashMap<>();
|
||||
headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
|
||||
|
||||
// replicate the request
|
||||
return clusterManager.applyRequest(HttpMethod.PUT, getAbsolutePath(), updateClientId(snippetEntity), getHeaders(headersToOverride)).getResponse();
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
serviceFacade.verifyUpdateSnippet(requestSnippetDTO);
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
// update the snippet
|
||||
final RevisionDTO revision = snippetEntity.getRevision();
|
||||
final ConfigurationSnapshot<SnippetDTO> controllerResponse = serviceFacade.updateSnippet(
|
||||
new Revision(revision.getVersion(), revision.getClientId()), snippetEntity.getSnippet());
|
||||
|
||||
// get the results
|
||||
final SnippetDTO snippet = controllerResponse.getConfiguration();
|
||||
|
||||
// always prune update responses
|
||||
snippet.setContents(null);
|
||||
|
||||
// get the updated revision
|
||||
final RevisionDTO updatedRevision = new RevisionDTO();
|
||||
updatedRevision.setClientId(revision.getClientId());
|
||||
updatedRevision.setVersion(controllerResponse.getVersion());
|
||||
|
||||
// build the response entity
|
||||
SnippetEntity entity = new SnippetEntity();
|
||||
entity.setRevision(updatedRevision);
|
||||
entity.setSnippet(populateRemainingSnippetContent(snippet));
|
||||
|
||||
if (controllerResponse.isNew()) {
|
||||
return clusterContext(generateCreatedResponse(URI.create(snippet.getUri()), entity)).build();
|
||||
} else {
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the specified snippet.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param version The revision is used to verify the client is working with
|
||||
* the latest version of the flow.
|
||||
* @param clientId Optional client id. If the client id is not specified, a
|
||||
* new one will be generated. This value (whether specified or generated) is
|
||||
* included in the response.
|
||||
* @param id The id of the snippet to remove.
|
||||
* @return A entity containing the client id and an updated revision.
|
||||
*/
|
||||
@DELETE
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("{id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Deletes a snippet",
|
||||
response = SnippetEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 removeSnippet(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "The revision is used to verify the client is working with the latest version of the flow.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(VERSION) LongParameter version,
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The snippet id.",
|
||||
required = true
|
||||
)
|
||||
@PathParam("id") String id) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.DELETE, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
serviceFacade.verifyDeleteSnippet(id);
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
// determine the specified version
|
||||
Long clientVersion = null;
|
||||
if (version != null) {
|
||||
clientVersion = version.getLong();
|
||||
}
|
||||
|
||||
// delete the specified snippet
|
||||
final ConfigurationSnapshot<Void> controllerResponse = serviceFacade.deleteSnippet(new Revision(clientVersion, clientId.getClientId()), id);
|
||||
|
||||
// get the updated revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
revision.setVersion(controllerResponse.getVersion());
|
||||
|
||||
// build the response entity
|
||||
SnippetEntity entity = new SnippetEntity();
|
||||
entity.setRevision(revision);
|
||||
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
// setters
|
||||
public void setServiceFacade(NiFiServiceFacade serviceFacade) {
|
||||
this.serviceFacade = serviceFacade;
|
||||
}
|
||||
|
||||
public void setClusterManager(WebClusterManager clusterManager) {
|
||||
this.clusterManager = clusterManager;
|
||||
}
|
||||
|
||||
public void setProcessorResource(ProcessorResource processorResource) {
|
||||
this.processorResource = processorResource;
|
||||
}
|
||||
|
||||
public void setInputPortResource(InputPortResource inputPortResource) {
|
||||
this.inputPortResource = inputPortResource;
|
||||
}
|
||||
|
||||
public void setOutputPortResource(OutputPortResource outputPortResource) {
|
||||
this.outputPortResource = outputPortResource;
|
||||
}
|
||||
|
||||
public void setFunnelResource(FunnelResource funnelResource) {
|
||||
this.funnelResource = funnelResource;
|
||||
}
|
||||
|
||||
public void setLabelResource(LabelResource labelResource) {
|
||||
this.labelResource = labelResource;
|
||||
}
|
||||
|
||||
public void setRemoteProcessGroupResource(RemoteProcessGroupResource remoteProcessGroupResource) {
|
||||
this.remoteProcessGroupResource = remoteProcessGroupResource;
|
||||
}
|
||||
|
||||
public void setConnectionResource(ConnectionResource connectionResource) {
|
||||
this.connectionResource = connectionResource;
|
||||
}
|
||||
|
||||
public void setProcessGroupResource(ProcessGroupResource processGroupResource) {
|
||||
this.processGroupResource = processGroupResource;
|
||||
}
|
||||
|
||||
public void setProperties(NiFiProperties properties) {
|
||||
this.properties = properties;
|
||||
}
|
||||
}
|
|
@ -23,6 +23,7 @@ import com.wordnik.swagger.annotations.ApiResponse;
|
|||
import com.wordnik.swagger.annotations.ApiResponses;
|
||||
import com.wordnik.swagger.annotations.Authorization;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.authorization.Authorizer;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.exception.UnknownNodeException;
|
||||
import org.apache.nifi.cluster.manager.impl.WebClusterManager;
|
||||
|
@ -52,15 +53,15 @@ import java.util.Set;
|
|||
*/
|
||||
@Path("/system-diagnostics")
|
||||
@Api(
|
||||
value = "/system-diagnostics",
|
||||
description = "Provides diagnostics for the system NiFi is running on"
|
||||
value = "/system-diagnostics",
|
||||
description = "Endpoint for accessing system diagnostics."
|
||||
)
|
||||
public class SystemDiagnosticsResource extends ApplicationResource {
|
||||
|
||||
private NiFiServiceFacade serviceFacade;
|
||||
private WebClusterManager clusterManager;
|
||||
private NiFiProperties properties;
|
||||
|
||||
private Authorizer authorizer;
|
||||
|
||||
/**
|
||||
* Gets the system diagnostics for this NiFi instance.
|
||||
|
@ -72,7 +73,7 @@ public class SystemDiagnosticsResource extends ApplicationResource {
|
|||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets the diagnostics for the system NiFi is running on",
|
||||
|
@ -161,6 +162,10 @@ public class SystemDiagnosticsResource extends ApplicationResource {
|
|||
this.clusterManager = clusterManager;
|
||||
}
|
||||
|
||||
public void setAuthorizer(Authorizer authorizer) {
|
||||
this.authorizer = authorizer;
|
||||
}
|
||||
|
||||
public void setProperties(NiFiProperties properties) {
|
||||
this.properties = properties;
|
||||
}
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.web.api;
|
||||
|
||||
import com.sun.jersey.multipart.FormDataParam;
|
||||
import com.wordnik.swagger.annotations.Api;
|
||||
import com.wordnik.swagger.annotations.ApiOperation;
|
||||
import com.wordnik.swagger.annotations.ApiParam;
|
||||
|
@ -30,7 +29,6 @@ import org.apache.nifi.web.NiFiServiceFacade;
|
|||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
import org.apache.nifi.web.api.dto.TemplateDTO;
|
||||
import org.apache.nifi.web.api.entity.TemplateEntity;
|
||||
import org.apache.nifi.web.api.entity.TemplatesEntity;
|
||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -39,10 +37,8 @@ import javax.servlet.http.HttpServletRequest;
|
|||
import javax.ws.rs.Consumes;
|
||||
import javax.ws.rs.DELETE;
|
||||
import javax.ws.rs.DefaultValue;
|
||||
import javax.ws.rs.FormParam;
|
||||
import javax.ws.rs.GET;
|
||||
import javax.ws.rs.HttpMethod;
|
||||
import javax.ws.rs.POST;
|
||||
import javax.ws.rs.Path;
|
||||
import javax.ws.rs.PathParam;
|
||||
import javax.ws.rs.Produces;
|
||||
|
@ -50,22 +46,16 @@ 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.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.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* RESTful endpoint for managing a Template.
|
||||
*/
|
||||
@Api(hidden = true)
|
||||
@Path("/templates")
|
||||
@Api(
|
||||
value = "/templates",
|
||||
description = "Endpoint for managing a Template."
|
||||
)
|
||||
public class TemplateResource extends ApplicationResource {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(TemplateResource.class);
|
||||
|
@ -90,279 +80,12 @@ public class TemplateResource extends ApplicationResource {
|
|||
/**
|
||||
* Populates the uri for the specified template.
|
||||
*/
|
||||
private TemplateDTO populateRemainingTemplateContent(TemplateDTO template) {
|
||||
public TemplateDTO populateRemainingTemplateContent(TemplateDTO template) {
|
||||
// populate the template uri
|
||||
template.setUri(generateResourceUri("controller", "templates", template.getId()));
|
||||
template.setUri(generateResourceUri("templates", template.getId()));
|
||||
return template;
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves all the of templates in this NiFi.
|
||||
*
|
||||
* @param clientId Optional client id. If the client id is not specified, a
|
||||
* new one will be generated. This value (whether specified or generated) is
|
||||
* included in the response.
|
||||
* @return A templatesEntity.
|
||||
*/
|
||||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("") // necessary due to bug in swagger
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Gets all templates",
|
||||
response = TemplatesEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Read Only", type = "ROLE_MONITOR"),
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM"),
|
||||
@Authorization(value = "Administrator", type = "ROLE_ADMIN")
|
||||
}
|
||||
)
|
||||
@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 = 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 getTemplates(
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.GET, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// get all the templates
|
||||
final Set<TemplateDTO> templates = populateRemainingTemplatesContent(serviceFacade.getTemplates());
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// create the response entity
|
||||
final TemplatesEntity entity = new TemplatesEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setTemplates(templates);
|
||||
entity.setGenerated(new Date());
|
||||
|
||||
// generate the response
|
||||
return clusterContext(generateOkResponse(entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new template based off of the specified template.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param clientId Optional client id. If the client id is not specified, a
|
||||
* new one will be generated. This value (whether specified or generated) is
|
||||
* included in the response.
|
||||
* @param name The name of the template.
|
||||
* @param description The description of the template.
|
||||
* @param snippetId The id of the snippet this template is based on.
|
||||
* @return A templateEntity
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.APPLICATION_FORM_URLENCODED)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Path("") // necessary due to bug in swagger
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
value = "Creates a template",
|
||||
response = TemplateEntity.class,
|
||||
authorizations = {
|
||||
@Authorization(value = "Data Flow Manager", type = "ROLE_DFM")
|
||||
}
|
||||
)
|
||||
@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 createTemplate(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@ApiParam(
|
||||
value = "If the client id is not specified, new one will be generated. This value (whether specified or generated) is included in the response.",
|
||||
required = false
|
||||
)
|
||||
@FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@ApiParam(
|
||||
value = "The template name.",
|
||||
required = true
|
||||
)
|
||||
@FormParam("name") String name,
|
||||
@ApiParam(
|
||||
value = "The template description.",
|
||||
required = false
|
||||
)
|
||||
@FormParam("description") String description,
|
||||
@ApiParam(
|
||||
value = "The id of the snippet whose contents will comprise the template.",
|
||||
required = true
|
||||
)
|
||||
@FormParam("snippetId") String snippetId) {
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), getRequestParameters(true), getHeaders()).getResponse();
|
||||
}
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
// create the template and generate the json
|
||||
final TemplateDTO template = serviceFacade.createTemplate(name, description, snippetId);
|
||||
populateRemainingTemplateContent(template);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// build the response entity
|
||||
final TemplateEntity entity = new TemplateEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setTemplate(template);
|
||||
|
||||
// build the response
|
||||
return clusterContext(generateCreatedResponse(URI.create(template.getUri()), entity)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Imports the specified template.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param clientId Optional client id. If the client id is not specified, a
|
||||
* new one will be generated. This value (whether specified or generated) is
|
||||
* included in the response.
|
||||
* @param in The template stream
|
||||
* @return A templateEntity or an errorResponse XML snippet.
|
||||
*/
|
||||
@POST
|
||||
@Consumes(MediaType.MULTIPART_FORM_DATA)
|
||||
@Produces(MediaType.APPLICATION_XML)
|
||||
@Path("") // necessary due to bug in swagger
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
public Response importTemplate(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
@FormDataParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
|
||||
@FormDataParam("template") InputStream in) {
|
||||
|
||||
// unmarshal the template
|
||||
final TemplateDTO template;
|
||||
try {
|
||||
JAXBContext context = JAXBContext.newInstance(TemplateDTO.class);
|
||||
Unmarshaller unmarshaller = context.createUnmarshaller();
|
||||
JAXBElement<TemplateDTO> templateElement = unmarshaller.unmarshal(new StreamSource(in), TemplateDTO.class);
|
||||
template = templateElement.getValue();
|
||||
} catch (JAXBException jaxbe) {
|
||||
logger.warn("An error occurred while parsing a template.", jaxbe);
|
||||
String responseXml = String.format("<errorResponse status=\"%s\" statusText=\"The specified template is not in a valid format.\"/>", Response.Status.BAD_REQUEST.getStatusCode());
|
||||
return Response.status(Response.Status.OK).entity(responseXml).type("application/xml").build();
|
||||
} catch (IllegalArgumentException iae) {
|
||||
logger.warn("Unable to import template.", iae);
|
||||
String responseXml = String.format("<errorResponse status=\"%s\" statusText=\"%s\"/>", Response.Status.BAD_REQUEST.getStatusCode(), iae.getMessage());
|
||||
return Response.status(Response.Status.OK).entity(responseXml).type("application/xml").build();
|
||||
} catch (Exception e) {
|
||||
logger.warn("An error occurred while importing a template.", e);
|
||||
String responseXml = String.format("<errorResponse status=\"%s\" statusText=\"Unable to import the specified template: %s\"/>",
|
||||
Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), e.getMessage());
|
||||
return Response.status(Response.Status.OK).entity(responseXml).type("application/xml").build();
|
||||
}
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
revision.setClientId(clientId.getClientId());
|
||||
|
||||
// build the response entity
|
||||
TemplateEntity entity = new TemplateEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setTemplate(template);
|
||||
|
||||
// replicate if cluster manager
|
||||
if (properties.isClusterManager()) {
|
||||
// change content type to JSON for serializing entity
|
||||
final Map<String, String> headersToOverride = new HashMap<>();
|
||||
headersToOverride.put("content-type", MediaType.APPLICATION_JSON);
|
||||
|
||||
// replicate the request
|
||||
return clusterManager.applyRequest(HttpMethod.POST, getAbsolutePath(), updateClientId(entity), getHeaders(headersToOverride)).getResponse();
|
||||
}
|
||||
|
||||
// otherwise import the template locally
|
||||
return importTemplate(httpServletRequest, entity);
|
||||
}
|
||||
|
||||
/**
|
||||
* Imports the specified template.
|
||||
*
|
||||
* @param httpServletRequest request
|
||||
* @param templateEntity A templateEntity.
|
||||
* @return A templateEntity.
|
||||
*/
|
||||
@POST
|
||||
@Consumes({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Produces(MediaType.APPLICATION_XML)
|
||||
@Path("") // necessary due to bug in swagger
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
public Response importTemplate(
|
||||
@Context HttpServletRequest httpServletRequest,
|
||||
TemplateEntity templateEntity) {
|
||||
|
||||
// handle expects request (usually from the cluster manager)
|
||||
final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
|
||||
if (expects != null) {
|
||||
return generateContinueResponse().build();
|
||||
}
|
||||
|
||||
try {
|
||||
// verify the template was specified
|
||||
if (templateEntity == null || templateEntity.getTemplate() == null) {
|
||||
throw new IllegalArgumentException("Template details must be specified.");
|
||||
}
|
||||
|
||||
// import the template
|
||||
final TemplateDTO template = serviceFacade.importTemplate(templateEntity.getTemplate());
|
||||
populateRemainingTemplateContent(template);
|
||||
|
||||
// create the revision
|
||||
final RevisionDTO revision = new RevisionDTO();
|
||||
if (templateEntity.getRevision() == null) {
|
||||
revision.setClientId(new ClientIdParameter().getClientId());
|
||||
} else {
|
||||
revision.setClientId(templateEntity.getRevision().getClientId());
|
||||
}
|
||||
|
||||
// build the response entity
|
||||
TemplateEntity entity = new TemplateEntity();
|
||||
entity.setRevision(revision);
|
||||
entity.setTemplate(template);
|
||||
|
||||
// build the response
|
||||
return clusterContext(generateCreatedResponse(URI.create(template.getUri()), entity)).build();
|
||||
} catch (IllegalArgumentException | IllegalStateException e) {
|
||||
logger.info("Unable to import template: " + e);
|
||||
String responseXml = String.format("<errorResponse status=\"%s\" statusText=\"%s\"/>", Response.Status.BAD_REQUEST.getStatusCode(), e.getMessage());
|
||||
return Response.status(Response.Status.OK).entity(responseXml).type("application/xml").build();
|
||||
} catch (Exception e) {
|
||||
logger.warn("An error occurred while importing a template.", e);
|
||||
String responseXml
|
||||
= String.format("<errorResponse status=\"%s\" statusText=\"Unable to import the specified template: %s\"/>", Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), e.getMessage());
|
||||
return Response.status(Response.Status.OK).entity(responseXml).type("application/xml").build();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves the specified template.
|
||||
*
|
||||
|
@ -375,7 +98,7 @@ public class TemplateResource extends ApplicationResource {
|
|||
@GET
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces(MediaType.APPLICATION_XML)
|
||||
@Path("{id}")
|
||||
@Path("{id}/download")
|
||||
// TODO - @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
|
||||
@ApiOperation(
|
||||
value = "Exports a template",
|
||||
|
@ -442,7 +165,7 @@ public class TemplateResource extends ApplicationResource {
|
|||
*/
|
||||
@DELETE
|
||||
@Consumes(MediaType.WILDCARD)
|
||||
@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@Path("{id}")
|
||||
// TODO - @PreAuthorize("hasRole('ROLE_DFM')")
|
||||
@ApiOperation(
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.nifi.action.details.PurgeDetails;
|
|||
import org.apache.nifi.annotation.behavior.Stateful;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.authorization.Resource;
|
||||
import org.apache.nifi.cluster.HeartbeatPayload;
|
||||
import org.apache.nifi.cluster.event.Event;
|
||||
import org.apache.nifi.cluster.manager.StatusMerger;
|
||||
|
@ -1479,6 +1480,7 @@ public final class DtoFactory {
|
|||
final FlowSnippetDTO dto = new FlowSnippetDTO();
|
||||
|
||||
for (final ProcessorNode procNode : group.getProcessors()) {
|
||||
// authorization check
|
||||
dto.getProcessors().add(createProcessorDto(procNode));
|
||||
}
|
||||
|
||||
|
@ -1960,6 +1962,19 @@ public final class DtoFactory {
|
|||
return dto;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a ResourceDTO from the specified Resource.
|
||||
*
|
||||
* @param resource resource
|
||||
* @return dto
|
||||
*/
|
||||
public ResourceDTO createResourceDto(final Resource resource) {
|
||||
final ResourceDTO dto = new ResourceDTO();
|
||||
dto.setIdentifier(resource.getIdentifier());
|
||||
dto.setName(resource.getName());
|
||||
return dto;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a ProcessorConfigDTO from the specified ProcessorNode.
|
||||
*
|
||||
|
|
|
@ -20,6 +20,9 @@ import org.apache.commons.collections4.CollectionUtils;
|
|||
import org.apache.commons.lang3.ClassUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.admin.service.KeyService;
|
||||
import org.apache.nifi.authorization.Resource;
|
||||
import org.apache.nifi.authorization.resource.ResourceFactory;
|
||||
import org.apache.nifi.authorization.resource.ResourceType;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.connectable.Connectable;
|
||||
|
@ -31,11 +34,15 @@ import org.apache.nifi.controller.ControllerService;
|
|||
import org.apache.nifi.controller.Counter;
|
||||
import org.apache.nifi.controller.FlowController;
|
||||
import org.apache.nifi.controller.ProcessorNode;
|
||||
import org.apache.nifi.controller.ReportingTaskNode;
|
||||
import org.apache.nifi.controller.ScheduledState;
|
||||
import org.apache.nifi.controller.Template;
|
||||
import org.apache.nifi.controller.label.Label;
|
||||
import org.apache.nifi.controller.queue.FlowFileQueue;
|
||||
import org.apache.nifi.controller.queue.QueueSize;
|
||||
import org.apache.nifi.controller.repository.ContentNotFoundException;
|
||||
import org.apache.nifi.controller.repository.claim.ContentDirection;
|
||||
import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||
import org.apache.nifi.controller.status.ConnectionStatus;
|
||||
import org.apache.nifi.controller.status.PortStatus;
|
||||
import org.apache.nifi.controller.status.ProcessGroupStatus;
|
||||
|
@ -80,6 +87,7 @@ import org.apache.nifi.web.NiFiCoreException;
|
|||
import org.apache.nifi.web.ResourceNotFoundException;
|
||||
import org.apache.nifi.web.api.dto.DocumentedTypeDTO;
|
||||
import org.apache.nifi.web.api.dto.DtoFactory;
|
||||
import org.apache.nifi.web.api.dto.TemplateDTO;
|
||||
import org.apache.nifi.web.api.dto.provenance.AttributeDTO;
|
||||
import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO;
|
||||
import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO;
|
||||
|
@ -706,6 +714,79 @@ public class ControllerFacade {
|
|||
return flowController.getSystemDiagnostics();
|
||||
}
|
||||
|
||||
public List<Resource> getResources() {
|
||||
final List<Resource> resources = new ArrayList<>();
|
||||
resources.add(ResourceFactory.getSystemResource());
|
||||
resources.add(ResourceFactory.getControllerResource());
|
||||
resources.add(ResourceFactory.getFlowResource());
|
||||
resources.add(ResourceFactory.getProvenanceResource());
|
||||
resources.add(ResourceFactory.getProxyResource());
|
||||
resources.add(ResourceFactory.getResourceResource());
|
||||
|
||||
final ProcessGroup root = flowController.getGroup(flowController.getRootGroupId());
|
||||
|
||||
// add each processor
|
||||
for (final ProcessorNode processor : root.findAllProcessors()) {
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.Processor, processor.getIdentifier(), processor.getName()));
|
||||
resources.add(ResourceFactory.getComponentProvenanceResource(ResourceType.Processor, processor.getIdentifier(), processor.getName()));
|
||||
}
|
||||
|
||||
// add each connection
|
||||
for (final Connection connection : root.findAllConnections()) {
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.Connection, connection.getIdentifier(), connection.getName()));
|
||||
resources.add(ResourceFactory.getFlowFileQueueResource(connection.getIdentifier(), connection.getName()));
|
||||
}
|
||||
|
||||
// add each label
|
||||
for (final Label label : root.findAllLabels()) {
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.Label, label.getIdentifier(), label.getValue()));
|
||||
}
|
||||
|
||||
// add each process group
|
||||
for (final ProcessGroup processGroup : root.findAllProcessGroups()) {
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.ProcessGroup, processGroup.getIdentifier(), processGroup.getName()));
|
||||
resources.add(ResourceFactory.getComponentProvenanceResource(ResourceType.ProcessGroup, processGroup.getIdentifier(), processGroup.getName()));
|
||||
}
|
||||
|
||||
// add each remote process group
|
||||
for (final RemoteProcessGroup remoteProcessGroup : root.findAllRemoteProcessGroups()) {
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.RemoteProcessGroup, remoteProcessGroup.getIdentifier(), remoteProcessGroup.getName()));
|
||||
resources.add(ResourceFactory.getComponentProvenanceResource(ResourceType.RemoteProcessGroup, remoteProcessGroup.getIdentifier(), remoteProcessGroup.getName()));
|
||||
}
|
||||
|
||||
// add each input port
|
||||
for (final Port inputPort : root.findAllInputPorts()) {
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.InputPort, inputPort.getIdentifier(), inputPort.getName()));
|
||||
resources.add(ResourceFactory.getComponentProvenanceResource(ResourceType.InputPort, inputPort.getIdentifier(), inputPort.getName()));
|
||||
}
|
||||
|
||||
// add each output port
|
||||
for (final Port outputPort : root.findAllOutputPorts()) {
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.OutputPort, outputPort.getIdentifier(), outputPort.getName()));
|
||||
resources.add(ResourceFactory.getComponentProvenanceResource(ResourceType.OutputPort, outputPort.getIdentifier(), outputPort.getName()));
|
||||
}
|
||||
|
||||
// add each controller service
|
||||
for (final ControllerServiceNode controllerService : flowController.getAllControllerServices()) {
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.ControllerService, controllerService.getIdentifier(), controllerService.getName()));
|
||||
}
|
||||
|
||||
// add each reporting task
|
||||
for (final ReportingTaskNode reportingTask : flowController.getAllReportingTasks()) {
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.ReportingTask, reportingTask.getIdentifier(), reportingTask.getName()));
|
||||
}
|
||||
|
||||
// add each template
|
||||
for (final Template template : flowController.getTemplates()) {
|
||||
final TemplateDTO details = template.getDetails();
|
||||
resources.add(ResourceFactory.getComponentResource(ResourceType.Template, details.getId(), details.getName()));
|
||||
}
|
||||
|
||||
// TODO - need token resource?
|
||||
// resources.add(ResourceFactory.getTokenResource());
|
||||
return resources;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the available options for searching provenance.
|
||||
*
|
||||
|
|
|
@ -65,16 +65,6 @@ public class NodeRequestFilter implements Filter {
|
|||
HttpServletRequest httpReq = (HttpServletRequest) req;
|
||||
HttpServletResponse httpResp = (HttpServletResponse) resp;
|
||||
|
||||
/*
|
||||
* If we're the cluster manager or we're sent head requests, continue.
|
||||
* Head requests are included because there exists a AJAX/XHR race
|
||||
* condition between the following requests:
|
||||
* HEAD /nifi-api/cluster
|
||||
* GET /nifi-api/controller/config
|
||||
* If the head request finishes first, then the UI JavaScript will display
|
||||
* a default error message and not display the error message given in this
|
||||
* filter for directly accessing connected nodes.
|
||||
*/
|
||||
if (properties.isClusterManager() || "HEAD".equalsIgnoreCase(httpReq.getMethod())) {
|
||||
filterChain.doFilter(req, resp);
|
||||
} else {
|
||||
|
|
|
@ -156,12 +156,23 @@
|
|||
</bean>
|
||||
|
||||
<!-- rest endpoints -->
|
||||
<bean id="controllerResource" class="org.apache.nifi.web.api.ControllerResource" scope="singleton">
|
||||
<bean id="flowResource" class="org.apache.nifi.web.api.FlowResource" scope="singleton">
|
||||
<property name="serviceFacade" ref="serviceFacade"/>
|
||||
<property name="properties" ref="nifiProperties"/>
|
||||
<property name="clusterManager" ref="clusterManager"/>
|
||||
</bean>
|
||||
<bean id="bulletinBoardResource" class="org.apache.nifi.web.api.BulletinBoardResource" scope="singleton">
|
||||
<bean id="resourceResource" class="org.apache.nifi.web.api.ResourceResource" scope="singleton">
|
||||
<property name="serviceFacade" ref="serviceFacade"/>
|
||||
<property name="properties" ref="nifiProperties"/>
|
||||
<property name="clusterManager" ref="clusterManager"/>
|
||||
</bean>
|
||||
<bean id="controllerResource" class="org.apache.nifi.web.api.ControllerResource" scope="singleton">
|
||||
<property name="serviceFacade" ref="serviceFacade"/>
|
||||
<property name="properties" ref="nifiProperties"/>
|
||||
<property name="clusterManager" ref="clusterManager"/>
|
||||
<property name="reportingTaskResource" ref="reportingTaskResource"/>
|
||||
</bean>
|
||||
<bean id="siteToSiteResource" class="org.apache.nifi.web.api.SiteToSiteResource" scope="singleton">
|
||||
<property name="serviceFacade" ref="serviceFacade"/>
|
||||
<property name="properties" ref="nifiProperties"/>
|
||||
<property name="clusterManager" ref="clusterManager"/>
|
||||
|
@ -192,6 +203,8 @@
|
|||
<property name="labelResource" ref="labelResource"/>
|
||||
<property name="remoteProcessGroupResource" ref="remoteProcessGroupResource"/>
|
||||
<property name="connectionResource" ref="connectionResource"/>
|
||||
<property name="templateResource" ref="templateResource"/>
|
||||
<property name="controllerServiceResource" ref="controllerServiceResource"/>
|
||||
</bean>
|
||||
<bean id="processorResource" class="org.apache.nifi.web.api.ProcessorResource" scope="singleton">
|
||||
<property name="serviceFacade" ref="serviceFacade"/>
|
||||
|
@ -203,6 +216,11 @@
|
|||
<property name="properties" ref="nifiProperties"/>
|
||||
<property name="clusterManager" ref="clusterManager"/>
|
||||
</bean>
|
||||
<bean id="flowfileQueueResource" class="org.apache.nifi.web.api.FlowFileQueueResource" scope="singleton">
|
||||
<property name="serviceFacade" ref="serviceFacade"/>
|
||||
<property name="properties" ref="nifiProperties"/>
|
||||
<property name="clusterManager" ref="clusterManager"/>
|
||||
</bean>
|
||||
<bean id="remoteProcessGroupResource" class="org.apache.nifi.web.api.RemoteProcessGroupResource" scope="singleton">
|
||||
<property name="serviceFacade" ref="serviceFacade"/>
|
||||
<property name="properties" ref="nifiProperties"/>
|
||||
|
@ -228,19 +246,6 @@
|
|||
<property name="properties" ref="nifiProperties"/>
|
||||
<property name="clusterManager" ref="clusterManager"/>
|
||||
</bean>
|
||||
<bean id="snippetResource" class="org.apache.nifi.web.api.SnippetResource" scope="singleton">
|
||||
<property name="serviceFacade" ref="serviceFacade"/>
|
||||
<property name="properties" ref="nifiProperties"/>
|
||||
<property name="clusterManager" ref="clusterManager"/>
|
||||
<property name="processorResource" ref="processorResource"/>
|
||||
<property name="inputPortResource" ref="inputPortResource"/>
|
||||
<property name="outputPortResource" ref="outputPortResource"/>
|
||||
<property name="funnelResource" ref="funnelResource"/>
|
||||
<property name="labelResource" ref="labelResource"/>
|
||||
<property name="remoteProcessGroupResource" ref="remoteProcessGroupResource"/>
|
||||
<property name="connectionResource" ref="connectionResource"/>
|
||||
<property name="processGroupResource" ref="processGroupResource"/>
|
||||
</bean>
|
||||
<bean id="historyResource" class="org.apache.nifi.web.api.HistoryResource" scope="singleton">
|
||||
<property name="serviceFacade" ref="serviceFacade"/>
|
||||
</bean>
|
||||
|
|
|
@ -135,7 +135,7 @@ $(document).ready(function () {
|
|||
if (top === window) {
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: '../nifi-api/controller/banners',
|
||||
url: '../nifi-api/flow/banners',
|
||||
dataType: 'json'
|
||||
}).then(function (response) {
|
||||
// ensure the banners response is specified
|
||||
|
@ -184,7 +184,7 @@ $(document).ready(function () {
|
|||
// get the about details
|
||||
var about = $.ajax({
|
||||
type: 'GET',
|
||||
url: '../nifi-api/controller/about',
|
||||
url: '../nifi-api/flow/about',
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
var aboutDetails = response.about;
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.web.security.anonymous;
|
||||
|
||||
import org.apache.nifi.admin.service.KeyService;
|
||||
import org.apache.nifi.user.NiFiUser;
|
||||
import org.apache.nifi.web.security.token.NiFiAuthenticationToken;
|
||||
import org.apache.nifi.web.security.user.NiFiUserDetails;
|
||||
|
@ -28,7 +27,7 @@ import org.springframework.security.web.authentication.AnonymousAuthenticationFi
|
|||
import javax.servlet.http.HttpServletRequest;
|
||||
|
||||
/**
|
||||
* Custom AnonymouseAuthenticationFilter used to grant additional authorities depending on the current operating mode.
|
||||
* Custom AnonymousAuthenticationFilter used to grant additional authorities depending on the current operating mode.
|
||||
*/
|
||||
public class NiFiAnonymousUserFilter extends AnonymousAuthenticationFilter {
|
||||
|
||||
|
@ -36,8 +35,6 @@ public class NiFiAnonymousUserFilter extends AnonymousAuthenticationFilter {
|
|||
|
||||
private static final String ANONYMOUS_KEY = "anonymousNifiKey";
|
||||
|
||||
private KeyService keyService;
|
||||
|
||||
public NiFiAnonymousUserFilter() {
|
||||
super(ANONYMOUS_KEY);
|
||||
}
|
||||
|
@ -47,9 +44,4 @@ public class NiFiAnonymousUserFilter extends AnonymousAuthenticationFilter {
|
|||
return new NiFiAuthenticationToken(new NiFiUserDetails(NiFiUser.ANONYMOUS));
|
||||
}
|
||||
|
||||
/* setters */
|
||||
public void setKeyService(KeyService keyService) {
|
||||
this.keyService = keyService;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -347,6 +347,7 @@
|
|||
<include>${staging.dir}/js/nf/nf-ajax-setup.js</include>
|
||||
<include>${staging.dir}/js/nf/provenance/nf-provenance.js</include>
|
||||
<include>${staging.dir}/js/nf/provenance/nf-provenance-table.js</include>
|
||||
<include>${staging.dir}/js/nf/provenance/nf-provenance-lineage.js</include>
|
||||
</includes>
|
||||
</aggregation>
|
||||
<aggregation>
|
||||
|
@ -361,6 +362,7 @@
|
|||
<include>${staging.dir}/js/nf/nf-ajax-setup.js</include>
|
||||
<include>${staging.dir}/js/nf/nf-processor-details.js</include>
|
||||
<include>${staging.dir}/js/nf/nf-connection-details.js</include>
|
||||
<include>${staging.dir}/js/nf/nf-status-history.js</include>
|
||||
<include>${staging.dir}/js/nf/summary/nf-summary.js</include>
|
||||
<include>${staging.dir}/js/nf/summary/nf-summary-table.js</include>
|
||||
<include>${staging.dir}/js/nf/summary/nf-cluster-search.js</include>
|
||||
|
|
|
@ -1,56 +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.web.filter;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import javax.servlet.Filter;
|
||||
import javax.servlet.FilterChain;
|
||||
import javax.servlet.FilterConfig;
|
||||
import javax.servlet.ServletException;
|
||||
import javax.servlet.ServletRequest;
|
||||
import javax.servlet.ServletResponse;
|
||||
import javax.servlet.http.HttpServletResponse;
|
||||
|
||||
/**
|
||||
* A filter to set a response header directing IE to use the most recent
|
||||
* document mode.
|
||||
*
|
||||
*/
|
||||
public class IeEdgeHeader implements Filter {
|
||||
|
||||
@Override
|
||||
public void doFilter(final ServletRequest req, final ServletResponse resp, final FilterChain filterChain)
|
||||
throws IOException, ServletException {
|
||||
|
||||
// add the header
|
||||
final HttpServletResponse response = (HttpServletResponse) resp;
|
||||
response.addHeader("X-UA-Compatible", "IE=edge");
|
||||
|
||||
// continue the chain
|
||||
filterChain.doFilter(req, resp);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(final FilterConfig config) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void destroy() {
|
||||
}
|
||||
|
||||
}
|
|
@ -20,7 +20,8 @@ nf.provenance.script.tags=<script type="text/javascript" src="js/nf/nf-namespace
|
|||
<script type="text/javascript" src="js/nf/nf-storage.js?${project.version}"></script>\n\
|
||||
<script type="text/javascript" src="js/nf/nf-ajax-setup.js?${project.version}"></script>\n\
|
||||
<script type="text/javascript" src="js/nf/provenance/nf-provenance.js?${project.version}"></script>\n\
|
||||
<script type="text/javascript" src="js/nf/provenance/nf-provenance-table.js?${project.version}"></script>
|
||||
<script type="text/javascript" src="js/nf/provenance/nf-provenance-table.js?${project.version}"></script>\n\
|
||||
<script type="text/javascript" src="js/nf/provenance/nf-provenance-lineage.js?${project.version}"></script>
|
||||
nf.provenance.style.tags=<link rel="stylesheet" href="css/reset.css?${project.version}" type="text/css" />\n\
|
||||
<link rel="stylesheet" href="css/main.css?${project.version}" type="text/css" />\n\
|
||||
<link rel="stylesheet" href="css/banner.css?${project.version}" type="text/css" />\n\
|
||||
|
|
|
@ -21,6 +21,7 @@ nf.summary.script.tags=<script type="text/javascript" src="js/nf/nf-namespace.js
|
|||
<script type="text/javascript" src="js/nf/nf-ajax-setup.js?${project.version}"></script>\n\
|
||||
<script type="text/javascript" src="js/nf/nf-processor-details.js?${project.version}"></script>\n\
|
||||
<script type="text/javascript" src="js/nf/nf-connection-details.js?${project.version}"></script>\n\
|
||||
<script type="text/javascript" src="js/nf/nf-status-history.js?${project.version}"></script>\n\
|
||||
<script type="text/javascript" src="js/nf/summary/nf-summary.js?${project.version}"></script>\n\
|
||||
<script type="text/javascript" src="js/nf/summary/nf-summary-table.js?${project.version}"></script>\n\
|
||||
<script type="text/javascript" src="js/nf/summary/nf-cluster-search.js?${project.version}"></script>
|
||||
|
|
|
@ -38,6 +38,7 @@
|
|||
<link rel="stylesheet" href="js/jquery/slickgrid/css/slick.grid.css" type="text/css" />
|
||||
<link rel="stylesheet" href="js/jquery/slickgrid/css/slick-default-theme.css" type="text/css" />
|
||||
<script type="text/javascript" src="js/codemirror/lib/codemirror-compressed.js"></script>
|
||||
<script type="text/javascript" src="js/d3/d3.min.js"></script>
|
||||
<script type="text/javascript" src="js/jquery/jquery-2.1.1.min.js"></script>
|
||||
<script type="text/javascript" src="js/jquery/ui-smoothness/jquery-ui-1.10.4.min.js"></script>
|
||||
<script type="text/javascript" src="js/jquery/jquery.base64.js"></script>
|
||||
|
|
|
@ -30,6 +30,7 @@
|
|||
<link rel="stylesheet" href="js/jquery/ui-smoothness/jquery-ui-1.10.4.min.css" type="text/css" />
|
||||
<link rel="stylesheet" href="js/jquery/slickgrid/css/slick.grid.css" type="text/css" />
|
||||
<link rel="stylesheet" href="js/jquery/slickgrid/css/slick-default-theme.css" type="text/css" />
|
||||
<script type="text/javascript" src="js/d3/d3.min.js"></script>
|
||||
<script type="text/javascript" src="js/jquery/jquery-2.1.1.min.js"></script>
|
||||
<script type="text/javascript" src="js/jquery/jquery.base64.js"></script>
|
||||
<script type="text/javascript" src="js/jquery/jquery.center.js"></script>
|
||||
|
|
|
@ -36,6 +36,7 @@
|
|||
<link rel="stylesheet" href="js/jquery/slickgrid/css/slick.grid.css" type="text/css" />
|
||||
<link rel="stylesheet" href="js/jquery/slickgrid/css/slick-default-theme.css" type="text/css" />
|
||||
<script type="text/javascript" src="js/codemirror/lib/codemirror-compressed.js"></script>
|
||||
<script type="text/javascript" src="js/d3/d3.min.js"></script>
|
||||
<script type="text/javascript" src="js/jquery/jquery-2.1.1.min.js"></script>
|
||||
<script type="text/javascript" src="js/jquery/ui-smoothness/jquery-ui-1.10.4.min.js"></script>
|
||||
<script type="text/javascript" src="js/jquery/jquery.base64.js"></script>
|
||||
|
|
|
@ -15,7 +15,9 @@
|
|||
limitations under the License.
|
||||
--%>
|
||||
<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %>
|
||||
<%@ taglib prefix="c" uri="http://java.sun.com/jsp/jstl/core" %>
|
||||
<div id="templates">
|
||||
<span id="template-group-id" class="hidden"><c:out value="${param.groupId}"/></span>
|
||||
<div id="templates-header-and-filter">
|
||||
<div id="templates-header-text">NiFi Templates</div>
|
||||
<div id="templates-filter-controls">
|
||||
|
@ -39,7 +41,7 @@
|
|||
<div id="template-browse-container">
|
||||
<div id="select-template-button" class="template-button">
|
||||
<span>Browse</span>
|
||||
<form id="template-upload-form" enctype="multipart/form-data" method="post" action="../nifi-api/controller/templates">
|
||||
<form id="template-upload-form" enctype="multipart/form-data" method="post">
|
||||
<input type="file" name="template" id="template-file-field"/>
|
||||
</form>
|
||||
</div>
|
||||
|
|
|
@ -136,15 +136,6 @@
|
|||
<url-pattern>/login</url-pattern>
|
||||
</servlet-mapping>
|
||||
|
||||
<filter>
|
||||
<filter-name>IeEdgeHeader</filter-name>
|
||||
<filter-class>org.apache.nifi.web.filter.IeEdgeHeader</filter-class>
|
||||
</filter>
|
||||
<filter-mapping>
|
||||
<filter-name>IeEdgeHeader</filter-name>
|
||||
<url-pattern>/*</url-pattern>
|
||||
</filter-mapping>
|
||||
|
||||
<welcome-file-list>
|
||||
<welcome-file>canvas.jsp</welcome-file>
|
||||
<welcome-file>/WEB-INF/pages/canvas.jsp</welcome-file>
|
||||
|
|
|
@ -810,7 +810,7 @@
|
|||
var promptForNewControllerService = function (gridContainer, grid, item, serviceType, configurationOptions) {
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: '../nifi-api/controller/controller-service-types',
|
||||
url: '../nifi-api/flow/controller-service-types',
|
||||
data: {
|
||||
serviceType: serviceType
|
||||
},
|
||||
|
|
|
@ -32,9 +32,9 @@ nf.BulletinBoard = (function () {
|
|||
maxBulletins: 1000,
|
||||
defaultFilterText: 'Filter',
|
||||
urls: {
|
||||
banners: '../nifi-api/controller/banners',
|
||||
controllerAbout: '../nifi-api/controller/about',
|
||||
bulletinBoard: '../nifi-api/bulletin-board'
|
||||
banners: '../nifi-api/flow/banners',
|
||||
about: '../nifi-api/flow/about',
|
||||
bulletinBoard: '../nifi-api/flow/bulletin-board'
|
||||
},
|
||||
styles: {
|
||||
filterList: 'bulletin-board-filter-list',
|
||||
|
@ -113,7 +113,7 @@ nf.BulletinBoard = (function () {
|
|||
// get the about details
|
||||
var getTitle = $.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.controllerAbout,
|
||||
url: config.urls.about,
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
var aboutDetails = response.about;
|
||||
|
|
|
@ -21,6 +21,7 @@ nf.Actions = (function () {
|
|||
|
||||
var config = {
|
||||
urls: {
|
||||
api: '../nifi-api',
|
||||
controller: '../nifi-api/controller'
|
||||
}
|
||||
};
|
||||
|
@ -516,7 +517,7 @@ nf.Actions = (function () {
|
|||
}
|
||||
};
|
||||
|
||||
updateResource(config.urls.controller + '/process-groups/' + encodeURIComponent(nf.Canvas.getGroupId()), entity).done(updateProcessGroup);
|
||||
updateResource(config.urls.api + '/process-groups/' + encodeURIComponent(nf.Canvas.getGroupId()), entity).done(updateProcessGroup);
|
||||
} else {
|
||||
var componentsToStart = selection.filter(function (d) {
|
||||
return nf.CanvasUtils.isRunnable(d3.select(this));
|
||||
|
@ -593,7 +594,7 @@ nf.Actions = (function () {
|
|||
}
|
||||
};
|
||||
|
||||
updateResource(config.urls.controller + '/process-groups/' + encodeURIComponent(nf.Canvas.getGroupId()), entity).done(updateProcessGroup);
|
||||
updateResource(config.urls.api + '/process-groups/' + encodeURIComponent(nf.Canvas.getGroupId()), entity).done(updateProcessGroup);
|
||||
} else {
|
||||
var componentsToStop = selection.filter(function (d) {
|
||||
return nf.CanvasUtils.isStoppable(d3.select(this));
|
||||
|
@ -1064,7 +1065,7 @@ nf.Actions = (function () {
|
|||
// issue the request to delete the flow files
|
||||
$.ajax({
|
||||
type: 'POST',
|
||||
url: connection.component.uri + '/drop-requests',
|
||||
url: '../nifi-api/flowfile-queues/' + connection.component.id + '/drop-requests',
|
||||
dataType: 'json',
|
||||
contentType: 'application/json'
|
||||
}).done(function(response) {
|
||||
|
@ -1251,17 +1252,19 @@ nf.Actions = (function () {
|
|||
// create the snippet
|
||||
nf.Snippet.create(snippetDetails).done(function (response) {
|
||||
var snippet = response.snippet;
|
||||
var createSnippetEntity = {
|
||||
'name': templateName,
|
||||
'description': templateDescription,
|
||||
'snippetId': snippet.id
|
||||
};
|
||||
|
||||
// create the template
|
||||
$.ajax({
|
||||
type: 'POST',
|
||||
url: config.urls.controller + '/templates',
|
||||
data: {
|
||||
name: templateName,
|
||||
description: templateDescription,
|
||||
snippetId: snippet.id
|
||||
},
|
||||
dataType: 'json'
|
||||
url: config.urls.api + '/process-groups/' + encodeURIComponent(nf.Canvas.getGroupId()) + '/templates',
|
||||
data: JSON.stringify(createSnippetEntity),
|
||||
dataType: 'json',
|
||||
contentType: 'application/json'
|
||||
}).done(function () {
|
||||
// show the confirmation dialog
|
||||
nf.Dialog.showOkDialog({
|
||||
|
@ -1361,7 +1364,7 @@ nf.Actions = (function () {
|
|||
}
|
||||
|
||||
// copy the snippet to the new location
|
||||
nf.Snippet.copy(snippet.id, nf.Canvas.getGroupId(), origin).done(function (copyResponse) {
|
||||
nf.Snippet.copy(snippet.id, origin).done(function (copyResponse) {
|
||||
var snippetContents = copyResponse.contents;
|
||||
|
||||
// update the graph accordingly
|
||||
|
|
|
@ -23,8 +23,7 @@ nf.CanvasHeader = (function () {
|
|||
|
||||
var config = {
|
||||
urls: {
|
||||
helpDocument: '../nifi-docs/documentation',
|
||||
controllerAbout: '../nifi-api/controller/about'
|
||||
helpDocument: '../nifi-docs/documentation'
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -61,7 +60,9 @@ nf.CanvasHeader = (function () {
|
|||
|
||||
// mouse over for the templates link
|
||||
nf.Common.addHoverEffect('#templates-link', 'templates-link', 'templates-link-hover').click(function () {
|
||||
nf.Shell.showPage('templates');
|
||||
nf.Shell.showPage('templates?' + $.param({
|
||||
groupId: nf.Canvas.getGroupId()
|
||||
}));
|
||||
});
|
||||
|
||||
// mouse over for the flow settings link
|
||||
|
@ -93,18 +94,6 @@ nf.CanvasHeader = (function () {
|
|||
nf.CanvasHeader.reloadAndClearWarnings();
|
||||
});
|
||||
|
||||
// get the about details
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.controllerAbout,
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
var aboutDetails = response.about;
|
||||
// set the document title and the about title
|
||||
document.title = aboutDetails.title;
|
||||
$('#nf-version').text(aboutDetails.version);
|
||||
}).fail(nf.Common.handleAjaxError);
|
||||
|
||||
// configure the about dialog
|
||||
$('#nf-about').modal({
|
||||
overlayBackground: true,
|
||||
|
|
|
@ -38,8 +38,7 @@ nf.CanvasToolbox = (function () {
|
|||
urls: {
|
||||
api: '../nifi-api',
|
||||
controller: '../nifi-api/controller',
|
||||
processorTypes: '../nifi-api/controller/processor-types',
|
||||
templates: '../nifi-api/controller/templates'
|
||||
processorTypes: '../nifi-api/flow/processor-types'
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -758,7 +757,7 @@ nf.CanvasToolbox = (function () {
|
|||
var promptForTemplate = function (pt) {
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.templates,
|
||||
url: config.urls.api + '/process-groups/' + encodeURIComponent(nf.Canvas.getGroupId()) + '/templates',
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
var templates = response.templates;
|
||||
|
@ -823,20 +822,20 @@ nf.CanvasToolbox = (function () {
|
|||
* @argument {object} pt The point that the template was dropped
|
||||
*/
|
||||
var createTemplate = function (templateId, pt) {
|
||||
var revision = nf.Client.getRevision();
|
||||
var instantiateTemplateInstance = {
|
||||
'revision': nf.Client.getRevision(),
|
||||
'templateId': templateId,
|
||||
'originX': pt.x,
|
||||
'originY': pt.y
|
||||
};
|
||||
|
||||
// create a new instance of the new template
|
||||
$.ajax({
|
||||
type: 'POST',
|
||||
url: config.urls.api + '/process-groups/' + encodeURIComponent(nf.Canvas.getGroupId()) + '/template-instance',
|
||||
data: {
|
||||
version: revision.version,
|
||||
clientId: revision.clientId,
|
||||
templateId: templateId,
|
||||
originX: pt.x,
|
||||
originY: pt.y
|
||||
},
|
||||
dataType: 'json'
|
||||
data: JSON.stringify(instantiateTemplateInstance),
|
||||
dataType: 'json',
|
||||
contentType: 'application/json'
|
||||
}).done(function (response) {
|
||||
// update the revision
|
||||
nf.Client.setRevision(response.revision);
|
||||
|
|
|
@ -77,28 +77,22 @@ nf.Canvas = (function () {
|
|||
var config = {
|
||||
urls: {
|
||||
api: '../nifi-api',
|
||||
identity: '../nifi-api/controller/identity',
|
||||
identity: '../nifi-api/flow/identity',
|
||||
authorities: '../nifi-api/controller/authorities',
|
||||
kerberos: '../nifi-api/access/kerberos',
|
||||
revision: '../nifi-api/controller/revision',
|
||||
status: '../nifi-api/controller/status',
|
||||
bulletinBoard: '../nifi-api/bulletin-board',
|
||||
banners: '../nifi-api/controller/banners',
|
||||
revision: '../nifi-api/flow/revision',
|
||||
status: '../nifi-api/flow/status',
|
||||
bulletinBoard: '../nifi-api/flow/bulletin-board',
|
||||
banners: '../nifi-api/flow/banners',
|
||||
controller: '../nifi-api/controller',
|
||||
controllerConfig: '../nifi-api/controller/config',
|
||||
about: '../nifi-api/flow/about',
|
||||
accessConfig: '../nifi-api/access/config',
|
||||
cluster: '../nifi-api/cluster',
|
||||
d3Script: 'js/d3/d3.min.js'
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Loads D3.
|
||||
*/
|
||||
var loadD3 = function () {
|
||||
return nf.Common.cachedScript(config.urls.d3Script);
|
||||
};
|
||||
|
||||
/**
|
||||
* Starts polling for the revision.
|
||||
*
|
||||
|
@ -834,7 +828,7 @@ nf.Canvas = (function () {
|
|||
return $.Deferred(function (deferred) {
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.api + '/process-groups/' + encodeURIComponent(processGroupId) + '/status',
|
||||
url: config.urls.api + '/flow/process-groups/' + encodeURIComponent(processGroupId) + '/status',
|
||||
data: {
|
||||
recursive: false
|
||||
},
|
||||
|
@ -1020,6 +1014,15 @@ nf.Canvas = (function () {
|
|||
dataType: 'json'
|
||||
});
|
||||
|
||||
// get the about details
|
||||
var aboutXhr = $.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.about,
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
|
||||
}).fail(nf.Common.handleAjaxError);
|
||||
|
||||
// get the login config
|
||||
var loginXhr = $.ajax({
|
||||
type: 'GET',
|
||||
|
@ -1046,9 +1049,10 @@ nf.Canvas = (function () {
|
|||
}).promise();
|
||||
|
||||
// ensure the config requests are loaded
|
||||
$.when(configXhr, loginXhr, userXhr).done(function (configResult, loginResult) {
|
||||
$.when(configXhr, loginXhr, aboutXhr, userXhr).done(function (configResult, loginResult, aboutResult) {
|
||||
var configResponse = configResult[0];
|
||||
var loginResponse = loginResult[0];
|
||||
var aboutResponse = aboutResult[0];
|
||||
|
||||
// calculate the canvas offset
|
||||
var canvasContainer = $('#canvas-container');
|
||||
|
@ -1057,10 +1061,15 @@ nf.Canvas = (function () {
|
|||
// get the config details
|
||||
var configDetails = configResponse.config;
|
||||
var loginDetails = loginResponse.config;
|
||||
|
||||
var aboutDetails = aboutResponse.about;
|
||||
|
||||
// set the document title and the about title
|
||||
document.title = aboutDetails.title;
|
||||
$('#nf-version').text(aboutDetails.version);
|
||||
|
||||
// store the content viewer url if available
|
||||
if (!nf.Common.isBlank(configDetails.contentViewerUrl)) {
|
||||
$('#nifi-content-viewer-url').text(configDetails.contentViewerUrl);
|
||||
if (!nf.Common.isBlank(aboutDetails.contentViewerUrl)) {
|
||||
$('#nifi-content-viewer-url').text(aboutDetails.contentViewerUrl);
|
||||
}
|
||||
|
||||
// when both request complete, load the application
|
||||
|
@ -1071,64 +1080,62 @@ nf.Canvas = (function () {
|
|||
// initialize whether site to site is secure
|
||||
secureSiteToSite = configDetails.siteToSiteSecure;
|
||||
|
||||
// load d3
|
||||
loadD3().done(function () {
|
||||
nf.Storage.init();
|
||||
// init storage
|
||||
nf.Storage.init();
|
||||
|
||||
// initialize the application
|
||||
initCanvas();
|
||||
nf.Canvas.View.init();
|
||||
nf.ContextMenu.init();
|
||||
nf.CanvasToolbar.init();
|
||||
nf.CanvasToolbox.init();
|
||||
nf.CanvasHeader.init(loginDetails.supportsLogin);
|
||||
nf.GraphControl.init();
|
||||
nf.Search.init();
|
||||
nf.Settings.init();
|
||||
nf.Actions.init();
|
||||
nf.QueueListing.init();
|
||||
nf.ComponentState.init();
|
||||
// initialize the application
|
||||
initCanvas();
|
||||
nf.Canvas.View.init();
|
||||
nf.ContextMenu.init();
|
||||
nf.CanvasToolbar.init();
|
||||
nf.CanvasToolbox.init();
|
||||
nf.CanvasHeader.init(loginDetails.supportsLogin);
|
||||
nf.GraphControl.init();
|
||||
nf.Search.init();
|
||||
nf.Settings.init();
|
||||
nf.Actions.init();
|
||||
nf.QueueListing.init();
|
||||
nf.ComponentState.init();
|
||||
|
||||
// initialize the component behaviors
|
||||
nf.Draggable.init();
|
||||
nf.Selectable.init();
|
||||
nf.Connectable.init();
|
||||
// initialize the component behaviors
|
||||
nf.Draggable.init();
|
||||
nf.Selectable.init();
|
||||
nf.Connectable.init();
|
||||
|
||||
// initialize the chart
|
||||
nf.StatusHistory.init(configDetails.timeOffset);
|
||||
// initialize the chart
|
||||
nf.StatusHistory.init(configDetails.timeOffset);
|
||||
|
||||
// initialize the birdseye
|
||||
nf.Birdseye.init();
|
||||
// initialize the birdseye
|
||||
nf.Birdseye.init();
|
||||
|
||||
// initialize components
|
||||
nf.ConnectionConfiguration.init();
|
||||
nf.ControllerService.init();
|
||||
nf.ReportingTask.init();
|
||||
nf.ProcessorConfiguration.init();
|
||||
nf.ProcessGroupConfiguration.init();
|
||||
nf.RemoteProcessGroupConfiguration.init();
|
||||
nf.RemoteProcessGroupPorts.init();
|
||||
nf.PortConfiguration.init();
|
||||
nf.LabelConfiguration.init();
|
||||
nf.ProcessorDetails.init();
|
||||
nf.ProcessGroupDetails.init();
|
||||
nf.PortDetails.init();
|
||||
nf.ConnectionDetails.init();
|
||||
nf.RemoteProcessGroupDetails.init();
|
||||
nf.GoTo.init();
|
||||
nf.Graph.init().done(function () {
|
||||
// determine the split between the polling
|
||||
var pollingSplit = autoRefreshIntervalSeconds / 2;
|
||||
// initialize components
|
||||
nf.ConnectionConfiguration.init();
|
||||
nf.ControllerService.init();
|
||||
nf.ReportingTask.init();
|
||||
nf.ProcessorConfiguration.init();
|
||||
nf.ProcessGroupConfiguration.init();
|
||||
nf.RemoteProcessGroupConfiguration.init();
|
||||
nf.RemoteProcessGroupPorts.init();
|
||||
nf.PortConfiguration.init();
|
||||
nf.LabelConfiguration.init();
|
||||
nf.ProcessorDetails.init();
|
||||
nf.ProcessGroupDetails.init();
|
||||
nf.PortDetails.init();
|
||||
nf.ConnectionDetails.init();
|
||||
nf.RemoteProcessGroupDetails.init();
|
||||
nf.GoTo.init();
|
||||
nf.Graph.init().done(function () {
|
||||
// determine the split between the polling
|
||||
var pollingSplit = autoRefreshIntervalSeconds / 2;
|
||||
|
||||
// register the revision and status polling
|
||||
startRevisionPolling(autoRefreshIntervalSeconds);
|
||||
setTimeout(function () {
|
||||
startStatusPolling(autoRefreshIntervalSeconds);
|
||||
}, pollingSplit * 1000);
|
||||
// register the revision and status polling
|
||||
startRevisionPolling(autoRefreshIntervalSeconds);
|
||||
setTimeout(function () {
|
||||
startStatusPolling(autoRefreshIntervalSeconds);
|
||||
}, pollingSplit * 1000);
|
||||
|
||||
// hide the splash screen
|
||||
nf.Canvas.hideSplash();
|
||||
}).fail(nf.Common.handleAjaxError);
|
||||
// hide the splash screen
|
||||
nf.Canvas.hideSplash();
|
||||
}).fail(nf.Common.handleAjaxError);
|
||||
}).fail(nf.Common.handleAjaxError);
|
||||
}).fail(nf.Common.handleAjaxError);
|
||||
|
|
|
@ -25,7 +25,7 @@ nf.ConnectionConfiguration = (function () {
|
|||
var config = {
|
||||
urls: {
|
||||
api: '../nifi-api',
|
||||
prioritizers: '../nifi-api/controller/prioritizers'
|
||||
prioritizers: '../nifi-api/flow/prioritizers'
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
@ -548,7 +548,7 @@ nf.ControllerService = (function () {
|
|||
|
||||
return $.ajax({
|
||||
type: 'GET',
|
||||
url: '../nifi-api/bulletin-board',
|
||||
url: '../nifi-api/flow/bulletin-board',
|
||||
data: {
|
||||
sourceId: ids
|
||||
},
|
||||
|
|
|
@ -380,7 +380,7 @@ nf.QueueListing = (function () {
|
|||
// issue the request to list the flow files
|
||||
$.ajax({
|
||||
type: 'POST',
|
||||
url: connection.component.uri + '/listing-requests',
|
||||
url: '../nifi-api/flowfile-queues/' + connection.component.id + '/listing-requests',
|
||||
dataType: 'json',
|
||||
contentType: 'application/json'
|
||||
}).done(function(response) {
|
||||
|
|
|
@ -19,7 +19,7 @@ nf.Search = (function () {
|
|||
var config = {
|
||||
search: 'Search',
|
||||
urls: {
|
||||
search: '../nifi-api/controller/search-results'
|
||||
search: '../nifi-api/flow/search-results'
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
@ -27,12 +27,12 @@ nf.Settings = (function () {
|
|||
filterList: 'filter-list'
|
||||
},
|
||||
urls: {
|
||||
api: '../nifi-api',
|
||||
controllerConfig: '../nifi-api/controller/config',
|
||||
controllerArchive: '../nifi-api/controller/archive',
|
||||
controllerServiceTypes: '../nifi-api/controller/controller-service-types',
|
||||
controllerServices: '../nifi-api/controller-services',
|
||||
reportingTaskTypes: '../nifi-api/controller/reporting-task-types',
|
||||
reportingTasks: '../nifi-api/reporting-tasks'
|
||||
controllerServiceTypes: '../nifi-api/flow/controller-service-types',
|
||||
reportingTaskTypes: '../nifi-api/flow/reporting-task-types',
|
||||
reportingTasks: '../nifi-api/controller/reporting-tasks'
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -80,19 +80,20 @@ nf.Settings = (function () {
|
|||
|
||||
// register the click listener for the save button
|
||||
$('#settings-save').click(function () {
|
||||
var revision = nf.Client.getRevision();
|
||||
|
||||
// marshal the configuration details
|
||||
var configuration = marshalConfiguration();
|
||||
configuration['version'] = revision.version;
|
||||
configuration['clientId'] = revision.clientId;
|
||||
var entity = {
|
||||
'revision': nf.Client.getRevision(),
|
||||
'config': configuration
|
||||
};
|
||||
|
||||
// save the new configuration details
|
||||
$.ajax({
|
||||
type: 'PUT',
|
||||
url: config.urls.controllerConfig,
|
||||
data: configuration,
|
||||
dataType: 'json'
|
||||
data: JSON.stringify(entity),
|
||||
dataType: 'json',
|
||||
contentType: 'application/json'
|
||||
}).done(function (response) {
|
||||
// update the revision
|
||||
nf.Client.setRevision(response.revision);
|
||||
|
@ -316,7 +317,7 @@ nf.Settings = (function () {
|
|||
// add the new controller service
|
||||
var addService = $.ajax({
|
||||
type: 'POST',
|
||||
url: config.urls.controllerServices + '/' + encodeURIComponent(availability),
|
||||
url: config.urls.api + '/process-groups/' + encodeURIComponent(nf.Canvas.getGroupId) + '/controller-services/' + encodeURIComponent(availability),
|
||||
data: JSON.stringify(controllerServiceEntity),
|
||||
dataType: 'json',
|
||||
contentType: 'application/json'
|
||||
|
@ -863,7 +864,7 @@ nf.Settings = (function () {
|
|||
// get the controller services that are running on the nodes
|
||||
var nodeControllerServices = $.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.controllerServices + '/' + encodeURIComponent(config.node),
|
||||
url: config.urls.api + '/process-groups/' + encodeURIComponent(nf.Canvas.getGroupId) + '/controller-services/' + encodeURIComponent(config.node),
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
var nodeServices = response.controllerServices;
|
||||
|
@ -881,7 +882,7 @@ nf.Settings = (function () {
|
|||
if (nf.Canvas.isClustered()) {
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.controllerServices + '/' + encodeURIComponent(config.ncm),
|
||||
url: config.urls.api + '/process-groups/' + encodeURIComponent(nf.Canvas.getGroupId) + '/controller-services/' + encodeURIComponent(config.ncm),
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
var ncmServices = response.controllerServices;
|
||||
|
|
|
@ -21,7 +21,6 @@ nf.Snippet = (function () {
|
|||
|
||||
var config = {
|
||||
urls: {
|
||||
snippets: '../nifi-api/controller/snippets',
|
||||
processGroups: '../nifi-api/process-groups'
|
||||
}
|
||||
};
|
||||
|
@ -37,14 +36,14 @@ nf.Snippet = (function () {
|
|||
var snippet = {
|
||||
parentGroupId: nf.Canvas.getGroupId(),
|
||||
linked: nf.Common.isDefinedAndNotNull(linked) ? linked : false,
|
||||
processorIds: [],
|
||||
funnelIds: [],
|
||||
inputPortIds: [],
|
||||
outputPortIds: [],
|
||||
remoteProcessGroupIds: [],
|
||||
processGroupIds: [],
|
||||
connectionIds: [],
|
||||
labelIds: []
|
||||
processors: [],
|
||||
funnels: [],
|
||||
inputPorts: [],
|
||||
outputPorts: [],
|
||||
remoteProcessGroups: [],
|
||||
processGroups: [],
|
||||
connections: [],
|
||||
labels: []
|
||||
};
|
||||
|
||||
// go through each component and identify its type
|
||||
|
@ -52,21 +51,21 @@ nf.Snippet = (function () {
|
|||
var selected = d3.select(this);
|
||||
|
||||
if (nf.CanvasUtils.isProcessor(selected)) {
|
||||
snippet.processorIds.push(d.component.id);
|
||||
snippet.processors.push(d.component.id);
|
||||
} else if (nf.CanvasUtils.isFunnel(selected)) {
|
||||
snippet.funnelIds.push(d.component.id);
|
||||
snippet.funnels.push(d.component.id);
|
||||
} else if (nf.CanvasUtils.isLabel(selected)) {
|
||||
snippet.labelIds.push(d.component.id);
|
||||
snippet.labels.push(d.component.id);
|
||||
} else if (nf.CanvasUtils.isInputPort(selected)) {
|
||||
snippet.inputPortIds.push(d.component.id);
|
||||
snippet.inputPorts.push(d.component.id);
|
||||
} else if (nf.CanvasUtils.isOutputPort(selected)) {
|
||||
snippet.outputPortIds.push(d.component.id);
|
||||
snippet.outputPorts.push(d.component.id);
|
||||
} else if (nf.CanvasUtils.isProcessGroup(selected)) {
|
||||
snippet.processGroupIds.push(d.component.id);
|
||||
snippet.processGroups.push(d.component.id);
|
||||
} else if (nf.CanvasUtils.isRemoteProcessGroup(selected)) {
|
||||
snippet.remoteProcessGroupIds.push(d.component.id);
|
||||
snippet.remoteProcessGroups.push(d.component.id);
|
||||
} else if (nf.CanvasUtils.isConnection(selected)) {
|
||||
snippet.connectionIds.push(d.component.id);
|
||||
snippet.connections.push(d.component.id);
|
||||
}
|
||||
});
|
||||
|
||||
|
@ -79,16 +78,17 @@ nf.Snippet = (function () {
|
|||
* @argument {object} snippet The snippet
|
||||
*/
|
||||
create: function (snippet) {
|
||||
var revision = nf.Client.getRevision();
|
||||
var snippetEntity = {
|
||||
'revision': nf.Client.getRevision(),
|
||||
'snippet': snippet
|
||||
};
|
||||
|
||||
return $.ajax({
|
||||
type: 'POST',
|
||||
url: config.urls.snippets,
|
||||
data: $.extend({
|
||||
version: revision.version,
|
||||
clientId: revision.clientId
|
||||
}, snippet),
|
||||
dataType: 'json'
|
||||
url: config.urls.processGroups + '/' + encodeURIComponent(nf.Canvas.getGroupId()) + '/snippets',
|
||||
data: JSON.stringify(snippetEntity),
|
||||
dataType: 'json',
|
||||
contentType: 'application/json'
|
||||
}).done(function (response) {
|
||||
// update the revision
|
||||
nf.Client.setRevision(response.revision);
|
||||
|
@ -99,23 +99,22 @@ nf.Snippet = (function () {
|
|||
* Copies the snippet to the specified group and origin.
|
||||
*
|
||||
* @argument {string} snippetId The snippet id
|
||||
* @argument {string} groupId The group id
|
||||
* @argument {object} origin The origin
|
||||
*/
|
||||
copy: function (snippetId, groupId, origin) {
|
||||
var revision = nf.Client.getRevision();
|
||||
copy: function (snippetId, origin) {
|
||||
var copySnippetRequestEntity = {
|
||||
'revision': nf.Client.getRevision(),
|
||||
'snippetId': snippetId,
|
||||
'originX': origin.x,
|
||||
'originY': origin.y
|
||||
};
|
||||
|
||||
return $.ajax({
|
||||
type: 'POST',
|
||||
url: config.urls.processGroups + '/' + encodeURIComponent(groupId) + '/snippet-instance',
|
||||
data: {
|
||||
version: revision.version,
|
||||
clientId: revision.clientId,
|
||||
snippetId: snippetId,
|
||||
originX: origin.x,
|
||||
originY: origin.y
|
||||
},
|
||||
dataType: 'json'
|
||||
url: config.urls.processGroups + '/' + encodeURIComponent(nf.Canvas.getGroupId()) + '/snippet-instance',
|
||||
data: JSON.stringify(copySnippetRequestEntity),
|
||||
dataType: 'json',
|
||||
contentType: 'application/json'
|
||||
}).done(function (response) {
|
||||
// update the revision
|
||||
nf.Client.setRevision(response.revision);
|
||||
|
@ -132,7 +131,7 @@ nf.Snippet = (function () {
|
|||
|
||||
return $.ajax({
|
||||
type: 'DELETE',
|
||||
url: config.urls.snippets + '/' + encodeURIComponent(snippetId) + '?' + $.param({
|
||||
url: config.urls.processGroups + '/' + encodeURIComponent(nf.Canvas.getGroupId()) + '/snippets/' + encodeURIComponent(snippetId) + '?' + $.param({
|
||||
version: revision.version,
|
||||
clientId: revision.clientId
|
||||
})
|
||||
|
@ -149,17 +148,20 @@ nf.Snippet = (function () {
|
|||
* @argument {string} newGroupId The new group id
|
||||
*/
|
||||
move: function (snippetId, newGroupId) {
|
||||
var revision = nf.Client.getRevision();
|
||||
var snippetEntity = {
|
||||
'revision': nf.Client.getRevision(),
|
||||
'snippet': {
|
||||
'id': snippetId,
|
||||
'parentGroupId': newGroupId
|
||||
}
|
||||
};
|
||||
|
||||
return $.ajax({
|
||||
type: 'PUT',
|
||||
url: config.urls.snippets + '/' + encodeURIComponent(snippetId),
|
||||
data: {
|
||||
version: revision.version,
|
||||
clientId: revision.clientId,
|
||||
parentGroupId: newGroupId
|
||||
},
|
||||
dataType: 'json'
|
||||
url: config.urls.processGroups + '/' + encodeURIComponent(nf.Canvas.getGroupId()) + '/snippets/' + encodeURIComponent(snippetId),
|
||||
data: JSON.stringify(snippetEntity),
|
||||
dataType: 'json',
|
||||
contentType: 'application/json'
|
||||
}).done(function (response) {
|
||||
// update the revision
|
||||
nf.Client.setRevision(response.revision);
|
||||
|
@ -172,17 +174,20 @@ nf.Snippet = (function () {
|
|||
* @argument {string} snippetId The snippet id
|
||||
*/
|
||||
unlink: function (snippetId) {
|
||||
var revision = nf.Client.getRevision();
|
||||
var snippetEntity = {
|
||||
'revision': nf.Client.getRevision(),
|
||||
'snippet': {
|
||||
'id': snippetId,
|
||||
'linked': false
|
||||
}
|
||||
};
|
||||
|
||||
return $.ajax({
|
||||
type: 'PUT',
|
||||
url: config.urls.snippets + '/' + encodeURIComponent(snippetId),
|
||||
data: {
|
||||
version: revision.version,
|
||||
clientId: revision.clientId,
|
||||
linked: false
|
||||
},
|
||||
dataType: 'json'
|
||||
url: config.urls.processGroups + '/' + encodeURIComponent(nf.Canvas.getGroupId()) + '/snippets/' + encodeURIComponent(snippetId),
|
||||
data: JSON.stringify(snippetEntity),
|
||||
dataType: 'json',
|
||||
contentType: 'application/json'
|
||||
}).done(function (response) {
|
||||
// update the revision
|
||||
nf.Client.setRevision(response.revision);
|
||||
|
@ -195,17 +200,20 @@ nf.Snippet = (function () {
|
|||
* @argument {string} snippetId The snippet id
|
||||
*/
|
||||
link: function (snippetId) {
|
||||
var revision = nf.Client.getRevision();
|
||||
var snippetEntity = {
|
||||
'revision': nf.Client.getRevision(),
|
||||
'snippet': {
|
||||
'id': snippetId,
|
||||
'linked': true
|
||||
}
|
||||
};
|
||||
|
||||
return $.ajax({
|
||||
type: 'PUT',
|
||||
url: config.urls.snippets + '/' + encodeURIComponent(snippetId),
|
||||
data: {
|
||||
version: revision.version,
|
||||
clientId: revision.clientId,
|
||||
linked: true
|
||||
},
|
||||
dataType: 'json'
|
||||
url: config.urls.processGroups + '/' + encodeURIComponent(nf.Canvas.getGroupId()) + '/snippets/' + encodeURIComponent(snippetId),
|
||||
data: JSON.stringify(snippetEntity),
|
||||
dataType: 'json',
|
||||
contentType: 'application/json'
|
||||
}).done(function (response) {
|
||||
// update the revision
|
||||
nf.Client.setRevision(response.revision);
|
||||
|
|
|
@ -29,8 +29,8 @@ nf.Cluster = (function () {
|
|||
*/
|
||||
var config = {
|
||||
urls: {
|
||||
banners: '../nifi-api/controller/banners',
|
||||
controllerAbout: '../nifi-api/controller/about',
|
||||
banners: '../nifi-api/flow/banners',
|
||||
about: '../nifi-api/flow/about',
|
||||
authorities: '../nifi-api/controller/authorities'
|
||||
}
|
||||
};
|
||||
|
@ -140,7 +140,7 @@ nf.Cluster = (function () {
|
|||
// get the about details
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.controllerAbout,
|
||||
url: config.urls.about,
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
var aboutDetails = response.about;
|
||||
|
|
|
@ -29,8 +29,8 @@ nf.Counters = (function () {
|
|||
*/
|
||||
var config = {
|
||||
urls: {
|
||||
banners: '../nifi-api/controller/banners',
|
||||
controllerAbout: '../nifi-api/controller/about',
|
||||
banners: '../nifi-api/flow/banners',
|
||||
about: '../nifi-api/flow/about',
|
||||
authorities: '../nifi-api/controller/authorities'
|
||||
}
|
||||
};
|
||||
|
@ -140,7 +140,7 @@ nf.Counters = (function () {
|
|||
// get the about details
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.controllerAbout,
|
||||
url: config.urls.about,
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
var aboutDetails = response.about;
|
||||
|
|
|
@ -29,8 +29,8 @@ nf.History = (function () {
|
|||
*/
|
||||
var config = {
|
||||
urls: {
|
||||
banners: '../nifi-api/controller/banners',
|
||||
controllerAbout: '../nifi-api/controller/about',
|
||||
banners: '../nifi-api/flow/banners',
|
||||
about: '../nifi-api/flow/about',
|
||||
authorities: '../nifi-api/controller/authorities'
|
||||
}
|
||||
};
|
||||
|
@ -142,7 +142,7 @@ nf.History = (function () {
|
|||
// get the about details
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.controllerAbout,
|
||||
url: config.urls.about,
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
var aboutDetails = response.about;
|
||||
|
|
|
@ -27,7 +27,7 @@ nf.Login = (function () {
|
|||
|
||||
var config = {
|
||||
urls: {
|
||||
identity: '../nifi-api/controller/identity',
|
||||
identity: '../nifi-api/flow/identity',
|
||||
users: '../nifi-api/controller/users',
|
||||
token: '../nifi-api/access/token',
|
||||
accessStatus: '../nifi-api/access',
|
||||
|
|
|
@ -123,19 +123,6 @@ nf.Common = (function () {
|
|||
nf.Common.authorities = roles;
|
||||
},
|
||||
|
||||
/**
|
||||
* Loads a script at the specified URL. Supports caching the script on the browser.
|
||||
*
|
||||
* @param {string} url
|
||||
*/
|
||||
cachedScript: function (url) {
|
||||
return $.ajax({
|
||||
dataType: 'script',
|
||||
cache: true,
|
||||
url: url
|
||||
});
|
||||
},
|
||||
|
||||
/**
|
||||
* Automatically refresh tokens by checking once an hour if its going to expire soon.
|
||||
*/
|
||||
|
|
|
@ -33,8 +33,7 @@ nf.StatusHistory = (function () {
|
|||
label: 'Label'
|
||||
},
|
||||
urls: {
|
||||
api: '../nifi-api',
|
||||
processGroups: '../nifi-api/process-groups/'
|
||||
api: '../nifi-api'
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -1100,7 +1099,7 @@ nf.StatusHistory = (function () {
|
|||
showConnectionChart: function (groupId, connectionId, selectedDescriptor) {
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.api + '/connections/' + encodeURIComponent(connectionId) + '/status/history',
|
||||
url: config.urls.api + '/flow/connections/' + encodeURIComponent(connectionId) + '/status/history',
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
handleStatusHistoryResponse(groupId, connectionId, response.statusHistory, config.type.connection, selectedDescriptor);
|
||||
|
@ -1117,7 +1116,7 @@ nf.StatusHistory = (function () {
|
|||
showProcessorChart: function (groupId, processorId, selectedDescriptor) {
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.api + '/processors/' + encodeURIComponent(processorId) + '/status/history',
|
||||
url: config.urls.api + '/flow/processors/' + encodeURIComponent(processorId) + '/status/history',
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
handleStatusHistoryResponse(groupId, processorId, response.statusHistory, config.type.processor, selectedDescriptor);
|
||||
|
@ -1134,7 +1133,7 @@ nf.StatusHistory = (function () {
|
|||
showProcessGroupChart: function (groupId, processGroupId, selectedDescriptor) {
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.processGroups + encodeURIComponent(processGroupId) + '/status/history',
|
||||
url: config.urls.api + '/flow/process-groups/' + encodeURIComponent(processGroupId) + '/status/history',
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
handleStatusHistoryResponse(groupId, processGroupId, response.statusHistory, config.type.processGroup, selectedDescriptor);
|
||||
|
@ -1151,7 +1150,7 @@ nf.StatusHistory = (function () {
|
|||
showRemoteProcessGroupChart: function (groupId, remoteProcessGroupId, selectedDescriptor) {
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.api + '/remote-process-groups/' + encodeURIComponent(remoteProcessGroupId) + '/status/history',
|
||||
url: config.urls.api + '/flow/remote-process-groups/' + encodeURIComponent(remoteProcessGroupId) + '/status/history',
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
handleStatusHistoryResponse(groupId, remoteProcessGroupId, response.statusHistory, config.type.remoteProcessGroup, selectedDescriptor);
|
||||
|
|
|
@ -25,8 +25,8 @@ nf.ProvenanceLineage = (function () {
|
|||
var config = {
|
||||
sliderTickCount: 75,
|
||||
urls: {
|
||||
lineage: '../nifi-api/controller/provenance/lineage',
|
||||
events: '../nifi-api/controller/provenance/events/'
|
||||
lineage: '../nifi-api/provenance/lineage',
|
||||
events: '../nifi-api/provenance/events/'
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -123,11 +123,18 @@ nf.ProvenanceLineage = (function () {
|
|||
* @returns {deferred}
|
||||
*/
|
||||
var submitLineage = function (lineageRequest) {
|
||||
var lineageEntity = {
|
||||
'lineage': {
|
||||
'request': lineageRequest
|
||||
}
|
||||
};
|
||||
|
||||
return $.ajax({
|
||||
type: 'POST',
|
||||
url: config.urls.lineage,
|
||||
data: lineageRequest,
|
||||
dataType: 'json'
|
||||
data: JSON.stringify(lineageEntity),
|
||||
dataType: 'json',
|
||||
contentType: 'application/json'
|
||||
}).fail(nf.Common.handleAjaxError);
|
||||
};
|
||||
|
||||
|
|
|
@ -32,9 +32,9 @@ nf.ProvenanceTable = (function () {
|
|||
hidden: 'hidden'
|
||||
},
|
||||
urls: {
|
||||
searchOptions: '../nifi-api/controller/provenance/search-options',
|
||||
replays: '../nifi-api/controller/provenance/replays',
|
||||
provenance: '../nifi-api/controller/provenance',
|
||||
searchOptions: '../nifi-api/provenance/search-options',
|
||||
replays: '../nifi-api/provenance/replays',
|
||||
provenance: '../nifi-api/provenance',
|
||||
cluster: '../nifi-api/cluster',
|
||||
d3Script: 'js/d3/d3.min.js',
|
||||
lineageScript: 'js/nf/provenance/nf-provenance-lineage.js',
|
||||
|
@ -48,29 +48,6 @@ nf.ProvenanceTable = (function () {
|
|||
*/
|
||||
var cachedQuery = {};
|
||||
|
||||
/**
|
||||
* Loads the lineage capabilities when the current browser supports SVG.
|
||||
*/
|
||||
var loadLineageCapabilities = function () {
|
||||
return $.Deferred(function (deferred) {
|
||||
if (nf.Common.SUPPORTS_SVG) {
|
||||
nf.Common.cachedScript(config.urls.d3Script).done(function () {
|
||||
nf.Common.cachedScript(config.urls.lineageScript).done(function () {
|
||||
// initialize the lineage graph
|
||||
nf.ProvenanceLineage.init();
|
||||
deferred.resolve();
|
||||
}).fail(function () {
|
||||
deferred.reject();
|
||||
});
|
||||
}).fail(function () {
|
||||
deferred.reject();
|
||||
});
|
||||
} else {
|
||||
deferred.resolve();
|
||||
}
|
||||
}).promise();
|
||||
};
|
||||
|
||||
/**
|
||||
* Downloads the content for the provenance event that is currently loaded in the specified direction.
|
||||
*
|
||||
|
@ -121,7 +98,7 @@ nf.ProvenanceTable = (function () {
|
|||
var eventId = $('#provenance-event-id').text();
|
||||
|
||||
// build the uri to the data
|
||||
var dataUri = controllerUri + '/provenance/events/' + encodeURIComponent(eventId) + '/content/' + encodeURIComponent(direction);
|
||||
var dataUri = controllerUri + 'provenance/events/' + encodeURIComponent(eventId) + '/content/' + encodeURIComponent(direction);
|
||||
|
||||
// generate tokens as necessary
|
||||
var getAccessTokens = $.Deferred(function (deferred) {
|
||||
|
@ -280,21 +257,22 @@ nf.ProvenanceTable = (function () {
|
|||
if (nf.Common.isDFM()) {
|
||||
// replay
|
||||
$('#replay-content').on('click', function () {
|
||||
var parameters = {
|
||||
eventId: $('#provenance-event-id').text()
|
||||
var replayEntity = {
|
||||
'eventId': $('#provenance-event-id').text()
|
||||
};
|
||||
|
||||
// conditionally include the cluster node id
|
||||
var clusterNodeId = $('#provenance-event-cluster-node-id').text();
|
||||
if (!nf.Common.isBlank(clusterNodeId)) {
|
||||
parameters['clusterNodeId'] = clusterNodeId;
|
||||
replayEntity['clusterNodeId'] = clusterNodeId;
|
||||
}
|
||||
|
||||
$.ajax({
|
||||
type: 'POST',
|
||||
url: config.urls.replays,
|
||||
data: parameters,
|
||||
dataType: 'json'
|
||||
data: JSON.stringify(replayEntity),
|
||||
dataType: 'json',
|
||||
contentType: 'application/json'
|
||||
}).done(function (response) {
|
||||
nf.Dialog.showOkDialog({
|
||||
dialogContent: 'Successfully submitted replay request.',
|
||||
|
@ -432,7 +410,7 @@ nf.ProvenanceTable = (function () {
|
|||
}
|
||||
|
||||
// add the search criteria
|
||||
search = $.extend(getSearchCriteria(), search);
|
||||
search['searchTerms'] = getSearchCriteria();
|
||||
|
||||
// reload the table
|
||||
nf.ProvenanceTable.loadProvenanceTable(search);
|
||||
|
@ -522,7 +500,7 @@ nf.ProvenanceTable = (function () {
|
|||
|
||||
// if the field isn't blank include it in the search
|
||||
if (!nf.Common.isBlank(searchValue)) {
|
||||
searchCriteria['search[' + fieldId + ']'] = searchValue;
|
||||
searchCriteria[fieldId] = searchValue;
|
||||
}
|
||||
});
|
||||
return searchCriteria;
|
||||
|
@ -861,13 +839,21 @@ nf.ProvenanceTable = (function () {
|
|||
* @returns {deferred}
|
||||
*/
|
||||
var submitProvenance = function (provenance) {
|
||||
var provenanceEntity = {
|
||||
'provenance': {
|
||||
'request': $.extend({
|
||||
maxResults: config.maxResults
|
||||
}, provenance)
|
||||
}
|
||||
};
|
||||
|
||||
// submit the provenance request
|
||||
return $.ajax({
|
||||
type: 'POST',
|
||||
url: config.urls.provenance,
|
||||
data: $.extend({
|
||||
maxResults: config.maxResults
|
||||
}, provenance),
|
||||
dataType: 'json'
|
||||
data: JSON.stringify(provenanceEntity),
|
||||
dataType: 'json',
|
||||
contentType: 'application/json'
|
||||
}).fail(nf.Common.handleAjaxError);
|
||||
};
|
||||
|
||||
|
@ -1020,15 +1006,16 @@ nf.ProvenanceTable = (function () {
|
|||
deferred.reject();
|
||||
nf.Common.handleAjaxError(xhr, status, error);
|
||||
};
|
||||
|
||||
// load the lineage capabilities
|
||||
loadLineageCapabilities().done(function () {
|
||||
initDetailsDialog();
|
||||
initProvenanceQueryDialog();
|
||||
initProvenanceTable(isClustered);
|
||||
initSearchDialog(isClustered).done(function () {
|
||||
deferred.resolve();
|
||||
}).fail(failure);
|
||||
|
||||
// initialize the lineage view
|
||||
nf.ProvenanceLineage.init();
|
||||
|
||||
// initialize the table view
|
||||
initDetailsDialog();
|
||||
initProvenanceQueryDialog();
|
||||
initProvenanceTable(isClustered);
|
||||
initSearchDialog(isClustered).done(function () {
|
||||
deferred.resolve();
|
||||
}).fail(failure);
|
||||
}).promise();
|
||||
},
|
||||
|
|
|
@ -30,9 +30,8 @@ nf.Provenance = (function () {
|
|||
var config = {
|
||||
urls: {
|
||||
cluster: '../nifi-api/cluster',
|
||||
banners: '../nifi-api/controller/banners',
|
||||
config: '../nifi-api/controller/config',
|
||||
controllerAbout: '../nifi-api/controller/about',
|
||||
banners: '../nifi-api/flow/banners',
|
||||
about: '../nifi-api/flow/about',
|
||||
authorities: '../nifi-api/controller/authorities'
|
||||
}
|
||||
};
|
||||
|
@ -68,21 +67,27 @@ nf.Provenance = (function () {
|
|||
/**
|
||||
* Loads the controller configuration.
|
||||
*/
|
||||
var loadControllerConfig = function () {
|
||||
return $.ajax({
|
||||
var loadAbout = function () {
|
||||
// get the about details
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.config,
|
||||
url: config.urls.about,
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
var config = response.config;
|
||||
var aboutDetails = response.about;
|
||||
var provenanceTitle = aboutDetails.title + ' Data Provenance';
|
||||
|
||||
// store the controller name
|
||||
$('#nifi-controller-uri').text(config.uri);
|
||||
$('#nifi-controller-uri').text(aboutDetails.uri);
|
||||
|
||||
// store the content viewer url if available
|
||||
if (!nf.Common.isBlank(config.contentViewerUrl)) {
|
||||
$('#nifi-content-viewer-url').text(config.contentViewerUrl);
|
||||
if (!nf.Common.isBlank(aboutDetails.contentViewerUrl)) {
|
||||
$('#nifi-content-viewer-url').text(aboutDetails.contentViewerUrl);
|
||||
}
|
||||
|
||||
// set the document title and the about title
|
||||
document.title = provenanceTitle;
|
||||
$('#provenance-header-text').text(provenanceTitle);
|
||||
}).fail(nf.Common.handleAjaxError);
|
||||
};
|
||||
|
||||
|
@ -177,10 +182,10 @@ nf.Provenance = (function () {
|
|||
nf.Storage.init();
|
||||
|
||||
// load the users authorities and detect if the NiFi is clustered
|
||||
$.when(loadControllerConfig(), loadAuthorities(), detectedCluster()).done(function () {
|
||||
$.when(loadAbout(), loadAuthorities(), detectedCluster()).done(function () {
|
||||
// create the provenance table
|
||||
nf.ProvenanceTable.init(isClustered).done(function () {
|
||||
var search = {};
|
||||
var searchTerms = {};
|
||||
|
||||
// look for a processor id in the query search
|
||||
var initialComponentId = $('#intial-component-query').text();
|
||||
|
@ -189,9 +194,7 @@ nf.Provenance = (function () {
|
|||
$('input.searchable-component-id').val(initialComponentId);
|
||||
|
||||
// build the search criteria
|
||||
search = $.extend(search, {
|
||||
'search[ProcessorID]': initialComponentId
|
||||
});
|
||||
searchTerms['ProcessorID'] = initialComponentId;
|
||||
}
|
||||
|
||||
// look for a flowfile uuid in the query search
|
||||
|
@ -201,32 +204,24 @@ nf.Provenance = (function () {
|
|||
$('input.searchable-flowfile-uuid').val(initialFlowFileUuid);
|
||||
|
||||
// build the search criteria
|
||||
search = $.extend(search, {
|
||||
'search[FlowFileUUID]': initialFlowFileUuid
|
||||
searchTerms['FlowFileUUID'] = initialFlowFileUuid;
|
||||
}
|
||||
|
||||
// load the provenance table
|
||||
if ($.isEmptyObject(searchTerms)) {
|
||||
// load the provenance table
|
||||
nf.ProvenanceTable.loadProvenanceTable();
|
||||
} else {
|
||||
// load the provenance table
|
||||
nf.ProvenanceTable.loadProvenanceTable({
|
||||
'searchTerms': searchTerms
|
||||
});
|
||||
}
|
||||
|
||||
// load the provenance table
|
||||
nf.ProvenanceTable.loadProvenanceTable(search);
|
||||
|
||||
// once the table is initialized, finish initializing the page
|
||||
initializeProvenancePage().done(function () {
|
||||
// configure the initial grid height
|
||||
nf.ProvenanceTable.resetTableSize();
|
||||
|
||||
// get the about details
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.controllerAbout,
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
var aboutDetails = response.about;
|
||||
var provenanceTitle = aboutDetails.title + ' Data Provenance';
|
||||
|
||||
// set the document title and the about title
|
||||
document.title = provenanceTitle;
|
||||
$('#provenance-header-text').text(provenanceTitle);
|
||||
}).fail(nf.Common.handleAjaxError);
|
||||
});
|
||||
});
|
||||
});
|
||||
|
|
|
@ -21,9 +21,7 @@ nf.ClusterSearch = (function () {
|
|||
var config = {
|
||||
search: 'Search nodes',
|
||||
urls: {
|
||||
clusterSearch: '../nifi-api/cluster/search-results',
|
||||
status: '../nifi-api/process-groups/root/status',
|
||||
systemDiagnostics: '../nifi-api/system-diagnostics'
|
||||
clusterSearch: '../nifi-api/cluster/search-results'
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
@ -28,49 +28,12 @@ nf.SummaryTable = (function () {
|
|||
},
|
||||
urls: {
|
||||
api: '../nifi-api',
|
||||
status: '../nifi-api/process-groups/root/status',
|
||||
processGroups: '../nifi-api/process-groups/',
|
||||
status: '../nifi-api/flow/process-groups/root/status',
|
||||
systemDiagnostics: '../nifi-api/system-diagnostics',
|
||||
controllerConfig: '../nifi-api/controller/config',
|
||||
d3Script: 'js/d3/d3.min.js',
|
||||
statusHistory: 'js/nf/nf-status-history.js'
|
||||
controllerConfig: '../nifi-api/controller/config'
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Loads the lineage capabilities when the current browser supports SVG.
|
||||
*/
|
||||
var loadChartCapabilities = function () {
|
||||
return $.Deferred(function (deferred) {
|
||||
if (nf.Common.SUPPORTS_SVG) {
|
||||
nf.Common.cachedScript(config.urls.d3Script).done(function () {
|
||||
// get the controller config to get the server offset
|
||||
var configRequest = $.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.controllerConfig,
|
||||
dataType: 'json'
|
||||
});
|
||||
|
||||
// get the config details and load the chart script
|
||||
$.when(configRequest, nf.Common.cachedScript(config.urls.statusHistory)).done(function (response) {
|
||||
var configResponse = response[0];
|
||||
var configDetails = configResponse.config;
|
||||
|
||||
// initialize the chart
|
||||
nf.StatusHistory.init(configDetails.timeOffset);
|
||||
deferred.resolve();
|
||||
}).fail(function () {
|
||||
deferred.reject();
|
||||
});
|
||||
}).fail(function () {
|
||||
deferred.reject();
|
||||
});
|
||||
} else {
|
||||
deferred.resolve();
|
||||
}
|
||||
}).promise();
|
||||
};
|
||||
|
||||
/**
|
||||
* Goes to the specified component if possible.
|
||||
*
|
||||
|
@ -2162,7 +2125,7 @@ nf.SummaryTable = (function () {
|
|||
// get the summary
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.api + '/processors/' + encodeURIComponent(processorId) + '/status',
|
||||
url: config.urls.api + '/flow/processors/' + encodeURIComponent(processorId) + '/status',
|
||||
data: {
|
||||
nodewise: true
|
||||
},
|
||||
|
@ -2219,7 +2182,7 @@ nf.SummaryTable = (function () {
|
|||
// get the summary
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.api + '/connections/' + encodeURIComponent(connectionId) + '/status',
|
||||
url: config.urls.api + '/flow/connections/' + encodeURIComponent(connectionId) + '/status',
|
||||
data: {
|
||||
nodewise: true
|
||||
},
|
||||
|
@ -2272,7 +2235,7 @@ nf.SummaryTable = (function () {
|
|||
// get the summary
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.processGroups + encodeURIComponent(processGroupId) + '/status',
|
||||
url: config.urls.api + '/flow/process-groups/' + encodeURIComponent(processGroupId) + '/status',
|
||||
data: {
|
||||
nodewise: true,
|
||||
recursive: false
|
||||
|
@ -2332,7 +2295,7 @@ nf.SummaryTable = (function () {
|
|||
// get the summary
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.api + '/input-ports/' + encodeURIComponent(inputPortId) + '/status',
|
||||
url: config.urls.api + '/flow/input-ports/' + encodeURIComponent(inputPortId) + '/status',
|
||||
data: {
|
||||
nodewise: true
|
||||
},
|
||||
|
@ -2384,7 +2347,7 @@ nf.SummaryTable = (function () {
|
|||
// get the summary
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.api + '/output-ports/' + encodeURIComponent(outputPortId) + '/status',
|
||||
url: config.urls.api + '/flow/output-ports/' + encodeURIComponent(outputPortId) + '/status',
|
||||
data: {
|
||||
nodewise: true
|
||||
},
|
||||
|
@ -2436,7 +2399,7 @@ nf.SummaryTable = (function () {
|
|||
// get the summary
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.api + '/remote-process-groups/' + encodeURIComponent(remoteProcessGroupId) + '/status',
|
||||
url: config.urls.api + '/flow/remote-process-groups/' + encodeURIComponent(remoteProcessGroupId) + '/status',
|
||||
data: {
|
||||
nodewise: true
|
||||
},
|
||||
|
@ -2492,7 +2455,17 @@ nf.SummaryTable = (function () {
|
|||
*/
|
||||
init: function (isClustered) {
|
||||
return $.Deferred(function (deferred) {
|
||||
loadChartCapabilities().done(function () {
|
||||
// get the controller config to get the server offset
|
||||
var configRequest = $.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.controllerConfig,
|
||||
dataType: 'json'
|
||||
}).done(function (configResponse) {
|
||||
var configDetails = configResponse.config;
|
||||
|
||||
// initialize the chart
|
||||
nf.StatusHistory.init(configDetails.timeOffset);
|
||||
|
||||
// initialize the processor/connection details dialog
|
||||
nf.ProcessorDetails.init(false);
|
||||
nf.ConnectionDetails.init(false);
|
||||
|
|
|
@ -29,8 +29,8 @@ nf.Summary = (function () {
|
|||
*/
|
||||
var config = {
|
||||
urls: {
|
||||
banners: '../nifi-api/controller/banners',
|
||||
controllerAbout: '../nifi-api/controller/about',
|
||||
banners: '../nifi-api/flow/banners',
|
||||
about: '../nifi-api/flow/about',
|
||||
cluster: '../nifi-api/cluster'
|
||||
}
|
||||
};
|
||||
|
@ -142,7 +142,7 @@ nf.Summary = (function () {
|
|||
// get the about details
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.controllerAbout,
|
||||
url: config.urls.about,
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
var aboutDetails = response.about;
|
||||
|
|
|
@ -28,11 +28,16 @@ nf.TemplatesTable = (function () {
|
|||
filterList: 'templates-filter-list'
|
||||
},
|
||||
urls: {
|
||||
templates: '../nifi-api/controller/templates',
|
||||
api: '../nifi-api',
|
||||
downloadToken: '../nifi-api/access/download-token'
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* the current group id
|
||||
*/
|
||||
var groupId;
|
||||
|
||||
/**
|
||||
* Sorts the specified data using the specified sort details.
|
||||
*
|
||||
|
@ -68,7 +73,7 @@ nf.TemplatesTable = (function () {
|
|||
dialogContent: 'Delete template \'' + nf.Common.escapeHtml(template.name) + '\'?',
|
||||
overlayBackground: false,
|
||||
yesHandler: function () {
|
||||
deleteTemplate(template.id);
|
||||
deleteTemplate(template);
|
||||
}
|
||||
});
|
||||
};
|
||||
|
@ -76,17 +81,17 @@ nf.TemplatesTable = (function () {
|
|||
/**
|
||||
* Deletes the template with the specified id.
|
||||
*
|
||||
* @argument {string} templateId The template id
|
||||
* @argument {string} template The template
|
||||
*/
|
||||
var deleteTemplate = function (templateId) {
|
||||
var deleteTemplate = function (template) {
|
||||
$.ajax({
|
||||
type: 'DELETE',
|
||||
url: config.urls.templates + '/' + encodeURIComponent(templateId),
|
||||
url: template.uri,
|
||||
dataType: 'json'
|
||||
}).done(function () {
|
||||
var templatesGrid = $('#templates-table').data('gridInstance');
|
||||
var templatesData = templatesGrid.getData();
|
||||
templatesData.deleteItem(templateId);
|
||||
templatesData.deleteItem(template.id);
|
||||
|
||||
// update the total number of templates
|
||||
$('#total-templates').text(templatesData.getItems().length);
|
||||
|
@ -167,9 +172,9 @@ nf.TemplatesTable = (function () {
|
|||
|
||||
// open the url
|
||||
if ($.isEmptyObject(parameters)) {
|
||||
window.open(config.urls.templates + '/' + encodeURIComponent(template.id));
|
||||
window.open(template.uri + '/download');
|
||||
} else {
|
||||
window.open(config.urls.templates + '/' + encodeURIComponent(template.id) + '?' + $.param(parameters));
|
||||
window.open(template.uri + '/download' + '?' + $.param(parameters));
|
||||
}
|
||||
}).fail(function () {
|
||||
nf.Dialog.showOkDialog({
|
||||
|
@ -332,9 +337,17 @@ nf.TemplatesTable = (function () {
|
|||
* Load the processor templates table.
|
||||
*/
|
||||
loadTemplatesTable: function () {
|
||||
groupId = $('#template-group-id').text();
|
||||
if (nf.Common.isUndefined(groupId) || nf.Common.isNull(groupId)) {
|
||||
nf.Dialog.showOkDialog({
|
||||
overlayBackground: false,
|
||||
content: 'Group id not specified.'
|
||||
});
|
||||
}
|
||||
|
||||
return $.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.templates,
|
||||
url: config.urls.api + '/process-groups/' + encodeURIComponent(groupId) + '/templates',
|
||||
data: {
|
||||
verbose: false
|
||||
},
|
||||
|
|
|
@ -29,12 +29,17 @@ nf.Templates = (function () {
|
|||
*/
|
||||
var config = {
|
||||
urls: {
|
||||
banners: '../nifi-api/controller/banners',
|
||||
controllerAbout: '../nifi-api/controller/about',
|
||||
banners: '../nifi-api/flow/banners',
|
||||
about: '../nifi-api/flow/about',
|
||||
authorities: '../nifi-api/controller/authorities'
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* the current group id
|
||||
*/
|
||||
var groupId;
|
||||
|
||||
/**
|
||||
* Loads the current users authorities.
|
||||
*/
|
||||
|
@ -102,6 +107,7 @@ nf.Templates = (function () {
|
|||
|
||||
// initialize the form
|
||||
var templateForm = $('#template-upload-form').ajaxForm({
|
||||
url: '../nifi-api/process-groups/' + encodeURIComponent(groupId) + '/templates/upload',
|
||||
dataType: 'xml',
|
||||
success: function (response, statusText, xhr, form) {
|
||||
// see if the import was successful
|
||||
|
@ -205,6 +211,15 @@ nf.Templates = (function () {
|
|||
*/
|
||||
init: function () {
|
||||
nf.Storage.init();
|
||||
|
||||
// ensure the group id is specified
|
||||
groupId = $('#template-group-id').text();
|
||||
if (nf.Common.isUndefined(groupId) || nf.Common.isNull(groupId)) {
|
||||
nf.Dialog.showOkDialog({
|
||||
overlayBackground: false,
|
||||
content: 'Group id not specified.'
|
||||
});
|
||||
}
|
||||
|
||||
// load the users authorities
|
||||
loadAuthorities().done(function () {
|
||||
|
@ -222,7 +237,7 @@ nf.Templates = (function () {
|
|||
// get the about details
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.controllerAbout,
|
||||
url: config.urls.about,
|
||||
dataType: 'json'
|
||||
}).done(function (response) {
|
||||
var aboutDetails = response.about;
|
||||
|
|
Loading…
Reference in New Issue