mirror of https://github.com/apache/nifi.git
NIFI-4436:
- Added the import dialog for importing a versioned flow into a new process group. - Added the change version dialog for upgrading/downgrading a versioned flow.
This commit is contained in:
parent
6aa8b5c61c
commit
696d583b14
|
@ -32,7 +32,6 @@ public class FlowConfigurationDTO {
|
||||||
private Boolean supportsManagedAuthorizer;
|
private Boolean supportsManagedAuthorizer;
|
||||||
private Boolean supportsConfigurableAuthorizer;
|
private Boolean supportsConfigurableAuthorizer;
|
||||||
private Boolean supportsConfigurableUsersAndGroups;
|
private Boolean supportsConfigurableUsersAndGroups;
|
||||||
private Boolean supportsFlowVersioning;
|
|
||||||
private Long autoRefreshIntervalSeconds;
|
private Long autoRefreshIntervalSeconds;
|
||||||
|
|
||||||
private Date currentTime;
|
private Date currentTime;
|
||||||
|
@ -129,17 +128,4 @@ public class FlowConfigurationDTO {
|
||||||
this.timeOffset = timeOffset;
|
this.timeOffset = timeOffset;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return whether this NiFi is configured for support flow versioning
|
|
||||||
*/
|
|
||||||
@ApiModelProperty(
|
|
||||||
value = "Whether this NiFi supports flow versioning."
|
|
||||||
)
|
|
||||||
public Boolean getSupportsFlowVersioning() {
|
|
||||||
return supportsFlowVersioning;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setSupportsFlowVersioning(Boolean supportsFlowVersioning) {
|
|
||||||
this.supportsFlowVersioning = supportsFlowVersioning;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,6 +27,7 @@ public class VersionControlInformationDTO {
|
||||||
private String registryId;
|
private String registryId;
|
||||||
private String bucketId;
|
private String bucketId;
|
||||||
private String flowId;
|
private String flowId;
|
||||||
|
private String flowName;
|
||||||
private Integer version;
|
private Integer version;
|
||||||
private Boolean modified;
|
private Boolean modified;
|
||||||
private Boolean current;
|
private Boolean current;
|
||||||
|
@ -67,6 +68,15 @@ public class VersionControlInformationDTO {
|
||||||
this.flowId = flowId;
|
this.flowId = flowId;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ApiModelProperty("The name of the flow")
|
||||||
|
public String getFlowName() {
|
||||||
|
return flowName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setFlowName(String flowName) {
|
||||||
|
this.flowName = flowName;
|
||||||
|
}
|
||||||
|
|
||||||
@ApiModelProperty("The version of the flow")
|
@ApiModelProperty("The version of the flow")
|
||||||
public Integer getVersion() {
|
public Integer getVersion() {
|
||||||
return version;
|
return version;
|
||||||
|
|
|
@ -38,6 +38,8 @@ public class CurrentUserEntity extends Entity {
|
||||||
private PermissionsDTO systemPermissions;
|
private PermissionsDTO systemPermissions;
|
||||||
private PermissionsDTO restrictedComponentsPermissions;
|
private PermissionsDTO restrictedComponentsPermissions;
|
||||||
|
|
||||||
|
private boolean canVersionFlows;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return the user identity being serialized
|
* @return the user identity being serialized
|
||||||
*/
|
*/
|
||||||
|
@ -145,4 +147,16 @@ public class CurrentUserEntity extends Entity {
|
||||||
public void setRestrictedComponentsPermissions(PermissionsDTO restrictedComponentsPermissions) {
|
public void setRestrictedComponentsPermissions(PermissionsDTO restrictedComponentsPermissions) {
|
||||||
this.restrictedComponentsPermissions = restrictedComponentsPermissions;
|
this.restrictedComponentsPermissions = restrictedComponentsPermissions;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return whether the current user can version flows
|
||||||
|
*/
|
||||||
|
@ApiModelProperty("Whether the current user can version flows.")
|
||||||
|
public boolean isCanVersionFlows() {
|
||||||
|
return canVersionFlows;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setCanVersionFlows(boolean canVersionFlows) {
|
||||||
|
this.canVersionFlows = canVersionFlows;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,48 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.nifi.web.api.entity;
|
||||||
|
|
||||||
|
import io.swagger.annotations.ApiModelProperty;
|
||||||
|
import org.apache.nifi.web.api.dto.RevisionDTO;
|
||||||
|
import org.apache.nifi.web.api.dto.VersionedFlowDTO;
|
||||||
|
|
||||||
|
import javax.xml.bind.annotation.XmlRootElement;
|
||||||
|
|
||||||
|
@XmlRootElement(name = "versionedFlow")
|
||||||
|
public class StartVersionControlRequestEntity extends Entity {
|
||||||
|
private VersionedFlowDTO versionedFlow;
|
||||||
|
private RevisionDTO processGroupRevision;
|
||||||
|
|
||||||
|
@ApiModelProperty("The versioned flow")
|
||||||
|
public VersionedFlowDTO getVersionedFlow() {
|
||||||
|
return versionedFlow;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setVersionedFlow(VersionedFlowDTO versionedFLow) {
|
||||||
|
this.versionedFlow = versionedFLow;
|
||||||
|
}
|
||||||
|
|
||||||
|
@ApiModelProperty("The Revision of the Process Group under Version Control")
|
||||||
|
public RevisionDTO getProcessGroupRevision() {
|
||||||
|
return processGroupRevision;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setProcessGroupRevision(final RevisionDTO revision) {
|
||||||
|
this.processGroupRevision = revision;
|
||||||
|
}
|
||||||
|
}
|
|
@ -18,7 +18,6 @@
|
||||||
package org.apache.nifi.web.api.entity;
|
package org.apache.nifi.web.api.entity;
|
||||||
|
|
||||||
import io.swagger.annotations.ApiModelProperty;
|
import io.swagger.annotations.ApiModelProperty;
|
||||||
import org.apache.nifi.web.api.dto.RevisionDTO;
|
|
||||||
import org.apache.nifi.web.api.dto.VersionedFlowDTO;
|
import org.apache.nifi.web.api.dto.VersionedFlowDTO;
|
||||||
|
|
||||||
import javax.xml.bind.annotation.XmlRootElement;
|
import javax.xml.bind.annotation.XmlRootElement;
|
||||||
|
@ -26,7 +25,6 @@ import javax.xml.bind.annotation.XmlRootElement;
|
||||||
@XmlRootElement(name = "versionedFlow")
|
@XmlRootElement(name = "versionedFlow")
|
||||||
public class VersionedFlowEntity extends Entity {
|
public class VersionedFlowEntity extends Entity {
|
||||||
private VersionedFlowDTO versionedFlow;
|
private VersionedFlowDTO versionedFlow;
|
||||||
private RevisionDTO processGroupRevision;
|
|
||||||
|
|
||||||
@ApiModelProperty("The versioned flow")
|
@ApiModelProperty("The versioned flow")
|
||||||
public VersionedFlowDTO getVersionedFlow() {
|
public VersionedFlowDTO getVersionedFlow() {
|
||||||
|
@ -36,13 +34,4 @@ public class VersionedFlowEntity extends Entity {
|
||||||
public void setVersionedFlow(VersionedFlowDTO versionedFLow) {
|
public void setVersionedFlow(VersionedFlowDTO versionedFLow) {
|
||||||
this.versionedFlow = versionedFLow;
|
this.versionedFlow = versionedFLow;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ApiModelProperty("The Revision of the Process Group under Version Control")
|
|
||||||
public RevisionDTO getProcessGroupRevision() {
|
|
||||||
return processGroupRevision;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setProcessGroupRevision(final RevisionDTO revision) {
|
|
||||||
this.processGroupRevision = revision;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,47 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.nifi.web.api.entity;
|
||||||
|
|
||||||
|
import io.swagger.annotations.ApiModelProperty;
|
||||||
|
import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
|
||||||
|
|
||||||
|
import javax.xml.bind.annotation.XmlRootElement;
|
||||||
|
|
||||||
|
@XmlRootElement(name = "versionedFlowSnapshotMetadata")
|
||||||
|
public class VersionedFlowSnapshotMetadataEntity extends Entity {
|
||||||
|
private VersionedFlowSnapshotMetadata versionedFlowSnapshotMetadata;
|
||||||
|
private String registryId;
|
||||||
|
|
||||||
|
@ApiModelProperty("The collection of versioned flow snapshot metadata")
|
||||||
|
public VersionedFlowSnapshotMetadata getVersionedFlowSnapshotMetadata() {
|
||||||
|
return versionedFlowSnapshotMetadata;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setVersionedFlowMetadata(VersionedFlowSnapshotMetadata versionedFlowSnapshotMetadata) {
|
||||||
|
this.versionedFlowSnapshotMetadata = versionedFlowSnapshotMetadata;
|
||||||
|
}
|
||||||
|
|
||||||
|
@ApiModelProperty("The ID of the Registry that this flow belongs to")
|
||||||
|
public String getRegistryId() {
|
||||||
|
return registryId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setRegistryId(String registryId) {
|
||||||
|
this.registryId = registryId;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,38 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.nifi.web.api.entity;
|
||||||
|
|
||||||
|
import javax.xml.bind.annotation.XmlRootElement;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
@XmlRootElement(name = "versionedFlowSnapshotMetadataSetEntity")
|
||||||
|
public class VersionedFlowSnapshotMetadataSetEntity extends Entity {
|
||||||
|
|
||||||
|
private Set<VersionedFlowSnapshotMetadataEntity> versionedFlowSnapshotMetadataSet;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return collection of VersionedFlowSnapshotMetadataEntity's that are being serialized
|
||||||
|
*/
|
||||||
|
public Set<VersionedFlowSnapshotMetadataEntity> getVersionedFlowSnapshotMetadataSet() {
|
||||||
|
return versionedFlowSnapshotMetadataSet;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setVersionedFlowSnapshotMetadataSet(Set<VersionedFlowSnapshotMetadataEntity> versionedFlowSnapshotMetadataSet) {
|
||||||
|
this.versionedFlowSnapshotMetadataSet = versionedFlowSnapshotMetadataSet;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,38 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.nifi.web.api.entity;
|
||||||
|
|
||||||
|
import javax.xml.bind.annotation.XmlRootElement;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
@XmlRootElement(name = "versionedFlowsEntity")
|
||||||
|
public class VersionedFlowsEntity extends Entity {
|
||||||
|
|
||||||
|
private Set<VersionedFlowEntity> versionedFlows;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return collection of VersionedEntity's that are being serialized
|
||||||
|
*/
|
||||||
|
public Set<VersionedFlowEntity> getVersionedFlows() {
|
||||||
|
return versionedFlows;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setVersionedFlows(Set<VersionedFlowEntity> versionedFlows) {
|
||||||
|
this.versionedFlows = versionedFlows;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -77,6 +77,7 @@ import org.apache.nifi.web.api.entity.AccessPolicyEntity;
|
||||||
import org.apache.nifi.web.api.entity.ActionEntity;
|
import org.apache.nifi.web.api.entity.ActionEntity;
|
||||||
import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
|
import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
|
||||||
import org.apache.nifi.web.api.entity.AffectedComponentEntity;
|
import org.apache.nifi.web.api.entity.AffectedComponentEntity;
|
||||||
|
import org.apache.nifi.web.api.entity.BucketEntity;
|
||||||
import org.apache.nifi.web.api.entity.BulletinEntity;
|
import org.apache.nifi.web.api.entity.BulletinEntity;
|
||||||
import org.apache.nifi.web.api.entity.ConnectionEntity;
|
import org.apache.nifi.web.api.entity.ConnectionEntity;
|
||||||
import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
|
import org.apache.nifi.web.api.entity.ConnectionStatusEntity;
|
||||||
|
@ -103,6 +104,7 @@ import org.apache.nifi.web.api.entity.RemoteProcessGroupStatusEntity;
|
||||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||||
import org.apache.nifi.web.api.entity.ScheduleComponentsEntity;
|
import org.apache.nifi.web.api.entity.ScheduleComponentsEntity;
|
||||||
import org.apache.nifi.web.api.entity.SnippetEntity;
|
import org.apache.nifi.web.api.entity.SnippetEntity;
|
||||||
|
import org.apache.nifi.web.api.entity.StartVersionControlRequestEntity;
|
||||||
import org.apache.nifi.web.api.entity.StatusHistoryEntity;
|
import org.apache.nifi.web.api.entity.StatusHistoryEntity;
|
||||||
import org.apache.nifi.web.api.entity.TemplateEntity;
|
import org.apache.nifi.web.api.entity.TemplateEntity;
|
||||||
import org.apache.nifi.web.api.entity.UserEntity;
|
import org.apache.nifi.web.api.entity.UserEntity;
|
||||||
|
@ -111,6 +113,7 @@ import org.apache.nifi.web.api.entity.VariableRegistryEntity;
|
||||||
import org.apache.nifi.web.api.entity.VersionControlComponentMappingEntity;
|
import org.apache.nifi.web.api.entity.VersionControlComponentMappingEntity;
|
||||||
import org.apache.nifi.web.api.entity.VersionControlInformationEntity;
|
import org.apache.nifi.web.api.entity.VersionControlInformationEntity;
|
||||||
import org.apache.nifi.web.api.entity.VersionedFlowEntity;
|
import org.apache.nifi.web.api.entity.VersionedFlowEntity;
|
||||||
|
import org.apache.nifi.web.api.entity.VersionedFlowSnapshotMetadataEntity;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
|
@ -1300,7 +1303,7 @@ public interface NiFiServiceFacade {
|
||||||
* @return a VersionControlComponentMappingEntity that contains the information needed to notify a Process Group where it is tracking to and map
|
* @return a VersionControlComponentMappingEntity that contains the information needed to notify a Process Group where it is tracking to and map
|
||||||
* component ID's to their Versioned Component ID's
|
* component ID's to their Versioned Component ID's
|
||||||
*/
|
*/
|
||||||
VersionControlComponentMappingEntity registerFlowWithFlowRegistry(String groupId, VersionedFlowEntity requestEntity);
|
VersionControlComponentMappingEntity registerFlowWithFlowRegistry(String groupId, StartVersionControlRequestEntity requestEntity);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Adds the given snapshot to the already existing Versioned Flow, which resides in the given Flow Registry with the given id
|
* Adds the given snapshot to the already existing Versioned Flow, which resides in the given Flow Registry with the given id
|
||||||
|
@ -1854,7 +1857,7 @@ public interface NiFiServiceFacade {
|
||||||
* @param registryDTO The registry DTO
|
* @param registryDTO The registry DTO
|
||||||
* @return The reporting task DTO
|
* @return The reporting task DTO
|
||||||
*/
|
*/
|
||||||
RegistryEntity createRegistry(Revision revision, RegistryDTO registryDTO);
|
RegistryEntity createRegistryClient(Revision revision, RegistryDTO registryDTO);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets a registry with the specified id.
|
* Gets a registry with the specified id.
|
||||||
|
@ -1862,14 +1865,52 @@ public interface NiFiServiceFacade {
|
||||||
* @param registryId id
|
* @param registryId id
|
||||||
* @return entity
|
* @return entity
|
||||||
*/
|
*/
|
||||||
RegistryEntity getRegistry(String registryId);
|
RegistryEntity getRegistryClient(String registryId);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets all registries.
|
* Returns all registry clients.
|
||||||
*
|
*
|
||||||
|
* @return registry clients
|
||||||
|
*/
|
||||||
|
Set<RegistryEntity> getRegistryClients();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets all registries for the current user.
|
||||||
|
*
|
||||||
|
* @param user current user
|
||||||
* @return registries
|
* @return registries
|
||||||
*/
|
*/
|
||||||
Set<RegistryEntity> getRegistries();
|
Set<RegistryEntity> getRegistriesForUser(NiFiUser user);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets all buckets for a given registry.
|
||||||
|
*
|
||||||
|
* @param registryId registry id
|
||||||
|
* @param user current user
|
||||||
|
* @return the buckets
|
||||||
|
*/
|
||||||
|
Set<BucketEntity> getBucketsForUser(String registryId, NiFiUser user);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets the flows for the current user for the specified registry and bucket.
|
||||||
|
*
|
||||||
|
* @param registryId registry id
|
||||||
|
* @param bucketId bucket id
|
||||||
|
* @param user current user
|
||||||
|
* @return the flows
|
||||||
|
*/
|
||||||
|
Set<VersionedFlowEntity> getFlowsForUser(String registryId, String bucketId, NiFiUser user);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets the versions of the specified registry, bucket, and flow for the current user.
|
||||||
|
*
|
||||||
|
* @param registryId registry id
|
||||||
|
* @param bucketId bucket id
|
||||||
|
* @param flowId flow id
|
||||||
|
* @param user current user
|
||||||
|
* @return the versions of the flow
|
||||||
|
*/
|
||||||
|
Set<VersionedFlowSnapshotMetadataEntity> getFlowVersionsForUser(String registryId, String bucketId, String flowId, NiFiUser user);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Updates the specified registry using the specified revision.
|
* Updates the specified registry using the specified revision.
|
||||||
|
@ -1878,7 +1919,7 @@ public interface NiFiServiceFacade {
|
||||||
* @param registryDTO the registry dto
|
* @param registryDTO the registry dto
|
||||||
* @return the updated registry registry entity
|
* @return the updated registry registry entity
|
||||||
*/
|
*/
|
||||||
RegistryEntity updateRegistry(Revision revision, RegistryDTO registryDTO);
|
RegistryEntity updateRegistryClient(Revision revision, RegistryDTO registryDTO);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deletes the specified registry using the specified revision.
|
* Deletes the specified registry using the specified revision.
|
||||||
|
@ -1887,7 +1928,14 @@ public interface NiFiServiceFacade {
|
||||||
* @param registryId id
|
* @param registryId id
|
||||||
* @return the deleted registry entity
|
* @return the deleted registry entity
|
||||||
*/
|
*/
|
||||||
RegistryEntity deleteRegistry(Revision revision, String registryId);
|
RegistryEntity deleteRegistryClient(Revision revision, String registryId);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Verifies the specified registry can be removed.
|
||||||
|
*
|
||||||
|
* @param registryId the registry id
|
||||||
|
*/
|
||||||
|
void verifyDeleteRegistry(String registryId);
|
||||||
|
|
||||||
// ----------------------------------------
|
// ----------------------------------------
|
||||||
// History methods
|
// History methods
|
||||||
|
|
|
@ -16,32 +16,8 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.nifi.web;
|
package org.apache.nifi.web;
|
||||||
|
|
||||||
import java.io.IOException;
|
import com.google.common.collect.Sets;
|
||||||
import java.nio.charset.StandardCharsets;
|
import org.apache.commons.collections4.CollectionUtils;
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.Comparator;
|
|
||||||
import java.util.Date;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.LinkedHashMap;
|
|
||||||
import java.util.LinkedHashSet;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.ListIterator;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.UUID;
|
|
||||||
import java.util.function.Function;
|
|
||||||
import java.util.function.Supplier;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
import javax.ws.rs.WebApplicationException;
|
|
||||||
import javax.ws.rs.core.Response;
|
|
||||||
|
|
||||||
import org.apache.nifi.action.Action;
|
import org.apache.nifi.action.Action;
|
||||||
import org.apache.nifi.action.Component;
|
import org.apache.nifi.action.Component;
|
||||||
import org.apache.nifi.action.FlowChangeAction;
|
import org.apache.nifi.action.FlowChangeAction;
|
||||||
|
@ -111,15 +87,17 @@ import org.apache.nifi.history.History;
|
||||||
import org.apache.nifi.history.HistoryQuery;
|
import org.apache.nifi.history.HistoryQuery;
|
||||||
import org.apache.nifi.history.PreviousValue;
|
import org.apache.nifi.history.PreviousValue;
|
||||||
import org.apache.nifi.registry.ComponentVariableRegistry;
|
import org.apache.nifi.registry.ComponentVariableRegistry;
|
||||||
|
import org.apache.nifi.registry.bucket.Bucket;
|
||||||
import org.apache.nifi.registry.flow.FlowRegistry;
|
import org.apache.nifi.registry.flow.FlowRegistry;
|
||||||
import org.apache.nifi.registry.flow.FlowRegistryClient;
|
import org.apache.nifi.registry.flow.FlowRegistryClient;
|
||||||
import org.apache.nifi.registry.flow.VersionedFlowCoordinates;
|
|
||||||
import org.apache.nifi.registry.flow.UnknownResourceException;
|
import org.apache.nifi.registry.flow.UnknownResourceException;
|
||||||
import org.apache.nifi.registry.flow.VersionControlInformation;
|
import org.apache.nifi.registry.flow.VersionControlInformation;
|
||||||
import org.apache.nifi.registry.flow.VersionedComponent;
|
import org.apache.nifi.registry.flow.VersionedComponent;
|
||||||
import org.apache.nifi.registry.flow.VersionedConnection;
|
import org.apache.nifi.registry.flow.VersionedConnection;
|
||||||
import org.apache.nifi.registry.flow.VersionedFlow;
|
import org.apache.nifi.registry.flow.VersionedFlow;
|
||||||
|
import org.apache.nifi.registry.flow.VersionedFlowCoordinates;
|
||||||
import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
|
import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
|
||||||
|
import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
|
||||||
import org.apache.nifi.registry.flow.VersionedProcessGroup;
|
import org.apache.nifi.registry.flow.VersionedProcessGroup;
|
||||||
import org.apache.nifi.registry.flow.diff.ComparableDataFlow;
|
import org.apache.nifi.registry.flow.diff.ComparableDataFlow;
|
||||||
import org.apache.nifi.registry.flow.diff.DifferenceType;
|
import org.apache.nifi.registry.flow.diff.DifferenceType;
|
||||||
|
@ -144,6 +122,7 @@ import org.apache.nifi.util.NiFiProperties;
|
||||||
import org.apache.nifi.web.api.dto.AccessPolicyDTO;
|
import org.apache.nifi.web.api.dto.AccessPolicyDTO;
|
||||||
import org.apache.nifi.web.api.dto.AccessPolicySummaryDTO;
|
import org.apache.nifi.web.api.dto.AccessPolicySummaryDTO;
|
||||||
import org.apache.nifi.web.api.dto.AffectedComponentDTO;
|
import org.apache.nifi.web.api.dto.AffectedComponentDTO;
|
||||||
|
import org.apache.nifi.web.api.dto.BucketDTO;
|
||||||
import org.apache.nifi.web.api.dto.BulletinBoardDTO;
|
import org.apache.nifi.web.api.dto.BulletinBoardDTO;
|
||||||
import org.apache.nifi.web.api.dto.BulletinDTO;
|
import org.apache.nifi.web.api.dto.BulletinDTO;
|
||||||
import org.apache.nifi.web.api.dto.BulletinQueryDTO;
|
import org.apache.nifi.web.api.dto.BulletinQueryDTO;
|
||||||
|
@ -215,6 +194,7 @@ import org.apache.nifi.web.api.entity.AccessPolicySummaryEntity;
|
||||||
import org.apache.nifi.web.api.entity.ActionEntity;
|
import org.apache.nifi.web.api.entity.ActionEntity;
|
||||||
import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
|
import org.apache.nifi.web.api.entity.ActivateControllerServicesEntity;
|
||||||
import org.apache.nifi.web.api.entity.AffectedComponentEntity;
|
import org.apache.nifi.web.api.entity.AffectedComponentEntity;
|
||||||
|
import org.apache.nifi.web.api.entity.BucketEntity;
|
||||||
import org.apache.nifi.web.api.entity.BulletinEntity;
|
import org.apache.nifi.web.api.entity.BulletinEntity;
|
||||||
import org.apache.nifi.web.api.entity.ComponentReferenceEntity;
|
import org.apache.nifi.web.api.entity.ComponentReferenceEntity;
|
||||||
import org.apache.nifi.web.api.entity.ConnectionEntity;
|
import org.apache.nifi.web.api.entity.ConnectionEntity;
|
||||||
|
@ -253,7 +233,9 @@ import org.apache.nifi.web.api.entity.VariableEntity;
|
||||||
import org.apache.nifi.web.api.entity.VariableRegistryEntity;
|
import org.apache.nifi.web.api.entity.VariableRegistryEntity;
|
||||||
import org.apache.nifi.web.api.entity.VersionControlComponentMappingEntity;
|
import org.apache.nifi.web.api.entity.VersionControlComponentMappingEntity;
|
||||||
import org.apache.nifi.web.api.entity.VersionControlInformationEntity;
|
import org.apache.nifi.web.api.entity.VersionControlInformationEntity;
|
||||||
|
import org.apache.nifi.web.api.entity.StartVersionControlRequestEntity;
|
||||||
import org.apache.nifi.web.api.entity.VersionedFlowEntity;
|
import org.apache.nifi.web.api.entity.VersionedFlowEntity;
|
||||||
|
import org.apache.nifi.web.api.entity.VersionedFlowSnapshotMetadataEntity;
|
||||||
import org.apache.nifi.web.controller.ControllerFacade;
|
import org.apache.nifi.web.controller.ControllerFacade;
|
||||||
import org.apache.nifi.web.dao.AccessPolicyDAO;
|
import org.apache.nifi.web.dao.AccessPolicyDAO;
|
||||||
import org.apache.nifi.web.dao.ConnectionDAO;
|
import org.apache.nifi.web.dao.ConnectionDAO;
|
||||||
|
@ -282,7 +264,30 @@ import org.apache.nifi.web.util.SnippetUtils;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import com.google.common.collect.Sets;
|
import javax.ws.rs.WebApplicationException;
|
||||||
|
import javax.ws.rs.core.Response;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.Date;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
|
import java.util.LinkedHashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.ListIterator;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.UUID;
|
||||||
|
import java.util.function.Function;
|
||||||
|
import java.util.function.Supplier;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implementation of NiFiServiceFacade that performs revision checking.
|
* Implementation of NiFiServiceFacade that performs revision checking.
|
||||||
|
@ -2268,7 +2273,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public RegistryEntity createRegistry(Revision revision, RegistryDTO registryDTO) {
|
public RegistryEntity createRegistryClient(Revision revision, RegistryDTO registryDTO) {
|
||||||
final NiFiUser user = NiFiUserUtils.getNiFiUser();
|
final NiFiUser user = NiFiUserUtils.getNiFiUser();
|
||||||
|
|
||||||
// read lock on the containing group
|
// read lock on the containing group
|
||||||
|
@ -2292,7 +2297,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public RegistryEntity getRegistry(final String registryId) {
|
public RegistryEntity getRegistryClient(final String registryId) {
|
||||||
final FlowRegistry registry = registryDAO.getFlowRegistry(registryId);
|
final FlowRegistry registry = registryDAO.getFlowRegistry(registryId);
|
||||||
return createRegistryEntity(registry);
|
return createRegistryEntity(registry);
|
||||||
}
|
}
|
||||||
|
@ -2319,15 +2324,90 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
||||||
return entity;
|
return entity;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private BucketEntity createBucketEntity(final Bucket bucket) {
|
||||||
|
if (bucket == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
final BucketDTO dto = new BucketDTO();
|
||||||
|
dto.setId(bucket.getIdentifier());
|
||||||
|
dto.setName(bucket.getName());
|
||||||
|
dto.setDescription(bucket.getDescription());
|
||||||
|
dto.setCreated(bucket.getCreatedTimestamp());
|
||||||
|
|
||||||
|
final BucketEntity entity = new BucketEntity();
|
||||||
|
entity.setBucket(dto);
|
||||||
|
|
||||||
|
return entity;
|
||||||
|
}
|
||||||
|
|
||||||
|
private VersionedFlowEntity createVersionedFlowEntity(final String registryId, final VersionedFlow versionedFlow) {
|
||||||
|
if (versionedFlow == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
final VersionedFlowDTO dto = new VersionedFlowDTO();
|
||||||
|
dto.setRegistryId(registryId);
|
||||||
|
dto.setBucketId(versionedFlow.getBucketIdentifier());
|
||||||
|
dto.setFlowId(versionedFlow.getIdentifier());
|
||||||
|
dto.setFlowName(versionedFlow.getName());
|
||||||
|
dto.setDescription(versionedFlow.getDescription());
|
||||||
|
|
||||||
|
final VersionedFlowEntity entity = new VersionedFlowEntity();
|
||||||
|
entity.setVersionedFlow(dto);
|
||||||
|
|
||||||
|
return entity;
|
||||||
|
}
|
||||||
|
|
||||||
|
private VersionedFlowSnapshotMetadataEntity createVersionedFlowSnapshotMetadataEntity(final String registryId, final VersionedFlowSnapshotMetadata metadata) {
|
||||||
|
if (metadata == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
final VersionedFlowSnapshotMetadataEntity entity = new VersionedFlowSnapshotMetadataEntity();
|
||||||
|
entity.setRegistryId(registryId);
|
||||||
|
entity.setVersionedFlowMetadata(metadata);
|
||||||
|
|
||||||
|
return entity;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Set<RegistryEntity> getRegistries() {
|
public Set<RegistryEntity> getRegistryClients() {
|
||||||
return registryDAO.getFlowRegistries().stream()
|
return registryDAO.getFlowRegistries().stream()
|
||||||
.map(this::createRegistryEntity)
|
.map(this::createRegistryEntity)
|
||||||
.collect(Collectors.toSet());
|
.collect(Collectors.toSet());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public RegistryEntity updateRegistry(Revision revision, RegistryDTO registryDTO) {
|
public Set<RegistryEntity> getRegistriesForUser(final NiFiUser user) {
|
||||||
|
return registryDAO.getFlowRegistriesForUser(user).stream()
|
||||||
|
.map(this::createRegistryEntity)
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<BucketEntity> getBucketsForUser(final String registryId, final NiFiUser user) {
|
||||||
|
return registryDAO.getBucketsForUser(registryId, user).stream()
|
||||||
|
.map(this::createBucketEntity)
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<VersionedFlowEntity> getFlowsForUser(String registryId, String bucketId, NiFiUser user) {
|
||||||
|
return registryDAO.getFlowsForUser(registryId, bucketId, user).stream()
|
||||||
|
.map(vf -> createVersionedFlowEntity(registryId, vf))
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<VersionedFlowSnapshotMetadataEntity> getFlowVersionsForUser(String registryId, String bucketId, String flowId, NiFiUser user) {
|
||||||
|
return registryDAO.getFlowVersionsForUser(registryId, bucketId, flowId, user).stream()
|
||||||
|
.map(md -> createVersionedFlowSnapshotMetadataEntity(registryId, md))
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RegistryEntity updateRegistryClient(Revision revision, RegistryDTO registryDTO) {
|
||||||
final RevisionClaim revisionClaim = new StandardRevisionClaim(revision);
|
final RevisionClaim revisionClaim = new StandardRevisionClaim(revision);
|
||||||
final NiFiUser user = NiFiUserUtils.getNiFiUser();
|
final NiFiUser user = NiFiUserUtils.getNiFiUser();
|
||||||
|
|
||||||
|
@ -2350,7 +2430,12 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public RegistryEntity deleteRegistry(final Revision revision, final String registryId) {
|
public void verifyDeleteRegistry(String registryId) {
|
||||||
|
processGroupDAO.verifyDeleteFlowRegistry(registryId);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public RegistryEntity deleteRegistryClient(final Revision revision, final String registryId) {
|
||||||
final RevisionClaim claim = new StandardRevisionClaim(revision);
|
final RevisionClaim claim = new StandardRevisionClaim(revision);
|
||||||
final NiFiUser user = NiFiUserUtils.getNiFiUser();
|
final NiFiUser user = NiFiUserUtils.getNiFiUser();
|
||||||
|
|
||||||
|
@ -3340,6 +3425,10 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
||||||
entity.setPoliciesPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getPolicies()));
|
entity.setPoliciesPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getPolicies()));
|
||||||
entity.setSystemPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getSystem()));
|
entity.setSystemPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getSystem()));
|
||||||
entity.setRestrictedComponentsPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getRestrictedComponents()));
|
entity.setRestrictedComponentsPermissions(dtoFactory.createPermissionsDto(authorizableLookup.getRestrictedComponents()));
|
||||||
|
|
||||||
|
// TODO - update to be user specific
|
||||||
|
entity.setCanVersionFlows(CollectionUtils.isNotEmpty(flowRegistryClient.getRegistryIdentifiers()));
|
||||||
|
|
||||||
return entity;
|
return entity;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -3538,7 +3627,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public VersionControlComponentMappingEntity registerFlowWithFlowRegistry(final String groupId, final VersionedFlowEntity requestEntity) {
|
public VersionControlComponentMappingEntity registerFlowWithFlowRegistry(final String groupId, final StartVersionControlRequestEntity requestEntity) {
|
||||||
// Create a VersionedProcessGroup snapshot of the flow as it is currently.
|
// Create a VersionedProcessGroup snapshot of the flow as it is currently.
|
||||||
final InstantiatedVersionedProcessGroup versionedProcessGroup = createFlowSnapshot(groupId);
|
final InstantiatedVersionedProcessGroup versionedProcessGroup = createFlowSnapshot(groupId);
|
||||||
|
|
||||||
|
@ -3584,6 +3673,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
||||||
vci.setBucketId(registeredFlow.getBucketIdentifier());
|
vci.setBucketId(registeredFlow.getBucketIdentifier());
|
||||||
vci.setCurrent(true);
|
vci.setCurrent(true);
|
||||||
vci.setFlowId(registeredFlow.getIdentifier());
|
vci.setFlowId(registeredFlow.getIdentifier());
|
||||||
|
vci.setFlowName(registeredFlow.getName());
|
||||||
vci.setGroupId(groupId);
|
vci.setGroupId(groupId);
|
||||||
vci.setModified(false);
|
vci.setModified(false);
|
||||||
vci.setRegistryId(registryId);
|
vci.setRegistryId(registryId);
|
||||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.nifi.web.api.entity.ControllerServiceEntity;
|
||||||
import org.apache.nifi.web.api.entity.Entity;
|
import org.apache.nifi.web.api.entity.Entity;
|
||||||
import org.apache.nifi.web.api.entity.HistoryEntity;
|
import org.apache.nifi.web.api.entity.HistoryEntity;
|
||||||
import org.apache.nifi.web.api.entity.NodeEntity;
|
import org.apache.nifi.web.api.entity.NodeEntity;
|
||||||
|
import org.apache.nifi.web.api.entity.RegistriesEntity;
|
||||||
import org.apache.nifi.web.api.entity.RegistryEntity;
|
import org.apache.nifi.web.api.entity.RegistryEntity;
|
||||||
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
import org.apache.nifi.web.api.entity.ReportingTaskEntity;
|
||||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||||
|
@ -69,6 +70,7 @@ import javax.ws.rs.core.MediaType;
|
||||||
import javax.ws.rs.core.Response;
|
import javax.ws.rs.core.Response;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* RESTful endpoint for managing a Flow Controller.
|
* RESTful endpoint for managing a Flow Controller.
|
||||||
|
@ -93,7 +95,7 @@ public class ControllerResource extends ApplicationResource {
|
||||||
* @return dtos
|
* @return dtos
|
||||||
*/
|
*/
|
||||||
public RegistryEntity populateRemainingRegistryEntityContent(final RegistryEntity registryEntity) {
|
public RegistryEntity populateRemainingRegistryEntityContent(final RegistryEntity registryEntity) {
|
||||||
registryEntity.setUri(generateResourceUri("controller", "registries", registryEntity.getId()));
|
registryEntity.setUri(generateResourceUri("controller", "registry-clients", registryEntity.getId()));
|
||||||
return registryEntity;
|
return registryEntity;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -310,6 +312,36 @@ public class ControllerResource extends ApplicationResource {
|
||||||
// registries
|
// registries
|
||||||
// ----------
|
// ----------
|
||||||
|
|
||||||
|
@GET
|
||||||
|
@Consumes(MediaType.WILDCARD)
|
||||||
|
@Produces(MediaType.APPLICATION_JSON)
|
||||||
|
@Path("registry-clients")
|
||||||
|
@ApiOperation(value = "Gets the listing of available registry clients", response = RegistriesEntity.class, authorizations = {
|
||||||
|
@Authorization(value = "Read - /flow")
|
||||||
|
})
|
||||||
|
@ApiResponses(value = {
|
||||||
|
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
|
||||||
|
@ApiResponse(code = 401, message = "Client could not be authenticated."),
|
||||||
|
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
|
||||||
|
@ApiResponse(code = 404, message = "The specified resource could not be found."),
|
||||||
|
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
|
||||||
|
})
|
||||||
|
public Response getRegistryClients() {
|
||||||
|
authorizeController(RequestAction.READ);
|
||||||
|
|
||||||
|
if (isReplicateRequest()) {
|
||||||
|
return replicate(HttpMethod.GET);
|
||||||
|
}
|
||||||
|
|
||||||
|
final Set<RegistryEntity> registries = serviceFacade.getRegistryClients();
|
||||||
|
registries.forEach(registry -> populateRemainingRegistryEntityContent(registry));
|
||||||
|
|
||||||
|
final RegistriesEntity registryEntities = new RegistriesEntity();
|
||||||
|
registryEntities.setRegistries(registries);
|
||||||
|
|
||||||
|
return generateOkResponse(registryEntities).build();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new Registry.
|
* Creates a new Registry.
|
||||||
*
|
*
|
||||||
|
@ -320,9 +352,9 @@ public class ControllerResource extends ApplicationResource {
|
||||||
@POST
|
@POST
|
||||||
@Consumes(MediaType.APPLICATION_JSON)
|
@Consumes(MediaType.APPLICATION_JSON)
|
||||||
@Produces(MediaType.APPLICATION_JSON)
|
@Produces(MediaType.APPLICATION_JSON)
|
||||||
@Path("registries")
|
@Path("registry-clients")
|
||||||
@ApiOperation(
|
@ApiOperation(
|
||||||
value = "Creates a new registry",
|
value = "Creates a new registry client",
|
||||||
response = RegistryEntity.class,
|
response = RegistryEntity.class,
|
||||||
authorizations = {
|
authorizations = {
|
||||||
@Authorization(value = "Write - /controller")
|
@Authorization(value = "Write - /controller")
|
||||||
|
@ -336,8 +368,7 @@ public class ControllerResource extends ApplicationResource {
|
||||||
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
|
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
|
public Response createRegistryClient(
|
||||||
public Response createRegistry(
|
|
||||||
@Context final HttpServletRequest httpServletRequest,
|
@Context final HttpServletRequest httpServletRequest,
|
||||||
@ApiParam(
|
@ApiParam(
|
||||||
value = "The registry configuration details.",
|
value = "The registry configuration details.",
|
||||||
|
@ -376,7 +407,7 @@ public class ControllerResource extends ApplicationResource {
|
||||||
|
|
||||||
// create the reporting task and generate the json
|
// create the reporting task and generate the json
|
||||||
final Revision revision = getRevision(registryEntity, registry.getId());
|
final Revision revision = getRevision(registryEntity, registry.getId());
|
||||||
final RegistryEntity entity = serviceFacade.createRegistry(revision, registry);
|
final RegistryEntity entity = serviceFacade.createRegistryClient(revision, registry);
|
||||||
populateRemainingRegistryEntityContent(entity);
|
populateRemainingRegistryEntityContent(entity);
|
||||||
|
|
||||||
// build the response
|
// build the response
|
||||||
|
@ -394,9 +425,9 @@ public class ControllerResource extends ApplicationResource {
|
||||||
@GET
|
@GET
|
||||||
@Consumes(MediaType.WILDCARD)
|
@Consumes(MediaType.WILDCARD)
|
||||||
@Produces(MediaType.APPLICATION_JSON)
|
@Produces(MediaType.APPLICATION_JSON)
|
||||||
@Path("/registries/{id}")
|
@Path("/registry-clients/{id}")
|
||||||
@ApiOperation(
|
@ApiOperation(
|
||||||
value = "Gets a registry",
|
value = "Gets a registry client",
|
||||||
response = RegistryEntity.class,
|
response = RegistryEntity.class,
|
||||||
authorizations = {
|
authorizations = {
|
||||||
@Authorization(value = "Read - /controller")
|
@Authorization(value = "Read - /controller")
|
||||||
|
@ -426,7 +457,7 @@ public class ControllerResource extends ApplicationResource {
|
||||||
authorizeController(RequestAction.READ);
|
authorizeController(RequestAction.READ);
|
||||||
|
|
||||||
// get the registry
|
// get the registry
|
||||||
final RegistryEntity entity = serviceFacade.getRegistry(id);
|
final RegistryEntity entity = serviceFacade.getRegistryClient(id);
|
||||||
populateRemainingRegistryEntityContent(entity);
|
populateRemainingRegistryEntityContent(entity);
|
||||||
|
|
||||||
return generateOkResponse(entity).build();
|
return generateOkResponse(entity).build();
|
||||||
|
@ -443,9 +474,9 @@ public class ControllerResource extends ApplicationResource {
|
||||||
@PUT
|
@PUT
|
||||||
@Consumes(MediaType.APPLICATION_JSON)
|
@Consumes(MediaType.APPLICATION_JSON)
|
||||||
@Produces(MediaType.APPLICATION_JSON)
|
@Produces(MediaType.APPLICATION_JSON)
|
||||||
@Path("/registries/{id}")
|
@Path("/registry-clients/{id}")
|
||||||
@ApiOperation(
|
@ApiOperation(
|
||||||
value = "Updates a registry",
|
value = "Updates a registry client",
|
||||||
response = RegistryEntity.class,
|
response = RegistryEntity.class,
|
||||||
authorizations = {
|
authorizations = {
|
||||||
@Authorization(value = "Write - /controller")
|
@Authorization(value = "Write - /controller")
|
||||||
|
@ -460,7 +491,7 @@ public class ControllerResource extends ApplicationResource {
|
||||||
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
|
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
public Response updateControllerService(
|
public Response updateRegistryClient(
|
||||||
@Context HttpServletRequest httpServletRequest,
|
@Context HttpServletRequest httpServletRequest,
|
||||||
@ApiParam(
|
@ApiParam(
|
||||||
value = "The registry id.",
|
value = "The registry id.",
|
||||||
|
@ -505,7 +536,7 @@ public class ControllerResource extends ApplicationResource {
|
||||||
final RegistryDTO registry = registryEntity.getComponent();
|
final RegistryDTO registry = registryEntity.getComponent();
|
||||||
|
|
||||||
// update the controller service
|
// update the controller service
|
||||||
final RegistryEntity entity = serviceFacade.updateRegistry(revision, registry);
|
final RegistryEntity entity = serviceFacade.updateRegistryClient(revision, registry);
|
||||||
populateRemainingRegistryEntityContent(entity);
|
populateRemainingRegistryEntityContent(entity);
|
||||||
|
|
||||||
return generateOkResponse(entity).build();
|
return generateOkResponse(entity).build();
|
||||||
|
@ -528,9 +559,9 @@ public class ControllerResource extends ApplicationResource {
|
||||||
@DELETE
|
@DELETE
|
||||||
@Consumes(MediaType.WILDCARD)
|
@Consumes(MediaType.WILDCARD)
|
||||||
@Produces(MediaType.APPLICATION_JSON)
|
@Produces(MediaType.APPLICATION_JSON)
|
||||||
@Path("/registries/{id}")
|
@Path("/registry-clients/{id}")
|
||||||
@ApiOperation(
|
@ApiOperation(
|
||||||
value = "Deletes a reistry",
|
value = "Deletes a registry client",
|
||||||
response = RegistryEntity.class,
|
response = RegistryEntity.class,
|
||||||
authorizations = {
|
authorizations = {
|
||||||
@Authorization(value = "Write - /controller")
|
@Authorization(value = "Write - /controller")
|
||||||
|
@ -545,7 +576,7 @@ public class ControllerResource extends ApplicationResource {
|
||||||
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
|
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
public Response deleteRegistry(
|
public Response deleteRegistryClient(
|
||||||
@Context HttpServletRequest httpServletRequest,
|
@Context HttpServletRequest httpServletRequest,
|
||||||
@ApiParam(
|
@ApiParam(
|
||||||
value = "The revision is used to verify the client is working with the latest version of the flow.",
|
value = "The revision is used to verify the client is working with the latest version of the flow.",
|
||||||
|
@ -579,10 +610,10 @@ public class ControllerResource extends ApplicationResource {
|
||||||
lookup -> {
|
lookup -> {
|
||||||
authorizeController(RequestAction.WRITE);
|
authorizeController(RequestAction.WRITE);
|
||||||
},
|
},
|
||||||
null,
|
() -> serviceFacade.verifyDeleteRegistry(id),
|
||||||
(revision, registryEntity) -> {
|
(revision, registryEntity) -> {
|
||||||
// delete the specified registry
|
// delete the specified registry
|
||||||
final RegistryEntity entity = serviceFacade.deleteRegistry(revision, registryEntity.getId());
|
final RegistryEntity entity = serviceFacade.deleteRegistryClient(revision, registryEntity.getId());
|
||||||
return generateOkResponse(entity).build();
|
return generateOkResponse(entity).build();
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
|
|
@ -39,18 +39,13 @@ import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||||
import org.apache.nifi.controller.service.ControllerServiceState;
|
import org.apache.nifi.controller.service.ControllerServiceState;
|
||||||
import org.apache.nifi.groups.ProcessGroup;
|
import org.apache.nifi.groups.ProcessGroup;
|
||||||
import org.apache.nifi.nar.NarClassLoaders;
|
import org.apache.nifi.nar.NarClassLoaders;
|
||||||
import org.apache.nifi.registry.bucket.Bucket;
|
|
||||||
import org.apache.nifi.registry.flow.FlowRegistry;
|
|
||||||
import org.apache.nifi.registry.flow.FlowRegistryClient;
|
|
||||||
import org.apache.nifi.util.NiFiProperties;
|
import org.apache.nifi.util.NiFiProperties;
|
||||||
import org.apache.nifi.web.IllegalClusterResourceRequestException;
|
import org.apache.nifi.web.IllegalClusterResourceRequestException;
|
||||||
import org.apache.nifi.web.NiFiCoreException;
|
|
||||||
import org.apache.nifi.web.NiFiServiceFacade;
|
import org.apache.nifi.web.NiFiServiceFacade;
|
||||||
import org.apache.nifi.web.ResourceNotFoundException;
|
import org.apache.nifi.web.ResourceNotFoundException;
|
||||||
import org.apache.nifi.web.Revision;
|
import org.apache.nifi.web.Revision;
|
||||||
import org.apache.nifi.web.api.dto.AboutDTO;
|
import org.apache.nifi.web.api.dto.AboutDTO;
|
||||||
import org.apache.nifi.web.api.dto.BannerDTO;
|
import org.apache.nifi.web.api.dto.BannerDTO;
|
||||||
import org.apache.nifi.web.api.dto.BucketDTO;
|
|
||||||
import org.apache.nifi.web.api.dto.BulletinBoardDTO;
|
import org.apache.nifi.web.api.dto.BulletinBoardDTO;
|
||||||
import org.apache.nifi.web.api.dto.BulletinQueryDTO;
|
import org.apache.nifi.web.api.dto.BulletinQueryDTO;
|
||||||
import org.apache.nifi.web.api.dto.ClusterDTO;
|
import org.apache.nifi.web.api.dto.ClusterDTO;
|
||||||
|
@ -102,6 +97,10 @@ import org.apache.nifi.web.api.entity.SearchResultsEntity;
|
||||||
import org.apache.nifi.web.api.entity.StatusHistoryEntity;
|
import org.apache.nifi.web.api.entity.StatusHistoryEntity;
|
||||||
import org.apache.nifi.web.api.entity.TemplateEntity;
|
import org.apache.nifi.web.api.entity.TemplateEntity;
|
||||||
import org.apache.nifi.web.api.entity.TemplatesEntity;
|
import org.apache.nifi.web.api.entity.TemplatesEntity;
|
||||||
|
import org.apache.nifi.web.api.entity.VersionedFlowEntity;
|
||||||
|
import org.apache.nifi.web.api.entity.VersionedFlowSnapshotMetadataEntity;
|
||||||
|
import org.apache.nifi.web.api.entity.VersionedFlowSnapshotMetadataSetEntity;
|
||||||
|
import org.apache.nifi.web.api.entity.VersionedFlowsEntity;
|
||||||
import org.apache.nifi.web.api.request.BulletinBoardPatternParameter;
|
import org.apache.nifi.web.api.request.BulletinBoardPatternParameter;
|
||||||
import org.apache.nifi.web.api.request.DateTimeParameter;
|
import org.apache.nifi.web.api.request.DateTimeParameter;
|
||||||
import org.apache.nifi.web.api.request.IntegerParameter;
|
import org.apache.nifi.web.api.request.IntegerParameter;
|
||||||
|
@ -121,7 +120,6 @@ import javax.ws.rs.WebApplicationException;
|
||||||
import javax.ws.rs.core.Context;
|
import javax.ws.rs.core.Context;
|
||||||
import javax.ws.rs.core.MediaType;
|
import javax.ws.rs.core.MediaType;
|
||||||
import javax.ws.rs.core.Response;
|
import javax.ws.rs.core.Response;
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
import java.util.EnumSet;
|
import java.util.EnumSet;
|
||||||
|
@ -158,11 +156,8 @@ public class FlowResource extends ApplicationResource {
|
||||||
private TemplateResource templateResource;
|
private TemplateResource templateResource;
|
||||||
private ProcessGroupResource processGroupResource;
|
private ProcessGroupResource processGroupResource;
|
||||||
private ControllerServiceResource controllerServiceResource;
|
private ControllerServiceResource controllerServiceResource;
|
||||||
private ControllerResource controllerResource;
|
|
||||||
private ReportingTaskResource reportingTaskResource;
|
private ReportingTaskResource reportingTaskResource;
|
||||||
|
|
||||||
private FlowRegistryClient flowRegistryClient;
|
|
||||||
|
|
||||||
public FlowResource() {
|
public FlowResource() {
|
||||||
super();
|
super();
|
||||||
}
|
}
|
||||||
|
@ -1351,12 +1346,7 @@ public class FlowResource extends ApplicationResource {
|
||||||
public Response getRegistries() {
|
public Response getRegistries() {
|
||||||
authorizeFlow();
|
authorizeFlow();
|
||||||
|
|
||||||
if (isReplicateRequest()) {
|
final Set<RegistryEntity> registries = serviceFacade.getRegistriesForUser(NiFiUserUtils.getNiFiUser());
|
||||||
return replicate(HttpMethod.GET);
|
|
||||||
}
|
|
||||||
|
|
||||||
final Set<RegistryEntity> registries = serviceFacade.getRegistries();
|
|
||||||
registries.forEach(registry -> controllerResource.populateRemainingRegistryEntityContent(registry));
|
|
||||||
|
|
||||||
final RegistriesEntity registryEntities = new RegistriesEntity();
|
final RegistriesEntity registryEntities = new RegistriesEntity();
|
||||||
registryEntities.setRegistries(registries);
|
registryEntities.setRegistries(registries);
|
||||||
|
@ -1387,39 +1377,89 @@ public class FlowResource extends ApplicationResource {
|
||||||
|
|
||||||
authorizeFlow();
|
authorizeFlow();
|
||||||
|
|
||||||
try {
|
final Set<BucketEntity> buckets = serviceFacade.getBucketsForUser(id, NiFiUserUtils.getNiFiUser());
|
||||||
final FlowRegistry flowRegistry = flowRegistryClient.getFlowRegistry(id);
|
|
||||||
if (flowRegistry == null) {
|
|
||||||
throw new IllegalArgumentException("The specified registry id is unknown to this NiFi.");
|
|
||||||
}
|
|
||||||
|
|
||||||
final Set<Bucket> userBuckets = flowRegistry.getBuckets(NiFiUserUtils.getNiFiUser());
|
final BucketsEntity bucketsEntity = new BucketsEntity();
|
||||||
|
bucketsEntity.setBuckets(buckets);
|
||||||
|
|
||||||
final BucketsEntity bucketsEntity = new BucketsEntity();
|
return generateOkResponse(bucketsEntity).build();
|
||||||
|
}
|
||||||
|
|
||||||
if (userBuckets != null) {
|
@GET
|
||||||
|
@Consumes(MediaType.WILDCARD)
|
||||||
|
@Produces(MediaType.APPLICATION_JSON)
|
||||||
|
@Path("registries/{registry-id}/buckets/{bucket-id}/flows")
|
||||||
|
@ApiOperation(value = "Gets the flows from the specified registry and bucket for the current user", response = BucketsEntity.class, authorizations = {
|
||||||
|
@Authorization(value = "Read - /flow")
|
||||||
|
})
|
||||||
|
@ApiResponses(value = {
|
||||||
|
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
|
||||||
|
@ApiResponse(code = 401, message = "Client could not be authenticated."),
|
||||||
|
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
|
||||||
|
@ApiResponse(code = 404, message = "The specified resource could not be found."),
|
||||||
|
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
|
||||||
|
})
|
||||||
|
public Response getFlows(
|
||||||
|
@ApiParam(
|
||||||
|
value = "The registry id.",
|
||||||
|
required = true
|
||||||
|
)
|
||||||
|
@PathParam("registry-id") String registryId,
|
||||||
|
@ApiParam(
|
||||||
|
value = "The bucket id.",
|
||||||
|
required = true
|
||||||
|
)
|
||||||
|
@PathParam("bucket-id") String bucketId) {
|
||||||
|
|
||||||
final Set<BucketEntity> bucketSet = new HashSet<>();
|
authorizeFlow();
|
||||||
for (final Bucket userBucket : userBuckets) {
|
|
||||||
final BucketDTO bucket = new BucketDTO();
|
|
||||||
bucket.setId(userBucket.getIdentifier());
|
|
||||||
bucket.setName(userBucket.getName());
|
|
||||||
bucket.setDescription(userBucket.getDescription());
|
|
||||||
bucket.setCreated(userBucket.getCreatedTimestamp());
|
|
||||||
|
|
||||||
final BucketEntity bucketEntity = new BucketEntity();
|
final Set<VersionedFlowEntity> versionedFlows = serviceFacade.getFlowsForUser(registryId, bucketId, NiFiUserUtils.getNiFiUser());
|
||||||
bucketEntity.setBucket(bucket);
|
|
||||||
|
|
||||||
bucketSet.add(bucketEntity);
|
final VersionedFlowsEntity versionedFlowsEntity = new VersionedFlowsEntity();
|
||||||
}
|
versionedFlowsEntity.setVersionedFlows(versionedFlows);
|
||||||
|
|
||||||
bucketsEntity.setBuckets(bucketSet);
|
return generateOkResponse(versionedFlowsEntity).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
return generateOkResponse(bucketsEntity).build();
|
@GET
|
||||||
} catch (final IOException ioe) {
|
@Consumes(MediaType.WILDCARD)
|
||||||
throw new NiFiCoreException("Unable to obtain bucket listing: " + ioe.getMessage(), ioe);
|
@Produces(MediaType.APPLICATION_JSON)
|
||||||
}
|
@Path("registries/{registry-id}/buckets/{bucket-id}/flows/{flow-id}/versions")
|
||||||
|
@ApiOperation(value = "Gets the flow versions from the specified registry and bucket for the specified flow for the current user", response = BucketsEntity.class, authorizations = {
|
||||||
|
@Authorization(value = "Read - /flow")
|
||||||
|
})
|
||||||
|
@ApiResponses(value = {
|
||||||
|
@ApiResponse(code = 400, message = "NiFi was unable to complete the request because it was invalid. The request should not be retried without modification."),
|
||||||
|
@ApiResponse(code = 401, message = "Client could not be authenticated."),
|
||||||
|
@ApiResponse(code = 403, message = "Client is not authorized to make this request."),
|
||||||
|
@ApiResponse(code = 404, message = "The specified resource could not be found."),
|
||||||
|
@ApiResponse(code = 409, message = "The request was valid but NiFi was not in the appropriate state to process it. Retrying the same request later may be successful.")
|
||||||
|
})
|
||||||
|
public Response getVersions(
|
||||||
|
@ApiParam(
|
||||||
|
value = "The registry id.",
|
||||||
|
required = true
|
||||||
|
)
|
||||||
|
@PathParam("registry-id") String registryId,
|
||||||
|
@ApiParam(
|
||||||
|
value = "The bucket id.",
|
||||||
|
required = true
|
||||||
|
)
|
||||||
|
@PathParam("bucket-id") String bucketId,
|
||||||
|
@ApiParam(
|
||||||
|
value = "The flow id.",
|
||||||
|
required = true
|
||||||
|
)
|
||||||
|
@PathParam("flow-id") String flowId) {
|
||||||
|
|
||||||
|
authorizeFlow();
|
||||||
|
|
||||||
|
final Set<VersionedFlowSnapshotMetadataEntity> versionedFlowSnapshotMetadataSet = serviceFacade.getFlowVersionsForUser(registryId, bucketId, flowId, NiFiUserUtils.getNiFiUser());
|
||||||
|
|
||||||
|
final VersionedFlowSnapshotMetadataSetEntity versionedFlowSnapshotMetadataSetEntity = new VersionedFlowSnapshotMetadataSetEntity();
|
||||||
|
versionedFlowSnapshotMetadataSetEntity.setVersionedFlowSnapshotMetadataSet(versionedFlowSnapshotMetadataSet);
|
||||||
|
|
||||||
|
return generateOkResponse(versionedFlowSnapshotMetadataSetEntity).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
// --------------
|
// --------------
|
||||||
|
@ -2629,10 +2669,6 @@ public class FlowResource extends ApplicationResource {
|
||||||
this.processGroupResource = processGroupResource;
|
this.processGroupResource = processGroupResource;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setControllerResource(ControllerResource controllerResource) {
|
|
||||||
this.controllerResource = controllerResource;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setControllerServiceResource(ControllerServiceResource controllerServiceResource) {
|
public void setControllerServiceResource(ControllerServiceResource controllerServiceResource) {
|
||||||
this.controllerServiceResource = controllerServiceResource;
|
this.controllerServiceResource = controllerServiceResource;
|
||||||
}
|
}
|
||||||
|
@ -2644,8 +2680,4 @@ public class FlowResource extends ApplicationResource {
|
||||||
public void setAuthorizer(Authorizer authorizer) {
|
public void setAuthorizer(Authorizer authorizer) {
|
||||||
this.authorizer = authorizer;
|
this.authorizer = authorizer;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setFlowRegistryClient(FlowRegistryClient flowRegistryClient) {
|
|
||||||
this.flowRegistryClient = flowRegistryClient;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,39 +17,12 @@
|
||||||
|
|
||||||
package org.apache.nifi.web.api;
|
package org.apache.nifi.web.api;
|
||||||
|
|
||||||
import java.io.IOException;
|
import io.swagger.annotations.Api;
|
||||||
import java.net.URI;
|
import io.swagger.annotations.ApiOperation;
|
||||||
import java.net.URISyntaxException;
|
import io.swagger.annotations.ApiParam;
|
||||||
import java.util.Collections;
|
import io.swagger.annotations.ApiResponse;
|
||||||
import java.util.Date;
|
import io.swagger.annotations.ApiResponses;
|
||||||
import java.util.HashMap;
|
import io.swagger.annotations.Authorization;
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.LinkedHashSet;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.UUID;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
import java.util.function.Consumer;
|
|
||||||
import java.util.function.Function;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
import javax.ws.rs.Consumes;
|
|
||||||
import javax.ws.rs.DELETE;
|
|
||||||
import javax.ws.rs.DefaultValue;
|
|
||||||
import javax.ws.rs.GET;
|
|
||||||
import javax.ws.rs.HttpMethod;
|
|
||||||
import javax.ws.rs.POST;
|
|
||||||
import javax.ws.rs.PUT;
|
|
||||||
import javax.ws.rs.Path;
|
|
||||||
import javax.ws.rs.PathParam;
|
|
||||||
import javax.ws.rs.Produces;
|
|
||||||
import javax.ws.rs.QueryParam;
|
|
||||||
import javax.ws.rs.core.MediaType;
|
|
||||||
import javax.ws.rs.core.MultivaluedHashMap;
|
|
||||||
import javax.ws.rs.core.Response;
|
|
||||||
import javax.ws.rs.core.Response.Status;
|
|
||||||
|
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.nifi.authorization.AuthorizableLookup;
|
import org.apache.nifi.authorization.AuthorizableLookup;
|
||||||
import org.apache.nifi.authorization.Authorizer;
|
import org.apache.nifi.authorization.Authorizer;
|
||||||
|
@ -58,6 +31,7 @@ import org.apache.nifi.authorization.resource.Authorizable;
|
||||||
import org.apache.nifi.authorization.user.NiFiUser;
|
import org.apache.nifi.authorization.user.NiFiUser;
|
||||||
import org.apache.nifi.authorization.user.NiFiUserUtils;
|
import org.apache.nifi.authorization.user.NiFiUserUtils;
|
||||||
import org.apache.nifi.cluster.manager.NodeResponse;
|
import org.apache.nifi.cluster.manager.NodeResponse;
|
||||||
|
import org.apache.nifi.controller.FlowController;
|
||||||
import org.apache.nifi.controller.ScheduledState;
|
import org.apache.nifi.controller.ScheduledState;
|
||||||
import org.apache.nifi.controller.service.ControllerServiceState;
|
import org.apache.nifi.controller.service.ControllerServiceState;
|
||||||
import org.apache.nifi.registry.flow.ComponentType;
|
import org.apache.nifi.registry.flow.ComponentType;
|
||||||
|
@ -83,7 +57,7 @@ import org.apache.nifi.web.api.entity.AffectedComponentEntity;
|
||||||
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
|
import org.apache.nifi.web.api.entity.ProcessGroupEntity;
|
||||||
import org.apache.nifi.web.api.entity.VersionControlComponentMappingEntity;
|
import org.apache.nifi.web.api.entity.VersionControlComponentMappingEntity;
|
||||||
import org.apache.nifi.web.api.entity.VersionControlInformationEntity;
|
import org.apache.nifi.web.api.entity.VersionControlInformationEntity;
|
||||||
import org.apache.nifi.web.api.entity.VersionedFlowEntity;
|
import org.apache.nifi.web.api.entity.StartVersionControlRequestEntity;
|
||||||
import org.apache.nifi.web.api.entity.VersionedFlowSnapshotEntity;
|
import org.apache.nifi.web.api.entity.VersionedFlowSnapshotEntity;
|
||||||
import org.apache.nifi.web.api.entity.VersionedFlowUpdateRequestEntity;
|
import org.apache.nifi.web.api.entity.VersionedFlowUpdateRequestEntity;
|
||||||
import org.apache.nifi.web.api.request.ClientIdParameter;
|
import org.apache.nifi.web.api.request.ClientIdParameter;
|
||||||
|
@ -96,12 +70,37 @@ import org.apache.nifi.web.util.Pause;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import io.swagger.annotations.Api;
|
import javax.ws.rs.Consumes;
|
||||||
import io.swagger.annotations.ApiOperation;
|
import javax.ws.rs.DELETE;
|
||||||
import io.swagger.annotations.ApiParam;
|
import javax.ws.rs.DefaultValue;
|
||||||
import io.swagger.annotations.ApiResponse;
|
import javax.ws.rs.GET;
|
||||||
import io.swagger.annotations.ApiResponses;
|
import javax.ws.rs.HttpMethod;
|
||||||
import io.swagger.annotations.Authorization;
|
import javax.ws.rs.POST;
|
||||||
|
import javax.ws.rs.PUT;
|
||||||
|
import javax.ws.rs.Path;
|
||||||
|
import javax.ws.rs.PathParam;
|
||||||
|
import javax.ws.rs.Produces;
|
||||||
|
import javax.ws.rs.QueryParam;
|
||||||
|
import javax.ws.rs.core.MediaType;
|
||||||
|
import javax.ws.rs.core.MultivaluedHashMap;
|
||||||
|
import javax.ws.rs.core.Response;
|
||||||
|
import javax.ws.rs.core.Response.Status;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.net.URI;
|
||||||
|
import java.net.URISyntaxException;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Date;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.LinkedHashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.UUID;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.function.Consumer;
|
||||||
|
import java.util.function.Function;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
@Path("/versions")
|
@Path("/versions")
|
||||||
@Api(value = "/versions", description = "Endpoint for managing version control for a flow")
|
@Api(value = "/versions", description = "Endpoint for managing version control for a flow")
|
||||||
|
@ -368,7 +367,7 @@ public class VersionsResource extends ApplicationResource {
|
||||||
})
|
})
|
||||||
public Response startVersionControl(
|
public Response startVersionControl(
|
||||||
@ApiParam("The process group id.") @PathParam("id") final String groupId,
|
@ApiParam("The process group id.") @PathParam("id") final String groupId,
|
||||||
@ApiParam(value = "The versioned flow details.", required = true) final VersionedFlowEntity requestEntity) throws IOException {
|
@ApiParam(value = "The versioned flow details.", required = true) final StartVersionControlRequestEntity requestEntity) throws IOException {
|
||||||
|
|
||||||
// Verify the request
|
// Verify the request
|
||||||
final RevisionDTO revisionDto = requestEntity.getProcessGroupRevision();
|
final RevisionDTO revisionDto = requestEntity.getProcessGroupRevision();
|
||||||
|
@ -390,6 +389,12 @@ public class VersionsResource extends ApplicationResource {
|
||||||
throw new IllegalArgumentException("The Registry ID must be supplied.");
|
throw new IllegalArgumentException("The Registry ID must be supplied.");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// ensure we're not attempting to version the root group
|
||||||
|
final ProcessGroupEntity root = serviceFacade.getProcessGroup(FlowController.ROOT_GROUP_ID_ALIAS);
|
||||||
|
if (root.getId().equals(groupId)) {
|
||||||
|
throw new IllegalArgumentException("The Root Process Group cannot be versioned.");
|
||||||
|
}
|
||||||
|
|
||||||
if (isReplicateRequest()) {
|
if (isReplicateRequest()) {
|
||||||
// We first have to obtain a "lock" on all nodes in the cluster so that multiple Version Control requests
|
// We first have to obtain a "lock" on all nodes in the cluster so that multiple Version Control requests
|
||||||
// are not being made simultaneously. We do this by making a POST to /nifi-api/versions/start-requests.
|
// are not being made simultaneously. We do this by making a POST to /nifi-api/versions/start-requests.
|
||||||
|
@ -688,6 +693,7 @@ public class VersionsResource extends ApplicationResource {
|
||||||
versionControlInfoDto.setBucketId(snapshotMetadata.getBucketIdentifier());
|
versionControlInfoDto.setBucketId(snapshotMetadata.getBucketIdentifier());
|
||||||
versionControlInfoDto.setCurrent(true);
|
versionControlInfoDto.setCurrent(true);
|
||||||
versionControlInfoDto.setFlowId(snapshotMetadata.getFlowIdentifier());
|
versionControlInfoDto.setFlowId(snapshotMetadata.getFlowIdentifier());
|
||||||
|
versionControlInfoDto.setFlowName(snapshotMetadata.getFlowName());
|
||||||
versionControlInfoDto.setGroupId(groupId);
|
versionControlInfoDto.setGroupId(groupId);
|
||||||
versionControlInfoDto.setModified(false);
|
versionControlInfoDto.setModified(false);
|
||||||
versionControlInfoDto.setVersion(snapshotMetadata.getVersion());
|
versionControlInfoDto.setVersion(snapshotMetadata.getVersion());
|
||||||
|
@ -1139,12 +1145,13 @@ public class VersionsResource extends ApplicationResource {
|
||||||
updateRequestDto.setLastUpdated(new Date());
|
updateRequestDto.setLastUpdated(new Date());
|
||||||
updateRequestDto.setProcessGroupId(groupId);
|
updateRequestDto.setProcessGroupId(groupId);
|
||||||
updateRequestDto.setRequestId(requestId);
|
updateRequestDto.setRequestId(requestId);
|
||||||
updateRequestDto.setUri(generateResourceUri("versions", "update-requests", requestId));
|
updateRequestDto.setUri(generateResourceUri("versions", "revert-requests", requestId));
|
||||||
|
|
||||||
final VersionedFlowUpdateRequestEntity updateRequestEntity = new VersionedFlowUpdateRequestEntity();
|
final VersionedFlowUpdateRequestEntity updateRequestEntity = new VersionedFlowUpdateRequestEntity();
|
||||||
updateRequestEntity.setProcessGroupRevision(revisionDto);
|
updateRequestEntity.setProcessGroupRevision(revisionDto);
|
||||||
updateRequestEntity.setRequest(updateRequestDto);
|
updateRequestEntity.setRequest(updateRequestDto);
|
||||||
|
|
||||||
|
request.markComplete(currentVersionEntity);
|
||||||
return generateOkResponse(updateRequestEntity).build();
|
return generateOkResponse(updateRequestEntity).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -16,7 +16,6 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.nifi.web.api.dto;
|
package org.apache.nifi.web.api.dto;
|
||||||
|
|
||||||
import org.apache.commons.collections4.CollectionUtils;
|
|
||||||
import org.apache.commons.lang3.ClassUtils;
|
import org.apache.commons.lang3.ClassUtils;
|
||||||
import org.apache.commons.lang3.StringUtils;
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.nifi.action.Action;
|
import org.apache.nifi.action.Action;
|
||||||
|
@ -245,7 +244,6 @@ public final class DtoFactory {
|
||||||
dto.setSupportsManagedAuthorizer(AuthorizerCapabilityDetection.isManagedAuthorizer(authorizer));
|
dto.setSupportsManagedAuthorizer(AuthorizerCapabilityDetection.isManagedAuthorizer(authorizer));
|
||||||
dto.setSupportsConfigurableUsersAndGroups(AuthorizerCapabilityDetection.isConfigurableUserGroupProvider(authorizer));
|
dto.setSupportsConfigurableUsersAndGroups(AuthorizerCapabilityDetection.isConfigurableUserGroupProvider(authorizer));
|
||||||
dto.setSupportsConfigurableAuthorizer(AuthorizerCapabilityDetection.isConfigurableAccessPolicyProvider(authorizer));
|
dto.setSupportsConfigurableAuthorizer(AuthorizerCapabilityDetection.isConfigurableAccessPolicyProvider(authorizer));
|
||||||
dto.setSupportsFlowVersioning(CollectionUtils.isNotEmpty(flowRegistryClient.getRegistryIdentifiers()));
|
|
||||||
|
|
||||||
final Date now = new Date();
|
final Date now = new Date();
|
||||||
dto.setTimeOffset(TimeZone.getDefault().getOffset(now.getTime()));
|
dto.setTimeOffset(TimeZone.getDefault().getOffset(now.getTime()));
|
||||||
|
@ -2191,6 +2189,8 @@ public final class DtoFactory {
|
||||||
dto.setRegistryId(versionControlInfo.getRegistryIdentifier());
|
dto.setRegistryId(versionControlInfo.getRegistryIdentifier());
|
||||||
dto.setBucketId(versionControlInfo.getBucketIdentifier());
|
dto.setBucketId(versionControlInfo.getBucketIdentifier());
|
||||||
dto.setFlowId(versionControlInfo.getFlowIdentifier());
|
dto.setFlowId(versionControlInfo.getFlowIdentifier());
|
||||||
|
// TODO - need to get flow name here
|
||||||
|
dto.setFlowName(group.getName());
|
||||||
dto.setVersion(versionControlInfo.getVersion());
|
dto.setVersion(versionControlInfo.getVersion());
|
||||||
dto.setCurrent(versionControlInfo.getCurrent().orElse(null));
|
dto.setCurrent(versionControlInfo.getCurrent().orElse(null));
|
||||||
dto.setModified(versionControlInfo.getModified().orElse(null));
|
dto.setModified(versionControlInfo.getModified().orElse(null));
|
||||||
|
@ -3409,6 +3409,7 @@ public final class DtoFactory {
|
||||||
copy.setRegistryId(original.getRegistryId());
|
copy.setRegistryId(original.getRegistryId());
|
||||||
copy.setBucketId(original.getBucketId());
|
copy.setBucketId(original.getBucketId());
|
||||||
copy.setFlowId(original.getFlowId());
|
copy.setFlowId(original.getFlowId());
|
||||||
|
copy.setFlowName(original.getFlowName());
|
||||||
copy.setVersion(original.getVersion());
|
copy.setVersion(original.getVersion());
|
||||||
copy.setCurrent(original.getCurrent());
|
copy.setCurrent(original.getCurrent());
|
||||||
copy.setModified(original.getModified());
|
copy.setModified(original.getModified());
|
||||||
|
|
|
@ -158,6 +158,13 @@ public interface ProcessGroupDAO {
|
||||||
*/
|
*/
|
||||||
void verifyDelete(String groupId);
|
void verifyDelete(String groupId);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Verifies the specified registry can be removed.
|
||||||
|
*
|
||||||
|
* @param registryId registry id
|
||||||
|
*/
|
||||||
|
void verifyDeleteFlowRegistry(String registryId);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deletes the specified process group.
|
* Deletes the specified process group.
|
||||||
*
|
*
|
||||||
|
|
|
@ -17,11 +17,15 @@
|
||||||
|
|
||||||
package org.apache.nifi.web.dao;
|
package org.apache.nifi.web.dao;
|
||||||
|
|
||||||
import java.util.Set;
|
import org.apache.nifi.authorization.user.NiFiUser;
|
||||||
|
import org.apache.nifi.registry.bucket.Bucket;
|
||||||
import org.apache.nifi.registry.flow.FlowRegistry;
|
import org.apache.nifi.registry.flow.FlowRegistry;
|
||||||
|
import org.apache.nifi.registry.flow.VersionedFlow;
|
||||||
|
import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
|
||||||
import org.apache.nifi.web.api.dto.RegistryDTO;
|
import org.apache.nifi.web.api.dto.RegistryDTO;
|
||||||
|
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
public interface RegistryDAO {
|
public interface RegistryDAO {
|
||||||
|
|
||||||
FlowRegistry createFlowRegistry(RegistryDTO registryDto);
|
FlowRegistry createFlowRegistry(RegistryDTO registryDto);
|
||||||
|
@ -30,6 +34,14 @@ public interface RegistryDAO {
|
||||||
|
|
||||||
Set<FlowRegistry> getFlowRegistries();
|
Set<FlowRegistry> getFlowRegistries();
|
||||||
|
|
||||||
|
Set<FlowRegistry> getFlowRegistriesForUser(NiFiUser user);
|
||||||
|
|
||||||
|
Set<Bucket> getBucketsForUser(String registry, NiFiUser user);
|
||||||
|
|
||||||
|
Set<VersionedFlow> getFlowsForUser(String registryId, String bucketId, NiFiUser user);
|
||||||
|
|
||||||
|
Set<VersionedFlowSnapshotMetadata> getFlowVersionsForUser(String registryId, String bucketId, String flowId, NiFiUser user);
|
||||||
|
|
||||||
FlowRegistry removeFlowRegistry(String registryId);
|
FlowRegistry removeFlowRegistry(String registryId);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,15 +17,21 @@
|
||||||
|
|
||||||
package org.apache.nifi.web.dao.impl;
|
package org.apache.nifi.web.dao.impl;
|
||||||
|
|
||||||
import java.util.Set;
|
import org.apache.nifi.authorization.user.NiFiUser;
|
||||||
import java.util.stream.Collectors;
|
import org.apache.nifi.registry.bucket.Bucket;
|
||||||
|
|
||||||
import org.apache.nifi.registry.flow.FlowRegistry;
|
import org.apache.nifi.registry.flow.FlowRegistry;
|
||||||
import org.apache.nifi.registry.flow.FlowRegistryClient;
|
import org.apache.nifi.registry.flow.FlowRegistryClient;
|
||||||
|
import org.apache.nifi.registry.flow.VersionedFlow;
|
||||||
|
import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
|
||||||
|
import org.apache.nifi.web.NiFiCoreException;
|
||||||
import org.apache.nifi.web.ResourceNotFoundException;
|
import org.apache.nifi.web.ResourceNotFoundException;
|
||||||
import org.apache.nifi.web.api.dto.RegistryDTO;
|
import org.apache.nifi.web.api.dto.RegistryDTO;
|
||||||
import org.apache.nifi.web.dao.RegistryDAO;
|
import org.apache.nifi.web.dao.RegistryDAO;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
public class FlowRegistryDAO implements RegistryDAO {
|
public class FlowRegistryDAO implements RegistryDAO {
|
||||||
private FlowRegistryClient flowRegistryClient;
|
private FlowRegistryClient flowRegistryClient;
|
||||||
|
|
||||||
|
@ -51,6 +57,53 @@ public class FlowRegistryDAO implements RegistryDAO {
|
||||||
.collect(Collectors.toSet());
|
.collect(Collectors.toSet());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<FlowRegistry> getFlowRegistriesForUser(final NiFiUser user) {
|
||||||
|
// TODO - implement to be user specific
|
||||||
|
return getFlowRegistries();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<Bucket> getBucketsForUser(final String registryId, final NiFiUser user) {
|
||||||
|
try {
|
||||||
|
final FlowRegistry flowRegistry = flowRegistryClient.getFlowRegistry(registryId);
|
||||||
|
if (flowRegistry == null) {
|
||||||
|
throw new IllegalArgumentException("The specified registry id is unknown to this NiFi.");
|
||||||
|
}
|
||||||
|
|
||||||
|
return flowRegistry.getBuckets(user);
|
||||||
|
} catch (final IOException ioe) {
|
||||||
|
throw new NiFiCoreException("Unable to obtain bucket listing: " + ioe.getMessage(), ioe);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<VersionedFlow> getFlowsForUser(String registryId, String bucketId, NiFiUser user) {
|
||||||
|
final Set<Bucket> bucketsForUser = getBucketsForUser(registryId, user);
|
||||||
|
|
||||||
|
// TODO - implement getBucket(bucketId, user)
|
||||||
|
final Bucket bucket = bucketsForUser.stream().filter(b -> b.getIdentifier().equals(bucketId)).findFirst().orElse(null);
|
||||||
|
if (bucket == null) {
|
||||||
|
throw new IllegalArgumentException("The specified bucket is not available.");
|
||||||
|
}
|
||||||
|
|
||||||
|
return bucket.getVersionedFlows();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<VersionedFlowSnapshotMetadata> getFlowVersionsForUser(String registryId, String bucketId, String flowId, NiFiUser user) {
|
||||||
|
final Set<VersionedFlow> flowsForUser = getFlowsForUser(registryId, bucketId, user);
|
||||||
|
|
||||||
|
// TODO - implement getFlow(bucketId, flowId, user)
|
||||||
|
final VersionedFlow versionedFlow = flowsForUser.stream().filter(vf -> vf.getIdentifier().equals(flowId)).findFirst().orElse(null);
|
||||||
|
if (versionedFlow == null) {
|
||||||
|
throw new IllegalArgumentException("The specified flow is not available.");
|
||||||
|
}
|
||||||
|
|
||||||
|
return versionedFlow.getSnapshotMetadata();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FlowRegistry removeFlowRegistry(final String registryId) {
|
public FlowRegistry removeFlowRegistry(final String registryId) {
|
||||||
final FlowRegistry registry = flowRegistryClient.removeFlowRegistry(registryId);
|
final FlowRegistry registry = flowRegistryClient.removeFlowRegistry(registryId);
|
||||||
|
|
|
@ -39,9 +39,11 @@ import org.apache.nifi.web.dao.ProcessGroupDAO;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGroupDAO {
|
public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGroupDAO {
|
||||||
|
|
||||||
|
@ -294,6 +296,26 @@ public class StandardProcessGroupDAO extends ComponentDAO implements ProcessGrou
|
||||||
group.verifyCanDelete();
|
group.verifyCanDelete();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void verifyDeleteFlowRegistry(String registryId) {
|
||||||
|
final ProcessGroup rootGroup = flowController.getRootGroup();
|
||||||
|
|
||||||
|
final VersionControlInformation versionControlInformation = rootGroup.getVersionControlInformation();
|
||||||
|
if (versionControlInformation != null && versionControlInformation.getRegistryIdentifier().equals(registryId)) {
|
||||||
|
throw new IllegalStateException("The Registry cannot be removed because a Process Group currently under version control is tracking to it.");
|
||||||
|
}
|
||||||
|
|
||||||
|
final Set<VersionControlInformation> trackedVersionControlInformation = rootGroup.findAllProcessGroups().stream()
|
||||||
|
.map(group -> group.getVersionControlInformation())
|
||||||
|
.filter(Objects::nonNull)
|
||||||
|
.filter(vci -> vci.getRegistryIdentifier().equals(registryId))
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
|
||||||
|
if (!trackedVersionControlInformation.isEmpty()) {
|
||||||
|
throw new IllegalStateException("The Registry cannot be removed because a Process Group currently under version control is tracking to it.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void deleteProcessGroup(String processGroupId) {
|
public void deleteProcessGroup(String processGroupId) {
|
||||||
// get the group
|
// get the group
|
||||||
|
|
|
@ -213,7 +213,6 @@
|
||||||
<property name="remoteProcessGroupResource" ref="remoteProcessGroupResource"/>
|
<property name="remoteProcessGroupResource" ref="remoteProcessGroupResource"/>
|
||||||
<property name="connectionResource" ref="connectionResource"/>
|
<property name="connectionResource" ref="connectionResource"/>
|
||||||
<property name="templateResource" ref="templateResource"/>
|
<property name="templateResource" ref="templateResource"/>
|
||||||
<property name="controllerResource" ref="controllerResource"/>
|
|
||||||
<property name="controllerServiceResource" ref="controllerServiceResource"/>
|
<property name="controllerServiceResource" ref="controllerServiceResource"/>
|
||||||
<property name="reportingTaskResource" ref="reportingTaskResource"/>
|
<property name="reportingTaskResource" ref="reportingTaskResource"/>
|
||||||
<property name="processGroupResource" ref="processGroupResource"/>
|
<property name="processGroupResource" ref="processGroupResource"/>
|
||||||
|
@ -221,7 +220,6 @@
|
||||||
<property name="clusterCoordinator" ref="clusterCoordinator"/>
|
<property name="clusterCoordinator" ref="clusterCoordinator"/>
|
||||||
<property name="requestReplicator" ref="requestReplicator" />
|
<property name="requestReplicator" ref="requestReplicator" />
|
||||||
<property name="flowController" ref="flowController" />
|
<property name="flowController" ref="flowController" />
|
||||||
<property name="flowRegistryClient" ref="flowRegistryClient" />
|
|
||||||
</bean>
|
</bean>
|
||||||
<bean id="resourceResource" class="org.apache.nifi.web.api.ResourceResource" scope="singleton">
|
<bean id="resourceResource" class="org.apache.nifi.web.api.ResourceResource" scope="singleton">
|
||||||
<property name="serviceFacade" ref="serviceFacade"/>
|
<property name="serviceFacade" ref="serviceFacade"/>
|
||||||
|
|
|
@ -116,6 +116,7 @@
|
||||||
<jsp:include page="/WEB-INF/partials/canvas/fill-color-dialog.jsp"/>
|
<jsp:include page="/WEB-INF/partials/canvas/fill-color-dialog.jsp"/>
|
||||||
<jsp:include page="/WEB-INF/partials/canvas/connections-dialog.jsp"/>
|
<jsp:include page="/WEB-INF/partials/canvas/connections-dialog.jsp"/>
|
||||||
<jsp:include page="/WEB-INF/partials/canvas/save-flow-version-dialog.jsp"/>
|
<jsp:include page="/WEB-INF/partials/canvas/save-flow-version-dialog.jsp"/>
|
||||||
|
<jsp:include page="/WEB-INF/partials/canvas/import-flow-version-dialog.jsp"/>
|
||||||
<jsp:include page="/WEB-INF/partials/canvas/registry-configuration-dialog.jsp"/>
|
<jsp:include page="/WEB-INF/partials/canvas/registry-configuration-dialog.jsp"/>
|
||||||
<div id="canvas-container" class="unselectable"></div>
|
<div id="canvas-container" class="unselectable"></div>
|
||||||
<div id="canvas-tooltips">
|
<div id="canvas-tooltips">
|
||||||
|
|
|
@ -0,0 +1,44 @@
|
||||||
|
<%--
|
||||||
|
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
contributor license agreements. See the NOTICE file distributed with
|
||||||
|
this work for additional information regarding copyright ownership.
|
||||||
|
The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
(the "License"); you may not use this file except in compliance with
|
||||||
|
the License. You may obtain a copy of the License at
|
||||||
|
|
||||||
|
http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
|
||||||
|
Unless required by applicable law or agreed to in writing, software
|
||||||
|
distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
See the License for the specific language governing permissions and
|
||||||
|
limitations under the License.
|
||||||
|
--%>
|
||||||
|
<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %>
|
||||||
|
<div id="import-flow-version-dialog" layout="column" class="hidden large-dialog">
|
||||||
|
<div class="dialog-content">
|
||||||
|
<div class="setting">
|
||||||
|
<div class="setting-name">Registry</div>
|
||||||
|
<div class="setting-field">
|
||||||
|
<div id="import-flow-version-registry-combo"></div>
|
||||||
|
<div id="import-flow-version-registry" class="hidden"></div>
|
||||||
|
</div>
|
||||||
|
</div>
|
||||||
|
<div class="setting">
|
||||||
|
<div class="setting-name">Location</div>
|
||||||
|
<div class="setting-field">
|
||||||
|
<div id="import-flow-version-bucket-combo"></div>
|
||||||
|
<div id="import-flow-version-bucket" class="hidden"></div>
|
||||||
|
</div>
|
||||||
|
</div>
|
||||||
|
<div class="setting">
|
||||||
|
<div class="setting-name">Name</div>
|
||||||
|
<div class="setting-field">
|
||||||
|
<span id="import-flow-version-process-group-id" class="hidden"></span>
|
||||||
|
<div id="import-flow-version-name-combo"></div>
|
||||||
|
<div id="import-flow-version-name" class="hidden"></div>
|
||||||
|
</div>
|
||||||
|
</div>
|
||||||
|
<div id="import-flow-version-table"></div>
|
||||||
|
</div>
|
||||||
|
</div>
|
|
@ -128,13 +128,13 @@
|
||||||
<div class="button-spacer-large"> </div>
|
<div class="button-spacer-large"> </div>
|
||||||
<div id="operate-template" class="action-button" title="Create Template">
|
<div id="operate-template" class="action-button" title="Create Template">
|
||||||
<button ng-click="appCtrl.nf.Actions['template'](appCtrl.nf.CanvasUtils.getSelection());"
|
<button ng-click="appCtrl.nf.Actions['template'](appCtrl.nf.CanvasUtils.getSelection());"
|
||||||
ng-disabled="!(appCtrl.nf.CanvasUtils.canWrite() && (appCtrl.nf.CanvasUtils.getSelection().empty() || appCtrl.nf.CanvasUtils.canRead(appCtrl.nf.CanvasUtils.getSelection())));">
|
ng-disabled="!(appCtrl.nf.CanvasUtils.canWriteCurrentGroup() && (appCtrl.nf.CanvasUtils.getSelection().empty() || appCtrl.nf.CanvasUtils.canRead(appCtrl.nf.CanvasUtils.getSelection())));">
|
||||||
<div class="graph-control-action-icon icon icon-template-save"></div></button>
|
<div class="graph-control-action-icon icon icon-template-save"></div></button>
|
||||||
</div>
|
</div>
|
||||||
<div class="button-spacer-small"> </div>
|
<div class="button-spacer-small"> </div>
|
||||||
<div id="operate-template-upload" class="action-button" title="Upload Template">
|
<div id="operate-template-upload" class="action-button" title="Upload Template">
|
||||||
<button ng-click="appCtrl.nf.Actions['uploadTemplate']();"
|
<button ng-click="appCtrl.nf.Actions['uploadTemplate']();"
|
||||||
ng-disabled="!(appCtrl.nf.CanvasUtils.canWrite() && appCtrl.nf.CanvasUtils.getSelection().empty());">
|
ng-disabled="!(appCtrl.nf.CanvasUtils.canWriteCurrentGroup() && appCtrl.nf.CanvasUtils.getSelection().empty());">
|
||||||
<div class="graph-control-action-icon icon icon-template-import"></div></button>
|
<div class="graph-control-action-icon icon icon-template-import"></div></button>
|
||||||
</div>
|
</div>
|
||||||
<div class="clear"></div>
|
<div class="clear"></div>
|
||||||
|
|
|
@ -23,5 +23,8 @@
|
||||||
<input id="new-process-group-name" type="text"/>
|
<input id="new-process-group-name" type="text"/>
|
||||||
</div>
|
</div>
|
||||||
</div>
|
</div>
|
||||||
|
<div class="setting">
|
||||||
|
<span id="import-process-group-link" class="link"><i class="fa fa-cloud-download" aria-hidden="true" style="margin-left: 5px; margin-right: 5px;"></i>Import version...</span>
|
||||||
|
</div>
|
||||||
</div>
|
</div>
|
||||||
</div>
|
</div>
|
|
@ -22,7 +22,7 @@
|
||||||
<div class="fa fa-question-circle" alt="Info" title="Specify the remote target NiFi URLs. Multiple URLs can be specified in comma-separated format. Different protocols cannot be mixed. If remote NiFi is a cluster, two or more node URLs are recommended for better connection establishment availability."></div>
|
<div class="fa fa-question-circle" alt="Info" title="Specify the remote target NiFi URLs. Multiple URLs can be specified in comma-separated format. Different protocols cannot be mixed. If remote NiFi is a cluster, two or more node URLs are recommended for better connection establishment availability."></div>
|
||||||
</div>
|
</div>
|
||||||
<div class="setting-field">
|
<div class="setting-field">
|
||||||
<input id="new-remote-process-group-uris" type="text" placeholder="https://remotehost:8080/nifi"/>
|
<input id="new-remote-process-group-uris" type="text" placeholder="https://remotehost:8443/nifi"/>
|
||||||
</div>
|
</div>
|
||||||
</div>
|
</div>
|
||||||
<div class="setting">
|
<div class="setting">
|
||||||
|
|
|
@ -25,9 +25,9 @@
|
||||||
</div>
|
</div>
|
||||||
</div>
|
</div>
|
||||||
<div class="setting">
|
<div class="setting">
|
||||||
<div class="setting-name">Location</div>
|
<div class="setting-name">URL</div>
|
||||||
<div class="setting-field">
|
<div class="setting-field">
|
||||||
<input type="text" id="registry-location" class="setting-input"/>
|
<input type="text" id="registry-location" class="setting-input" placeholder="https://remotehost:8443"/>
|
||||||
</div>
|
</div>
|
||||||
</div>
|
</div>
|
||||||
<div class="setting">
|
<div class="setting">
|
||||||
|
|
|
@ -20,34 +20,36 @@
|
||||||
<div class="setting">
|
<div class="setting">
|
||||||
<div class="setting-name">Registry</div>
|
<div class="setting-name">Registry</div>
|
||||||
<div class="setting-field">
|
<div class="setting-field">
|
||||||
<div id="flow-version-registry-combo"></div>
|
<div id="save-flow-version-registry-combo" class="hidden"></div>
|
||||||
<div id="flow-version-registry" class="hidden"></div>
|
<div id="save-flow-version-registry" class="hidden"></div>
|
||||||
</div>
|
</div>
|
||||||
</div>
|
</div>
|
||||||
<div class="setting">
|
<div class="setting">
|
||||||
<div class="setting-name">Location</div>
|
<div class="setting-name">Location</div>
|
||||||
<div class="setting-field">
|
<div class="setting-field">
|
||||||
<div id="flow-version-bucket-combo"></div>
|
<div id="save-flow-version-bucket-combo" class="hidden"></div>
|
||||||
<div id="flow-version-bucket" class="hidden"></div>
|
<div id="save-flow-version-bucket" class="hidden"></div>
|
||||||
</div>
|
</div>
|
||||||
</div>
|
</div>
|
||||||
<div class="setting">
|
<div class="setting">
|
||||||
<div class="setting-name">Name</div>
|
<div class="setting-name">Name</div>
|
||||||
<div class="setting-field">
|
<div class="setting-field">
|
||||||
<span id="flow-version-process-group-id" class="hidden"></span>
|
<span id="save-flow-version-process-group-id" class="hidden"></span>
|
||||||
<input type="text" id="flow-version-name" class="setting-input"/>
|
<input type="text" id="save-flow-version-name-field" class="setting-input hidden"/>
|
||||||
|
<div id="save-flow-version-name" class="hidden"></div>
|
||||||
</div>
|
</div>
|
||||||
</div>
|
</div>
|
||||||
<div class="setting">
|
<div class="setting">
|
||||||
<div class="setting-name">Description</div>
|
<div class="setting-name">Description</div>
|
||||||
<div class="setting-field">
|
<div class="setting-field">
|
||||||
<textarea id="flow-version-description" class="setting-input"></textarea>
|
<textarea id="save-flow-version-description-field" class="setting-input hidden"></textarea>
|
||||||
|
<div id="save-flow-version-description" class="hidden"></div>
|
||||||
</div>
|
</div>
|
||||||
</div>
|
</div>
|
||||||
<div class="setting">
|
<div class="setting">
|
||||||
<div class="setting-name">Change Comments</div>
|
<div class="setting-name">Comments</div>
|
||||||
<div class="setting-field">
|
<div class="setting-field">
|
||||||
<textarea id="flow-version-change-comments" class="setting-input"></textarea>
|
<textarea id="save-flow-version-change-comments" class="setting-input"></textarea>
|
||||||
</div>
|
</div>
|
||||||
</div>
|
</div>
|
||||||
</div>
|
</div>
|
||||||
|
|
|
@ -213,13 +213,27 @@ div.progress-label {
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
Flow Version
|
Save Flow Version
|
||||||
*/
|
*/
|
||||||
|
|
||||||
#flow-version-description, #flow-version-change-comments {
|
#save-flow-version-description-field, #save-flow-version-change-comments {
|
||||||
height: 85px;
|
height: 85px;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
Import Flow Version
|
||||||
|
*/
|
||||||
|
|
||||||
|
#import-flow-version-table {
|
||||||
|
overflow: hidden;
|
||||||
|
position: absolute;
|
||||||
|
top: 202px;
|
||||||
|
left: 0px;
|
||||||
|
right: 0px;
|
||||||
|
bottom: 0px;
|
||||||
|
height: 225px;
|
||||||
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
Variable Registry
|
Variable Registry
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -24,9 +24,10 @@
|
||||||
'nf.Birdseye',
|
'nf.Birdseye',
|
||||||
'nf.Graph',
|
'nf.Graph',
|
||||||
'nf.CanvasUtils',
|
'nf.CanvasUtils',
|
||||||
'nf.ErrorHandler'],
|
'nf.ErrorHandler',
|
||||||
function ($, nfClient, nfBirdseye, nfGraph, nfCanvasUtils, nfErrorHandler) {
|
'nf.Common'],
|
||||||
return (nf.ng.GroupComponent = factory($, nfClient, nfBirdseye, nfGraph, nfCanvasUtils, nfErrorHandler));
|
function ($, nfClient, nfBirdseye, nfGraph, nfCanvasUtils, nfErrorHandler, nfCommon) {
|
||||||
|
return (nf.ng.GroupComponent = factory($, nfClient, nfBirdseye, nfGraph, nfCanvasUtils, nfErrorHandler, nfCommon));
|
||||||
});
|
});
|
||||||
} else if (typeof exports === 'object' && typeof module === 'object') {
|
} else if (typeof exports === 'object' && typeof module === 'object') {
|
||||||
module.exports = (nf.ng.GroupComponent =
|
module.exports = (nf.ng.GroupComponent =
|
||||||
|
@ -35,16 +36,18 @@
|
||||||
require('nf.Birdseye'),
|
require('nf.Birdseye'),
|
||||||
require('nf.Graph'),
|
require('nf.Graph'),
|
||||||
require('nf.CanvasUtils'),
|
require('nf.CanvasUtils'),
|
||||||
require('nf.ErrorHandler')));
|
require('nf.ErrorHandler'),
|
||||||
|
require('nf.Common')));
|
||||||
} else {
|
} else {
|
||||||
nf.ng.GroupComponent = factory(root.$,
|
nf.ng.GroupComponent = factory(root.$,
|
||||||
root.nf.Client,
|
root.nf.Client,
|
||||||
root.nf.Birdseye,
|
root.nf.Birdseye,
|
||||||
root.nf.Graph,
|
root.nf.Graph,
|
||||||
root.nf.CanvasUtils,
|
root.nf.CanvasUtils,
|
||||||
root.nf.ErrorHandler);
|
root.nf.ErrorHandler,
|
||||||
|
root.nf.Common);
|
||||||
}
|
}
|
||||||
}(this, function ($, nfClient, nfBirdseye, nfGraph, nfCanvasUtils, nfErrorHandler) {
|
}(this, function ($, nfClient, nfBirdseye, nfGraph, nfCanvasUtils, nfErrorHandler, nfCommon) {
|
||||||
'use strict';
|
'use strict';
|
||||||
|
|
||||||
return function (serviceProvider) {
|
return function (serviceProvider) {
|
||||||
|
@ -126,6 +129,7 @@
|
||||||
handler: {
|
handler: {
|
||||||
close: function () {
|
close: function () {
|
||||||
$('#new-process-group-name').val('');
|
$('#new-process-group-name').val('');
|
||||||
|
$('#new-process-group-dialog').removeData('pt');
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -145,9 +149,24 @@
|
||||||
* Show the modal.
|
* Show the modal.
|
||||||
*/
|
*/
|
||||||
show: function () {
|
show: function () {
|
||||||
|
if (nfCommon.canVersionFlows()) {
|
||||||
|
$('#import-process-group-link').show();
|
||||||
|
} else {
|
||||||
|
$('#import-process-group-link').hide();
|
||||||
|
}
|
||||||
|
|
||||||
this.getElement().modal('show');
|
this.getElement().modal('show');
|
||||||
},
|
},
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Stores the pt.
|
||||||
|
*
|
||||||
|
* @param pt
|
||||||
|
*/
|
||||||
|
storePt: function (pt) {
|
||||||
|
$('#new-process-group-dialog').data('pt', pt);
|
||||||
|
},
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Hide the modal.
|
* Hide the modal.
|
||||||
*/
|
*/
|
||||||
|
@ -255,6 +274,7 @@
|
||||||
}]);
|
}]);
|
||||||
|
|
||||||
// show the dialog
|
// show the dialog
|
||||||
|
groupComponent.modal.storePt(pt);
|
||||||
groupComponent.modal.show();
|
groupComponent.modal.show();
|
||||||
|
|
||||||
// set up the focus and key handlers
|
// set up the focus and key handlers
|
||||||
|
|
|
@ -1254,14 +1254,14 @@
|
||||||
},
|
},
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reverts outstanding changes.
|
* Reverts local changes.
|
||||||
*/
|
*/
|
||||||
revertFlowChanges: function (selection) {
|
revertLocalChanges: function (selection) {
|
||||||
if (selection.empty()) {
|
if (selection.empty()) {
|
||||||
nfFlowVersion.revertFlowChanges(nfCanvasUtils.getGroupId());
|
nfFlowVersion.revertLocalChanges(nfCanvasUtils.getGroupId());
|
||||||
} else if (selection.size() === 1) {
|
} else if (selection.size() === 1) {
|
||||||
var selectionData = selection.datum();
|
var selectionData = selection.datum();
|
||||||
nfFlowVersion.revertFlowChanges(selectionData.id);
|
nfFlowVersion.revertLocalChanges(selectionData.id);
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
|
|
||||||
|
@ -1269,18 +1269,25 @@
|
||||||
* Changes the flow version.
|
* Changes the flow version.
|
||||||
*/
|
*/
|
||||||
changeFlowVersion: function (selection) {
|
changeFlowVersion: function (selection) {
|
||||||
|
if (selection.empty()) {
|
||||||
|
nfFlowVersion.showChangeFlowVersionDialog(nfCanvasUtils.getGroupId());
|
||||||
|
} else if (selection.size() === 1) {
|
||||||
|
var selectionData = selection.datum();
|
||||||
|
if (nfCanvasUtils.isProcessGroup(selection)) {
|
||||||
|
nfFlowVersion.showChangeFlowVersionDialog(selectionData.id);
|
||||||
|
}
|
||||||
|
}
|
||||||
},
|
},
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Disconnects a Process Group from flow versioning.
|
* Disconnects a Process Group from flow versioning.
|
||||||
*/
|
*/
|
||||||
disconnectFlowVersioning: function (selection) {
|
stopVersionControl: function (selection) {
|
||||||
if (selection.empty()) {
|
if (selection.empty()) {
|
||||||
nfFlowVersion.disconnectFlowVersioning(nfCanvasUtils.getGroupId());
|
nfFlowVersion.stopVersionControl(nfCanvasUtils.getGroupId());
|
||||||
} else if (selection.size() === 1) {
|
} else if (selection.size() === 1) {
|
||||||
var selectionData = selection.datum();
|
var selectionData = selection.datum();
|
||||||
nfFlowVersion.disconnectFlowVersioning(selectionData.id);
|
nfFlowVersion.stopVersionControl(selectionData.id);
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
|
|
||||||
|
|
|
@ -337,7 +337,6 @@
|
||||||
nfCanvas.setManagedAuthorizer(configDetails.supportsManagedAuthorizer);
|
nfCanvas.setManagedAuthorizer(configDetails.supportsManagedAuthorizer);
|
||||||
nfCanvas.setConfigurableAuthorizer(configDetails.supportsConfigurableAuthorizer);
|
nfCanvas.setConfigurableAuthorizer(configDetails.supportsConfigurableAuthorizer);
|
||||||
nfCanvas.setConfigurableUsersAndGroups(configDetails.supportsConfigurableUsersAndGroups);
|
nfCanvas.setConfigurableUsersAndGroups(configDetails.supportsConfigurableUsersAndGroups);
|
||||||
nfCanvas.setSupportsFlowVersioning(configDetails.supportsFlowVersioning);
|
|
||||||
|
|
||||||
// init nfStorage
|
// init nfStorage
|
||||||
nfStorage.init();
|
nfStorage.init();
|
||||||
|
@ -356,7 +355,7 @@
|
||||||
nfQueueListing.init();
|
nfQueueListing.init();
|
||||||
nfVariableRegistry.init();
|
nfVariableRegistry.init();
|
||||||
nfComponentState.init();
|
nfComponentState.init();
|
||||||
nfFlowVersion.init();
|
nfFlowVersion.init(configDetails.timeOffset);
|
||||||
nfComponentVersion.init(nfSettings);
|
nfComponentVersion.init(nfSettings);
|
||||||
|
|
||||||
// initialize the component behaviors
|
// initialize the component behaviors
|
||||||
|
|
|
@ -1819,13 +1819,6 @@
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns whether this NiFi supports flow versioning.
|
|
||||||
*/
|
|
||||||
supportsFlowVersioning: function () {
|
|
||||||
return nfCanvas.supportsFlowVersioning();
|
|
||||||
},
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns whether the authorizer is managed.
|
* Returns whether the authorizer is managed.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -85,7 +85,6 @@
|
||||||
var permissions = null;
|
var permissions = null;
|
||||||
var parentGroupId = null;
|
var parentGroupId = null;
|
||||||
var managedAuthorizer = false;
|
var managedAuthorizer = false;
|
||||||
var supportsFlowVersioning = false;
|
|
||||||
var configurableAuthorizer = false;
|
var configurableAuthorizer = false;
|
||||||
var configurableUsersAndGroups = false;
|
var configurableUsersAndGroups = false;
|
||||||
var svg = null;
|
var svg = null;
|
||||||
|
@ -909,23 +908,6 @@
|
||||||
return managedAuthorizer;
|
return managedAuthorizer;
|
||||||
},
|
},
|
||||||
|
|
||||||
/**
|
|
||||||
* Set whether this NiFi supports flow versioning.
|
|
||||||
*
|
|
||||||
* @param bool Whether this NiFi supports flow versioning
|
|
||||||
*/
|
|
||||||
setSupportsFlowVersioning: function (bool) {
|
|
||||||
supportsFlowVersioning = bool;
|
|
||||||
},
|
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
* @returns {boolean}
|
|
||||||
*/
|
|
||||||
supportsFlowVersioning: function () {
|
|
||||||
return supportsFlowVersioning;
|
|
||||||
},
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Set whether the authorizer is configurable.
|
* Set whether the authorizer is configurable.
|
||||||
*
|
*
|
||||||
|
|
|
@ -375,11 +375,17 @@
|
||||||
* @param selection
|
* @param selection
|
||||||
*/
|
*/
|
||||||
var supportsFlowVersioning = function (selection) {
|
var supportsFlowVersioning = function (selection) {
|
||||||
if (nfCanvasUtils.supportsFlowVersioning() === false) {
|
if (nfCommon.canVersionFlows() === false) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (selection.empty()) {
|
if (selection.empty()) {
|
||||||
|
// prevent versioning of the root group
|
||||||
|
if (nfCanvasUtils.getParentGroupId() === null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
// if not root group, ensure adequate permissions
|
||||||
return nfCanvasUtils.canReadCurrentGroup() && nfCanvasUtils.canWriteCurrentGroup();
|
return nfCanvasUtils.canReadCurrentGroup() && nfCanvasUtils.canWriteCurrentGroup();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -632,13 +638,13 @@
|
||||||
{id: 'variable-registry-menu-item', condition: hasVariables, menuItem: {clazz: 'fa', text: 'Variables', action: 'openVariableRegistry'}},
|
{id: 'variable-registry-menu-item', condition: hasVariables, menuItem: {clazz: 'fa', text: 'Variables', action: 'openVariableRegistry'}},
|
||||||
{separator: true},
|
{separator: true},
|
||||||
{id: 'version-menu-item', groupMenuItem: {clazz: 'fa', text: 'Version'}, menuItems: [
|
{id: 'version-menu-item', groupMenuItem: {clazz: 'fa', text: 'Version'}, menuItems: [
|
||||||
{id: 'start-version-control-menu-item', condition: supportsStartFlowVersioning, menuItem: {clazz: 'fa fa-floppy-o', text: 'Start version control', action: 'saveFlowVersion'}},
|
{id: 'start-version-control-menu-item', condition: supportsStartFlowVersioning, menuItem: {clazz: 'fa fa-upload', text: 'Start version control', action: 'saveFlowVersion'}},
|
||||||
{separator: true},
|
{separator: true},
|
||||||
{id: 'commit-menu-item', condition: supportsStopFlowVersioning, menuItem: {clazz: 'fa fa-floppy-o', text: 'Commit local changes', action: 'saveFlowVersion'}},
|
{id: 'commit-menu-item', condition: supportsStopFlowVersioning, menuItem: {clazz: 'fa fa-upload', text: 'Commit local changes', action: 'saveFlowVersion'}},
|
||||||
{id: 'revert-menu-item', condition: supportsStopFlowVersioning, menuItem: {clazz: 'fa', text: 'Revert local changes', action: 'revertFlowChanges'}},
|
{id: 'revert-menu-item', condition: supportsStopFlowVersioning, menuItem: {clazz: 'fa fa-undo', text: 'Revert local changes', action: 'revertLocalChanges'}},
|
||||||
{id: 'change-version-menu-item', condition: supportsStopFlowVersioning, menuItem: {clazz: 'fa', text: 'Change version', action: 'changeFlowVersion'}},
|
{id: 'change-version-menu-item', condition: supportsStopFlowVersioning, menuItem: {clazz: 'fa', text: 'Change version', action: 'changeFlowVersion'}},
|
||||||
{separator: true},
|
{separator: true},
|
||||||
{id: 'stop-version-control-menu-item', condition: supportsStopFlowVersioning, menuItem: {clazz: 'fa', text: 'Stop version control', action: 'disconnectFlowVersioning'}}
|
{id: 'stop-version-control-menu-item', condition: supportsStopFlowVersioning, menuItem: {clazz: 'fa', text: 'Stop version control', action: 'stopVersionControl'}}
|
||||||
]},
|
]},
|
||||||
{separator: true},
|
{separator: true},
|
||||||
{id: 'enter-group-menu-item', condition: isProcessGroup, menuItem: {clazz: 'fa fa-sign-in', text: 'Enter group', action: 'enterGroup'}},
|
{id: 'enter-group-menu-item', condition: isProcessGroup, menuItem: {clazz: 'fa fa-sign-in', text: 'Enter group', action: 'enterGroup'}},
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -82,8 +82,7 @@
|
||||||
reportingTaskTypes: '../nifi-api/flow/reporting-task-types',
|
reportingTaskTypes: '../nifi-api/flow/reporting-task-types',
|
||||||
createReportingTask: '../nifi-api/controller/reporting-tasks',
|
createReportingTask: '../nifi-api/controller/reporting-tasks',
|
||||||
reportingTasks: '../nifi-api/flow/reporting-tasks',
|
reportingTasks: '../nifi-api/flow/reporting-tasks',
|
||||||
createRegistry: '../nifi-api/controller/registries',
|
registries: '../nifi-api/controller/registry-clients'
|
||||||
registries: '../nifi-api/flow/registries'
|
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -486,7 +485,7 @@
|
||||||
// add the new registry
|
// add the new registry
|
||||||
var addRegistry = $.ajax({
|
var addRegistry = $.ajax({
|
||||||
type: 'POST',
|
type: 'POST',
|
||||||
url: config.urls.createRegistry,
|
url: config.urls.registries,
|
||||||
data: JSON.stringify(registryEntity),
|
data: JSON.stringify(registryEntity),
|
||||||
dataType: 'json',
|
dataType: 'json',
|
||||||
contentType: 'application/json'
|
contentType: 'application/json'
|
||||||
|
@ -877,7 +876,6 @@
|
||||||
// initialize the registry configuration dialog
|
// initialize the registry configuration dialog
|
||||||
$('#registry-configuration-dialog').modal({
|
$('#registry-configuration-dialog').modal({
|
||||||
scrollableContentStyle: 'scrollable',
|
scrollableContentStyle: 'scrollable',
|
||||||
headerText: 'Add Registry',
|
|
||||||
handler: {
|
handler: {
|
||||||
close: function () {
|
close: function () {
|
||||||
$('#registry-id').text('');
|
$('#registry-id').text('');
|
||||||
|
@ -1351,7 +1349,7 @@
|
||||||
$('#registry-description').val(registryEntity.component.description);
|
$('#registry-description').val(registryEntity.component.description);
|
||||||
|
|
||||||
// show the dialog
|
// show the dialog
|
||||||
$('#registry-configuration-dialog').modal('setButtonModel', [{
|
$('#registry-configuration-dialog').modal('setHeaderText', 'Edit Registry Client').modal('setButtonModel', [{
|
||||||
buttonText: 'Update',
|
buttonText: 'Update',
|
||||||
color: {
|
color: {
|
||||||
base: '#728E9B',
|
base: '#728E9B',
|
||||||
|
@ -1597,7 +1595,7 @@
|
||||||
name: 'Reporting Tasks',
|
name: 'Reporting Tasks',
|
||||||
tabContentId: 'reporting-tasks-tab-content'
|
tabContentId: 'reporting-tasks-tab-content'
|
||||||
}, {
|
}, {
|
||||||
name: 'Registries',
|
name: 'Registry Clients',
|
||||||
tabContentId: 'registries-tab-content'
|
tabContentId: 'registries-tab-content'
|
||||||
}],
|
}],
|
||||||
select: function () {
|
select: function () {
|
||||||
|
@ -1625,9 +1623,9 @@
|
||||||
} else if (tab === 'Reporting Tasks') {
|
} else if (tab === 'Reporting Tasks') {
|
||||||
$('#settings-save').hide();
|
$('#settings-save').hide();
|
||||||
return 'Create a new reporting task';
|
return 'Create a new reporting task';
|
||||||
} else if (tab === 'Registries') {
|
} else if (tab === 'Registry Clients') {
|
||||||
$('#settings-save').hide();
|
$('#settings-save').hide();
|
||||||
return 'Register a new registry';
|
return 'Register a new registry client';
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
} else {
|
} else {
|
||||||
|
@ -1637,7 +1635,7 @@
|
||||||
|
|
||||||
if (tab === 'Reporting Task Controller Services') {
|
if (tab === 'Reporting Task Controller Services') {
|
||||||
$('#controller-cs-availability').show();
|
$('#controller-cs-availability').show();
|
||||||
} else if (tab === 'Reporting Tasks' || tab === 'Registries') {
|
} else if (tab === 'Reporting Tasks' || tab === 'Registry Clients') {
|
||||||
$('#controller-cs-availability').hide();
|
$('#controller-cs-availability').hide();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1676,8 +1674,8 @@
|
||||||
|
|
||||||
// set the initial focus
|
// set the initial focus
|
||||||
$('#reporting-task-type-filter').focus();
|
$('#reporting-task-type-filter').focus();
|
||||||
} else if (selectedTab === 'Registries') {
|
} else if (selectedTab === 'Registry Clients') {
|
||||||
$('#registry-configuration-dialog').modal('setButtonModel', [{
|
$('#registry-configuration-dialog').modal('setHeaderText', 'Add Registry Client').modal('setButtonModel', [{
|
||||||
buttonText: 'Add',
|
buttonText: 'Add',
|
||||||
color: {
|
color: {
|
||||||
base: '#728E9B',
|
base: '#728E9B',
|
||||||
|
@ -1702,6 +1700,9 @@
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}]).modal('show');
|
}]).modal('show');
|
||||||
|
|
||||||
|
// set the initial focus
|
||||||
|
$('#registry-name').focus();
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
|
|
|
@ -556,6 +556,17 @@
|
||||||
return null;
|
return null;
|
||||||
},
|
},
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Determines whether the current user can version flows.
|
||||||
|
*/
|
||||||
|
canVersionFlows: function () {
|
||||||
|
if (nfCommon.isDefinedAndNotNull(nfCommon.currentUser)) {
|
||||||
|
return nfCommon.currentUser.canVersionFlows === true;
|
||||||
|
} else {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
},
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Determines whether the current user can access provenance.
|
* Determines whether the current user can access provenance.
|
||||||
*
|
*
|
||||||
|
|
Loading…
Reference in New Issue