mirror of https://github.com/apache/nifi.git
NIFI-2007:
- Restoring bulletin functionality. - Ensuring appropriate merging of bulletins in clustered responses.
This commit is contained in:
parent
548561ed4c
commit
0b437e09a7
|
@ -19,7 +19,6 @@ package org.apache.nifi.controller.status;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
|
@ -554,18 +553,6 @@ public class ProcessGroupStatus implements Cloneable {
|
|||
merged.setSentContentSize(merged.getSentContentSize() + statusToMerge.getSentContentSize());
|
||||
merged.setSentCount(merged.getSentCount() + statusToMerge.getSentCount());
|
||||
merged.setActiveThreadCount(merged.getActiveThreadCount() + statusToMerge.getActiveThreadCount());
|
||||
|
||||
List<String> mergedAuthenticationIssues = merged.getAuthorizationIssues();
|
||||
if (mergedAuthenticationIssues == null) {
|
||||
mergedAuthenticationIssues = new ArrayList<>();
|
||||
}
|
||||
|
||||
final List<String> nodeAuthorizationIssues = statusToMerge.getAuthorizationIssues();
|
||||
if (nodeAuthorizationIssues != null && !nodeAuthorizationIssues.isEmpty()) {
|
||||
mergedAuthenticationIssues.addAll(nodeAuthorizationIssues);
|
||||
}
|
||||
|
||||
merged.setAuthorizationIssues(mergedAuthenticationIssues);
|
||||
}
|
||||
|
||||
target.setRemoteProcessGroupStatus(mergedRemoteGroupMap.values());
|
||||
|
|
|
@ -16,9 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.controller.status;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
|
@ -40,7 +37,6 @@ public class RemoteProcessGroupStatus implements Cloneable {
|
|||
private Integer inactiveRemotePortCount;
|
||||
|
||||
private long averageLineageDuration;
|
||||
private List<String> authorizationIssues = new ArrayList<>();
|
||||
|
||||
public String getTargetUri() {
|
||||
return uri;
|
||||
|
@ -138,14 +134,6 @@ public class RemoteProcessGroupStatus implements Cloneable {
|
|||
this.inactiveRemotePortCount = inactiveRemotePortCount;
|
||||
}
|
||||
|
||||
public List<String> getAuthorizationIssues() {
|
||||
return new ArrayList<>(authorizationIssues);
|
||||
}
|
||||
|
||||
public void setAuthorizationIssues(List<String> authorizationIssues) {
|
||||
this.authorizationIssues = new ArrayList<>(Objects.requireNonNull(authorizationIssues));
|
||||
}
|
||||
|
||||
public long getAverageLineageDuration() {
|
||||
return averageLineageDuration;
|
||||
}
|
||||
|
@ -178,7 +166,6 @@ public class RemoteProcessGroupStatus implements Cloneable {
|
|||
clonedObj.activeRemotePortCount = activeRemotePortCount;
|
||||
clonedObj.inactiveRemotePortCount = inactiveRemotePortCount;
|
||||
clonedObj.averageLineageDuration = averageLineageDuration;
|
||||
clonedObj.authorizationIssues = getAuthorizationIssues();
|
||||
return clonedObj;
|
||||
}
|
||||
|
||||
|
@ -209,8 +196,6 @@ public class RemoteProcessGroupStatus implements Cloneable {
|
|||
builder.append(activeRemotePortCount);
|
||||
builder.append(", inactiveRemotePortCount=");
|
||||
builder.append(inactiveRemotePortCount);
|
||||
builder.append(", authenticationIssues=");
|
||||
builder.append(authorizationIssues);
|
||||
builder.append("]");
|
||||
return builder.toString();
|
||||
}
|
||||
|
|
|
@ -25,6 +25,8 @@ import java.util.List;
|
|||
*/
|
||||
public interface BulletinRepository {
|
||||
|
||||
public static final int MAX_BULLETINS_PER_COMPONENT = 5;
|
||||
|
||||
/**
|
||||
* Adds a Bulletin to the repository.
|
||||
*
|
||||
|
@ -50,6 +52,14 @@ public interface BulletinRepository {
|
|||
*/
|
||||
List<Bulletin> findBulletins(BulletinQuery bulletinQuery);
|
||||
|
||||
/**
|
||||
* Finds all bulletins for the specified source component.
|
||||
*
|
||||
* @param sourceId id of the source
|
||||
* @return bulletins for the given source
|
||||
*/
|
||||
List<Bulletin> findBulletinsForSource(String sourceId);
|
||||
|
||||
/**
|
||||
* Finds all bulletins for the specified group.
|
||||
*
|
||||
|
|
|
@ -16,12 +16,12 @@
|
|||
*/
|
||||
package org.apache.nifi.util;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.nifi.reporting.Bulletin;
|
||||
import org.apache.nifi.reporting.BulletinQuery;
|
||||
import org.apache.nifi.reporting.BulletinRepository;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class MockBulletinRepository implements BulletinRepository {
|
||||
|
||||
@Override
|
||||
|
@ -48,6 +48,12 @@ public class MockBulletinRepository implements BulletinRepository {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Bulletin> findBulletinsForSource(String sourceId) {
|
||||
// TODO: Implement
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Bulletin> findBulletinsForGroupBySource(String groupId) {
|
||||
// TODO: Implement
|
||||
|
|
|
@ -17,11 +17,12 @@
|
|||
package org.apache.nifi.web.api.dto;
|
||||
|
||||
import com.wordnik.swagger.annotations.ApiModelProperty;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import org.apache.nifi.web.api.dto.util.DateTimeAdapter;
|
||||
|
||||
import javax.xml.bind.annotation.XmlType;
|
||||
import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
|
||||
import org.apache.nifi.web.api.dto.util.DateTimeAdapter;
|
||||
import java.util.Collection;
|
||||
import java.util.Date;
|
||||
|
||||
/**
|
||||
* Details of a remote process group in this NiFi.
|
||||
|
@ -42,7 +43,7 @@ public class RemoteProcessGroupDTO extends ComponentDTO {
|
|||
private String proxyUser;
|
||||
private String proxyPassword;
|
||||
|
||||
private List<String> authorizationIssues;
|
||||
private Collection<String> authorizationIssues;
|
||||
private Boolean transmitting;
|
||||
|
||||
private Integer inputPortCount;
|
||||
|
@ -116,11 +117,11 @@ public class RemoteProcessGroupDTO extends ComponentDTO {
|
|||
@ApiModelProperty(
|
||||
value = "Any remote authorization issues for the remote process group."
|
||||
)
|
||||
public List<String> getAuthorizationIssues() {
|
||||
public Collection<String> getAuthorizationIssues() {
|
||||
return authorizationIssues;
|
||||
}
|
||||
|
||||
public void setAuthorizationIssues(List<String> authorizationIssues) {
|
||||
public void setAuthorizationIssues(Collection<String> authorizationIssues) {
|
||||
this.authorizationIssues = authorizationIssues;
|
||||
}
|
||||
|
||||
|
|
|
@ -18,10 +18,12 @@ package org.apache.nifi.web.api.entity;
|
|||
|
||||
import com.wordnik.swagger.annotations.ApiModelProperty;
|
||||
import org.apache.nifi.web.api.dto.AccessPolicyDTO;
|
||||
import org.apache.nifi.web.api.dto.BulletinDTO;
|
||||
import org.apache.nifi.web.api.dto.PositionDTO;
|
||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A base type for request/response entities.
|
||||
|
@ -33,6 +35,7 @@ public class ComponentEntity extends Entity {
|
|||
private String id;
|
||||
private PositionDTO position;
|
||||
private AccessPolicyDTO accessPolicy;
|
||||
private List<BulletinDTO> bulletins;
|
||||
|
||||
/**
|
||||
* @return revision for this request/response
|
||||
|
@ -96,6 +99,22 @@ public class ComponentEntity extends Entity {
|
|||
this.accessPolicy = accessPolicy;
|
||||
}
|
||||
|
||||
/**
|
||||
* The bulletins for this component.
|
||||
*
|
||||
* @return The bulletins
|
||||
*/
|
||||
@ApiModelProperty(
|
||||
value = "The bulletins for this component."
|
||||
)
|
||||
public List<BulletinDTO> getBulletins() {
|
||||
return bulletins;
|
||||
}
|
||||
|
||||
public void setBulletins(List<BulletinDTO> bulletins) {
|
||||
this.bulletins = bulletins;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return id.hashCode();
|
||||
|
|
|
@ -17,18 +17,11 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import javax.ws.rs.core.StreamingOutput;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.BulletinBoardEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ComponentStateEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionStatusEndpiontMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ConnectionsEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServiceEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServiceReferenceEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ControllerServicesEndpointMerger;
|
||||
|
@ -38,9 +31,13 @@ import org.apache.nifi.cluster.coordination.http.endpoints.DropRequestEndpiontMe
|
|||
import org.apache.nifi.cluster.coordination.http.endpoints.FlowMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.FlowSnippetEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.GroupStatusEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.InputPortsEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ListFlowFilesEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.OutputPortsEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.PortEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.PortStatusEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ProcessGroupEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ProcessGroupsEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorStatusEndpointMerger;
|
||||
import org.apache.nifi.cluster.coordination.http.endpoints.ProcessorsEndpointMerger;
|
||||
|
@ -59,6 +56,14 @@ import org.apache.nifi.stream.io.NullOutputStream;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.ws.rs.core.StreamingOutput;
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class StandardHttpResponseMerger implements HttpResponseMerger {
|
||||
private Logger logger = LoggerFactory.getLogger(StandardHttpResponseMerger.class);
|
||||
|
||||
|
@ -72,9 +77,15 @@ public class StandardHttpResponseMerger implements HttpResponseMerger {
|
|||
endpointMergers.add(new RemoteProcessGroupStatusEndpointMerger());
|
||||
endpointMergers.add(new ProcessorEndpointMerger());
|
||||
endpointMergers.add(new ProcessorsEndpointMerger());
|
||||
endpointMergers.add(new ConnectionEndpointMerger());
|
||||
endpointMergers.add(new ConnectionsEndpointMerger());
|
||||
endpointMergers.add(new PortEndpointMerger());
|
||||
endpointMergers.add(new InputPortsEndpointMerger());
|
||||
endpointMergers.add(new OutputPortsEndpointMerger());
|
||||
endpointMergers.add(new RemoteProcessGroupEndpointMerger());
|
||||
endpointMergers.add(new RemoteProcessGroupsEndpointMerger());
|
||||
endpointMergers.add(new ProcessGroupEndpointMerger());
|
||||
endpointMergers.add(new ProcessGroupsEndpointMerger());
|
||||
endpointMergers.add(new FlowSnippetEndpointMerger());
|
||||
endpointMergers.add(new ProvenanceQueryEndpointMerger());
|
||||
endpointMergers.add(new ProvenanceEventEndpointMerger());
|
||||
|
|
|
@ -17,18 +17,16 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.Entity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public abstract class AbstractSingleDTOEndpoint<EntityType extends Entity, DtoType> implements EndpointResponseMerger {
|
||||
|
||||
@Override
|
||||
|
@ -51,44 +49,6 @@ public abstract class AbstractSingleDTOEndpoint<EntityType extends Entity, DtoTy
|
|||
return new NodeResponse(clientResponse, responseEntity);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Normalizes the validation errors by prepending the corresponding nodes when the error does not exist across all nodes.
|
||||
*
|
||||
* @param validationErrorMap map
|
||||
* @param totalNodes total
|
||||
* @return normalized errors
|
||||
*/
|
||||
protected Set<String> normalizedMergedValidationErrors(final Map<String, Set<NodeIdentifier>> validationErrorMap, int totalNodes) {
|
||||
final Set<String> normalizedValidationErrors = new HashSet<>();
|
||||
for (final Map.Entry<String, Set<NodeIdentifier>> validationEntry : validationErrorMap.entrySet()) {
|
||||
final String msg = validationEntry.getKey();
|
||||
final Set<NodeIdentifier> nodeIds = validationEntry.getValue();
|
||||
|
||||
if (nodeIds.size() == totalNodes) {
|
||||
normalizedValidationErrors.add(msg);
|
||||
} else {
|
||||
nodeIds.forEach(id -> normalizedValidationErrors.add(id.getApiAddress() + ":" + id.getApiPort() + " -- " + msg));
|
||||
}
|
||||
}
|
||||
return normalizedValidationErrors;
|
||||
}
|
||||
|
||||
/**
|
||||
* Merges the validation errors into the specified map, recording the corresponding node identifier.
|
||||
*
|
||||
* @param validationErrorMap map
|
||||
* @param nodeId id
|
||||
* @param nodeValidationErrors errors
|
||||
*/
|
||||
protected void mergeValidationErrors(final Map<String, Set<NodeIdentifier>> validationErrorMap, final NodeIdentifier nodeId, final Collection<String> nodeValidationErrors) {
|
||||
if (nodeValidationErrors != null) {
|
||||
nodeValidationErrors.stream().forEach(
|
||||
err -> validationErrorMap.computeIfAbsent(err, k -> new HashSet<NodeIdentifier>())
|
||||
.add(nodeId));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the class that represents the type of Entity that is expected by this response mapper
|
||||
*/
|
||||
|
|
|
@ -17,18 +17,16 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.Entity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public abstract class AbstractSingleEntityEndpoint<EntityType extends Entity> implements EndpointResponseMerger {
|
||||
@Override
|
||||
public final NodeResponse merge(final URI uri, final String method, final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses, final NodeResponse clientResponse) {
|
||||
|
@ -48,43 +46,6 @@ public abstract class AbstractSingleEntityEndpoint<EntityType extends Entity> im
|
|||
return new NodeResponse(clientResponse, responseEntity);
|
||||
}
|
||||
|
||||
/**
|
||||
* Merges the validation errors into the specified map, recording the corresponding node identifier.
|
||||
*
|
||||
* @param validationErrorMap map
|
||||
* @param nodeId id
|
||||
* @param nodeValidationErrors errors
|
||||
*/
|
||||
protected void mergeValidationErrors(final Map<String, Set<NodeIdentifier>> validationErrorMap, final NodeIdentifier nodeId, final Collection<String> nodeValidationErrors) {
|
||||
if (nodeValidationErrors != null) {
|
||||
nodeValidationErrors.stream().forEach(
|
||||
err -> validationErrorMap.computeIfAbsent(err, k -> new HashSet<NodeIdentifier>())
|
||||
.add(nodeId));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Normalizes the validation errors by prepending the corresponding nodes when the error does not exist across all nodes.
|
||||
*
|
||||
* @param validationErrorMap map
|
||||
* @param totalNodes total
|
||||
* @return normalized errors
|
||||
*/
|
||||
protected Set<String> normalizedMergedValidationErrors(final Map<String, Set<NodeIdentifier>> validationErrorMap, int totalNodes) {
|
||||
final Set<String> normalizedValidationErrors = new HashSet<>();
|
||||
for (final Map.Entry<String, Set<NodeIdentifier>> validationEntry : validationErrorMap.entrySet()) {
|
||||
final String msg = validationEntry.getKey();
|
||||
final Set<NodeIdentifier> nodeIds = validationEntry.getValue();
|
||||
|
||||
if (nodeIds.size() == totalNodes) {
|
||||
normalizedValidationErrors.add(msg);
|
||||
} else {
|
||||
nodeIds.forEach(id -> normalizedValidationErrors.add(id.getApiAddress() + ":" + id.getApiPort() + " -- " + msg));
|
||||
}
|
||||
}
|
||||
return normalizedValidationErrors;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the class that represents the type of Entity that is expected by this response mapper
|
||||
*/
|
||||
|
|
|
@ -17,21 +17,21 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.cluster.manager.BulletinMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.BulletinBoardDTO;
|
||||
import org.apache.nifi.web.api.dto.BulletinDTO;
|
||||
import org.apache.nifi.web.api.entity.BulletinBoardEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class BulletinBoardEndpointMerger extends AbstractSingleDTOEndpoint<BulletinBoardEntity, BulletinBoardDTO> {
|
||||
public static final Pattern BULLETIN_BOARD_URI_PATTERN = Pattern.compile("/nifi-api/flow/bulletin-board");
|
||||
|
||||
|
@ -52,31 +52,16 @@ public class BulletinBoardEndpointMerger extends AbstractSingleDTOEndpoint<Bulle
|
|||
|
||||
@Override
|
||||
protected void mergeResponses(BulletinBoardDTO clientDto, Map<NodeIdentifier, BulletinBoardDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
|
||||
final List<BulletinDTO> bulletinDtos = new ArrayList<>();
|
||||
final Map<NodeIdentifier, List<BulletinDTO>> bulletinDtos = new HashMap<>();
|
||||
for (final Map.Entry<NodeIdentifier, BulletinBoardDTO> entry : dtoMap.entrySet()) {
|
||||
final NodeIdentifier nodeId = entry.getKey();
|
||||
final NodeIdentifier nodeIdentifier = entry.getKey();
|
||||
final BulletinBoardDTO boardDto = entry.getValue();
|
||||
final String nodeAddress = nodeId.getApiAddress() + ":" + nodeId.getApiPort();
|
||||
|
||||
for (final BulletinDTO bulletin : boardDto.getBulletins()) {
|
||||
bulletin.setNodeAddress(nodeAddress);
|
||||
bulletinDtos.add(bulletin);
|
||||
}
|
||||
boardDto.getBulletins().forEach(bulletin -> {
|
||||
bulletinDtos.computeIfAbsent(nodeIdentifier, nodeId -> new ArrayList<>()).add(bulletin);
|
||||
});
|
||||
}
|
||||
|
||||
Collections.sort(bulletinDtos, new Comparator<BulletinDTO>() {
|
||||
@Override
|
||||
public int compare(final BulletinDTO o1, final BulletinDTO o2) {
|
||||
final int timeComparison = o1.getTimestamp().compareTo(o2.getTimestamp());
|
||||
if (timeComparison != 0) {
|
||||
return timeComparison;
|
||||
}
|
||||
|
||||
return o1.getNodeAddress().compareTo(o2.getNodeAddress());
|
||||
}
|
||||
});
|
||||
|
||||
clientDto.setBulletins(bulletinDtos);
|
||||
clientDto.setBulletins(BulletinMerger.mergeBulletins(bulletinDtos));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,58 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.ConnectionEntityMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ConnectionEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class ConnectionEndpointMerger extends AbstractSingleEntityEndpoint<ConnectionEntity> implements EndpointResponseMerger {
|
||||
public static final Pattern PROCESSORS_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/connections");
|
||||
public static final Pattern PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/connections/[a-f0-9\\-]{36}");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(final URI uri, final String method) {
|
||||
if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && (PROCESSOR_URI_PATTERN.matcher(uri.getPath()).matches())) {
|
||||
return true;
|
||||
} else if ("POST".equalsIgnoreCase(method) && PROCESSORS_URI_PATTERN.matcher(uri.getPath()).matches()) {
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Class<ConnectionEntity> getEntityClass() {
|
||||
return ConnectionEntity.class;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected void mergeResponses(final ConnectionEntity clientEntity, final Map<NodeIdentifier, ConnectionEntity> entityMap, final Set<NodeResponse> successfulResponses,
|
||||
final Set<NodeResponse> problematicResponses) {
|
||||
|
||||
ConnectionEntityMerger.mergeConnections(clientEntity, entityMap);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.ConnectionsEntityMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ConnectionEntity;
|
||||
import org.apache.nifi.web.api.entity.ConnectionsEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class ConnectionsEndpointMerger implements EndpointResponseMerger {
|
||||
public static final Pattern CONNECTIONS_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/connections");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(final URI uri, final String method) {
|
||||
return "GET".equalsIgnoreCase(method) && CONNECTIONS_URI_PATTERN.matcher(uri.getPath()).matches();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final NodeResponse merge(final URI uri, final String method, final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses, final NodeResponse clientResponse) {
|
||||
if (!canHandle(uri, method)) {
|
||||
throw new IllegalArgumentException("Cannot use Endpoint Mapper of type " + getClass().getSimpleName() + " to map responses for URI " + uri + ", HTTP Method " + method);
|
||||
}
|
||||
|
||||
final ConnectionsEntity responseEntity = clientResponse.getClientResponse().getEntity(ConnectionsEntity.class);
|
||||
final Set<ConnectionEntity> connectionEntities = responseEntity.getConnections();
|
||||
|
||||
final Map<String, Map<NodeIdentifier, ConnectionEntity>> entityMap = new HashMap<>();
|
||||
for (final NodeResponse nodeResponse : successfulResponses) {
|
||||
final ConnectionsEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(ConnectionsEntity.class);
|
||||
final Set<ConnectionEntity> nodeConnectionEntities = nodeResponseEntity.getConnections();
|
||||
|
||||
for (final ConnectionEntity nodeConnectionEntity : nodeConnectionEntities) {
|
||||
final NodeIdentifier nodeId = nodeResponse.getNodeId();
|
||||
Map<NodeIdentifier, ConnectionEntity> innerMap = entityMap.get(nodeId);
|
||||
if (innerMap == null) {
|
||||
innerMap = new HashMap<>();
|
||||
entityMap.put(nodeConnectionEntity.getId(), innerMap);
|
||||
}
|
||||
|
||||
innerMap.put(nodeResponse.getNodeId(), nodeConnectionEntity);
|
||||
}
|
||||
}
|
||||
|
||||
ConnectionsEntityMerger.mergeConnections(connectionEntities, entityMap);
|
||||
|
||||
// create a new client response
|
||||
return new NodeResponse(clientResponse, responseEntity);
|
||||
}
|
||||
}
|
|
@ -17,29 +17,27 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.ControllerServiceEntityMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.controller.service.ControllerServiceState;
|
||||
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
|
||||
import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class ControllerServiceEndpointMerger extends AbstractSingleDTOEndpoint<ControllerServiceEntity, ControllerServiceDTO> {
|
||||
public static final String CONTROLLER_SERVICES_URI = "/nifi-api/controller-services/node";
|
||||
public static final Pattern CONTROLLER_SERVICE_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/node/[a-f0-9\\-]{36}");
|
||||
public class ControllerServiceEndpointMerger extends AbstractSingleEntityEndpoint<ControllerServiceEntity> implements EndpointResponseMerger {
|
||||
public static final String CONTROLLER_CONTROLLER_SERVICES_URI = "/nifi-api/controller/controller-services";
|
||||
public static final Pattern PROCESS_GROUPS_CONTROLLER_SERVICES_URI = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/controller-services");
|
||||
public static final Pattern CONTROLLER_SERVICE_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/[a-f0-9\\-]{36}");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(URI uri, 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_CONTROLLER_SERVICES_URI.equals(uri.getPath()) || PROCESS_GROUPS_CONTROLLER_SERVICES_URI.matcher(uri.getPath()).matches())) {
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -52,95 +50,9 @@ public class ControllerServiceEndpointMerger extends AbstractSingleDTOEndpoint<C
|
|||
}
|
||||
|
||||
@Override
|
||||
protected ControllerServiceDTO getDto(ControllerServiceEntity entity) {
|
||||
return entity.getComponent();
|
||||
protected void mergeResponses(ControllerServiceEntity clientEntity, Map<NodeIdentifier, ControllerServiceEntity> entityMap,
|
||||
Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
|
||||
|
||||
ControllerServiceEntityMerger.mergeControllerServices(clientEntity, entityMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void mergeResponses(ControllerServiceDTO clientDto, Map<NodeIdentifier, ControllerServiceDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
|
||||
final Map<String, Set<NodeIdentifier>> validationErrorMap = new HashMap<>();
|
||||
final Set<ControllerServiceReferencingComponentEntity> referencingComponents = clientDto.getReferencingComponents();
|
||||
final Map<NodeIdentifier, Set<ControllerServiceReferencingComponentEntity>> nodeReferencingComponentsMap = new HashMap<>();
|
||||
|
||||
String state = null;
|
||||
for (final Map.Entry<NodeIdentifier, ControllerServiceDTO> nodeEntry : dtoMap.entrySet()) {
|
||||
final NodeIdentifier nodeId = nodeEntry.getKey();
|
||||
final ControllerServiceDTO nodeControllerService = nodeEntry.getValue();
|
||||
|
||||
if (state == null) {
|
||||
if (ControllerServiceState.DISABLING.name().equals(nodeControllerService.getState())) {
|
||||
state = ControllerServiceState.DISABLING.name();
|
||||
} else if (ControllerServiceState.ENABLING.name().equals(nodeControllerService.getState())) {
|
||||
state = ControllerServiceState.ENABLING.name();
|
||||
}
|
||||
}
|
||||
|
||||
nodeReferencingComponentsMap.put(nodeId, nodeControllerService.getReferencingComponents());
|
||||
|
||||
// merge the validation errors
|
||||
mergeValidationErrors(validationErrorMap, nodeId, nodeControllerService.getValidationErrors());
|
||||
}
|
||||
|
||||
// merge the referencing components
|
||||
mergeControllerServiceReferences(referencingComponents, nodeReferencingComponentsMap);
|
||||
|
||||
// store the 'transition' state is applicable
|
||||
if (state != null) {
|
||||
clientDto.setState(state);
|
||||
}
|
||||
|
||||
// set the merged the validation errors
|
||||
clientDto.setValidationErrors(normalizedMergedValidationErrors(validationErrorMap, dtoMap.size()));
|
||||
}
|
||||
|
||||
public static void mergeControllerServiceReferences(Set<ControllerServiceReferencingComponentEntity> referencingComponents,
|
||||
Map<NodeIdentifier, Set<ControllerServiceReferencingComponentEntity>> referencingComponentMap) {
|
||||
|
||||
final Map<String, Integer> activeThreadCounts = new HashMap<>();
|
||||
final Map<String, String> states = new HashMap<>();
|
||||
for (final Map.Entry<NodeIdentifier, Set<ControllerServiceReferencingComponentEntity>> nodeEntry : referencingComponentMap.entrySet()) {
|
||||
final Set<ControllerServiceReferencingComponentEntity> nodeReferencingComponents = nodeEntry.getValue();
|
||||
|
||||
// go through all the nodes referencing components
|
||||
if (nodeReferencingComponents != null) {
|
||||
for (final ControllerServiceReferencingComponentEntity nodeReferencingComponentEntity : nodeReferencingComponents) {
|
||||
final ControllerServiceReferencingComponentDTO nodeReferencingComponent = nodeReferencingComponentEntity.getComponent();
|
||||
|
||||
// handle active thread counts
|
||||
if (nodeReferencingComponent.getActiveThreadCount() != null && nodeReferencingComponent.getActiveThreadCount() > 0) {
|
||||
final Integer current = activeThreadCounts.get(nodeReferencingComponent.getId());
|
||||
if (current == null) {
|
||||
activeThreadCounts.put(nodeReferencingComponent.getId(), nodeReferencingComponent.getActiveThreadCount());
|
||||
} else {
|
||||
activeThreadCounts.put(nodeReferencingComponent.getId(), nodeReferencingComponent.getActiveThreadCount() + current);
|
||||
}
|
||||
}
|
||||
|
||||
// handle controller service state
|
||||
final String state = states.get(nodeReferencingComponent.getId());
|
||||
if (state == null) {
|
||||
if (ControllerServiceState.DISABLING.name().equals(nodeReferencingComponent.getState())) {
|
||||
states.put(nodeReferencingComponent.getId(), ControllerServiceState.DISABLING.name());
|
||||
} else if (ControllerServiceState.ENABLING.name().equals(nodeReferencingComponent.getState())) {
|
||||
states.put(nodeReferencingComponent.getId(), ControllerServiceState.ENABLING.name());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// go through each referencing components
|
||||
for (final ControllerServiceReferencingComponentEntity referencingComponent : referencingComponents) {
|
||||
final Integer activeThreadCount = activeThreadCounts.get(referencingComponent.getId());
|
||||
if (activeThreadCount != null) {
|
||||
referencingComponent.getComponent().setActiveThreadCount(activeThreadCount);
|
||||
}
|
||||
|
||||
final String state = states.get(referencingComponent.getId());
|
||||
if (state != null) {
|
||||
referencingComponent.getComponent().setState(state);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -17,18 +17,19 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.ControllerServiceEntityMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentsEntity;
|
||||
|
||||
public class ControllerServiceReferenceEndpointMerger implements EndpointResponseMerger {
|
||||
public static final Pattern CONTROLLER_SERVICE_REFERENCES_URI_PATTERN = Pattern.compile("/nifi-api/controller-services/node/[a-f0-9\\-]{36}/references");
|
||||
|
||||
|
@ -59,7 +60,7 @@ public class ControllerServiceReferenceEndpointMerger implements EndpointRespons
|
|||
resultsMap.put(nodeResponse.getNodeId(), nodeReferencingComponents);
|
||||
}
|
||||
|
||||
ControllerServiceEndpointMerger.mergeControllerServiceReferences(referencingComponents, resultsMap);
|
||||
ControllerServiceEntityMerger.mergeControllerServiceReferences(referencingComponents, resultsMap);
|
||||
|
||||
return new NodeResponse(clientResponse, responseEntity);
|
||||
}
|
||||
|
|
|
@ -17,47 +17,57 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.ControllerServicesEntityMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServicesEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class ControllerServicesEndpointMerger extends AbstractMultiEntityEndpoint<ControllerServicesEntity, ControllerServiceEntity> {
|
||||
public static final String CONTROLLER_SERVICES_URI = "/nifi-api/controller-services/node";
|
||||
public class ControllerServicesEndpointMerger implements EndpointResponseMerger {
|
||||
public static final String CONTROLLER_SERVICES_URI = "/nifi-api/flow/controller/controller-services";
|
||||
public static final Pattern PROCESS_GROUPS_CONTROLLER_SERVICES_URI = Pattern.compile("/nifi-api/flow/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/controller-services");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(URI uri, String method) {
|
||||
return "GET".equalsIgnoreCase(method) && CONTROLLER_SERVICES_URI.equals(uri.getPath());
|
||||
return "GET".equalsIgnoreCase(method) && (CONTROLLER_SERVICES_URI.equals(uri.getPath()) || PROCESS_GROUPS_CONTROLLER_SERVICES_URI.matcher(uri.getPath()).matches());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Class<ControllerServicesEntity> getEntityClass() {
|
||||
return ControllerServicesEntity.class;
|
||||
}
|
||||
public final NodeResponse merge(final URI uri, final String method, final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses, final NodeResponse clientResponse) {
|
||||
if (!canHandle(uri, method)) {
|
||||
throw new IllegalArgumentException("Cannot use Endpoint Mapper of type " + getClass().getSimpleName() + " to map responses for URI " + uri + ", HTTP Method " + method);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Set<ControllerServiceEntity> getDtos(ControllerServicesEntity entity) {
|
||||
return entity.getControllerServices();
|
||||
}
|
||||
final ControllerServicesEntity responseEntity = clientResponse.getClientResponse().getEntity(ControllerServicesEntity.class);
|
||||
final Set<ControllerServiceEntity> controllerServiceEntities = responseEntity.getControllerServices();
|
||||
|
||||
@Override
|
||||
protected String getComponentId(ControllerServiceEntity entity) {
|
||||
return entity.getComponent().getId();
|
||||
}
|
||||
final Map<String, Map<NodeIdentifier, ControllerServiceEntity>> entityMap = new HashMap<>();
|
||||
for (final NodeResponse nodeResponse : successfulResponses) {
|
||||
final ControllerServicesEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(ControllerServicesEntity.class);
|
||||
final Set<ControllerServiceEntity> nodeControllerServiceEntities = nodeResponseEntity.getControllerServices();
|
||||
|
||||
@Override
|
||||
protected void mergeResponses(ControllerServiceEntity entity, Map<NodeIdentifier, ControllerServiceEntity> entityMap,
|
||||
Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
|
||||
for (final ControllerServiceEntity nodeControllerServiceEntity : nodeControllerServiceEntities) {
|
||||
final NodeIdentifier nodeId = nodeResponse.getNodeId();
|
||||
Map<NodeIdentifier, ControllerServiceEntity> innerMap = entityMap.get(nodeId);
|
||||
if (innerMap == null) {
|
||||
innerMap = new HashMap<>();
|
||||
entityMap.put(nodeControllerServiceEntity.getId(), innerMap);
|
||||
}
|
||||
|
||||
new ControllerServiceEndpointMerger().mergeResponses(
|
||||
entity.getComponent(),
|
||||
entityMap.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().getComponent())),
|
||||
successfulResponses,
|
||||
problematicResponses);
|
||||
innerMap.put(nodeResponse.getNodeId(), nodeControllerServiceEntity);
|
||||
}
|
||||
}
|
||||
|
||||
ControllerServicesEntityMerger.mergeControllerServices(controllerServiceEntities, entityMap);
|
||||
|
||||
// create a new client response
|
||||
return new NodeResponse(clientResponse, responseEntity);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,16 +17,15 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.manager.ConnectionsEntityMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.StatusMerger;
|
||||
import org.apache.nifi.cluster.manager.PortsEntityMerger;
|
||||
import org.apache.nifi.cluster.manager.ProcessGroupsEntityMerger;
|
||||
import org.apache.nifi.cluster.manager.ProcessorsEntityMerger;
|
||||
import org.apache.nifi.cluster.manager.RemoteProcessGroupsEntityMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.flow.FlowDTO;
|
||||
import org.apache.nifi.web.api.dto.flow.ProcessGroupFlowDTO;
|
||||
import org.apache.nifi.web.api.dto.status.ConnectionStatusDTO;
|
||||
import org.apache.nifi.web.api.dto.status.PortStatusDTO;
|
||||
import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO;
|
||||
import org.apache.nifi.web.api.dto.status.ProcessorStatusDTO;
|
||||
import org.apache.nifi.web.api.dto.status.RemoteProcessGroupStatusDTO;
|
||||
import org.apache.nifi.web.api.entity.ConnectionEntity;
|
||||
import org.apache.nifi.web.api.entity.FunnelEntity;
|
||||
import org.apache.nifi.web.api.entity.LabelEntity;
|
||||
|
@ -68,23 +67,31 @@ public class FlowMerger extends AbstractSingleDTOEndpoint<ProcessGroupFlowEntity
|
|||
final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses) {
|
||||
|
||||
final FlowDTO flowDto = clientDto.getFlow();
|
||||
final Set<ConnectionEntity> clientConnections = flowDto.getConnections();
|
||||
final Set<ProcessorEntity> clientProcessors = flowDto.getProcessors();
|
||||
final Set<PortEntity> clientInputPorts = flowDto.getInputPorts();
|
||||
final Set<PortEntity> clientOutputPorts = flowDto.getOutputPorts();
|
||||
final Set<RemoteProcessGroupEntity> clientRemoteProcessGroups = flowDto.getRemoteProcessGroups();
|
||||
final Set<ProcessGroupEntity> clientProcessGroups = flowDto.getProcessGroups();
|
||||
|
||||
final Map<String, List<ConnectionEntity>> connections = new HashMap<>();
|
||||
final Map<String, Map<NodeIdentifier, ConnectionEntity>> connections = new HashMap<>();
|
||||
final Map<String, List<FunnelEntity>> funnels = new HashMap<>();
|
||||
final Map<String, List<PortEntity>> inputPorts = new HashMap<>();
|
||||
final Map<String, Map<NodeIdentifier, PortEntity>> inputPorts = new HashMap<>();
|
||||
final Map<String, List<LabelEntity>> labels = new HashMap<>();
|
||||
final Map<String, List<PortEntity>> outputPorts = new HashMap<>();
|
||||
final Map<String, List<ProcessorEntity>> processors = new HashMap<>();
|
||||
final Map<String, List<RemoteProcessGroupEntity>> rpgs = new HashMap<>();
|
||||
final Map<String, List<ProcessGroupEntity>> processGroups = new HashMap<>();
|
||||
final Map<String, Map<NodeIdentifier, PortEntity>> outputPorts = new HashMap<>();
|
||||
final Map<String, Map<NodeIdentifier, ProcessorEntity>> processors = new HashMap<>();
|
||||
final Map<String, Map<NodeIdentifier, RemoteProcessGroupEntity>> rpgs = new HashMap<>();
|
||||
final Map<String, Map<NodeIdentifier, ProcessGroupEntity>> processGroups = new HashMap<>();
|
||||
|
||||
// Create mapping of ComponentID -> all components with that ID (one per node)
|
||||
for (final ProcessGroupFlowDTO nodeGroupFlowDto : dtoMap.values()) {
|
||||
// Create mapping of ComponentID -> [nodeId, entity on that node]
|
||||
for (final Map.Entry<NodeIdentifier, ProcessGroupFlowDTO> nodeGroupFlowEntry : dtoMap.entrySet()) {
|
||||
final NodeIdentifier nodeIdentifier = nodeGroupFlowEntry.getKey();
|
||||
final ProcessGroupFlowDTO nodeGroupFlowDto = nodeGroupFlowEntry.getValue();
|
||||
final FlowDTO nodeFlowDto = nodeGroupFlowDto.getFlow();
|
||||
|
||||
// Merge connection statuses
|
||||
for (final ConnectionEntity entity : nodeFlowDto.getConnections()) {
|
||||
connections.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
|
||||
connections.computeIfAbsent(entity.getId(), id -> new HashMap<>()).computeIfAbsent(nodeIdentifier, nodeId -> entity);
|
||||
}
|
||||
|
||||
for (final FunnelEntity entity : nodeFlowDto.getFunnels()) {
|
||||
|
@ -92,11 +99,11 @@ public class FlowMerger extends AbstractSingleDTOEndpoint<ProcessGroupFlowEntity
|
|||
}
|
||||
|
||||
for (final PortEntity entity : nodeFlowDto.getInputPorts()) {
|
||||
inputPorts.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
|
||||
inputPorts.computeIfAbsent(entity.getId(), id -> new HashMap<>()).computeIfAbsent(nodeIdentifier, nodeId -> entity);
|
||||
}
|
||||
|
||||
for (final PortEntity entity : nodeFlowDto.getOutputPorts()) {
|
||||
outputPorts.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
|
||||
outputPorts.computeIfAbsent(entity.getId(), id -> new HashMap<>()).computeIfAbsent(nodeIdentifier, nodeId -> entity);
|
||||
}
|
||||
|
||||
for (final LabelEntity entity : nodeFlowDto.getLabels()) {
|
||||
|
@ -104,15 +111,15 @@ public class FlowMerger extends AbstractSingleDTOEndpoint<ProcessGroupFlowEntity
|
|||
}
|
||||
|
||||
for (final ProcessorEntity entity : nodeFlowDto.getProcessors()) {
|
||||
processors.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
|
||||
processors.computeIfAbsent(entity.getId(), id -> new HashMap<>()).computeIfAbsent(nodeIdentifier, nodeId -> entity);
|
||||
}
|
||||
|
||||
for (final RemoteProcessGroupEntity entity : nodeFlowDto.getRemoteProcessGroups()) {
|
||||
rpgs.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
|
||||
rpgs.computeIfAbsent(entity.getId(), id -> new HashMap<>()).computeIfAbsent(nodeIdentifier, nodeId -> entity);
|
||||
}
|
||||
|
||||
for (final ProcessGroupEntity entity : nodeFlowDto.getProcessGroups()) {
|
||||
processGroups.computeIfAbsent(entity.getId(), id -> new ArrayList<>()).add(entity);
|
||||
processGroups.computeIfAbsent(entity.getId(), id -> new HashMap<>()).computeIfAbsent(nodeIdentifier, nodeId -> entity);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -121,11 +128,7 @@ public class FlowMerger extends AbstractSingleDTOEndpoint<ProcessGroupFlowEntity
|
|||
//
|
||||
|
||||
// Merge connections
|
||||
final Set<ConnectionEntity> mergedConnections = new HashSet<>();
|
||||
for (final List<ConnectionEntity> connectionList : connections.values()) {
|
||||
mergedConnections.add(mergeConnections(connectionList));
|
||||
}
|
||||
flowDto.setConnections(mergedConnections);
|
||||
ConnectionsEntityMerger.mergeConnections(clientConnections, connections);
|
||||
|
||||
// Merge funnel statuses
|
||||
final Set<FunnelEntity> mergedFunnels = new HashSet<>();
|
||||
|
@ -135,18 +138,10 @@ public class FlowMerger extends AbstractSingleDTOEndpoint<ProcessGroupFlowEntity
|
|||
flowDto.setFunnels(mergedFunnels);
|
||||
|
||||
// Merge input ports
|
||||
final Set<PortEntity> mergedInputPorts = new HashSet<>();
|
||||
for (final List<PortEntity> portList : inputPorts.values()) {
|
||||
mergedInputPorts.add(mergePorts(portList));
|
||||
}
|
||||
flowDto.setInputPorts(mergedInputPorts);
|
||||
PortsEntityMerger.mergePorts(clientInputPorts, inputPorts);
|
||||
|
||||
// Merge output ports
|
||||
final Set<PortEntity> mergedOutputPorts = new HashSet<>();
|
||||
for (final List<PortEntity> portList : outputPorts.values()) {
|
||||
mergedOutputPorts.add(mergePorts(portList));
|
||||
}
|
||||
flowDto.setOutputPorts(mergedOutputPorts);
|
||||
PortsEntityMerger.mergePorts(clientOutputPorts, outputPorts);
|
||||
|
||||
// Merge labels
|
||||
final Set<LabelEntity> mergedLabels = new HashSet<>();
|
||||
|
@ -155,57 +150,14 @@ public class FlowMerger extends AbstractSingleDTOEndpoint<ProcessGroupFlowEntity
|
|||
}
|
||||
flowDto.setLabels(mergedLabels);
|
||||
|
||||
|
||||
// Merge processors
|
||||
final Set<ProcessorEntity> mergedProcessors = new HashSet<>();
|
||||
for (final List<ProcessorEntity> processorList : processors.values()) {
|
||||
mergedProcessors.add(mergeProcessors(processorList));
|
||||
}
|
||||
flowDto.setProcessors(mergedProcessors);
|
||||
|
||||
ProcessorsEntityMerger.mergeProcessors(clientProcessors, processors);
|
||||
|
||||
// Merge Remote Process Groups
|
||||
final Set<RemoteProcessGroupEntity> mergedRpgs = new HashSet<>();
|
||||
for (final List<RemoteProcessGroupEntity> rpgList : rpgs.values()) {
|
||||
mergedRpgs.add(mergeRemoteProcessGroups(rpgList));
|
||||
}
|
||||
flowDto.setRemoteProcessGroups(mergedRpgs);
|
||||
|
||||
RemoteProcessGroupsEntityMerger.mergeRemoteProcessGroups(clientRemoteProcessGroups, rpgs);
|
||||
|
||||
// Merge Process Groups
|
||||
final Set<ProcessGroupEntity> mergedGroups = new HashSet<>();
|
||||
for (final List<ProcessGroupEntity> groupList : processGroups.values()) {
|
||||
mergedGroups.add(mergeProcessGroups(groupList));
|
||||
}
|
||||
flowDto.setProcessGroups(mergedGroups);
|
||||
}
|
||||
|
||||
private ConnectionEntity mergeConnections(final List<ConnectionEntity> connections) {
|
||||
final ConnectionEntity merged = connections.get(0);
|
||||
final ConnectionStatusDTO statusDto = merged.getStatus();
|
||||
statusDto.setNodeSnapshots(null);
|
||||
|
||||
for (final ConnectionEntity entity : connections) {
|
||||
if (entity != merged) {
|
||||
StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
|
||||
}
|
||||
}
|
||||
|
||||
return merged;
|
||||
}
|
||||
|
||||
private PortEntity mergePorts(final List<PortEntity> ports) {
|
||||
final PortEntity merged = ports.get(0);
|
||||
final PortStatusDTO statusDto = merged.getStatus();
|
||||
statusDto.setNodeSnapshots(null);
|
||||
|
||||
for (final PortEntity entity : ports) {
|
||||
if (entity != merged) {
|
||||
StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
|
||||
}
|
||||
}
|
||||
|
||||
return merged;
|
||||
ProcessGroupsEntityMerger.mergeProcessGroups(clientProcessGroups, processGroups);
|
||||
}
|
||||
|
||||
private FunnelEntity mergeFunnels(final List<FunnelEntity> funnels) {
|
||||
|
@ -215,53 +167,4 @@ public class FlowMerger extends AbstractSingleDTOEndpoint<ProcessGroupFlowEntity
|
|||
private LabelEntity mergeLabels(final List<LabelEntity> labels) {
|
||||
return labels.get(0);
|
||||
}
|
||||
|
||||
private ProcessorEntity mergeProcessors(final List<ProcessorEntity> processors) {
|
||||
final ProcessorEntity merged = processors.get(0);
|
||||
final ProcessorStatusDTO statusDto = merged.getStatus();
|
||||
statusDto.setNodeSnapshots(null);
|
||||
|
||||
for (final ProcessorEntity entity : processors) {
|
||||
if (entity != merged) {
|
||||
StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
|
||||
}
|
||||
}
|
||||
|
||||
return merged;
|
||||
}
|
||||
|
||||
|
||||
private RemoteProcessGroupEntity mergeRemoteProcessGroups(final List<RemoteProcessGroupEntity> rpgs) {
|
||||
final RemoteProcessGroupEntity merged = rpgs.get(0);
|
||||
final RemoteProcessGroupStatusDTO statusDto = merged.getStatus();
|
||||
statusDto.setNodeSnapshots(null);
|
||||
|
||||
for (final RemoteProcessGroupEntity entity : rpgs) {
|
||||
if (entity != merged) {
|
||||
StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
|
||||
}
|
||||
}
|
||||
|
||||
return merged;
|
||||
}
|
||||
|
||||
private ProcessGroupEntity mergeProcessGroups(final List<ProcessGroupEntity> groups) {
|
||||
final ProcessGroupEntity merged = groups.get(0);
|
||||
final ProcessGroupStatusDTO statusDto = merged.getStatus();
|
||||
statusDto.setNodeSnapshots(null);
|
||||
|
||||
for (final ProcessGroupEntity entity : groups) {
|
||||
if (entity != merged) {
|
||||
StatusMerger.merge(merged.getStatus().getAggregateSnapshot(), entity.getStatus().getAggregateSnapshot());
|
||||
}
|
||||
}
|
||||
|
||||
// We merge only the statuses of the Process Groups. The child components are not
|
||||
// necessary for a FlowProcessGroupDTO, so we just ensure that they are null
|
||||
if (merged.getComponent() != null) {
|
||||
merged.getComponent().setContents(null);
|
||||
}
|
||||
|
||||
return merged;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.PortsEntityMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.InputPortsEntity;
|
||||
import org.apache.nifi.web.api.entity.PortEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class InputPortsEndpointMerger implements EndpointResponseMerger {
|
||||
public static final Pattern INPUT_PORTS_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/input-ports");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(final URI uri, final String method) {
|
||||
return "GET".equalsIgnoreCase(method) && INPUT_PORTS_URI_PATTERN.matcher(uri.getPath()).matches();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final NodeResponse merge(final URI uri, final String method, final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses, final NodeResponse clientResponse) {
|
||||
if (!canHandle(uri, method)) {
|
||||
throw new IllegalArgumentException("Cannot use Endpoint Mapper of type " + getClass().getSimpleName() + " to map responses for URI " + uri + ", HTTP Method " + method);
|
||||
}
|
||||
|
||||
final InputPortsEntity responseEntity = clientResponse.getClientResponse().getEntity(InputPortsEntity.class);
|
||||
final Set<PortEntity> portEntities = responseEntity.getInputPorts();
|
||||
|
||||
final Map<String, Map<NodeIdentifier, PortEntity>> entityMap = new HashMap<>();
|
||||
for (final NodeResponse nodeResponse : successfulResponses) {
|
||||
final InputPortsEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(InputPortsEntity.class);
|
||||
final Set<PortEntity> nodePortEntities = nodeResponseEntity.getInputPorts();
|
||||
|
||||
for (final PortEntity nodePortEntity : nodePortEntities) {
|
||||
final NodeIdentifier nodeId = nodeResponse.getNodeId();
|
||||
Map<NodeIdentifier, PortEntity> innerMap = entityMap.get(nodeId);
|
||||
if (innerMap == null) {
|
||||
innerMap = new HashMap<>();
|
||||
entityMap.put(nodePortEntity.getId(), innerMap);
|
||||
}
|
||||
|
||||
innerMap.put(nodeResponse.getNodeId(), nodePortEntity);
|
||||
}
|
||||
}
|
||||
|
||||
PortsEntityMerger.mergePorts(portEntities, entityMap);
|
||||
|
||||
// create a new client response
|
||||
return new NodeResponse(clientResponse, responseEntity);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.PortsEntityMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.OutputPortsEntity;
|
||||
import org.apache.nifi.web.api.entity.PortEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class OutputPortsEndpointMerger implements EndpointResponseMerger {
|
||||
public static final Pattern OUTPUT_PORTS_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/output-ports");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(final URI uri, final String method) {
|
||||
return "GET".equalsIgnoreCase(method) && OUTPUT_PORTS_URI_PATTERN.matcher(uri.getPath()).matches();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final NodeResponse merge(final URI uri, final String method, final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses, final NodeResponse clientResponse) {
|
||||
if (!canHandle(uri, method)) {
|
||||
throw new IllegalArgumentException("Cannot use Endpoint Mapper of type " + getClass().getSimpleName() + " to map responses for URI " + uri + ", HTTP Method " + method);
|
||||
}
|
||||
|
||||
final OutputPortsEntity responseEntity = clientResponse.getClientResponse().getEntity(OutputPortsEntity.class);
|
||||
final Set<PortEntity> portEntities = responseEntity.getOutputPorts();
|
||||
|
||||
final Map<String, Map<NodeIdentifier, PortEntity>> entityMap = new HashMap<>();
|
||||
for (final NodeResponse nodeResponse : successfulResponses) {
|
||||
final OutputPortsEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(OutputPortsEntity.class);
|
||||
final Set<PortEntity> nodePortEntities = nodeResponseEntity.getOutputPorts();
|
||||
|
||||
for (final PortEntity nodePortEntity : nodePortEntities) {
|
||||
final NodeIdentifier nodeId = nodeResponse.getNodeId();
|
||||
Map<NodeIdentifier, PortEntity> innerMap = entityMap.get(nodeId);
|
||||
if (innerMap == null) {
|
||||
innerMap = new HashMap<>();
|
||||
entityMap.put(nodePortEntity.getId(), innerMap);
|
||||
}
|
||||
|
||||
innerMap.put(nodeResponse.getNodeId(), nodePortEntity);
|
||||
}
|
||||
}
|
||||
|
||||
PortsEntityMerger.mergePorts(portEntities, entityMap);
|
||||
|
||||
// create a new client response
|
||||
return new NodeResponse(clientResponse, responseEntity);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,75 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.PortEntityMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.PortEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class PortEndpointMerger extends AbstractSingleEntityEndpoint<PortEntity> implements EndpointResponseMerger {
|
||||
public static final Pattern INPUT_PORTS_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/input-ports");
|
||||
public static final Pattern INPUT_PORT_URI_PATTERN = Pattern.compile("/nifi-api/input-ports/[a-f0-9\\-]{36}");
|
||||
|
||||
public static final Pattern OUTPUT_PORTS_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/output-ports");
|
||||
public static final Pattern OUTPUT_PORT_URI_PATTERN = Pattern.compile("/nifi-api/output-ports/[a-f0-9\\-]{36}");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(final URI uri, final String method) {
|
||||
return canHandleInputPort(uri, method) || canHandleOutputPort(uri, method);
|
||||
}
|
||||
|
||||
private boolean canHandleInputPort(final URI uri, final String method) {
|
||||
if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && (INPUT_PORT_URI_PATTERN.matcher(uri.getPath()).matches())) {
|
||||
return true;
|
||||
} else if ("POST".equalsIgnoreCase(method) && INPUT_PORTS_URI_PATTERN.matcher(uri.getPath()).matches()) {
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean canHandleOutputPort(final URI uri, final String method) {
|
||||
if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && (OUTPUT_PORT_URI_PATTERN.matcher(uri.getPath()).matches())) {
|
||||
return true;
|
||||
} else if ("POST".equalsIgnoreCase(method) && OUTPUT_PORTS_URI_PATTERN.matcher(uri.getPath()).matches()) {
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Class<PortEntity> getEntityClass() {
|
||||
return PortEntity.class;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected void mergeResponses(final PortEntity clientEntity, final Map<NodeIdentifier, PortEntity> entityMap,
|
||||
final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses) {
|
||||
|
||||
PortEntityMerger.mergePorts(clientEntity, entityMap);
|
||||
}
|
||||
}
|
|
@ -17,22 +17,18 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.ProcessGroupEntityMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.FlowSnippetDTO;
|
||||
import org.apache.nifi.web.api.dto.ProcessGroupDTO;
|
||||
import org.apache.nifi.web.api.dto.ProcessorDTO;
|
||||
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
|
||||
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
|
||||
|
||||
public class ProcessGroupEndpointMerger implements EndpointResponseMerger {
|
||||
public class ProcessGroupEndpointMerger extends AbstractSingleEntityEndpoint<ProcessGroupEntity> implements EndpointResponseMerger {
|
||||
public static final Pattern PROCESS_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))");
|
||||
|
||||
@Override
|
||||
|
@ -41,67 +37,12 @@ public class ProcessGroupEndpointMerger implements EndpointResponseMerger {
|
|||
}
|
||||
|
||||
@Override
|
||||
public NodeResponse merge(final URI uri, final String method, final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses, final NodeResponse clientResponse) {
|
||||
if (!canHandle(uri, method)) {
|
||||
throw new IllegalArgumentException("Cannot use Endpoint Mapper of type " + getClass().getSimpleName() + " to map responses for URI " + uri + ", HTTP Method " + method);
|
||||
}
|
||||
|
||||
final ProcessGroupEntity responseEntity = clientResponse.getClientResponse().getEntity(ProcessGroupEntity.class);
|
||||
final ProcessGroupDTO responseDto = responseEntity.getComponent();
|
||||
|
||||
final FlowSnippetDTO contents = responseDto.getContents();
|
||||
if (contents == null) {
|
||||
return new NodeResponse(clientResponse, responseEntity);
|
||||
} else {
|
||||
final Map<String, Map<NodeIdentifier, ProcessorDTO>> processorMap = new HashMap<>();
|
||||
final Map<String, Map<NodeIdentifier, RemoteProcessGroupDTO>> remoteProcessGroupMap = new HashMap<>();
|
||||
|
||||
for (final NodeResponse nodeResponse : successfulResponses) {
|
||||
final ProcessGroupEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(ProcessGroupEntity.class);
|
||||
final ProcessGroupDTO nodeProcessGroup = nodeResponseEntity.getComponent();
|
||||
|
||||
for (final ProcessorDTO nodeProcessor : nodeProcessGroup.getContents().getProcessors()) {
|
||||
Map<NodeIdentifier, ProcessorDTO> innerMap = processorMap.get(nodeProcessor.getId());
|
||||
if (innerMap == null) {
|
||||
innerMap = new HashMap<>();
|
||||
processorMap.put(nodeProcessor.getId(), innerMap);
|
||||
}
|
||||
|
||||
innerMap.put(nodeResponse.getNodeId(), nodeProcessor);
|
||||
}
|
||||
|
||||
for (final RemoteProcessGroupDTO nodeRemoteProcessGroup : nodeProcessGroup.getContents().getRemoteProcessGroups()) {
|
||||
Map<NodeIdentifier, RemoteProcessGroupDTO> innerMap = remoteProcessGroupMap.get(nodeRemoteProcessGroup.getId());
|
||||
if (innerMap == null) {
|
||||
innerMap = new HashMap<>();
|
||||
remoteProcessGroupMap.put(nodeRemoteProcessGroup.getId(), innerMap);
|
||||
}
|
||||
|
||||
innerMap.put(nodeResponse.getNodeId(), nodeRemoteProcessGroup);
|
||||
}
|
||||
}
|
||||
|
||||
final ProcessorEndpointMerger procMerger = new ProcessorEndpointMerger();
|
||||
for (final ProcessorDTO processor : contents.getProcessors()) {
|
||||
final String procId = processor.getId();
|
||||
final Map<NodeIdentifier, ProcessorDTO> mergeMap = processorMap.get(procId);
|
||||
|
||||
procMerger.mergeResponses(processor, mergeMap, successfulResponses, problematicResponses);
|
||||
}
|
||||
|
||||
final RemoteProcessGroupEndpointMerger rpgMerger = new RemoteProcessGroupEndpointMerger();
|
||||
for (final RemoteProcessGroupDTO remoteProcessGroup : contents.getRemoteProcessGroups()) {
|
||||
if (remoteProcessGroup.getContents() != null) {
|
||||
final String remoteProcessGroupId = remoteProcessGroup.getId();
|
||||
final Map<NodeIdentifier, RemoteProcessGroupDTO> mergeMap = remoteProcessGroupMap.get(remoteProcessGroupId);
|
||||
|
||||
rpgMerger.mergeResponses(remoteProcessGroup, mergeMap, successfulResponses, problematicResponses);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// create a new client response
|
||||
return new NodeResponse(clientResponse, responseEntity);
|
||||
protected Class<ProcessGroupEntity> getEntityClass() {
|
||||
return ProcessGroupEntity.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void mergeResponses(ProcessGroupEntity clientEntity, Map<NodeIdentifier, ProcessGroupEntity> entityMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
|
||||
ProcessGroupEntityMerger.mergeProcessGroups(clientEntity, entityMap);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.ProcessGroupsEntityMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
|
||||
import org.apache.nifi.web.api.entity.ProcessGroupsEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class ProcessGroupsEndpointMerger implements EndpointResponseMerger {
|
||||
public static final Pattern PROCESS_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/process-groups");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(final URI uri, final String method) {
|
||||
return "GET".equalsIgnoreCase(method) && PROCESS_GROUPS_URI_PATTERN.matcher(uri.getPath()).matches();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final NodeResponse merge(final URI uri, final String method, final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses, final NodeResponse clientResponse) {
|
||||
if (!canHandle(uri, method)) {
|
||||
throw new IllegalArgumentException("Cannot use Endpoint Mapper of type " + getClass().getSimpleName() + " to map responses for URI " + uri + ", HTTP Method " + method);
|
||||
}
|
||||
|
||||
final ProcessGroupsEntity responseEntity = clientResponse.getClientResponse().getEntity(ProcessGroupsEntity.class);
|
||||
final Set<ProcessGroupEntity> processGroupEntities = responseEntity.getProcessGroups();
|
||||
|
||||
final Map<String, Map<NodeIdentifier, ProcessGroupEntity>> entityMap = new HashMap<>();
|
||||
for (final NodeResponse nodeResponse : successfulResponses) {
|
||||
final ProcessGroupsEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(ProcessGroupsEntity.class);
|
||||
final Set<ProcessGroupEntity> nodeProcessGroupEntities = nodeResponseEntity.getProcessGroups();
|
||||
|
||||
for (final ProcessGroupEntity nodeProcessGroupEntity : nodeProcessGroupEntities) {
|
||||
final NodeIdentifier nodeId = nodeResponse.getNodeId();
|
||||
Map<NodeIdentifier, ProcessGroupEntity> innerMap = entityMap.get(nodeId);
|
||||
if (innerMap == null) {
|
||||
innerMap = new HashMap<>();
|
||||
entityMap.put(nodeProcessGroupEntity.getId(), innerMap);
|
||||
}
|
||||
|
||||
innerMap.put(nodeResponse.getNodeId(), nodeProcessGroupEntity);
|
||||
}
|
||||
}
|
||||
|
||||
ProcessGroupsEntityMerger.mergeProcessGroups(processGroupEntities, entityMap);
|
||||
|
||||
// create a new client response
|
||||
return new NodeResponse(clientResponse, responseEntity);
|
||||
}
|
||||
}
|
|
@ -17,28 +17,24 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.ProcessorEntityMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ProcessorEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.StatusMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.ProcessorDTO;
|
||||
import org.apache.nifi.web.api.entity.ProcessorEntity;
|
||||
|
||||
public class ProcessorEndpointMerger extends AbstractSingleEntityEndpoint<ProcessorEntity> implements EndpointResponseMerger {
|
||||
public static final Pattern PROCESSORS_URI_PATTERN = Pattern.compile("/nifi-api/processors");
|
||||
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 CLUSTER_PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/cluster/processors/[a-f0-9\\-]{36}");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(final URI uri, final String method) {
|
||||
if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method))
|
||||
&& (PROCESSOR_URI_PATTERN.matcher(uri.getPath()).matches() || CLUSTER_PROCESSOR_URI_PATTERN.matcher(uri.getPath()).matches())) {
|
||||
if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && (PROCESSOR_URI_PATTERN.matcher(uri.getPath()).matches())) {
|
||||
return true;
|
||||
} else if ("POST".equalsIgnoreCase(method) && PROCESSORS_URI_PATTERN.matcher(uri.getPath()).matches()) {
|
||||
return true;
|
||||
|
@ -53,42 +49,10 @@ public class ProcessorEndpointMerger extends AbstractSingleEntityEndpoint<Proces
|
|||
}
|
||||
|
||||
|
||||
protected void mergeResponses(final ProcessorDTO clientDto, final Map<NodeIdentifier, ProcessorDTO> dtoMap, final Set<NodeResponse> successfulResponses,
|
||||
final Set<NodeResponse> problematicResponses) {
|
||||
final Map<String, Set<NodeIdentifier>> validationErrorMap = new HashMap<>();
|
||||
|
||||
for (final Map.Entry<NodeIdentifier, ProcessorDTO> nodeEntry : dtoMap.entrySet()) {
|
||||
final NodeIdentifier nodeId = nodeEntry.getKey();
|
||||
final ProcessorDTO nodeProcessor = nodeEntry.getValue();
|
||||
|
||||
// merge the validation errors
|
||||
mergeValidationErrors(validationErrorMap, nodeId, nodeProcessor.getValidationErrors());
|
||||
}
|
||||
|
||||
// set the merged the validation errors
|
||||
clientDto.setValidationErrors(normalizedMergedValidationErrors(validationErrorMap, dtoMap.size()));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void mergeResponses(final ProcessorEntity clientEntity, final Map<NodeIdentifier, ProcessorEntity> entityMap, final Set<NodeResponse> successfulResponses,
|
||||
final Set<NodeResponse> problematicResponses) {
|
||||
|
||||
final ProcessorDTO clientDto = clientEntity.getComponent();
|
||||
final Map<NodeIdentifier, ProcessorDTO> dtoMap = new HashMap<>();
|
||||
for (final Map.Entry<NodeIdentifier, ProcessorEntity> entry : entityMap.entrySet()) {
|
||||
final ProcessorEntity nodeProcEntity = entry.getValue();
|
||||
final ProcessorDTO nodeProcDto = nodeProcEntity.getComponent();
|
||||
dtoMap.put(entry.getKey(), nodeProcDto);
|
||||
}
|
||||
|
||||
for (final Map.Entry<NodeIdentifier, ProcessorEntity> entry : entityMap.entrySet()) {
|
||||
final NodeIdentifier nodeId = entry.getKey();
|
||||
final ProcessorEntity entity = entry.getValue();
|
||||
if (entity != clientEntity) {
|
||||
StatusMerger.merge(clientEntity.getStatus(), entity.getStatus(), nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
|
||||
}
|
||||
}
|
||||
|
||||
mergeResponses(clientDto, dtoMap, successfulResponses, problematicResponses);
|
||||
ProcessorEntityMerger.mergeProcessors(clientEntity, entityMap);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,21 +17,21 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.ProcessorsEntityMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ProcessorEntity;
|
||||
import org.apache.nifi.web.api.entity.ProcessorsEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.ProcessorDTO;
|
||||
import org.apache.nifi.web.api.entity.ProcessorEntity;
|
||||
import org.apache.nifi.web.api.entity.ProcessorsEntity;
|
||||
|
||||
public class ProcessorsEndpointMerger implements EndpointResponseMerger {
|
||||
public static final Pattern PROCESSORS_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors");
|
||||
public static final Pattern PROCESSORS_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(final URI uri, final String method) {
|
||||
|
@ -47,30 +47,24 @@ public class ProcessorsEndpointMerger implements EndpointResponseMerger {
|
|||
final ProcessorsEntity responseEntity = clientResponse.getClientResponse().getEntity(ProcessorsEntity.class);
|
||||
final Set<ProcessorEntity> processorEntities = responseEntity.getProcessors();
|
||||
|
||||
final Map<String, Map<NodeIdentifier, ProcessorDTO>> dtoMap = new HashMap<>();
|
||||
final Map<String, Map<NodeIdentifier, ProcessorEntity>> entityMap = new HashMap<>();
|
||||
for (final NodeResponse nodeResponse : successfulResponses) {
|
||||
final ProcessorsEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(ProcessorsEntity.class);
|
||||
final Set<ProcessorEntity> nodeProcessorEntities = nodeResponseEntity.getProcessors();
|
||||
|
||||
for (final ProcessorEntity nodeProcessorEntity : nodeProcessorEntities) {
|
||||
final NodeIdentifier nodeId = nodeResponse.getNodeId();
|
||||
Map<NodeIdentifier, ProcessorDTO> innerMap = dtoMap.get(nodeId);
|
||||
Map<NodeIdentifier, ProcessorEntity> innerMap = entityMap.get(nodeId);
|
||||
if (innerMap == null) {
|
||||
innerMap = new HashMap<>();
|
||||
dtoMap.put(nodeProcessorEntity.getId(), innerMap);
|
||||
entityMap.put(nodeProcessorEntity.getId(), innerMap);
|
||||
}
|
||||
|
||||
innerMap.put(nodeResponse.getNodeId(), nodeProcessorEntity.getComponent());
|
||||
innerMap.put(nodeResponse.getNodeId(), nodeProcessorEntity);
|
||||
}
|
||||
}
|
||||
|
||||
final ProcessorEndpointMerger procMerger = new ProcessorEndpointMerger();
|
||||
for (final ProcessorEntity entity : processorEntities) {
|
||||
final String componentId = entity.getId();
|
||||
final Map<NodeIdentifier, ProcessorDTO> mergeMap = dtoMap.get(componentId);
|
||||
|
||||
procMerger.mergeResponses(entity.getComponent(), mergeMap, successfulResponses, problematicResponses);
|
||||
}
|
||||
ProcessorsEntityMerger.mergeProcessors(processorEntities, entityMap);
|
||||
|
||||
// create a new client response
|
||||
return new NodeResponse(clientResponse, responseEntity);
|
||||
|
|
|
@ -17,24 +17,19 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.RemoteProcessGroupEntityMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO;
|
||||
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
|
||||
import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
|
||||
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
|
||||
|
||||
public class RemoteProcessGroupEndpointMerger extends AbstractSingleDTOEndpoint<RemoteProcessGroupEntity, RemoteProcessGroupDTO> {
|
||||
public static final Pattern REMOTE_PROCESS_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/remote-process-groups");
|
||||
public class RemoteProcessGroupEndpointMerger extends AbstractSingleEntityEndpoint<RemoteProcessGroupEntity> implements EndpointResponseMerger {
|
||||
public static final Pattern REMOTE_PROCESS_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/remote-process-groups");
|
||||
public static final Pattern REMOTE_PROCESS_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/remote-process-groups/[a-f0-9\\-]{36}");
|
||||
|
||||
@Override
|
||||
|
@ -54,76 +49,9 @@ public class RemoteProcessGroupEndpointMerger extends AbstractSingleDTOEndpoint<
|
|||
}
|
||||
|
||||
@Override
|
||||
protected RemoteProcessGroupDTO getDto(final RemoteProcessGroupEntity entity) {
|
||||
return entity.getComponent();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void mergeResponses(RemoteProcessGroupDTO clientDto, Map<NodeIdentifier, RemoteProcessGroupDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
|
||||
final RemoteProcessGroupContentsDTO remoteProcessGroupContents = clientDto.getContents();
|
||||
|
||||
Boolean mergedIsTargetSecure = null;
|
||||
final List<String> mergedAuthorizationIssues = new ArrayList<>();
|
||||
final Set<RemoteProcessGroupPortDTO> mergedInputPorts = new HashSet<>();
|
||||
final Set<RemoteProcessGroupPortDTO> mergedOutputPorts = new HashSet<>();
|
||||
|
||||
for (final Map.Entry<NodeIdentifier, RemoteProcessGroupDTO> nodeEntry : dtoMap.entrySet()) {
|
||||
final NodeIdentifier nodeId = nodeEntry.getKey();
|
||||
final RemoteProcessGroupDTO nodeRemoteProcessGroupDto = nodeEntry.getValue();
|
||||
|
||||
// merge the issues
|
||||
final List<String> nodeAuthorizationIssues = nodeRemoteProcessGroupDto.getAuthorizationIssues();
|
||||
if (nodeAuthorizationIssues != null && !nodeAuthorizationIssues.isEmpty()) {
|
||||
for (final String nodeAuthorizationIssue : nodeAuthorizationIssues) {
|
||||
mergedAuthorizationIssues.add(nodeId.getApiAddress() + ":" + nodeId.getApiPort() + " -- " + nodeAuthorizationIssue);
|
||||
}
|
||||
}
|
||||
|
||||
// use the first target secure flag since they will all be the same
|
||||
final Boolean nodeIsTargetSecure = nodeRemoteProcessGroupDto.isTargetSecure();
|
||||
if (mergedIsTargetSecure == null) {
|
||||
mergedIsTargetSecure = nodeIsTargetSecure;
|
||||
}
|
||||
|
||||
// merge the ports in the contents
|
||||
final RemoteProcessGroupContentsDTO nodeRemoteProcessGroupContentsDto = nodeRemoteProcessGroupDto.getContents();
|
||||
if (remoteProcessGroupContents != null && nodeRemoteProcessGroupContentsDto != null) {
|
||||
if (nodeRemoteProcessGroupContentsDto.getInputPorts() != null) {
|
||||
mergedInputPorts.addAll(nodeRemoteProcessGroupContentsDto.getInputPorts());
|
||||
}
|
||||
if (nodeRemoteProcessGroupContentsDto.getOutputPorts() != null) {
|
||||
mergedOutputPorts.addAll(nodeRemoteProcessGroupContentsDto.getOutputPorts());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (remoteProcessGroupContents != null) {
|
||||
if (!mergedInputPorts.isEmpty()) {
|
||||
remoteProcessGroupContents.setInputPorts(mergedInputPorts);
|
||||
}
|
||||
if (!mergedOutputPorts.isEmpty()) {
|
||||
remoteProcessGroupContents.setOutputPorts(mergedOutputPorts);
|
||||
}
|
||||
}
|
||||
|
||||
if (mergedIsTargetSecure != null) {
|
||||
clientDto.setTargetSecure(mergedIsTargetSecure);
|
||||
}
|
||||
|
||||
if (!mergedAuthorizationIssues.isEmpty()) {
|
||||
clientDto.setAuthorizationIssues(mergedAuthorizationIssues);
|
||||
}
|
||||
}
|
||||
|
||||
protected void mergeResponses(RemoteProcessGroupEntity clientEntity, Map<NodeIdentifier, RemoteProcessGroupEntity> entityMap,
|
||||
Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
|
||||
Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
|
||||
|
||||
final RemoteProcessGroupDTO clientDto = clientEntity.getComponent();
|
||||
final Map<NodeIdentifier, RemoteProcessGroupDTO> dtoMap = new HashMap<>();
|
||||
for (final Map.Entry<NodeIdentifier, RemoteProcessGroupEntity> entry : entityMap.entrySet()) {
|
||||
dtoMap.put(entry.getKey(), entry.getValue().getComponent());
|
||||
}
|
||||
|
||||
mergeResponses(clientDto, dtoMap, successfulResponses, problematicResponses);
|
||||
RemoteProcessGroupEntityMerger.mergeRemoteProcessGroups(clientEntity, entityMap);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,21 +17,21 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.RemoteProcessGroupsEntityMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
|
||||
import org.apache.nifi.web.api.entity.RemoteProcessGroupsEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
|
||||
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
|
||||
import org.apache.nifi.web.api.entity.RemoteProcessGroupsEntity;
|
||||
|
||||
public class RemoteProcessGroupsEndpointMerger implements EndpointResponseMerger {
|
||||
public static final Pattern REMOTE_PROCESS_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/remote-process-groups");
|
||||
public static final Pattern REMOTE_PROCESS_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/remote-process-groups");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(final URI uri, final String method) {
|
||||
|
@ -47,30 +47,24 @@ public class RemoteProcessGroupsEndpointMerger implements EndpointResponseMerger
|
|||
final RemoteProcessGroupsEntity responseEntity = clientResponse.getClientResponse().getEntity(RemoteProcessGroupsEntity.class);
|
||||
final Set<RemoteProcessGroupEntity> rpgEntities = responseEntity.getRemoteProcessGroups();
|
||||
|
||||
final Map<String, Map<NodeIdentifier, RemoteProcessGroupDTO>> dtoMap = new HashMap<>();
|
||||
final Map<String, Map<NodeIdentifier, RemoteProcessGroupEntity>> entityMap = new HashMap<>();
|
||||
for (final NodeResponse nodeResponse : successfulResponses) {
|
||||
final RemoteProcessGroupsEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(RemoteProcessGroupsEntity.class);
|
||||
final Set<RemoteProcessGroupEntity> nodeRpgEntities = nodeResponseEntity.getRemoteProcessGroups();
|
||||
|
||||
for (final RemoteProcessGroupEntity nodeRpgEntity : nodeRpgEntities) {
|
||||
final NodeIdentifier nodeId = nodeResponse.getNodeId();
|
||||
Map<NodeIdentifier, RemoteProcessGroupDTO> innerMap = dtoMap.get(nodeId);
|
||||
Map<NodeIdentifier, RemoteProcessGroupEntity> innerMap = entityMap.get(nodeId);
|
||||
if (innerMap == null) {
|
||||
innerMap = new HashMap<>();
|
||||
dtoMap.put(nodeRpgEntity.getId(), innerMap);
|
||||
entityMap.put(nodeRpgEntity.getId(), innerMap);
|
||||
}
|
||||
|
||||
innerMap.put(nodeResponse.getNodeId(), nodeRpgEntity.getComponent());
|
||||
innerMap.put(nodeResponse.getNodeId(), nodeRpgEntity);
|
||||
}
|
||||
}
|
||||
|
||||
final RemoteProcessGroupEndpointMerger rpgMerger = new RemoteProcessGroupEndpointMerger();
|
||||
for (final RemoteProcessGroupEntity entity : rpgEntities) {
|
||||
final String componentId = entity.getId();
|
||||
final Map<NodeIdentifier, RemoteProcessGroupDTO> mergeMap = dtoMap.get(componentId);
|
||||
|
||||
rpgMerger.mergeResponses(entity.getComponent(), mergeMap, successfulResponses, problematicResponses);
|
||||
}
|
||||
RemoteProcessGroupsEntityMerger.mergeRemoteProcessGroups(rpgEntities, entityMap);
|
||||
|
||||
// create a new client response
|
||||
return new NodeResponse(clientResponse, responseEntity);
|
||||
|
|
|
@ -17,20 +17,20 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.ReportingTaskEntityMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.ReportingTaskDTO;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
public class ReportingTaskEndpointMerger extends AbstractSingleDTOEndpoint<ReportingTaskEntity, ReportingTaskDTO> {
|
||||
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 class ReportingTaskEndpointMerger extends AbstractSingleEntityEndpoint<ReportingTaskEntity> implements EndpointResponseMerger {
|
||||
public static final String REPORTING_TASKS_URI = "/nifi-api/controller/reporting-tasks";
|
||||
public static final Pattern REPORTING_TASK_URI_PATTERN = Pattern.compile("/nifi-api/reporting-tasks/[a-f0-9\\-]{36}");
|
||||
|
||||
@Override
|
||||
public boolean canHandle(URI uri, String method) {
|
||||
|
@ -49,32 +49,7 @@ public class ReportingTaskEndpointMerger extends AbstractSingleDTOEndpoint<Repor
|
|||
}
|
||||
|
||||
@Override
|
||||
protected ReportingTaskDTO getDto(ReportingTaskEntity entity) {
|
||||
return entity.getComponent();
|
||||
protected void mergeResponses(ReportingTaskEntity clientEntity, Map<NodeIdentifier, ReportingTaskEntity> entityMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
|
||||
ReportingTaskEntityMerger.mergeReportingTasks(clientEntity, entityMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void mergeResponses(ReportingTaskDTO clientDto, Map<NodeIdentifier, ReportingTaskDTO> dtoMap, Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
|
||||
final Map<String, Set<NodeIdentifier>> validationErrorMap = new HashMap<>();
|
||||
|
||||
int activeThreadCount = 0;
|
||||
for (final Map.Entry<NodeIdentifier, ReportingTaskDTO> nodeEntry : dtoMap.entrySet()) {
|
||||
final NodeIdentifier nodeId = nodeEntry.getKey();
|
||||
final ReportingTaskDTO nodeReportingTask = nodeEntry.getValue();
|
||||
|
||||
if (nodeReportingTask.getActiveThreadCount() != null) {
|
||||
activeThreadCount += nodeReportingTask.getActiveThreadCount();
|
||||
}
|
||||
|
||||
// merge the validation errors
|
||||
mergeValidationErrors(validationErrorMap, nodeId, nodeReportingTask.getValidationErrors());
|
||||
}
|
||||
|
||||
// set the merged active thread counts
|
||||
clientDto.setActiveThreadCount(activeThreadCount);
|
||||
|
||||
// set the merged the validation errors
|
||||
clientDto.setValidationErrors(normalizedMergedValidationErrors(validationErrorMap, dtoMap.size()));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -17,18 +17,20 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import org.apache.nifi.cluster.coordination.http.EndpointResponseMerger;
|
||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||
import org.apache.nifi.cluster.manager.ReportingTasksEntityMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
import org.apache.nifi.web.api.entity.ReportingTasksEntity;
|
||||
|
||||
import java.net.URI;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class ReportingTasksEndpointMerger extends AbstractMultiEntityEndpoint<ReportingTasksEntity, ReportingTaskEntity> {
|
||||
public static final String REPORTING_TASKS_URI = "/nifi-api/controller/reporting-tasks/node";
|
||||
public class ReportingTasksEndpointMerger implements EndpointResponseMerger {
|
||||
public static final String REPORTING_TASKS_URI = "/nifi-api/controller/reporting-tasks";
|
||||
|
||||
@Override
|
||||
public boolean canHandle(URI uri, String method) {
|
||||
|
@ -36,28 +38,34 @@ public class ReportingTasksEndpointMerger extends AbstractMultiEntityEndpoint<Re
|
|||
}
|
||||
|
||||
@Override
|
||||
protected Class<ReportingTasksEntity> getEntityClass() {
|
||||
return ReportingTasksEntity.class;
|
||||
}
|
||||
public final NodeResponse merge(final URI uri, final String method, final Set<NodeResponse> successfulResponses, final Set<NodeResponse> problematicResponses, final NodeResponse clientResponse) {
|
||||
if (!canHandle(uri, method)) {
|
||||
throw new IllegalArgumentException("Cannot use Endpoint Mapper of type " + getClass().getSimpleName() + " to map responses for URI " + uri + ", HTTP Method " + method);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Set<ReportingTaskEntity> getDtos(ReportingTasksEntity entity) {
|
||||
return entity.getReportingTasks();
|
||||
}
|
||||
final ReportingTasksEntity responseEntity = clientResponse.getClientResponse().getEntity(ReportingTasksEntity.class);
|
||||
final Set<ReportingTaskEntity> reportingTasksEntities = responseEntity.getReportingTasks();
|
||||
|
||||
@Override
|
||||
protected String getComponentId(ReportingTaskEntity entity) {
|
||||
return entity.getComponent().getId();
|
||||
}
|
||||
final Map<String, Map<NodeIdentifier, ReportingTaskEntity>> entityMap = new HashMap<>();
|
||||
for (final NodeResponse nodeResponse : successfulResponses) {
|
||||
final ReportingTasksEntity nodeResponseEntity = nodeResponse == clientResponse ? responseEntity : nodeResponse.getClientResponse().getEntity(ReportingTasksEntity.class);
|
||||
final Set<ReportingTaskEntity> nodeReportingTaskEntities = nodeResponseEntity.getReportingTasks();
|
||||
|
||||
@Override
|
||||
protected void mergeResponses(ReportingTaskEntity entity, Map<NodeIdentifier, ReportingTaskEntity> entityMap,
|
||||
Set<NodeResponse> successfulResponses, Set<NodeResponse> problematicResponses) {
|
||||
for (final ReportingTaskEntity nodeReportingTaskEntity : nodeReportingTaskEntities) {
|
||||
final NodeIdentifier nodeId = nodeResponse.getNodeId();
|
||||
Map<NodeIdentifier, ReportingTaskEntity> innerMap = entityMap.get(nodeId);
|
||||
if (innerMap == null) {
|
||||
innerMap = new HashMap<>();
|
||||
entityMap.put(nodeReportingTaskEntity.getId(), innerMap);
|
||||
}
|
||||
|
||||
new ReportingTaskEndpointMerger().mergeResponses(
|
||||
entity.getComponent(),
|
||||
entityMap.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().getComponent())),
|
||||
successfulResponses,
|
||||
problematicResponses);
|
||||
innerMap.put(nodeResponse.getNodeId(), nodeReportingTaskEntity);
|
||||
}
|
||||
}
|
||||
|
||||
ReportingTasksEntityMerger.mergeReportingTasks(reportingTasksEntities, entityMap);
|
||||
|
||||
// create a new client response
|
||||
return new NodeResponse(clientResponse, responseEntity);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,85 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.BulletinDTO;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public final class BulletinMerger {
|
||||
|
||||
private BulletinMerger() {}
|
||||
|
||||
/**
|
||||
* Merges the validation errors.
|
||||
*
|
||||
* @param bulletins bulletins
|
||||
*/
|
||||
public static List<BulletinDTO> mergeBulletins(final Map<NodeIdentifier, List<BulletinDTO>> bulletins) {
|
||||
final List<BulletinDTO> bulletinDtos = new ArrayList<>();
|
||||
|
||||
for (final Map.Entry<NodeIdentifier, List<BulletinDTO>> entry : bulletins.entrySet()) {
|
||||
final NodeIdentifier nodeId = entry.getKey();
|
||||
final List<BulletinDTO> nodeBulletins = entry.getValue();
|
||||
final String nodeAddress = nodeId.getApiAddress() + ":" + nodeId.getApiPort();
|
||||
|
||||
for (final BulletinDTO bulletin : nodeBulletins) {
|
||||
bulletin.setNodeAddress(nodeAddress);
|
||||
bulletinDtos.add(bulletin);
|
||||
}
|
||||
}
|
||||
|
||||
Collections.sort(bulletinDtos, (BulletinDTO o1, BulletinDTO o2) -> {
|
||||
final int timeComparison = o1.getTimestamp().compareTo(o2.getTimestamp());
|
||||
if (timeComparison != 0) {
|
||||
return timeComparison;
|
||||
}
|
||||
|
||||
return o1.getNodeAddress().compareTo(o2.getNodeAddress());
|
||||
});
|
||||
|
||||
return bulletinDtos;
|
||||
}
|
||||
|
||||
/**
|
||||
* Normalizes the validation errors.
|
||||
*
|
||||
* @param validationErrorMap validation errors for each node
|
||||
* @param totalNodes total number of nodes
|
||||
* @return the normalized validation errors
|
||||
*/
|
||||
public static Set<String> normalizedMergedValidationErrors(final Map<String, Set<NodeIdentifier>> validationErrorMap, int totalNodes) {
|
||||
final Set<String> normalizedValidationErrors = new HashSet<>();
|
||||
for (final Map.Entry<String, Set<NodeIdentifier>> validationEntry : validationErrorMap.entrySet()) {
|
||||
final String msg = validationEntry.getKey();
|
||||
final Set<NodeIdentifier> nodeIds = validationEntry.getValue();
|
||||
|
||||
if (nodeIds.size() == totalNodes) {
|
||||
normalizedValidationErrors.add(msg);
|
||||
} else {
|
||||
nodeIds.forEach(id -> normalizedValidationErrors.add(id.getApiAddress() + ":" + id.getApiPort() + " -- " + msg));
|
||||
}
|
||||
}
|
||||
return normalizedValidationErrors;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.BulletinDTO;
|
||||
import org.apache.nifi.web.api.entity.ComponentEntity;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class ComponentEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges the ComponentEntity responses.
|
||||
*
|
||||
* @param clientEntity the entity being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergeComponents(final ComponentEntity clientEntity, final Map<NodeIdentifier, ? extends ComponentEntity> entityMap) {
|
||||
final Map<NodeIdentifier, List<BulletinDTO>> bulletinDtos = new HashMap<>();
|
||||
for (final Map.Entry<NodeIdentifier, ? extends ComponentEntity> entry : entityMap.entrySet()) {
|
||||
final NodeIdentifier nodeIdentifier = entry.getKey();
|
||||
final ComponentEntity entity = entry.getValue();
|
||||
|
||||
// consider the bulletins if present and authorized
|
||||
if (entity.getBulletins() != null) {
|
||||
entity.getBulletins().forEach(bulletin -> {
|
||||
bulletinDtos.computeIfAbsent(nodeIdentifier, nodeId -> new ArrayList<>()).add(bulletin);
|
||||
});
|
||||
}
|
||||
}
|
||||
clientEntity.setBulletins(BulletinMerger.mergeBulletins(bulletinDtos));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ConnectionEntity;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
public class ConnectionEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges the ConnectionEntity responses.
|
||||
*
|
||||
* @param clientEntity the entity being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergeConnections(final ConnectionEntity clientEntity, final Map<NodeIdentifier, ConnectionEntity> entityMap) {
|
||||
for (final Map.Entry<NodeIdentifier, ConnectionEntity> entry : entityMap.entrySet()) {
|
||||
final NodeIdentifier nodeId = entry.getKey();
|
||||
final ConnectionEntity entity = entry.getValue();
|
||||
if (entity != clientEntity) {
|
||||
StatusMerger.merge(clientEntity.getStatus(), entity.getStatus(), nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ConnectionEntity;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class ConnectionsEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges multiple ConnectionEntity responses.
|
||||
*
|
||||
* @param connectionEntities entities being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergeConnections(final Set<ConnectionEntity> connectionEntities, final Map<String, Map<NodeIdentifier, ConnectionEntity>> entityMap) {
|
||||
for (final ConnectionEntity entity : connectionEntities) {
|
||||
ConnectionEntityMerger.mergeConnections(entity, entityMap.get(entity.getId()));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,146 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.controller.service.ControllerServiceState;
|
||||
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
|
||||
import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceReferencingComponentEntity;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class ControllerServiceEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges the ControllerServiceEntity responses.
|
||||
*
|
||||
* @param clientEntity the entity being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergeControllerServices(final ControllerServiceEntity clientEntity, final Map<NodeIdentifier, ControllerServiceEntity> entityMap) {
|
||||
final ControllerServiceDTO clientDto = clientEntity.getComponent();
|
||||
final Map<NodeIdentifier, ControllerServiceDTO> dtoMap = new HashMap<>();
|
||||
for (final Map.Entry<NodeIdentifier, ControllerServiceEntity> entry : entityMap.entrySet()) {
|
||||
final ControllerServiceEntity nodeControllerServiceEntity = entry.getValue();
|
||||
final ControllerServiceDTO nodeControllerServiceDto = nodeControllerServiceEntity.getComponent();
|
||||
dtoMap.put(entry.getKey(), nodeControllerServiceDto);
|
||||
}
|
||||
|
||||
ComponentEntityMerger.mergeComponents(clientEntity, entityMap);
|
||||
|
||||
mergeDtos(clientDto, dtoMap);
|
||||
}
|
||||
|
||||
private static void mergeDtos(final ControllerServiceDTO clientDto, final Map<NodeIdentifier, ControllerServiceDTO> dtoMap) {
|
||||
// if unauthorized for the client dto, simple return
|
||||
if (clientDto == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final Map<String, Set<NodeIdentifier>> validationErrorMap = new HashMap<>();
|
||||
final Set<ControllerServiceReferencingComponentEntity> referencingComponents = clientDto.getReferencingComponents();
|
||||
final Map<NodeIdentifier, Set<ControllerServiceReferencingComponentEntity>> nodeReferencingComponentsMap = new HashMap<>();
|
||||
|
||||
String state = null;
|
||||
for (final Map.Entry<NodeIdentifier, ControllerServiceDTO> nodeEntry : dtoMap.entrySet()) {
|
||||
final ControllerServiceDTO nodeControllerService = nodeEntry.getValue();
|
||||
|
||||
// consider the node controller service if authorized
|
||||
if (nodeControllerService != null) {
|
||||
final NodeIdentifier nodeId = nodeEntry.getKey();
|
||||
|
||||
if (state == null) {
|
||||
if (ControllerServiceState.DISABLING.name().equals(nodeControllerService.getState())) {
|
||||
state = ControllerServiceState.DISABLING.name();
|
||||
} else if (ControllerServiceState.ENABLING.name().equals(nodeControllerService.getState())) {
|
||||
state = ControllerServiceState.ENABLING.name();
|
||||
}
|
||||
}
|
||||
|
||||
nodeReferencingComponentsMap.put(nodeId, nodeControllerService.getReferencingComponents());
|
||||
|
||||
// merge the validation errors
|
||||
ErrorMerger.mergeErrors(validationErrorMap, nodeId, nodeControllerService.getValidationErrors());
|
||||
}
|
||||
}
|
||||
|
||||
// merge the referencing components
|
||||
mergeControllerServiceReferences(referencingComponents, nodeReferencingComponentsMap);
|
||||
|
||||
// store the 'transition' state is applicable
|
||||
if (state != null) {
|
||||
clientDto.setState(state);
|
||||
}
|
||||
|
||||
// set the merged the validation errors
|
||||
clientDto.setValidationErrors(ErrorMerger.normalizedMergedErrors(validationErrorMap, dtoMap.size()));
|
||||
}
|
||||
|
||||
public static void mergeControllerServiceReferences(Set<ControllerServiceReferencingComponentEntity> referencingComponents,
|
||||
Map<NodeIdentifier, Set<ControllerServiceReferencingComponentEntity>> referencingComponentMap) {
|
||||
|
||||
final Map<String, Integer> activeThreadCounts = new HashMap<>();
|
||||
final Map<String, String> states = new HashMap<>();
|
||||
for (final Map.Entry<NodeIdentifier, Set<ControllerServiceReferencingComponentEntity>> nodeEntry : referencingComponentMap.entrySet()) {
|
||||
final Set<ControllerServiceReferencingComponentEntity> nodeReferencingComponents = nodeEntry.getValue();
|
||||
|
||||
// go through all the nodes referencing components
|
||||
if (nodeReferencingComponents != null) {
|
||||
for (final ControllerServiceReferencingComponentEntity nodeReferencingComponentEntity : nodeReferencingComponents) {
|
||||
final ControllerServiceReferencingComponentDTO nodeReferencingComponent = nodeReferencingComponentEntity.getComponent();
|
||||
|
||||
// handle active thread counts
|
||||
if (nodeReferencingComponent.getActiveThreadCount() != null && nodeReferencingComponent.getActiveThreadCount() > 0) {
|
||||
final Integer current = activeThreadCounts.get(nodeReferencingComponent.getId());
|
||||
if (current == null) {
|
||||
activeThreadCounts.put(nodeReferencingComponent.getId(), nodeReferencingComponent.getActiveThreadCount());
|
||||
} else {
|
||||
activeThreadCounts.put(nodeReferencingComponent.getId(), nodeReferencingComponent.getActiveThreadCount() + current);
|
||||
}
|
||||
}
|
||||
|
||||
// handle controller service state
|
||||
final String state = states.get(nodeReferencingComponent.getId());
|
||||
if (state == null) {
|
||||
if (ControllerServiceState.DISABLING.name().equals(nodeReferencingComponent.getState())) {
|
||||
states.put(nodeReferencingComponent.getId(), ControllerServiceState.DISABLING.name());
|
||||
} else if (ControllerServiceState.ENABLING.name().equals(nodeReferencingComponent.getState())) {
|
||||
states.put(nodeReferencingComponent.getId(), ControllerServiceState.ENABLING.name());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// go through each referencing components
|
||||
for (final ControllerServiceReferencingComponentEntity referencingComponent : referencingComponents) {
|
||||
final Integer activeThreadCount = activeThreadCounts.get(referencingComponent.getId());
|
||||
if (activeThreadCount != null) {
|
||||
referencingComponent.getComponent().setActiveThreadCount(activeThreadCount);
|
||||
}
|
||||
|
||||
final String state = states.get(referencingComponent.getId());
|
||||
if (state != null) {
|
||||
referencingComponent.getComponent().setState(state);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class ControllerServicesEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges multiple ControllerServiceEntity responses.
|
||||
*
|
||||
* @param controllerServiceEntities entities being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergeControllerServices(final Set<ControllerServiceEntity> controllerServiceEntities, final Map<String, Map<NodeIdentifier, ControllerServiceEntity>> entityMap) {
|
||||
for (final ControllerServiceEntity entity : controllerServiceEntities) {
|
||||
ControllerServiceEntityMerger.mergeControllerServices(entity, entityMap.get(entity.getId()));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,66 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public final class ErrorMerger {
|
||||
|
||||
private ErrorMerger() {}
|
||||
|
||||
/**
|
||||
* Merges the validation or authorization errors.
|
||||
*
|
||||
* @param validationErrorMap errors for each node
|
||||
* @param nodeId node id
|
||||
* @param nodeErrors node errors
|
||||
*/
|
||||
public static void mergeErrors(final Map<String, Set<NodeIdentifier>> validationErrorMap, final NodeIdentifier nodeId, final Collection<String> nodeErrors) {
|
||||
if (nodeErrors != null) {
|
||||
nodeErrors.stream().forEach(
|
||||
err -> validationErrorMap.computeIfAbsent(err, k -> new HashSet<NodeIdentifier>())
|
||||
.add(nodeId));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Normalizes the validation errors.
|
||||
*
|
||||
* @param errorMap validation errors for each node
|
||||
* @param totalNodes total number of nodes
|
||||
* @return the normalized validation errors
|
||||
*/
|
||||
public static Set<String> normalizedMergedErrors(final Map<String, Set<NodeIdentifier>> errorMap, int totalNodes) {
|
||||
final Set<String> normalizedErrors = new HashSet<>();
|
||||
for (final Map.Entry<String, Set<NodeIdentifier>> validationEntry : errorMap.entrySet()) {
|
||||
final String msg = validationEntry.getKey();
|
||||
final Set<NodeIdentifier> nodeIds = validationEntry.getValue();
|
||||
|
||||
if (nodeIds.size() == totalNodes) {
|
||||
normalizedErrors.add(msg);
|
||||
} else {
|
||||
nodeIds.forEach(id -> normalizedErrors.add(id.getApiAddress() + ":" + id.getApiPort() + " -- " + msg));
|
||||
}
|
||||
}
|
||||
return normalizedErrors;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,78 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.PortDTO;
|
||||
import org.apache.nifi.web.api.entity.PortEntity;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class PortEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges the PortEntity responses.
|
||||
*
|
||||
* @param clientEntity the entity being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergePorts(final PortEntity clientEntity, final Map<NodeIdentifier, PortEntity> entityMap) {
|
||||
final PortDTO clientDto = clientEntity.getComponent();
|
||||
final Map<NodeIdentifier, PortDTO> dtoMap = new HashMap<>();
|
||||
for (final Map.Entry<NodeIdentifier, PortEntity> entry : entityMap.entrySet()) {
|
||||
final PortEntity nodePortEntity = entry.getValue();
|
||||
final PortDTO nodePortDto = nodePortEntity.getComponent();
|
||||
dtoMap.put(entry.getKey(), nodePortDto);
|
||||
}
|
||||
|
||||
for (final Map.Entry<NodeIdentifier, PortEntity> entry : entityMap.entrySet()) {
|
||||
final NodeIdentifier nodeId = entry.getKey();
|
||||
final PortEntity entity = entry.getValue();
|
||||
if (entity != clientEntity) {
|
||||
StatusMerger.merge(clientEntity.getStatus(), entity.getStatus(), nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
|
||||
}
|
||||
}
|
||||
|
||||
ComponentEntityMerger.mergeComponents(clientEntity, entityMap);
|
||||
|
||||
mergeDtos(clientDto, dtoMap);
|
||||
}
|
||||
|
||||
private static void mergeDtos(final PortDTO clientDto, final Map<NodeIdentifier, PortDTO> dtoMap) {
|
||||
// if unauthorized for the client dto, simple return
|
||||
if (clientDto == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final Map<String, Set<NodeIdentifier>> validationErrorMap = new HashMap<>();
|
||||
|
||||
for (final Map.Entry<NodeIdentifier, PortDTO> nodeEntry : dtoMap.entrySet()) {
|
||||
final PortDTO nodePort = nodeEntry.getValue();
|
||||
|
||||
// merge the validation errors if authorized
|
||||
if (nodePort != null) {
|
||||
final NodeIdentifier nodeId = nodeEntry.getKey();
|
||||
ErrorMerger.mergeErrors(validationErrorMap, nodeId, nodePort.getValidationErrors());
|
||||
}
|
||||
}
|
||||
|
||||
// set the merged the validation errors
|
||||
clientDto.setValidationErrors(ErrorMerger.normalizedMergedErrors(validationErrorMap, dtoMap.size()));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.PortEntity;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class PortsEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges multiple PortEntity responses.
|
||||
*
|
||||
* @param portEntities entities being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergePorts(final Set<PortEntity> portEntities, final Map<String, Map<NodeIdentifier, PortEntity>> entityMap) {
|
||||
for (final PortEntity entity : portEntities) {
|
||||
PortEntityMerger.mergePorts(entity, entityMap.get(entity.getId()));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,43 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
public class ProcessGroupEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges the ProcessorGroupEntity responses.
|
||||
*
|
||||
* @param clientEntity the entity being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergeProcessGroups(final ProcessGroupEntity clientEntity, final Map<NodeIdentifier, ProcessGroupEntity> entityMap) {
|
||||
for (final Map.Entry<NodeIdentifier, ProcessGroupEntity> entry : entityMap.entrySet()) {
|
||||
final NodeIdentifier nodeId = entry.getKey();
|
||||
final ProcessGroupEntity entity = entry.getValue();
|
||||
if (entity != clientEntity) {
|
||||
StatusMerger.merge(clientEntity.getStatus(), entity.getStatus(), nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
|
||||
}
|
||||
}
|
||||
|
||||
ComponentEntityMerger.mergeComponents(clientEntity, entityMap);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class ProcessGroupsEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges multiple ProcessGroupEntity responses.
|
||||
*
|
||||
* @param processGroupEntities entities being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergeProcessGroups(final Set<ProcessGroupEntity> processGroupEntities, final Map<String, Map<NodeIdentifier, ProcessGroupEntity>> entityMap) {
|
||||
for (final ProcessGroupEntity entity : processGroupEntities) {
|
||||
ProcessGroupEntityMerger.mergeProcessGroups(entity, entityMap.get(entity.getId()));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,78 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.ProcessorDTO;
|
||||
import org.apache.nifi.web.api.entity.ProcessorEntity;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class ProcessorEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges the ProcessorEntity responses.
|
||||
*
|
||||
* @param clientEntity the entity being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergeProcessors(final ProcessorEntity clientEntity, final Map<NodeIdentifier, ProcessorEntity> entityMap) {
|
||||
final ProcessorDTO clientDto = clientEntity.getComponent();
|
||||
final Map<NodeIdentifier, ProcessorDTO> dtoMap = new HashMap<>();
|
||||
for (final Map.Entry<NodeIdentifier, ProcessorEntity> entry : entityMap.entrySet()) {
|
||||
final ProcessorEntity nodeProcEntity = entry.getValue();
|
||||
final ProcessorDTO nodeProcDto = nodeProcEntity.getComponent();
|
||||
dtoMap.put(entry.getKey(), nodeProcDto);
|
||||
}
|
||||
|
||||
for (final Map.Entry<NodeIdentifier, ProcessorEntity> entry : entityMap.entrySet()) {
|
||||
final NodeIdentifier nodeId = entry.getKey();
|
||||
final ProcessorEntity entity = entry.getValue();
|
||||
if (entity != clientEntity) {
|
||||
StatusMerger.merge(clientEntity.getStatus(), entity.getStatus(), nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
|
||||
}
|
||||
}
|
||||
|
||||
ComponentEntityMerger.mergeComponents(clientEntity, entityMap);
|
||||
|
||||
mergeDtos(clientDto, dtoMap);
|
||||
}
|
||||
|
||||
private static void mergeDtos(final ProcessorDTO clientDto, final Map<NodeIdentifier, ProcessorDTO> dtoMap) {
|
||||
// if unauthorized for the client dto, simple return
|
||||
if (clientDto == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final Map<String, Set<NodeIdentifier>> validationErrorMap = new HashMap<>();
|
||||
|
||||
for (final Map.Entry<NodeIdentifier, ProcessorDTO> nodeEntry : dtoMap.entrySet()) {
|
||||
final ProcessorDTO nodeProcessor = nodeEntry.getValue();
|
||||
|
||||
// merge the validation errors, if authorized
|
||||
if (nodeProcessor != null) {
|
||||
final NodeIdentifier nodeId = nodeEntry.getKey();
|
||||
ErrorMerger.mergeErrors(validationErrorMap, nodeId, nodeProcessor.getValidationErrors());
|
||||
}
|
||||
}
|
||||
|
||||
// set the merged the validation errors
|
||||
clientDto.setValidationErrors(ErrorMerger.normalizedMergedErrors(validationErrorMap, dtoMap.size()));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ProcessorEntity;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class ProcessorsEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges multiple ProcessorEntity responses.
|
||||
*
|
||||
* @param processorEntities entities being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergeProcessors(final Set<ProcessorEntity> processorEntities, final Map<String, Map<NodeIdentifier, ProcessorEntity>> entityMap) {
|
||||
for (final ProcessorEntity entity : processorEntities) {
|
||||
ProcessorEntityMerger.mergeProcessors(entity, entityMap.get(entity.getId()));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,119 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO;
|
||||
import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
|
||||
import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
|
||||
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class RemoteProcessGroupEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges the RemoteProcessGroupEntity responses.
|
||||
*
|
||||
* @param clientEntity the entity being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergeRemoteProcessGroups(final RemoteProcessGroupEntity clientEntity, final Map<NodeIdentifier, RemoteProcessGroupEntity> entityMap) {
|
||||
final RemoteProcessGroupDTO clientDto = clientEntity.getComponent();
|
||||
final Map<NodeIdentifier, RemoteProcessGroupDTO> dtoMap = new HashMap<>();
|
||||
for (final Map.Entry<NodeIdentifier, RemoteProcessGroupEntity> entry : entityMap.entrySet()) {
|
||||
final RemoteProcessGroupEntity nodeProcEntity = entry.getValue();
|
||||
final RemoteProcessGroupDTO nodeProcDto = nodeProcEntity.getComponent();
|
||||
dtoMap.put(entry.getKey(), nodeProcDto);
|
||||
}
|
||||
|
||||
for (final Map.Entry<NodeIdentifier, RemoteProcessGroupEntity> entry : entityMap.entrySet()) {
|
||||
final NodeIdentifier nodeId = entry.getKey();
|
||||
final RemoteProcessGroupEntity entity = entry.getValue();
|
||||
if (entity != clientEntity) {
|
||||
StatusMerger.merge(clientEntity.getStatus(), entity.getStatus(), nodeId.getId(), nodeId.getApiAddress(), nodeId.getApiPort());
|
||||
}
|
||||
}
|
||||
|
||||
ComponentEntityMerger.mergeComponents(clientEntity, entityMap);
|
||||
|
||||
mergeDtos(clientDto, dtoMap);
|
||||
}
|
||||
|
||||
private static void mergeDtos(final RemoteProcessGroupDTO clientDto, final Map<NodeIdentifier, RemoteProcessGroupDTO> dtoMap) {
|
||||
// if unauthorized for the client dto, simple return
|
||||
if (clientDto == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final RemoteProcessGroupContentsDTO remoteProcessGroupContents = clientDto.getContents();
|
||||
|
||||
final Map<String, Set<NodeIdentifier>> authorizationErrorMap = new HashMap<>();
|
||||
Boolean mergedIsTargetSecure = null;
|
||||
final Set<RemoteProcessGroupPortDTO> mergedInputPorts = new HashSet<>();
|
||||
final Set<RemoteProcessGroupPortDTO> mergedOutputPorts = new HashSet<>();
|
||||
|
||||
for (final Map.Entry<NodeIdentifier, RemoteProcessGroupDTO> nodeEntry : dtoMap.entrySet()) {
|
||||
final RemoteProcessGroupDTO nodeRemoteProcessGroup = nodeEntry.getValue();
|
||||
|
||||
// consider the node remote process group when authorized
|
||||
if (nodeRemoteProcessGroup != null) {
|
||||
final NodeIdentifier nodeId = nodeEntry.getKey();
|
||||
|
||||
// merge the authorization errors
|
||||
ErrorMerger.mergeErrors(authorizationErrorMap, nodeId, nodeRemoteProcessGroup.getAuthorizationIssues());
|
||||
|
||||
// use the first target secure flag since they will all be the same
|
||||
final Boolean nodeIsTargetSecure = nodeRemoteProcessGroup.isTargetSecure();
|
||||
if (mergedIsTargetSecure == null) {
|
||||
mergedIsTargetSecure = nodeIsTargetSecure;
|
||||
}
|
||||
|
||||
// merge the ports in the contents
|
||||
final RemoteProcessGroupContentsDTO nodeRemoteProcessGroupContentsDto = nodeRemoteProcessGroup.getContents();
|
||||
if (remoteProcessGroupContents != null && nodeRemoteProcessGroupContentsDto != null) {
|
||||
if (nodeRemoteProcessGroupContentsDto.getInputPorts() != null) {
|
||||
mergedInputPorts.addAll(nodeRemoteProcessGroupContentsDto.getInputPorts());
|
||||
}
|
||||
if (nodeRemoteProcessGroupContentsDto.getOutputPorts() != null) {
|
||||
mergedOutputPorts.addAll(nodeRemoteProcessGroupContentsDto.getOutputPorts());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
if (remoteProcessGroupContents != null) {
|
||||
if (!mergedInputPorts.isEmpty()) {
|
||||
remoteProcessGroupContents.setInputPorts(mergedInputPorts);
|
||||
}
|
||||
if (!mergedOutputPorts.isEmpty()) {
|
||||
remoteProcessGroupContents.setOutputPorts(mergedOutputPorts);
|
||||
}
|
||||
}
|
||||
|
||||
if (mergedIsTargetSecure != null) {
|
||||
clientDto.setTargetSecure(mergedIsTargetSecure);
|
||||
}
|
||||
|
||||
// set the merged the validation errors
|
||||
clientDto.setAuthorizationIssues(ErrorMerger.normalizedMergedErrors(authorizationErrorMap, dtoMap.size()));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class RemoteProcessGroupsEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges multiple RemoteProcessGroupEntity responses.
|
||||
*
|
||||
* @param remoteProcessGroupEntities entities being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergeRemoteProcessGroups(final Set<RemoteProcessGroupEntity> remoteProcessGroupEntities, final Map<String, Map<NodeIdentifier, RemoteProcessGroupEntity>> entityMap) {
|
||||
for (final RemoteProcessGroupEntity entity : remoteProcessGroupEntities) {
|
||||
RemoteProcessGroupEntityMerger.mergeRemoteProcessGroups(entity, entityMap.get(entity.getId()));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,80 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.dto.ReportingTaskDTO;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class ReportingTaskEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges the ReportingTaskEntity responses.
|
||||
*
|
||||
* @param clientEntity the entity being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergeReportingTasks(final ReportingTaskEntity clientEntity, final Map<NodeIdentifier, ReportingTaskEntity> entityMap) {
|
||||
final ReportingTaskDTO clientDto = clientEntity.getComponent();
|
||||
final Map<NodeIdentifier, ReportingTaskDTO> dtoMap = new HashMap<>();
|
||||
for (final Map.Entry<NodeIdentifier, ReportingTaskEntity> entry : entityMap.entrySet()) {
|
||||
final ReportingTaskEntity nodeReportingTaskEntity = entry.getValue();
|
||||
final ReportingTaskDTO nodeReportingTaskDto = nodeReportingTaskEntity.getComponent();
|
||||
dtoMap.put(entry.getKey(), nodeReportingTaskDto);
|
||||
}
|
||||
|
||||
ComponentEntityMerger.mergeComponents(clientEntity, entityMap);
|
||||
|
||||
mergeDtos(clientDto, dtoMap);
|
||||
}
|
||||
|
||||
private static void mergeDtos(final ReportingTaskDTO clientDto, final Map<NodeIdentifier, ReportingTaskDTO> dtoMap) {
|
||||
// if unauthorized for the client dto, simple return
|
||||
if (clientDto == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final Map<String, Set<NodeIdentifier>> validationErrorMap = new HashMap<>();
|
||||
|
||||
int activeThreadCount = 0;
|
||||
for (final Map.Entry<NodeIdentifier, ReportingTaskDTO> nodeEntry : dtoMap.entrySet()) {
|
||||
final ReportingTaskDTO nodeReportingTask = nodeEntry.getValue();
|
||||
|
||||
// consider the node reporting task if authorized
|
||||
if (nodeReportingTask != null) {
|
||||
final NodeIdentifier nodeId = nodeEntry.getKey();
|
||||
|
||||
if (nodeReportingTask.getActiveThreadCount() != null) {
|
||||
activeThreadCount += nodeReportingTask.getActiveThreadCount();
|
||||
}
|
||||
|
||||
// merge the validation errors
|
||||
ErrorMerger.mergeErrors(validationErrorMap, nodeId, nodeReportingTask.getValidationErrors());
|
||||
}
|
||||
}
|
||||
|
||||
// set the merged active thread counts
|
||||
clientDto.setActiveThreadCount(activeThreadCount);
|
||||
|
||||
// set the merged the validation errors
|
||||
clientDto.setValidationErrors(ErrorMerger.normalizedMergedErrors(validationErrorMap, dtoMap.size()));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.cluster.manager;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public class ReportingTasksEntityMerger {
|
||||
|
||||
/**
|
||||
* Merges multiple ReportingTaskEntity responses.
|
||||
*
|
||||
* @param reportingTaskEntities entities being returned to the client
|
||||
* @param entityMap all node responses
|
||||
*/
|
||||
public static void mergeReportingTasks(final Set<ReportingTaskEntity> reportingTaskEntities, final Map<String, Map<NodeIdentifier, ReportingTaskEntity>> entityMap) {
|
||||
for (final ReportingTaskEntity entity : reportingTaskEntities) {
|
||||
ReportingTaskEntityMerger.mergeReportingTasks(entity, entityMap.get(entity.getId()));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -17,8 +17,9 @@
|
|||
|
||||
package org.apache.nifi.cluster.coordination.http.endpoints;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import org.apache.nifi.cluster.manager.ErrorMerger;
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
|
@ -26,8 +27,8 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.nifi.cluster.protocol.NodeIdentifier;
|
||||
import org.junit.Test;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class TestProcessorEndpointMerger {
|
||||
|
||||
|
@ -41,13 +42,13 @@ public class TestProcessorEndpointMerger {
|
|||
nodeValidationErrors1234.add("error 1");
|
||||
nodeValidationErrors1234.add("error 2");
|
||||
|
||||
merger.mergeValidationErrors(validationErrorMap, nodeId1234, nodeValidationErrors1234);
|
||||
ErrorMerger.mergeErrors(validationErrorMap, nodeId1234, nodeValidationErrors1234);
|
||||
|
||||
final NodeIdentifier nodeXyz = new NodeIdentifier("xyz", "localhost", 8000, "localhost", 8001, "localhost", 8002, 8003, false);
|
||||
final List<String> nodeValidationErrorsXyz = new ArrayList<>();
|
||||
nodeValidationErrorsXyz.add("error 1");
|
||||
|
||||
merger.mergeValidationErrors(validationErrorMap, nodeXyz, nodeValidationErrorsXyz);
|
||||
ErrorMerger.mergeErrors(validationErrorMap, nodeXyz, nodeValidationErrorsXyz);
|
||||
|
||||
assertEquals(2, validationErrorMap.size());
|
||||
|
||||
|
|
|
@ -16,40 +16,7 @@
|
|||
*/
|
||||
package org.apache.nifi.controller;
|
||||
|
||||
import static java.util.Objects.requireNonNull;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.LockSupport;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
|
||||
import com.sun.jersey.api.client.ClientHandlerException;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.action.Action;
|
||||
import org.apache.nifi.admin.service.AuditService;
|
||||
|
@ -231,7 +198,37 @@ import org.apache.zookeeper.server.quorum.QuorumPeerConfig.ConfigException;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.sun.jersey.api.client.ClientHandlerException;
|
||||
import javax.net.ssl.SSLContext;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.text.DateFormat;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.LockSupport;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import static java.util.Objects.requireNonNull;
|
||||
|
||||
public class FlowController implements EventAccess, ControllerServiceProvider, ReportingTaskProvider, QueueProvider, Authorizable {
|
||||
|
||||
|
@ -2484,10 +2481,6 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
|
|||
final int flowFilesOutOrRemoved = flowFilesTransferred + flowFilesRemoved;
|
||||
status.setAverageLineageDuration(flowFilesOutOrRemoved == 0 ? 0 : lineageMillis / flowFilesOutOrRemoved, TimeUnit.MILLISECONDS);
|
||||
|
||||
if (remoteGroup.getAuthorizationIssue() != null) {
|
||||
status.setAuthorizationIssues(Arrays.asList(remoteGroup.getAuthorizationIssue()));
|
||||
}
|
||||
|
||||
return status;
|
||||
}
|
||||
|
||||
|
|
|
@ -16,13 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.events;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.reporting.Bulletin;
|
||||
import org.apache.nifi.reporting.BulletinQuery;
|
||||
import org.apache.nifi.reporting.BulletinRepository;
|
||||
|
@ -30,6 +23,13 @@ import org.apache.nifi.reporting.ComponentType;
|
|||
import org.apache.nifi.util.RingBuffer;
|
||||
import org.apache.nifi.util.RingBuffer.Filter;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class VolatileBulletinRepository implements BulletinRepository {
|
||||
|
||||
private static final int CONTROLLER_BUFFER_SIZE = 10;
|
||||
|
@ -135,6 +135,11 @@ public class VolatileBulletinRepository implements BulletinRepository {
|
|||
return selected;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Bulletin> findBulletinsForSource(String sourceId) {
|
||||
return findBulletins(new BulletinQuery.Builder().sourceIdMatches(sourceId).limit(COMPONENT_BUFFER_SIZE).build());
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Bulletin> findBulletinsForGroupBySource(String groupId) {
|
||||
return findBulletinsForGroupBySource(groupId, COMPONENT_BUFFER_SIZE);
|
||||
|
|
|
@ -206,6 +206,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
|
||||
// revision manager
|
||||
private RevisionManager revisionManager;
|
||||
private BulletinRepository bulletinRepository;
|
||||
|
||||
// data access objects
|
||||
private ProcessorDAO processorDAO;
|
||||
|
@ -633,7 +634,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(processorNode);
|
||||
final ProcessorStatusDTO status = dtoFactory.createProcessorStatusDto(controllerFacade.getProcessorStatus(processorNode.getIdentifier()));
|
||||
return new UpdateResult<>(entityFactory.createProcessorEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status), false);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(processorNode.getIdentifier()));
|
||||
return new UpdateResult<>(entityFactory.createProcessorEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status, bulletins), false);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -772,7 +774,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(inputPortNode);
|
||||
final PortStatusDTO status = dtoFactory.createPortStatusDto(controllerFacade.getInputPortStatus(inputPortNode.getIdentifier()));
|
||||
return new UpdateResult<>(entityFactory.createPortEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status), false);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(inputPortNode.getIdentifier()));
|
||||
return new UpdateResult<>(entityFactory.createPortEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status, bulletins), false);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -790,7 +793,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(outputPortNode);
|
||||
final PortStatusDTO status = dtoFactory.createPortStatusDto(controllerFacade.getOutputPortStatus(outputPortNode.getIdentifier()));
|
||||
return new UpdateResult<>(entityFactory.createPortEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status), false);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(outputPortNode.getIdentifier()));
|
||||
return new UpdateResult<>(entityFactory.createPortEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status, bulletins), false);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -810,7 +814,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(remoteProcessGroupNode);
|
||||
final RevisionDTO updateRevision = dtoFactory.createRevisionDTO(snapshot.getLastModification());
|
||||
final RemoteProcessGroupStatusDTO status = dtoFactory.createRemoteProcessGroupStatusDto(controllerFacade.getRemoteProcessGroupStatus(remoteProcessGroupNode.getIdentifier()));
|
||||
return new UpdateResult<>(entityFactory.createRemoteProcessGroupEntity(snapshot.getComponent(), updateRevision, accessPolicy, status), false);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(remoteProcessGroupNode.getIdentifier()));
|
||||
return new UpdateResult<>(entityFactory.createRemoteProcessGroupEntity(snapshot.getComponent(), updateRevision, accessPolicy, status, bulletins), false);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -865,7 +870,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(processGroupNode);
|
||||
final RevisionDTO updatedRevision = dtoFactory.createRevisionDTO(snapshot.getLastModification());
|
||||
final ProcessGroupStatusDTO status = dtoFactory.createConciseProcessGroupStatusDto(controllerFacade.getProcessGroupStatus(processGroupNode.getIdentifier()));
|
||||
return new UpdateResult<>(entityFactory.createProcessGroupEntity(snapshot.getComponent(), updatedRevision, accessPolicy, status), false);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(processGroupNode.getIdentifier()));
|
||||
return new UpdateResult<>(entityFactory.createProcessGroupEntity(snapshot.getComponent(), updatedRevision, accessPolicy, status, bulletins), false);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1047,7 +1053,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
() -> processorDAO.deleteProcessor(processorId),
|
||||
dtoFactory.createProcessorDto(processor));
|
||||
|
||||
return entityFactory.createProcessorEntity(snapshot, null, null, null);
|
||||
return entityFactory.createProcessorEntity(snapshot, null, null, null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1146,7 +1152,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
() -> inputPortDAO.deletePort(inputPortId),
|
||||
dtoFactory.createPortDto(port));
|
||||
|
||||
return entityFactory.createPortEntity(snapshot, null, null, null);
|
||||
return entityFactory.createPortEntity(snapshot, null, null, null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1158,7 +1164,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
() -> outputPortDAO.deletePort(outputPortId),
|
||||
dtoFactory.createPortDto(port));
|
||||
|
||||
return entityFactory.createPortEntity(snapshot, null, null, null);
|
||||
return entityFactory.createPortEntity(snapshot, null, null, null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1170,7 +1176,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
() -> processGroupDAO.deleteProcessGroup(groupId),
|
||||
dtoFactory.createProcessGroupDto(processGroup));
|
||||
|
||||
return entityFactory.createProcessGroupEntity(snapshot, null, null, null);
|
||||
return entityFactory.createProcessGroupEntity(snapshot, null, null, null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1182,7 +1188,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
() -> remoteProcessGroupDAO.deleteRemoteProcessGroup(remoteProcessGroupId),
|
||||
dtoFactory.createRemoteProcessGroupDto(remoteProcessGroup));
|
||||
|
||||
return entityFactory.createRemoteProcessGroupEntity(snapshot, null, null, null);
|
||||
return entityFactory.createRemoteProcessGroupEntity(snapshot, null, null, null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1240,7 +1246,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final ProcessorNode processor = processorDAO.getProcessor(processorDTO.getId());
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(processor);
|
||||
final ProcessorStatusDTO status = dtoFactory.createProcessorStatusDto(controllerFacade.getProcessorStatus(processorDTO.getId()));
|
||||
return entityFactory.createProcessorEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(processorDTO.getId()));
|
||||
return entityFactory.createProcessorEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status, bulletins);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1406,7 +1413,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final Port port = inputPortDAO.getPort(inputPortDTO.getId());
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(port);
|
||||
final PortStatusDTO status = dtoFactory.createPortStatusDto(controllerFacade.getInputPortStatus(port.getIdentifier()));
|
||||
return entityFactory.createPortEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(port.getIdentifier()));
|
||||
return entityFactory.createPortEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status, bulletins);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1419,7 +1427,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final Port port = outputPortDAO.getPort(outputPortDTO.getId());
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(port);
|
||||
final PortStatusDTO status = dtoFactory.createPortStatusDto(controllerFacade.getOutputPortStatus(port.getIdentifier()));
|
||||
return entityFactory.createPortEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(port.getIdentifier()));
|
||||
return entityFactory.createPortEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status, bulletins);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1432,7 +1441,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final ProcessGroup processGroup = processGroupDAO.getProcessGroup(processGroupDTO.getId());
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(processGroup);
|
||||
final ProcessGroupStatusDTO status = dtoFactory.createConciseProcessGroupStatusDto(controllerFacade.getProcessGroupStatus(processGroup.getIdentifier()));
|
||||
return entityFactory.createProcessGroupEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(processGroup.getIdentifier()));
|
||||
return entityFactory.createProcessGroupEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status, bulletins);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1445,7 +1455,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final RemoteProcessGroup remoteProcessGroup = remoteProcessGroupDAO.getRemoteProcessGroup(remoteProcessGroupDTO.getId());
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(remoteProcessGroup);
|
||||
final RemoteProcessGroupStatusDTO status = dtoFactory.createRemoteProcessGroupStatusDto(controllerFacade.getRemoteProcessGroupStatus(remoteProcessGroup.getIdentifier()));
|
||||
return entityFactory.createRemoteProcessGroupEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(remoteProcessGroup.getIdentifier()));
|
||||
return entityFactory.createRemoteProcessGroupEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, status, bulletins);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1601,7 +1612,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(processor);
|
||||
final FlowModification lastMod = new FlowModification(incrementRevision(revision), modifier);
|
||||
final ProcessorStatusDTO status = dtoFactory.createProcessorStatusDto(controllerFacade.getProcessorStatus(processor.getIdentifier()));
|
||||
final ProcessorEntity entity = entityFactory.createProcessorEntity(updatedProcDto, dtoFactory.createRevisionDTO(lastMod), accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(processor.getIdentifier()));
|
||||
final ProcessorEntity entity = entityFactory.createProcessorEntity(updatedProcDto, dtoFactory.createRevisionDTO(lastMod), accessPolicy, status, bulletins);
|
||||
|
||||
return new StandardRevisionUpdate<>(entity, lastMod);
|
||||
}
|
||||
|
@ -1631,7 +1643,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
|
||||
final ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerServiceDTO.getId());
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(controllerService);
|
||||
return entityFactory.createControllerServiceEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(controllerServiceDTO.getId()));
|
||||
return entityFactory.createControllerServiceEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, bulletins);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1649,7 +1662,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
cs -> dtoFactory.createControllerServiceDto(cs));
|
||||
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(controllerService);
|
||||
return new UpdateResult<>(entityFactory.createControllerServiceEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy), false);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(controllerServiceDTO.getId()));
|
||||
return new UpdateResult<>(entityFactory.createControllerServiceEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, bulletins), false);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1811,7 +1825,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
() -> controllerServiceDAO.deleteControllerService(controllerServiceId),
|
||||
dtoFactory.createControllerServiceDto(controllerService));
|
||||
|
||||
return entityFactory.createControllerServiceEntity(snapshot, null, null);
|
||||
return entityFactory.createControllerServiceEntity(snapshot, null, null, null);
|
||||
}
|
||||
|
||||
|
||||
|
@ -1837,7 +1851,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final ReportingTaskDTO dto = dtoFactory.createReportingTaskDto(reportingTask);
|
||||
final FlowModification lastMod = new FlowModification(new Revision(0L, rev.getClientId(), dto.getId()), modifier);
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(reportingTask);
|
||||
return entityFactory.createReportingTaskEntity(dto, dtoFactory.createRevisionDTO(lastMod), accessPolicy);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(reportingTask.getIdentifier()));
|
||||
return entityFactory.createReportingTaskEntity(dto, dtoFactory.createRevisionDTO(lastMod), accessPolicy, bulletins);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -1856,7 +1871,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
rt -> dtoFactory.createReportingTaskDto(rt));
|
||||
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(reportingTask);
|
||||
return new UpdateResult<>(entityFactory.createReportingTaskEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy), false);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(reportingTask.getIdentifier()));
|
||||
return new UpdateResult<>(entityFactory.createReportingTaskEntity(snapshot.getComponent(), dtoFactory.createRevisionDTO(snapshot.getLastModification()), accessPolicy, bulletins), false);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1868,7 +1884,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
() -> reportingTaskDAO.deleteReportingTask(reportingTaskId),
|
||||
dtoFactory.createReportingTaskDto(reportingTask));
|
||||
|
||||
return entityFactory.createReportingTaskEntity(snapshot, null, null);
|
||||
return entityFactory.createReportingTaskEntity(snapshot, null, null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -2119,7 +2135,9 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(processor.getIdentifier()));
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(processor);
|
||||
final ProcessorStatusDTO status = dtoFactory.createProcessorStatusDto(controllerFacade.getProcessorStatus(processor.getIdentifier()));
|
||||
return entityFactory.createProcessorEntity(dtoFactory.createProcessorDto(processor), revision, accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins =
|
||||
dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(processor.getIdentifier()));
|
||||
return entityFactory.createProcessorEntity(dtoFactory.createProcessorDto(processor), revision, accessPolicy, status, bulletins);
|
||||
})
|
||||
.collect(Collectors.toSet());
|
||||
});
|
||||
|
@ -2177,7 +2195,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
|
||||
final RevisionDTO revision = dtoFactory.createRevisionDTO(rev);
|
||||
final ProcessorStatusDTO status = dtoFactory.createProcessorStatusDto(controllerFacade.getProcessorStatus(id));
|
||||
return entityFactory.createProcessorEntity(dtoFactory.createProcessorDto(processor), revision, dtoFactory.createAccessPolicyDto(processor), status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(id));
|
||||
return entityFactory.createProcessorEntity(dtoFactory.createProcessorDto(processor), revision, dtoFactory.createAccessPolicyDto(processor), status, bulletins);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -2215,9 +2234,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
.after(query.getAfter())
|
||||
.limit(query.getLimit());
|
||||
|
||||
// get the bulletin repository
|
||||
final BulletinRepository bulletinRepository = controllerFacade.getBulletinRepository();
|
||||
|
||||
// perform the query
|
||||
final List<Bulletin> results = bulletinRepository.findBulletins(queryBuilder.build());
|
||||
|
||||
|
@ -2432,7 +2448,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(port.getIdentifier()));
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(port);
|
||||
final PortStatusDTO status = dtoFactory.createPortStatusDto(controllerFacade.getInputPortStatus(port.getIdentifier()));
|
||||
return entityFactory.createPortEntity(dtoFactory.createPortDto(port), revision, accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(port.getIdentifier()));
|
||||
return entityFactory.createPortEntity(dtoFactory.createPortDto(port), revision, accessPolicy, status, bulletins);
|
||||
})
|
||||
.collect(Collectors.toSet());
|
||||
});
|
||||
|
@ -2452,7 +2469,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(port.getIdentifier()));
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(port);
|
||||
final PortStatusDTO status = dtoFactory.createPortStatusDto(controllerFacade.getOutputPortStatus(port.getIdentifier()));
|
||||
return entityFactory.createPortEntity(dtoFactory.createPortDto(port), revision, accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(port.getIdentifier()));
|
||||
return entityFactory.createPortEntity(dtoFactory.createPortDto(port), revision, accessPolicy, status, bulletins);
|
||||
})
|
||||
.collect(Collectors.toSet());
|
||||
});
|
||||
|
@ -2471,7 +2489,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(group.getIdentifier()));
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(group);
|
||||
final ProcessGroupStatusDTO status = dtoFactory.createConciseProcessGroupStatusDto(controllerFacade.getProcessGroupStatus(group.getIdentifier()));
|
||||
return entityFactory.createProcessGroupEntity(dtoFactory.createProcessGroupDto(group), revision, accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(group.getIdentifier()));
|
||||
return entityFactory.createProcessGroupEntity(dtoFactory.createProcessGroupDto(group), revision, accessPolicy, status, bulletins);
|
||||
})
|
||||
.collect(Collectors.toSet());
|
||||
});
|
||||
|
@ -2491,7 +2510,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(rpg.getIdentifier()));
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(rpg);
|
||||
final RemoteProcessGroupStatusDTO status = dtoFactory.createRemoteProcessGroupStatusDto(controllerFacade.getRemoteProcessGroupStatus(rpg.getIdentifier()));
|
||||
return entityFactory.createRemoteProcessGroupEntity(dtoFactory.createRemoteProcessGroupDto(rpg), revision, accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(rpg.getIdentifier()));
|
||||
return entityFactory.createRemoteProcessGroupEntity(dtoFactory.createRemoteProcessGroupDto(rpg), revision, accessPolicy, status, bulletins);
|
||||
})
|
||||
.collect(Collectors.toSet());
|
||||
});
|
||||
|
@ -2506,7 +2526,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final RevisionDTO revision = dtoFactory.createRevisionDTO(rev);
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(port);
|
||||
final PortStatusDTO status = dtoFactory.createPortStatusDto(controllerFacade.getInputPortStatus(inputPortId));
|
||||
return entityFactory.createPortEntity(dtoFactory.createPortDto(port), revision, accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(inputPortId));
|
||||
return entityFactory.createPortEntity(dtoFactory.createPortDto(port), revision, accessPolicy, status, bulletins);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -2524,7 +2545,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final RevisionDTO revision = dtoFactory.createRevisionDTO(rev);
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(port);
|
||||
final PortStatusDTO status = dtoFactory.createPortStatusDto(controllerFacade.getOutputPortStatus(outputPortId));
|
||||
return entityFactory.createPortEntity(dtoFactory.createPortDto(port), revision, accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(outputPortId));
|
||||
return entityFactory.createPortEntity(dtoFactory.createPortDto(port), revision, accessPolicy, status, bulletins);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -2542,7 +2564,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final RevisionDTO revision = dtoFactory.createRevisionDTO(rev);
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(rpg);
|
||||
final RemoteProcessGroupStatusDTO status = dtoFactory.createRemoteProcessGroupStatusDto(controllerFacade.getRemoteProcessGroupStatus(rpg.getIdentifier()));
|
||||
return entityFactory.createRemoteProcessGroupEntity(dtoFactory.createRemoteProcessGroupDto(rpg), revision, accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(rpg.getIdentifier()));
|
||||
return entityFactory.createRemoteProcessGroupEntity(dtoFactory.createRemoteProcessGroupDto(rpg), revision, accessPolicy, status, bulletins);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -2609,7 +2632,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final RevisionDTO revision = dtoFactory.createRevisionDTO(rev);
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(processGroup);
|
||||
final ProcessGroupStatusDTO status = dtoFactory.createConciseProcessGroupStatusDto(controllerFacade.getProcessGroupStatus(groupId));
|
||||
return entityFactory.createProcessGroupEntity(dtoFactory.createProcessGroupDto(processGroup), revision, accessPolicy, status);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(groupId));
|
||||
return entityFactory.createProcessGroupEntity(dtoFactory.createProcessGroupDto(processGroup), revision, accessPolicy, status, bulletins);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -2641,7 +2665,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
|
||||
final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(serviceNode.getIdentifier()));
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(serviceNode);
|
||||
return entityFactory.createControllerServiceEntity(dto, revision, accessPolicy);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(serviceNode.getIdentifier()));
|
||||
return entityFactory.createControllerServiceEntity(dto, revision, accessPolicy, bulletins);
|
||||
})
|
||||
.collect(Collectors.toSet());
|
||||
});
|
||||
|
@ -2661,7 +2686,9 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
final ControllerServiceReferencingComponentsEntity referencingComponentsEntity = createControllerServiceReferencingComponentsEntity(ref, Sets.newHashSet(controllerServiceId));
|
||||
dto.setReferencingComponents(referencingComponentsEntity.getControllerServiceReferencingComponents());
|
||||
|
||||
return entityFactory.createControllerServiceEntity(dto, revision, accessPolicy);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(controllerServiceId));
|
||||
|
||||
return entityFactory.createControllerServiceEntity(dto, revision, accessPolicy, bulletins);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -2699,7 +2726,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
.map(reportingTask -> {
|
||||
final RevisionDTO revision = dtoFactory.createRevisionDTO(revisionManager.getRevision(reportingTask.getIdentifier()));
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(reportingTask);
|
||||
return entityFactory.createReportingTaskEntity(dtoFactory.createReportingTaskDto(reportingTask), revision, accessPolicy);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(reportingTask.getIdentifier()));
|
||||
return entityFactory.createReportingTaskEntity(dtoFactory.createReportingTaskDto(reportingTask), revision, accessPolicy, bulletins);
|
||||
})
|
||||
.collect(Collectors.toSet());
|
||||
});
|
||||
|
@ -2713,7 +2741,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
|
||||
final RevisionDTO revision = dtoFactory.createRevisionDTO(rev);
|
||||
final AccessPolicyDTO accessPolicy = dtoFactory.createAccessPolicyDto(reportingTask);
|
||||
return entityFactory.createReportingTaskEntity(dtoFactory.createReportingTaskDto(reportingTask), revision, accessPolicy);
|
||||
final List<BulletinDTO> bulletins = dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForSource(reportingTaskId));
|
||||
return entityFactory.createReportingTaskEntity(dtoFactory.createReportingTaskDto(reportingTask), revision, accessPolicy, bulletins);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -2964,4 +2993,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
public void setHeartbeatMonitor(HeartbeatMonitor heartbeatMonitor) {
|
||||
this.heartbeatMonitor = heartbeatMonitor;
|
||||
}
|
||||
|
||||
public void setBulletinRepository(BulletinRepository bulletinRepository) {
|
||||
this.bulletinRepository = bulletinRepository;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -98,6 +98,7 @@ import org.apache.nifi.provenance.lineage.ProvenanceEventLineageNode;
|
|||
import org.apache.nifi.remote.RemoteGroupPort;
|
||||
import org.apache.nifi.remote.RootGroupPort;
|
||||
import org.apache.nifi.reporting.Bulletin;
|
||||
import org.apache.nifi.reporting.BulletinRepository;
|
||||
import org.apache.nifi.reporting.ReportingTask;
|
||||
import org.apache.nifi.scheduling.SchedulingStrategy;
|
||||
import org.apache.nifi.util.FormatUtils;
|
||||
|
@ -174,6 +175,7 @@ public final class DtoFactory {
|
|||
};
|
||||
public static final String SENSITIVE_VALUE_MASK = "********";
|
||||
|
||||
private BulletinRepository bulletinRepository;
|
||||
private ControllerServiceProvider controllerServiceProvider;
|
||||
private EntityFactory entityFactory;
|
||||
private Authorizer authorizer;
|
||||
|
@ -777,7 +779,6 @@ public final class DtoFactory {
|
|||
snapshot.setBytesSent(remoteProcessGroupStatus.getSentContentSize());
|
||||
snapshot.setFlowFilesReceived(remoteProcessGroupStatus.getReceivedCount());
|
||||
snapshot.setBytesReceived(remoteProcessGroupStatus.getReceivedContentSize());
|
||||
snapshot.setAuthorizationIssues(remoteProcessGroupStatus.getAuthorizationIssues());
|
||||
|
||||
StatusMerger.updatePrettyPrintedFields(snapshot);
|
||||
return dto;
|
||||
|
@ -1535,7 +1536,8 @@ public final class DtoFactory {
|
|||
() -> groupStatus.getInputPortStatus().stream().filter(inputPortStatus -> inputPort.getId().equals(inputPortStatus.getId())).findFirst().orElse(null),
|
||||
inputPortStatus -> createPortStatusDto(inputPortStatus)
|
||||
);
|
||||
flow.getInputPorts().add(entityFactory.createPortEntity(inputPort, revision, accessPolicy, status));
|
||||
final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(inputPort.getId()));
|
||||
flow.getInputPorts().add(entityFactory.createPortEntity(inputPort, revision, accessPolicy, status, bulletins));
|
||||
}
|
||||
|
||||
for (final PortDTO outputPort : snippet.getOutputPorts()) {
|
||||
|
@ -1545,7 +1547,8 @@ public final class DtoFactory {
|
|||
() -> groupStatus.getOutputPortStatus().stream().filter(outputPortStatus -> outputPort.getId().equals(outputPortStatus.getId())).findFirst().orElse(null),
|
||||
outputPortStatus -> createPortStatusDto(outputPortStatus)
|
||||
);
|
||||
flow.getOutputPorts().add(entityFactory.createPortEntity(outputPort, revision, accessPolicy, status));
|
||||
final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(outputPort.getId()));
|
||||
flow.getOutputPorts().add(entityFactory.createPortEntity(outputPort, revision, accessPolicy, status, bulletins));
|
||||
}
|
||||
|
||||
for (final LabelDTO label : snippet.getLabels()) {
|
||||
|
@ -1561,7 +1564,8 @@ public final class DtoFactory {
|
|||
() -> groupStatus.getProcessGroupStatus().stream().filter(processGroupStatus -> processGroup.getId().equals(processGroupStatus.getId())).findFirst().orElse(null),
|
||||
processGroupStatus -> createConciseProcessGroupStatusDto(processGroupStatus)
|
||||
);
|
||||
flow.getProcessGroups().add(entityFactory.createProcessGroupEntity(processGroup, revision, accessPolicy, status));
|
||||
final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(processGroup.getId()));
|
||||
flow.getProcessGroups().add(entityFactory.createProcessGroupEntity(processGroup, revision, accessPolicy, status, bulletins));
|
||||
}
|
||||
|
||||
for (final ProcessorDTO processor : snippet.getProcessors()) {
|
||||
|
@ -1571,7 +1575,8 @@ public final class DtoFactory {
|
|||
() -> groupStatus.getProcessorStatus().stream().filter(processorStatus -> processor.getId().equals(processorStatus.getId())).findFirst().orElse(null),
|
||||
processorStatus -> createProcessorStatusDto(processorStatus)
|
||||
);
|
||||
flow.getProcessors().add(entityFactory.createProcessorEntity(processor, revision, accessPolicy, status));
|
||||
final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(processor.getId()));
|
||||
flow.getProcessors().add(entityFactory.createProcessorEntity(processor, revision, accessPolicy, status, bulletins));
|
||||
}
|
||||
|
||||
for (final RemoteProcessGroupDTO remoteProcessGroup : snippet.getRemoteProcessGroups()) {
|
||||
|
@ -1581,7 +1586,8 @@ public final class DtoFactory {
|
|||
() -> groupStatus.getRemoteProcessGroupStatus().stream().filter(rpgStatus -> remoteProcessGroup.getId().equals(rpgStatus.getId())).findFirst().orElse(null),
|
||||
remoteProcessGroupStatus -> createRemoteProcessGroupStatusDto(remoteProcessGroupStatus)
|
||||
);
|
||||
flow.getRemoteProcessGroups().add(entityFactory.createRemoteProcessGroupEntity(remoteProcessGroup, revision, accessPolicy, status));
|
||||
final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(remoteProcessGroup.getId()));
|
||||
flow.getRemoteProcessGroups().add(entityFactory.createRemoteProcessGroupEntity(remoteProcessGroup, revision, accessPolicy, status, bulletins));
|
||||
}
|
||||
|
||||
return flow;
|
||||
|
@ -1608,7 +1614,8 @@ public final class DtoFactory {
|
|||
() -> groupStatus.getProcessorStatus().stream().filter(processorStatus -> procNode.getIdentifier().equals(processorStatus.getId())).findFirst().orElse(null),
|
||||
processorStatus -> createProcessorStatusDto(processorStatus)
|
||||
);
|
||||
dto.getProcessors().add(entityFactory.createProcessorEntity(createProcessorDto(procNode), revision, accessPolicy, status));
|
||||
final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(procNode.getIdentifier()));
|
||||
dto.getProcessors().add(entityFactory.createProcessorEntity(createProcessorDto(procNode), revision, accessPolicy, status, bulletins));
|
||||
}
|
||||
|
||||
for (final Connection connNode : group.getConnections()) {
|
||||
|
@ -1640,7 +1647,8 @@ public final class DtoFactory {
|
|||
() -> groupStatus.getProcessGroupStatus().stream().filter(processGroupStatus -> childGroup.getIdentifier().equals(processGroupStatus.getId())).findFirst().orElse(null),
|
||||
processGroupStatus -> createConciseProcessGroupStatusDto(processGroupStatus)
|
||||
);
|
||||
dto.getProcessGroups().add(entityFactory.createProcessGroupEntity(createProcessGroupDto(childGroup), revision, accessPolicy, status));
|
||||
final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(childGroup.getIdentifier()));
|
||||
dto.getProcessGroups().add(entityFactory.createProcessGroupEntity(createProcessGroupDto(childGroup), revision, accessPolicy, status, bulletins));
|
||||
}
|
||||
|
||||
for (final RemoteProcessGroup rpg : group.getRemoteProcessGroups()) {
|
||||
|
@ -1650,7 +1658,8 @@ public final class DtoFactory {
|
|||
() -> groupStatus.getRemoteProcessGroupStatus().stream().filter(remoteProcessGroupStatus -> rpg.getIdentifier().equals(remoteProcessGroupStatus.getId())).findFirst().orElse(null),
|
||||
remoteProcessGroupStatus -> createRemoteProcessGroupStatusDto(remoteProcessGroupStatus)
|
||||
);
|
||||
dto.getRemoteProcessGroups().add(entityFactory.createRemoteProcessGroupEntity(createRemoteProcessGroupDto(rpg), revision, accessPolicy, status));
|
||||
final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(rpg.getIdentifier()));
|
||||
dto.getRemoteProcessGroups().add(entityFactory.createRemoteProcessGroupEntity(createRemoteProcessGroupDto(rpg), revision, accessPolicy, status, bulletins));
|
||||
}
|
||||
|
||||
for (final Port inputPort : group.getInputPorts()) {
|
||||
|
@ -1660,7 +1669,8 @@ public final class DtoFactory {
|
|||
() -> groupStatus.getInputPortStatus().stream().filter(inputPortStatus -> inputPort.getIdentifier().equals(inputPortStatus.getId())).findFirst().orElse(null),
|
||||
inputPortStatus -> createPortStatusDto(inputPortStatus)
|
||||
);
|
||||
dto.getInputPorts().add(entityFactory.createPortEntity(createPortDto(inputPort), revision, accessPolicy, status));
|
||||
final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(inputPort.getIdentifier()));
|
||||
dto.getInputPorts().add(entityFactory.createPortEntity(createPortDto(inputPort), revision, accessPolicy, status, bulletins));
|
||||
}
|
||||
|
||||
for (final Port outputPort : group.getOutputPorts()) {
|
||||
|
@ -1670,7 +1680,8 @@ public final class DtoFactory {
|
|||
() -> groupStatus.getOutputPortStatus().stream().filter(outputPortStatus -> outputPort.getIdentifier().equals(outputPortStatus.getId())).findFirst().orElse(null),
|
||||
outputPortStatus -> createPortStatusDto(outputPortStatus)
|
||||
);
|
||||
dto.getOutputPorts().add(entityFactory.createPortEntity(createPortDto(outputPort), revision, accessPolicy, status));
|
||||
final List<BulletinDTO> bulletins = createBulletinDtos(bulletinRepository.findBulletinsForSource(outputPort.getIdentifier()));
|
||||
dto.getOutputPorts().add(entityFactory.createPortEntity(createPortDto(outputPort), revision, accessPolicy, status, bulletins));
|
||||
}
|
||||
|
||||
return dto;
|
||||
|
@ -2868,4 +2879,8 @@ public final class DtoFactory {
|
|||
public void setEntityFactory(EntityFactory entityFactory) {
|
||||
this.entityFactory = entityFactory;
|
||||
}
|
||||
|
||||
public void setBulletinRepository(BulletinRepository bulletinRepository) {
|
||||
this.bulletinRepository = bulletinRepository;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,6 +39,8 @@ import org.apache.nifi.web.api.entity.RemoteProcessGroupPortEntity;
|
|||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||
import org.apache.nifi.web.api.entity.SnippetEntity;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public final class EntityFactory {
|
||||
|
||||
public ControllerConfigurationEntity createControllerConfigurationEntity(final ControllerConfigurationDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy) {
|
||||
|
@ -61,7 +63,9 @@ public final class EntityFactory {
|
|||
return entity;
|
||||
}
|
||||
|
||||
public ProcessorEntity createProcessorEntity(final ProcessorDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy, final ProcessorStatusDTO status) {
|
||||
public ProcessorEntity createProcessorEntity(final ProcessorDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy,
|
||||
final ProcessorStatusDTO status, final List<BulletinDTO> bulletins) {
|
||||
|
||||
final ProcessorEntity entity = new ProcessorEntity();
|
||||
entity.setRevision(revision);
|
||||
if (dto != null) {
|
||||
|
@ -71,12 +75,13 @@ public final class EntityFactory {
|
|||
entity.setPosition(dto.getPosition());
|
||||
if (accessPolicy != null && accessPolicy.getCanRead()) {
|
||||
entity.setComponent(dto);
|
||||
entity.setBulletins(bulletins);
|
||||
}
|
||||
}
|
||||
return entity;
|
||||
}
|
||||
|
||||
public PortEntity createPortEntity(final PortDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy, final PortStatusDTO status) {
|
||||
public PortEntity createPortEntity(final PortDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy, final PortStatusDTO status, final List<BulletinDTO> bulletins) {
|
||||
final PortEntity entity = new PortEntity();
|
||||
entity.setRevision(revision);
|
||||
if (dto != null) {
|
||||
|
@ -87,12 +92,15 @@ public final class EntityFactory {
|
|||
entity.setPortType(dto.getType());
|
||||
if (accessPolicy != null && accessPolicy.getCanRead()) {
|
||||
entity.setComponent(dto);
|
||||
entity.setBulletins(bulletins);
|
||||
}
|
||||
}
|
||||
return entity;
|
||||
}
|
||||
|
||||
public ProcessGroupEntity createProcessGroupEntity(final ProcessGroupDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy, final ProcessGroupStatusDTO status) {
|
||||
public ProcessGroupEntity createProcessGroupEntity(final ProcessGroupDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy,
|
||||
final ProcessGroupStatusDTO status, final List<BulletinDTO> bulletins) {
|
||||
|
||||
final ProcessGroupEntity entity = new ProcessGroupEntity();
|
||||
entity.setRevision(revision);
|
||||
if (dto != null) {
|
||||
|
@ -110,6 +118,7 @@ public final class EntityFactory {
|
|||
entity.setInactiveRemotePortCount(dto.getInactiveRemotePortCount());
|
||||
if (accessPolicy != null && accessPolicy.getCanRead()) {
|
||||
entity.setComponent(dto);
|
||||
entity.setBulletins(bulletins);
|
||||
}
|
||||
}
|
||||
return entity;
|
||||
|
@ -170,8 +179,8 @@ public final class EntityFactory {
|
|||
return entity;
|
||||
}
|
||||
|
||||
public RemoteProcessGroupEntity createRemoteProcessGroupEntity(
|
||||
final RemoteProcessGroupDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy,final RemoteProcessGroupStatusDTO status) {
|
||||
public RemoteProcessGroupEntity createRemoteProcessGroupEntity(final RemoteProcessGroupDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy,
|
||||
final RemoteProcessGroupStatusDTO status, final List<BulletinDTO> bulletins) {
|
||||
|
||||
final RemoteProcessGroupEntity entity = new RemoteProcessGroupEntity();
|
||||
entity.setRevision(revision);
|
||||
|
@ -184,6 +193,7 @@ public final class EntityFactory {
|
|||
entity.setOutputPortCount(dto.getOutputPortCount());
|
||||
if (accessPolicy != null && accessPolicy.getCanRead()) {
|
||||
entity.setComponent(dto);
|
||||
entity.setBulletins(bulletins);
|
||||
}
|
||||
}
|
||||
return entity;
|
||||
|
@ -209,7 +219,7 @@ public final class EntityFactory {
|
|||
return entity;
|
||||
}
|
||||
|
||||
public ReportingTaskEntity createReportingTaskEntity(final ReportingTaskDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy) {
|
||||
public ReportingTaskEntity createReportingTaskEntity(final ReportingTaskDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy, final List<BulletinDTO> bulletins) {
|
||||
final ReportingTaskEntity entity = new ReportingTaskEntity();
|
||||
entity.setRevision(revision);
|
||||
if (dto != null) {
|
||||
|
@ -217,13 +227,14 @@ public final class EntityFactory {
|
|||
entity.setId(dto.getId());
|
||||
if (accessPolicy != null && accessPolicy.getCanRead()) {
|
||||
entity.setComponent(dto);
|
||||
entity.setBulletins(bulletins);
|
||||
}
|
||||
}
|
||||
|
||||
return entity;
|
||||
}
|
||||
|
||||
public ControllerServiceEntity createControllerServiceEntity(final ControllerServiceDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy) {
|
||||
public ControllerServiceEntity createControllerServiceEntity(final ControllerServiceDTO dto, final RevisionDTO revision, final AccessPolicyDTO accessPolicy, final List<BulletinDTO> bulletins) {
|
||||
final ControllerServiceEntity entity = new ControllerServiceEntity();
|
||||
entity.setRevision(revision);
|
||||
if (dto != null) {
|
||||
|
@ -232,6 +243,7 @@ public final class EntityFactory {
|
|||
entity.setPosition(dto.getPosition());
|
||||
if (accessPolicy != null && accessPolicy.getCanRead()) {
|
||||
entity.setComponent(dto);
|
||||
entity.setBulletins(bulletins);
|
||||
}
|
||||
}
|
||||
return entity;
|
||||
|
|
|
@ -16,27 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.web.controller;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.text.Collator;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TimeZone;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import javax.ws.rs.WebApplicationException;
|
||||
|
||||
import org.apache.commons.collections4.CollectionUtils;
|
||||
import org.apache.commons.lang3.ClassUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
@ -131,6 +110,26 @@ import org.apache.nifi.web.security.ProxiedEntitiesUtils;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import javax.ws.rs.WebApplicationException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.text.Collator;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TimeZone;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class ControllerFacade implements Authorizable {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(ControllerFacade.class);
|
||||
|
@ -140,6 +139,7 @@ public class ControllerFacade implements Authorizable {
|
|||
private FlowService flowService;
|
||||
private KeyService keyService;
|
||||
private ClusterCoordinator clusterCoordinator;
|
||||
private BulletinRepository bulletinRepository;
|
||||
|
||||
// properties
|
||||
private NiFiProperties properties;
|
||||
|
@ -479,7 +479,6 @@ public class ControllerFacade implements Authorizable {
|
|||
controllerStatus.setConnectedNodes(connectedNodeCount + " / " + totalNodeCount);
|
||||
}
|
||||
|
||||
final BulletinRepository bulletinRepository = getBulletinRepository();
|
||||
controllerStatus.setBulletins(dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForController()));
|
||||
|
||||
// get the controller service bulletins
|
||||
|
@ -662,15 +661,6 @@ public class ControllerFacade implements Authorizable {
|
|||
return status;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the BulletinRepository.
|
||||
*
|
||||
* @return the BulletinRepository
|
||||
*/
|
||||
public BulletinRepository getBulletinRepository() {
|
||||
return flowController.getBulletinRepository();
|
||||
}
|
||||
|
||||
/**
|
||||
* Saves the state of the flow controller.
|
||||
*
|
||||
|
@ -1706,4 +1696,8 @@ public class ControllerFacade implements Authorizable {
|
|||
public void setClusterCoordinator(ClusterCoordinator clusterCoordinator) {
|
||||
this.clusterCoordinator = clusterCoordinator;
|
||||
}
|
||||
|
||||
public void setBulletinRepository(BulletinRepository bulletinRepository) {
|
||||
this.bulletinRepository = bulletinRepository;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -52,6 +52,7 @@
|
|||
<property name="controllerServiceProvider" ref="controllerServiceProvider" />
|
||||
<property name="entityFactory" ref="entityFactory"/>
|
||||
<property name="authorizer" ref="authorizer"/>
|
||||
<property name="bulletinRepository" ref="bulletinRepository"/>
|
||||
</bean>
|
||||
|
||||
<!-- snippet utils -->
|
||||
|
@ -113,6 +114,7 @@
|
|||
<property name="clusterCoordinator" ref="clusterCoordinator" />
|
||||
<property name="keyService" ref="keyService"/>
|
||||
<property name="dtoFactory" ref="dtoFactory"/>
|
||||
<property name="bulletinRepository" ref="bulletinRepository"/>
|
||||
</bean>
|
||||
<bean id="serviceFacade" class="org.apache.nifi.web.StandardNiFiServiceFacade">
|
||||
<property name="properties" ref="nifiProperties"/>
|
||||
|
@ -138,6 +140,7 @@
|
|||
<property name="entityFactory" ref="entityFactory"/>
|
||||
<property name="clusterCoordinator" ref="clusterCoordinator"/>
|
||||
<property name="heartbeatMonitor" ref="heartbeatMonitor" />
|
||||
<property name="bulletinRepository" ref="bulletinRepository"/>
|
||||
</bean>
|
||||
|
||||
<!-- component ui extension configuration context -->
|
||||
|
|
|
@ -92,6 +92,10 @@
|
|||
background-color: #728E9B; /*base-color*/
|
||||
}
|
||||
|
||||
#bulletin-button.has-bulletins {
|
||||
background-color: #ba554a; /*warm-color*/
|
||||
}
|
||||
|
||||
#bulletin-button i.fa {
|
||||
color: #fff;
|
||||
font-size: 15px;
|
||||
|
|
|
@ -287,7 +287,15 @@ nf.ng.Canvas.FlowStatusCtrl = function (serviceProvider, $sanitize) {
|
|||
// no bulletins before, show icon and tips
|
||||
bulletinIcon.addClass('has-bulletins').qtip($.extend({
|
||||
content: newBulletins
|
||||
}, nf.CanvasUtils.config.systemTooltipConfig));
|
||||
}, nf.CanvasUtils.config.systemTooltipConfig, {
|
||||
position: {
|
||||
at: 'bottom left',
|
||||
my: 'top right',
|
||||
adjust: {
|
||||
x: 4
|
||||
}
|
||||
}
|
||||
}));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -110,10 +110,7 @@ nf.CanvasUtils = (function () {
|
|||
},
|
||||
position: {
|
||||
at: 'bottom right',
|
||||
my: 'top left',
|
||||
adjust: {
|
||||
method: 'flipinvert flipinvert'
|
||||
}
|
||||
my: 'top left'
|
||||
}
|
||||
}
|
||||
},
|
||||
|
@ -493,7 +490,7 @@ nf.CanvasUtils = (function () {
|
|||
}
|
||||
|
||||
// if there are bulletins show them, otherwise hide
|
||||
if (!nf.Common.isEmpty(d.status.bulletins)) {
|
||||
if (!nf.Common.isEmpty(d.bulletins)) {
|
||||
// update the tooltip
|
||||
selection.select('text.bulletin-icon')
|
||||
.each(function () {
|
||||
|
@ -505,7 +502,7 @@ nf.CanvasUtils = (function () {
|
|||
.attr('class', 'tooltip nifi-tooltip')
|
||||
.html(function () {
|
||||
// format the bulletins
|
||||
var bulletins = nf.Common.getFormattedBulletins(d.status.bulletins);
|
||||
var bulletins = nf.Common.getFormattedBulletins(d.bulletins);
|
||||
|
||||
// create the unordered list based off the formatted bulletins
|
||||
var list = nf.Common.formatUnorderedList(bulletins);
|
||||
|
@ -519,6 +516,12 @@ nf.CanvasUtils = (function () {
|
|||
// add the tooltip
|
||||
nf.CanvasUtils.canvasTooltip(tip, d3.select(this));
|
||||
});
|
||||
|
||||
// update the tooltip background
|
||||
selection.select('rect.bulletin-background').classed('has-bulletins', true);
|
||||
} else {
|
||||
// update the tooltip background
|
||||
selection.select('rect.bulletin-background').classed('has-bulletins', false);
|
||||
}
|
||||
},
|
||||
|
||||
|
|
|
@ -496,7 +496,7 @@ nf.ControllerServices = (function () {
|
|||
markup += '<img src="images/iconUsage.png" title="Usage" class="pointer controller-service-usage" style="margin-left: 6px; margin-top: 3px; float: left;" />';
|
||||
|
||||
var hasErrors = !nf.Common.isEmpty(dataContext.component.validationErrors);
|
||||
var hasBulletins = !nf.Common.isEmpty(dataContext.component.bulletins);
|
||||
var hasBulletins = !nf.Common.isEmpty(dataContext.bulletins);
|
||||
|
||||
if (hasErrors) {
|
||||
markup += '<img src="images/iconAlert.png" class="has-errors" style="margin-top: 4px; margin-left: 3px; float: left;" />';
|
||||
|
@ -689,7 +689,7 @@ nf.ControllerServices = (function () {
|
|||
var controllerServiceEntity = controllerServicesData.getItemById(taskId);
|
||||
|
||||
// format the tooltip
|
||||
var bulletins = nf.Common.getFormattedBulletins(controllerServiceEntity.component.bulletins);
|
||||
var bulletins = nf.Common.getFormattedBulletins(controllerServiceEntity.bulletins);
|
||||
var tooltip = nf.Common.formatUnorderedList(bulletins);
|
||||
|
||||
// show the tooltip
|
||||
|
|
|
@ -679,7 +679,7 @@ nf.ProcessGroup = (function () {
|
|||
'x': function () {
|
||||
return processGroupData.dimensions.width - 17;
|
||||
},
|
||||
'y': 50
|
||||
'y': 49
|
||||
})
|
||||
.text('\uf24a');
|
||||
}
|
||||
|
|
|
@ -467,9 +467,9 @@ nf.Processor = (function () {
|
|||
.attr({
|
||||
'class': 'bulletin-icon',
|
||||
'x': function (d) {
|
||||
return processorData.dimensions.width - 18;
|
||||
return processorData.dimensions.width - 17;
|
||||
},
|
||||
'y': 18
|
||||
'y': 17
|
||||
})
|
||||
.text('\uf24a');
|
||||
}
|
||||
|
|
|
@ -694,10 +694,10 @@ nf.RemoteProcessGroup = (function () {
|
|||
updated.select('text.remote-process-group-transmission-status')
|
||||
.text(function (d) {
|
||||
var icon = '';
|
||||
if (!nf.Common.isEmpty(d.status.aggregateSnapshot.authorizationIssues)) {
|
||||
icon = '\uf071';
|
||||
} else if (d.accessPolicy.canRead) {
|
||||
if (d.component.transmitting === true) {
|
||||
if (d.accessPolicy.canRead) {
|
||||
if (!nf.Common.isEmpty(d.component.authorizationIssues)) {
|
||||
icon = '\uf071';
|
||||
} else if (d.component.transmitting === true) {
|
||||
icon = '\uf140';
|
||||
} else {
|
||||
icon = '\ue80a';
|
||||
|
@ -707,15 +707,17 @@ nf.RemoteProcessGroup = (function () {
|
|||
})
|
||||
.attr('font-family', function (d) {
|
||||
var family = '';
|
||||
if (!nf.Common.isEmpty(d.status.aggregateSnapshot.authorizationIssues) || (d.accessPolicy.canRead && d.component.transmitting)) {
|
||||
family = 'FontAwesome';
|
||||
} else {
|
||||
family = 'flowfont';
|
||||
if (d.accessPolicy.canRead) {
|
||||
if (!nf.Common.isEmpty(d.component.authorizationIssues) || d.component.transmitting) {
|
||||
family = 'FontAwesome';
|
||||
} else {
|
||||
family = 'flowfont';
|
||||
}
|
||||
}
|
||||
return family;
|
||||
})
|
||||
.classed('has-authorization-errors', function (d) {
|
||||
return !nf.Common.isEmpty(d.status.aggregateSnapshot.authorizationIssues);
|
||||
return d.accessPolicy.canRead && !nf.Common.isEmpty(d.component.authorizationIssues);
|
||||
})
|
||||
.each(function (d) {
|
||||
// remove the existing tip if necessary
|
||||
|
@ -725,14 +727,14 @@ nf.RemoteProcessGroup = (function () {
|
|||
}
|
||||
|
||||
// if there are validation errors generate a tooltip
|
||||
if (!nf.Common.isEmpty(d.status.aggregateSnapshot.authorizationIssues)) {
|
||||
if (d.accessPolicy.canRead && !nf.Common.isEmpty(d.component.authorizationIssues)) {
|
||||
tip = d3.select('#remote-process-group-tooltips').append('div')
|
||||
.attr('id', function () {
|
||||
return 'authorization-issues-' + d.id;
|
||||
})
|
||||
.attr('class', 'tooltip nifi-tooltip')
|
||||
.html(function () {
|
||||
var list = nf.Common.formatUnorderedList(d.status.aggregateSnapshot.authorizationIssues);
|
||||
var list = nf.Common.formatUnorderedList(d.component.authorizationIssues);
|
||||
if (list === null || list.length === 0) {
|
||||
return '';
|
||||
} else {
|
||||
|
|
|
@ -582,7 +582,7 @@ nf.Settings = (function () {
|
|||
markup += '<img src="images/iconUsage.png" title="Usage" class="pointer reporting-task-usage" style="margin-left: 6px; margin-top: 3px;"/>';
|
||||
|
||||
var hasErrors = !nf.Common.isEmpty(dataContext.component.validationErrors);
|
||||
var hasBulletins = !nf.Common.isEmpty(dataContext.component.bulletins);
|
||||
var hasBulletins = !nf.Common.isEmpty(dataContext.bulletins);
|
||||
|
||||
if (hasErrors) {
|
||||
markup += '<img src="images/iconAlert.png" class="has-errors" style="margin-top: 4px; margin-left: 3px; float: left;" />';
|
||||
|
@ -775,7 +775,7 @@ nf.Settings = (function () {
|
|||
var reportingTaskEntity = reportingTasksData.getItemById(taskId);
|
||||
|
||||
// format the tooltip
|
||||
var bulletins = nf.Common.getFormattedBulletins(reportingTaskEntity.component.bulletins);
|
||||
var bulletins = nf.Common.getFormattedBulletins(reportingTaskEntity.bulletins);
|
||||
var tooltip = nf.Common.formatUnorderedList(bulletins);
|
||||
|
||||
// show the tooltip
|
||||
|
|
|
@ -1464,14 +1464,12 @@ nf.SummaryTable = (function () {
|
|||
// determine what to put in the mark up
|
||||
var transmissionClass = 'invalid';
|
||||
var transmissionLabel = 'Invalid';
|
||||
if (nf.Common.isEmpty(dataContext.authorizationIssues)) {
|
||||
if (value === 'Transmitting') {
|
||||
transmissionClass = 'transmitting';
|
||||
transmissionLabel = value;
|
||||
} else {
|
||||
transmissionClass = 'not-transmitting';
|
||||
transmissionLabel = 'Not Transmitting';
|
||||
}
|
||||
if (value === 'Transmitting') {
|
||||
transmissionClass = 'transmitting';
|
||||
transmissionLabel = value;
|
||||
} else {
|
||||
transmissionClass = 'not-transmitting';
|
||||
transmissionLabel = 'Not Transmitting';
|
||||
}
|
||||
|
||||
// generate the mark up
|
||||
|
@ -2423,8 +2421,7 @@ nf.SummaryTable = (function () {
|
|||
transmissionStatus: snapshot.transmissionStatus,
|
||||
sent: snapshot.sent,
|
||||
received: snapshot.received,
|
||||
activeThreadCount: snapshot.activeThreadCount,
|
||||
authorizationIssues: snapshot.authorizationIssues
|
||||
activeThreadCount: snapshot.activeThreadCount
|
||||
});
|
||||
});
|
||||
|
||||
|
|
Loading…
Reference in New Issue