You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by bs...@apache.org on 2023/10/24 12:08:31 UTC
[nifi] branch main updated: NIFI-12245: Removed deprecations around flow registry clients
This is an automated email from the ASF dual-hosted git repository.
bsimon pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git
The following commit(s) were added to refs/heads/main by this push:
new 1d5f2f626b NIFI-12245: Removed deprecations around flow registry clients
1d5f2f626b is described below
commit 1d5f2f626babb65d1d271bd46a638447e3f35c6f
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Thu Oct 19 08:38:25 2023 -0400
NIFI-12245: Removed deprecations around flow registry clients
NIFI-12245: Removed null check that is no longer necessary
NIFI-12245: Fixed checkstyle violation
Signed-off-by: Bence Simon <bs...@apache.org>
This closes #7899
---
.../apache/nifi/flow/VersionedFlowCoordinates.java | 21 +--
.../nifi/flow/VersionedFlowRegistryClient.java | 28 ----
.../java/org/apache/nifi/util/TestFormatUtils.java | 12 +-
.../flow/NifiRegistryFlowRegistryClient.java | 8 +-
.../nifi/web/api/dto/FlowRegistryClientDTO.java | 52 ++----
.../web/api/entity/FlowRegistryClientEntity.java | 6 -
.../html/HtmlDocumentationWriter.java | 32 ++--
.../nifi/controller/StandardProcessorNode.java | 1 -
.../StandardVersionedComponentSynchronizer.java | 6 +-
.../flow/StandardFlowRegistryClientNode.java | 2 +-
.../flow/mapping/NiFiRegistryFlowMapper.java | 96 +++++------
.../apache/nifi/util/FlowDifferenceFilters.java | 30 ----
.../StandardControllerServiceResolverTest.java | 6 +-
.../nifi/util/TestFlowDifferenceFilters.java | 47 ------
.../parent.json | 1 -
.../org/apache/nifi/controller/FlowController.java | 4 -
.../serialization/VersionedFlowSynchronizer.java | 48 +-----
.../TestWriteAheadFlowFileRepository.java | 3 +-
.../flow/mapping/NiFiRegistryFlowMapperTest.java | 25 ++-
.../apache/nifi/web/StandardNiFiServiceFacade.java | 5 -
.../apache/nifi/web/api/ControllerResource.java | 36 +----
.../java/org/apache/nifi/web/api/FlowResource.java | 7 -
.../nifi/web/dao/impl/StandardFlowRegistryDAO.java | 3 +-
.../main/webapp/js/nf/canvas/nf-flow-version.js | 2 +-
.../org/apache/nifi/ssl/SSLContextServiceTest.java | 18 ---
.../diff/ConciseEvolvingDifferenceDescriptor.java | 2 +-
.../registry/flow/diff/FlowDifferenceUtil.java | 50 ------
.../registry/flow/diff/StandardFlowComparator.java | 8 -
.../flow/diff/StaticDifferenceDescriptor.java | 2 +-
.../nifi/registry/service/RegistryService.java | 13 +-
.../service/alias/RegistryUrlAliasService.java | 175 ---------------------
.../nifi/registry/service/TestRegistryService.java | 5 +-
.../service/alias/RegistryUrlAliasServiceTest.java | 173 --------------------
.../apache/nifi/stateless/core/RegistryUtil.java | 16 +-
.../toolkit/cli/impl/command/CommandOption.java | 1 +
.../nifi/registry/CreateRegistryClient.java | 9 +-
.../impl/command/registry/flow/ImportAllFlows.java | 2 +-
.../impl/result/nifi/RegistryClientsResult.java | 16 +-
.../cli/impl/result/TestRegistryClientResult.java | 22 +--
39 files changed, 145 insertions(+), 848 deletions(-)
diff --git a/nifi-api/src/main/java/org/apache/nifi/flow/VersionedFlowCoordinates.java b/nifi-api/src/main/java/org/apache/nifi/flow/VersionedFlowCoordinates.java
index fe20dfe9b9..3d5fb2e47c 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flow/VersionedFlowCoordinates.java
+++ b/nifi-api/src/main/java/org/apache/nifi/flow/VersionedFlowCoordinates.java
@@ -24,7 +24,6 @@ import java.util.Objects;
public class VersionedFlowCoordinates {
private String registryId;
private String storageLocation;
- private String registryUrl;
private String bucketId;
private String flowId;
private int version;
@@ -48,17 +47,6 @@ public class VersionedFlowCoordinates {
this.storageLocation = storageLocation;
}
- @Deprecated
- @ApiModelProperty("The URL of the Flow Registry that contains the flow")
- public String getRegistryUrl() {
- return registryUrl;
- }
-
- @Deprecated
- public void setRegistryUrl(String registryUrl) {
- this.registryUrl = registryUrl;
- }
-
@ApiModelProperty("The UUID of the bucket that the flow resides in")
public String getBucketId() {
return bucketId;
@@ -97,7 +85,7 @@ public class VersionedFlowCoordinates {
@Override
public int hashCode() {
- return Objects.hash(registryId, storageLocation, registryUrl, bucketId, flowId, version);
+ return Objects.hash(registryId, storageLocation, bucketId, flowId, version);
}
@Override
@@ -108,16 +96,15 @@ public class VersionedFlowCoordinates {
if (obj == this) {
return true;
}
- if (!(obj instanceof VersionedFlowCoordinates)) {
+ if (!(obj instanceof final VersionedFlowCoordinates other)) {
return false;
}
- final VersionedFlowCoordinates other = (VersionedFlowCoordinates) obj;
- return Objects.equals(registryUrl, other.registryUrl) && Objects.equals(bucketId, other.bucketId) && Objects.equals(flowId, other.flowId) && Objects.equals(version, other.version);
+ return Objects.equals(storageLocation, other.storageLocation) && Objects.equals(bucketId, other.bucketId) && Objects.equals(flowId, other.flowId) && Objects.equals(version, other.version);
}
@Override
public String toString() {
- return "VersionedFlowCoordinates[bucketId=" + bucketId + ", flowId=" + flowId + ", version=" + version + ", registryUrl=" + registryUrl + "]";
+ return "VersionedFlowCoordinates[bucketId=" + bucketId + ", flowId=" + flowId + ", version=" + version + ", storageLocation=" + storageLocation + "]";
}
}
diff --git a/nifi-api/src/main/java/org/apache/nifi/flow/VersionedFlowRegistryClient.java b/nifi-api/src/main/java/org/apache/nifi/flow/VersionedFlowRegistryClient.java
index 24d189fe8e..b0212bd0e7 100644
--- a/nifi-api/src/main/java/org/apache/nifi/flow/VersionedFlowRegistryClient.java
+++ b/nifi-api/src/main/java/org/apache/nifi/flow/VersionedFlowRegistryClient.java
@@ -20,10 +20,6 @@ package org.apache.nifi.flow;
import io.swagger.annotations.ApiModelProperty;
public class VersionedFlowRegistryClient extends VersionedConfigurableExtension {
- @Deprecated
- private String id;
- @Deprecated
- private String url;
private String description;
private String annotationData;
@@ -32,30 +28,6 @@ public class VersionedFlowRegistryClient extends VersionedConfigurableExtension
return ComponentType.FLOW_REGISTRY_CLIENT;
}
- /**
- * @deprecated use {@link #getIdentifier()} instead.
- */
- @Deprecated
- @ApiModelProperty("The ID of the Registry. This method is deprecated. Use #getIdentifier instead.")
- public String getId() {
- return id;
- }
-
- public void setId(final String id) {
- this.id = id;
- }
-
- @Deprecated
- @ApiModelProperty("The URL for interacting with the registry")
- public String getUrl() {
- return url;
- }
-
- @Deprecated
- public void setUrl(final String url) {
- this.url = url;
- }
-
@ApiModelProperty("The description of the registry")
public String getDescription() {
return description;
diff --git a/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/util/TestFormatUtils.java b/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/util/TestFormatUtils.java
index 7c6b7e709f..454b086e26 100644
--- a/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/util/TestFormatUtils.java
+++ b/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/util/TestFormatUtils.java
@@ -46,7 +46,6 @@ public class TestFormatUtils {
private static final String KIEV_TIME_ZONE_ID = "Europe/Kiev";
private static final String UTC_TIME_ZONE_ID = ZoneOffset.UTC.getId();
- @SuppressWarnings("deprecation")
@ParameterizedTest
@MethodSource("getParse")
public void testParse(String value, TimeUnit desiredUnit, Long expected) {
@@ -63,14 +62,13 @@ public class TestFormatUtils {
Arguments.of("1 Hrs", TimeUnit.MINUTES, 60L));
}
- @SuppressWarnings("deprecation")
@ParameterizedTest
@ValueSource(strings = {"1 week", "1 wk", "1 w", "1 wks", "1 weeks"})
public void testGetTimeDurationShouldConvertWeeks(String week) {
assertEquals(7L, FormatUtils.getTimeDuration(week, TimeUnit.DAYS));
}
- @SuppressWarnings("deprecation")
+
@ParameterizedTest
@ValueSource(strings = {"-1 week", "-1 wk", "-1 w", "-1 weeks", "- 1 week"})
public void testGetTimeDurationShouldHandleNegativeWeeks(String week) {
@@ -82,7 +80,7 @@ public class TestFormatUtils {
/**
* Regression test
*/
- @SuppressWarnings("deprecation")
+
@ParameterizedTest
@ValueSource(strings = {"1 work", "1 wek", "1 k"})
public void testGetTimeDurationShouldHandleInvalidAbbreviations(String week) {
@@ -94,7 +92,7 @@ public class TestFormatUtils {
/**
* New feature test
*/
- @SuppressWarnings("deprecation")
+
@ParameterizedTest
@ValueSource(strings={"1week", "1wk", "1w", "1wks", "1weeks"})
public void testGetTimeDurationShouldHandleNoSpaceInInput(String week) {
@@ -104,7 +102,7 @@ public class TestFormatUtils {
/**
* New feature test
*/
- @SuppressWarnings("deprecation")
+
@ParameterizedTest
@ValueSource(strings={"10 ms", "10 millis", "10 milliseconds"})
public void testGetTimeDurationWithWholeNumbers(String whole){
@@ -114,7 +112,7 @@ public class TestFormatUtils {
/**
* New feature test
*/
- @SuppressWarnings("deprecation")
+
@ParameterizedTest
@ValueSource(strings={"0.010 s", "0.010 seconds"})
public void testGetTimeDurationWithDecimalNumbers(String decimal){
diff --git a/nifi-nar-bundles/nifi-flow-registry-client-bundle/nifi-flow-registry-client-services/src/main/java/org/apache/nifi/registry/flow/NifiRegistryFlowRegistryClient.java b/nifi-nar-bundles/nifi-flow-registry-client-bundle/nifi-flow-registry-client-services/src/main/java/org/apache/nifi/registry/flow/NifiRegistryFlowRegistryClient.java
index 8b7a45c801..d33e89ef08 100644
--- a/nifi-nar-bundles/nifi-flow-registry-client-bundle/nifi-flow-registry-client-services/src/main/java/org/apache/nifi/registry/flow/NifiRegistryFlowRegistryClient.java
+++ b/nifi-nar-bundles/nifi-flow-registry-client-bundle/nifi-flow-registry-client-services/src/main/java/org/apache/nifi/registry/flow/NifiRegistryFlowRegistryClient.java
@@ -43,6 +43,7 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
+import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
@@ -198,8 +199,11 @@ public class NifiRegistryFlowRegistryClient extends AbstractFlowRegistryClient {
try {
final FlowClient flowClient = getFlowClient(context);
- if (flowClient.getByBucket(flow.getBucketIdentifier()).stream().map(f -> f.getName()).collect(Collectors.toSet()).contains(flow.getName())) {
- throw new FlowAlreadyExistsException(String.format("Flow %s within bucket %s already exists", flow.getName(), flow.getBucketIdentifier()));
+ final List<VersionedFlow> versionedFlows = flowClient.getByBucket(flow.getBucketIdentifier());
+ final boolean matched = versionedFlows.stream()
+ .anyMatch(versionedFlow -> Objects.equals(versionedFlow.getName(), flow.getName()));
+ if (matched) {
+ throw new FlowAlreadyExistsException("Flow %s within bucket %s already exists".formatted(flow.getName(), flow.getBucketIdentifier()));
}
return NifiRegistryUtil.convert(flowClient.create(NifiRegistryUtil.convert(flow)));
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowRegistryClientDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowRegistryClientDTO.java
index fbc415e454..b522ca5db0 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowRegistryClientDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowRegistryClientDTO.java
@@ -36,9 +36,6 @@ public class FlowRegistryClientDTO {
private String name;
private String description;
- @Deprecated // URI is kept for handling legacy calls, but FlowRegistryClient implementations should depend on the properties.
- private String uri;
-
private String type;
private BundleDTO bundle;
@@ -83,17 +80,7 @@ public class FlowRegistryClientDTO {
this.description = description;
}
- @Deprecated
- public String getUri() {
- return uri;
- }
-
- @Deprecated
- public void setUri(String uri) {
- this.uri = uri;
- }
-
- @ApiModelProperty(value = "The type of the controller service.")
+ @ApiModelProperty(value = "The type of the registry client.")
public String getType() {
return type;
}
@@ -102,7 +89,7 @@ public class FlowRegistryClientDTO {
this.type = type;
}
- @ApiModelProperty(value = "The details of the artifact that bundled this processor type.")
+ @ApiModelProperty(value = "The details of the artifact that bundled this registry client type.")
public BundleDTO getBundle() {
return bundle;
}
@@ -111,7 +98,7 @@ public class FlowRegistryClientDTO {
this.bundle = bundle;
}
- @ApiModelProperty(value = "The properties of the controller service.")
+ @ApiModelProperty(value = "The properties of the registry client.")
public Map<String, String> getProperties() {
return properties;
}
@@ -120,7 +107,7 @@ public class FlowRegistryClientDTO {
this.properties = properties;
}
- @ApiModelProperty(value = "The descriptors for the controller service properties.")
+ @ApiModelProperty(value = "The descriptors for the registry client properties.")
public Map<String, PropertyDescriptorDTO> getDescriptors() {
return descriptors;
}
@@ -138,11 +125,8 @@ public class FlowRegistryClientDTO {
this.sensitiveDynamicPropertyNames = sensitiveDynamicPropertyNames;
}
- /**
- * @return whether this reporting task supports sensitive dynamic properties
- */
@ApiModelProperty(
- value = "Whether the reporting task supports sensitive dynamic properties."
+ value = "Whether the registry client supports sensitive dynamic properties."
)
public Boolean getSupportsSensitiveDynamicProperties() {
return supportsSensitiveDynamicProperties;
@@ -152,11 +136,8 @@ public class FlowRegistryClientDTO {
this.supportsSensitiveDynamicProperties = supportsSensitiveDynamicProperties;
}
- /**
- * @return whether this reporting task requires elevated privileges
- */
@ApiModelProperty(
- value = "Whether the reporting task requires elevated privileges."
+ value = "Whether the registry client requires elevated privileges."
)
public Boolean getRestricted() {
return restricted;
@@ -166,11 +147,8 @@ public class FlowRegistryClientDTO {
this.restricted = restricted;
}
- /**
- * @return Whether the reporting task has been deprecated.
- */
@ApiModelProperty(
- value = "Whether the reporting task has been deprecated."
+ value = "Whether the registry client has been deprecated."
)
public Boolean getDeprecated() {
return deprecated;
@@ -208,11 +186,8 @@ public class FlowRegistryClientDTO {
this.setMultipleVersionsAvailable = setMultipleVersionsAvailable;
}
- /**
- * @return currently configured annotation data for the reporting task
- */
@ApiModelProperty(
- value = "The annotation data for the repoting task. This is how the custom UI relays configuration to the reporting task."
+ value = "The annotation data for the registry client. This is how the custom UI relays configuration to the registry client."
)
public String getAnnotationData() {
return annotationData;
@@ -222,14 +197,9 @@ public class FlowRegistryClientDTO {
this.annotationData = annotationData;
}
- /**
- * Gets the validation errors from this reporting task. These validation errors represent the problems with the reporting task that must be resolved before it can be scheduled to run.
- *
- * @return The validation errors
- */
@ApiModelProperty(
- value = "Gets the validation errors from the reporting task. These validation errors represent the problems with the reporting task that must be resolved before "
- + "it can be scheduled to run."
+ value = "Gets the validation errors from the registry client. These validation errors represent the problems with the registry client that must be resolved before "
+ + "it can be used for interacting with the flow registry."
)
public Collection<String> getValidationErrors() {
return validationErrors;
@@ -239,7 +209,7 @@ public class FlowRegistryClientDTO {
this.validationErrors = validationErrors;
}
- @ApiModelProperty(value = "Indicates whether the Processor is valid, invalid, or still in the process of validating (i.e., it is unknown whether or not the Processor is valid)",
+ @ApiModelProperty(value = "Indicates whether the Registry Client is valid, invalid, or still in the process of validating (i.e., it is unknown whether or not the Registry Client is valid)",
accessMode = ApiModelProperty.AccessMode.READ_ONLY,
allowableValues = VALID + ", " + INVALID + ", " + VALIDATING)
public String getValidationStatus() {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/FlowRegistryClientEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/FlowRegistryClientEntity.java
index a427f63053..c35f32ed72 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/FlowRegistryClientEntity.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/FlowRegistryClientEntity.java
@@ -34,12 +34,6 @@ public class FlowRegistryClientEntity extends ComponentEntity implements Permiss
return registry;
}
- // Note: this duplication is needed for the {@code org.apache.nifi.web.api.FlowResource} to maintain backward compatibility}
- @Deprecated
- public FlowRegistryClientDTO getRegistry() {
- return registry;
- }
-
public void setComponent(FlowRegistryClientDTO component) {
this.registry = component;
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
index 853a233cfe..9bda0d5519 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-documentation/src/main/java/org/apache/nifi/documentation/html/HtmlDocumentationWriter.java
@@ -16,20 +16,6 @@
*/
package org.apache.nifi.documentation.html;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-import javax.xml.stream.FactoryConfigurationError;
-import javax.xml.stream.XMLOutputFactory;
-import javax.xml.stream.XMLStreamException;
-import javax.xml.stream.XMLStreamWriter;
import org.apache.nifi.annotation.behavior.DynamicProperties;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.InputRequirement;
@@ -63,6 +49,21 @@ import org.apache.nifi.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import javax.xml.stream.FactoryConfigurationError;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamWriter;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
/**
* Generates HTML documentation for a ConfigurableComponent. This class is used
* to generate documentation for ControllerService, ParameterProvider, and ReportingTask because
@@ -831,7 +832,8 @@ public class HtmlDocumentationWriter implements DocumentationWriter {
}
return false;
}
- @SuppressWarnings("deprecation")
+
+
private void writeDynamicProperties(final ConfigurableComponent configurableComponent,
final XMLStreamWriter xmlStreamWriter) throws XMLStreamException {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
index 17e4164616..6d5f093da1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java
@@ -355,7 +355,6 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
}
@Override
- @SuppressWarnings("deprecation")
public boolean isIsolated() {
return executionNode == ExecutionNode.PRIMARY;
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java
index 3a2a5f76fc..55b6f0fc01 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/flow/synchronization/StandardVersionedComponentSynchronizer.java
@@ -502,11 +502,7 @@ public class StandardVersionedComponentSynchronizer implements VersionedComponen
}
}
- final String location = coordinates.getStorageLocation() == null ? coordinates.getRegistryUrl() : coordinates.getStorageLocation();
- if (location == null) {
- return null;
- }
-
+ final String location = coordinates.getStorageLocation();
for (final FlowRegistryClientNode flowRegistryClientNode : context.getFlowManager().getAllFlowRegistryClients()) {
final boolean locationApplicable;
try {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClientNode.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClientNode.java
index a95e00e559..769b7b4434 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClientNode.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/StandardFlowRegistryClientNode.java
@@ -367,7 +367,7 @@ public final class StandardFlowRegistryClientNode extends AbstractComponentNode
private RegisteredFlowSnapshot fetchFlowContents(final FlowRegistryClientUserContext context, final VersionedFlowCoordinates coordinates,
final boolean fetchRemoteFlows) throws FlowRegistryException {
- final String storageLocation = coordinates.getStorageLocation() == null ? coordinates.getRegistryUrl() : coordinates.getStorageLocation();
+ final String storageLocation = coordinates.getStorageLocation();
final String bucketId = coordinates.getBucketId();
final String flowId = coordinates.getFlowId();
final int version = coordinates.getVersion();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
index b0ac1b8909..6920fe9e97 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapper.java
@@ -18,7 +18,6 @@
package org.apache.nifi.registry.flow.mapping;
import org.apache.commons.lang3.ClassUtils;
-import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.resource.ResourceCardinality;
@@ -105,7 +104,6 @@ import java.util.stream.Collectors;
public class NiFiRegistryFlowMapper {
private static final String ENCRYPTED_PREFIX = "enc{";
private static final String ENCRYPTED_SUFFIX = "}";
- private static final String REGISTRY_URL_DESCRIPTOR_NAME = "url";
private final ExtensionManager extensionManager;
private final FlowMappingOptions flowMappingOptions;
@@ -114,7 +112,7 @@ public class NiFiRegistryFlowMapper {
// we call #mapConnectable, instead of generating a new UUID for the ConnectableComponent, we can lookup the 'versioned'
// identifier based on the component's actual id. We do connections last, so that all components will already have been
// created before attempting to create the connection, where the ConnectableDTO is converted.
- private Map<String, String> versionedComponentIds = new HashMap<>();
+ private final Map<String, String> versionedComponentIds = new HashMap<>();
public NiFiRegistryFlowMapper(final ExtensionManager extensionManager) {
this(extensionManager, FlowMappingOptions.DEFAULT_OPTIONS);
@@ -156,74 +154,51 @@ public class NiFiRegistryFlowMapper {
// apply registry versioning according to the lambda below
// NOTE: lambda refers to registry client and map descendant boolean which will not change during recursion
- return mapGroup(group, serviceProvider, (processGroup, versionedGroup) -> {
- final VersionControlInformation versionControlInfo = processGroup.getVersionControlInformation();
- if (versionControlInfo != null) {
- final VersionedFlowCoordinates coordinates = new VersionedFlowCoordinates();
- final String registryId = versionControlInfo.getRegistryIdentifier();
- final FlowRegistryClientNode registry = flowManager.getFlowRegistryClient(registryId);
- if (registry == null) {
- throw new IllegalStateException("Process Group refers to a Flow Registry with ID " + registryId + " but no Flow Registry exists with that ID. Cannot resolve to a URL.");
- }
-
- if (flowMappingOptions.isMapFlowRegistryClientId()) {
- coordinates.setRegistryId(registryId);
- }
-
- coordinates.setRegistryUrl(getRegistryUrl(registry));
-
- final String storageLocation = determineStorageLocation(registry, versionControlInfo);
- coordinates.setStorageLocation(storageLocation);
- coordinates.setBucketId(versionControlInfo.getBucketIdentifier());
- coordinates.setFlowId(versionControlInfo.getFlowIdentifier());
- coordinates.setVersion(versionControlInfo.getVersion());
- versionedGroup.setVersionedFlowCoordinates(coordinates);
+ final BiFunction<ProcessGroup, VersionedProcessGroup, Boolean> applyVersionControlInfo =
+ (processGroup, versionedGroup) -> applyVersionControlInformation(processGroup, versionedGroup, flowManager, mapDescendantVersionedFlows);
- // We need to register the Port ID -> Versioned Component ID's in our versionedComponentIds member variable for all input & output ports.
- // Otherwise, we will not be able to lookup the port when connecting to it.
- for (final Port port : processGroup.getInputPorts()) {
- getId(port.getVersionedComponentId(), port.getIdentifier());
- }
- for (final Port port : processGroup.getOutputPorts()) {
- getId(port.getVersionedComponentId(), port.getIdentifier());
- }
+ return mapGroup(group, serviceProvider, applyVersionControlInfo);
+ }
- // If the Process Group itself is remotely versioned, then we don't want to include its contents
- // because the contents are remotely managed and not part of the versioning of this Process Group
- return mapDescendantVersionedFlows;
- }
+ private boolean applyVersionControlInformation(final ProcessGroup processGroup, final VersionedProcessGroup versionedGroup, final FlowManager flowManager,
+ final boolean mapDescendantVersionedFlows) {
+ final VersionControlInformation versionControlInfo = processGroup.getVersionControlInformation();
+ if (versionControlInfo == null) {
return true;
- });
- }
+ }
+ final VersionedFlowCoordinates coordinates = new VersionedFlowCoordinates();
+ final String registryId = versionControlInfo.getRegistryIdentifier();
+ final FlowRegistryClientNode registry = flowManager.getFlowRegistryClient(registryId);
+ if (registry == null) {
+ throw new IllegalStateException("Process Group refers to a Flow Registry with ID " + registryId + " but no Flow Registry exists with that ID. Cannot resolve to a URL.");
+ }
- private boolean isNiFiRegistryClient(final FlowRegistryClientNode clientNode) {
- return clientNode.getComponentType().endsWith("NifiRegistryFlowRegistryClient");
- }
+ if (flowMappingOptions.isMapFlowRegistryClientId()) {
+ coordinates.setRegistryId(registryId);
+ }
- // This is specific for the {@code NifiRegistryFlowRegistryClient}, purely for backward compatibility
- private String getRegistryUrl(final FlowRegistryClientNode registry) {
- return isNiFiRegistryClient(registry) ? registry.getRawPropertyValue(registry.getPropertyDescriptor(REGISTRY_URL_DESCRIPTOR_NAME)) : "";
- }
+ final String storageLocation = versionControlInfo.getStorageLocation();
+ coordinates.setStorageLocation(storageLocation);
+ coordinates.setBucketId(versionControlInfo.getBucketIdentifier());
+ coordinates.setFlowId(versionControlInfo.getFlowIdentifier());
+ coordinates.setVersion(versionControlInfo.getVersion());
+ versionedGroup.setVersionedFlowCoordinates(coordinates);
- private String determineStorageLocation(final FlowRegistryClientNode registryClient, final VersionControlInformation versionControlInformation) {
- final String explicitStorageLocation = versionControlInformation.getStorageLocation();
- if (!StringUtils.isEmpty(explicitStorageLocation)) {
- return explicitStorageLocation;
+ // We need to register the Port ID -> Versioned Component ID's in our versionedComponentIds member variable for all input & output ports.
+ // Otherwise, we will not be able to lookup the port when connecting to it.
+ for (final Port port : processGroup.getInputPorts()) {
+ getId(port.getVersionedComponentId(), port.getIdentifier());
}
-
- final String registryUrl = getRegistryUrl(registryClient);
- if (StringUtils.isEmpty(registryUrl)) {
- return "";
+ for (final Port port : processGroup.getOutputPorts()) {
+ getId(port.getVersionedComponentId(), port.getIdentifier());
}
- final String bucketId = versionControlInformation.getBucketIdentifier();
- final String flowId = versionControlInformation.getFlowIdentifier();
- final int version = versionControlInformation.getVersion();
- return String.format("%s/nifi-registry-api/buckets/%s/flows/%s/versions/%s", registryUrl, bucketId, flowId, version);
+ // If the Process Group itself is remotely versioned, then we don't want to include its contents
+ // because the contents are remotely managed and not part of the versioning of this Process Group
+ return mapDescendantVersionedFlows;
}
-
private InstantiatedVersionedProcessGroup mapGroup(final ProcessGroup group, final ControllerServiceProvider serviceProvider,
final BiFunction<ProcessGroup, VersionedProcessGroup, Boolean> applyVersionControlInfo) {
final Set<String> allIncludedGroupsIds = group.findAllProcessGroups().stream()
@@ -399,8 +374,7 @@ public class NiFiRegistryFlowMapper {
component.setComments(connectable.getComments());
final String groupId;
- if (connectable instanceof RemoteGroupPort) {
- final RemoteGroupPort port = (RemoteGroupPort) connectable;
+ if (connectable instanceof final RemoteGroupPort port) {
final RemoteProcessGroup rpg = port.getRemoteProcessGroup();
groupId = getIdOrThrow(rpg.getIdentifier(),
() -> new IllegalArgumentException("Unable to find the Versioned Component ID for Remote Process Group that " + connectable + " belongs to"));
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/util/FlowDifferenceFilters.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/util/FlowDifferenceFilters.java
index b673d94806..d8334d771b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/util/FlowDifferenceFilters.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/util/FlowDifferenceFilters.java
@@ -33,7 +33,6 @@ import org.apache.nifi.flow.VersionedProcessor;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.registry.flow.diff.DifferenceType;
import org.apache.nifi.registry.flow.diff.FlowDifference;
-import org.apache.nifi.registry.flow.diff.FlowDifferenceUtil;
import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedComponent;
import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedControllerService;
import org.apache.nifi.registry.flow.mapping.InstantiatedVersionedProcessor;
@@ -61,7 +60,6 @@ public class FlowDifferenceFilters {
|| isRpgUrlChange(difference)
|| isAddedOrRemovedRemotePort(difference)
|| isPublicPortNameChange(difference)
- || isIgnorableVersionedFlowCoordinateChange(difference)
|| isNewPropertyWithDefaultValue(difference, flowManager)
|| isNewRelationshipAutoTerminatedAndDefaulted(difference, localGroup, flowManager)
|| isScheduledStateNew(difference)
@@ -177,34 +175,6 @@ public class FlowDifferenceFilters {
return false;
}
- public static Predicate<FlowDifference> FILTER_IGNORABLE_VERSIONED_FLOW_COORDINATE_CHANGES = (fd) -> !isIgnorableVersionedFlowCoordinateChange(fd);
-
- public static boolean isIgnorableVersionedFlowCoordinateChange(final FlowDifference fd) {
- if (fd.getDifferenceType() == DifferenceType.VERSIONED_FLOW_COORDINATES_CHANGED) {
- final VersionedComponent componentA = fd.getComponentA();
- final VersionedComponent componentB = fd.getComponentB();
-
- if (componentA instanceof VersionedProcessGroup && componentB instanceof VersionedProcessGroup) {
- final VersionedProcessGroup versionedProcessGroupA = (VersionedProcessGroup) componentA;
- final VersionedProcessGroup versionedProcessGroupB = (VersionedProcessGroup) componentB;
-
- final VersionedFlowCoordinates coordinatesA = versionedProcessGroupA.getVersionedFlowCoordinates();
- final VersionedFlowCoordinates coordinatesB = versionedProcessGroupB.getVersionedFlowCoordinates();
-
- if (coordinatesA != null && coordinatesB != null) {
- if (coordinatesA.getStorageLocation() != null || coordinatesB.getStorageLocation() != null) {
- return false;
- }
-
- return !FlowDifferenceUtil.areRegistryStrictlyEqual(coordinatesA, coordinatesB)
- && FlowDifferenceUtil.areRegistryUrlsEqual(coordinatesA, coordinatesB)
- && coordinatesA.getVersion() == coordinatesB.getVersion();
- }
- }
- }
-
- return false;
- }
private static boolean isNewZIndexLabelConfigWithDefaultValue(final FlowDifference fd, final FlowManager flowManager) {
final Object valueA = fd.getValueA();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceResolverTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceResolverTest.java
index 89d54c5ef2..14ba112030 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceResolverTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceResolverTest.java
@@ -54,8 +54,6 @@ public class StandardControllerServiceResolverTest {
private static final String CHILD_REFERENCES_SERVICES_FROM_PARENT_LOCATION = BASE_SNAPSHOT_LOCATION + "/versioned-child-services-from-parent";
private static final String STANDARD_EXTERNAL_SERVICE_REFERENCE = BASE_SNAPSHOT_LOCATION + "/standard-external-service-reference";
- private Authorizer authorizer;
- private FlowManager flowManager;
private NiFiRegistryFlowMapper flowMapper;
private ControllerServiceProvider controllerServiceProvider;
private ControllerServiceApiLookup controllerServiceApiLookup;
@@ -69,8 +67,8 @@ public class StandardControllerServiceResolverTest {
@BeforeEach
public void setup() {
- authorizer = mock(Authorizer.class);
- flowManager = mock(FlowManager.class);
+ Authorizer authorizer = mock(Authorizer.class);
+ FlowManager flowManager = mock(FlowManager.class);
flowMapper = mock(NiFiRegistryFlowMapper.class);
controllerServiceProvider = mock(ControllerServiceProvider.class);
controllerServiceApiLookup = mock(ControllerServiceApiLookup.class);
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/util/TestFlowDifferenceFilters.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/util/TestFlowDifferenceFilters.java
index b87bebc01c..9ee689dd53 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/util/TestFlowDifferenceFilters.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/java/org/apache/nifi/util/TestFlowDifferenceFilters.java
@@ -19,9 +19,7 @@ package org.apache.nifi.util;
import org.apache.nifi.flow.ComponentType;
import org.apache.nifi.flow.ScheduledState;
import org.apache.nifi.flow.VersionedControllerService;
-import org.apache.nifi.flow.VersionedFlowCoordinates;
import org.apache.nifi.flow.VersionedPort;
-import org.apache.nifi.flow.VersionedProcessGroup;
import org.apache.nifi.flow.VersionedProcessor;
import org.apache.nifi.flow.VersionedRemoteGroupPort;
import org.apache.nifi.registry.flow.diff.DifferenceType;
@@ -81,51 +79,6 @@ public class TestFlowDifferenceFilters {
assertTrue(FlowDifferenceFilters.FILTER_ADDED_REMOVED_REMOTE_PORTS.test(flowDifference));
}
- @Test
- public void testFilterIgnorableVersionedCoordinateDifferencesWithIgnorableDifference() {
- VersionedFlowCoordinates coordinatesA = new VersionedFlowCoordinates();
- coordinatesA.setRegistryUrl("http://localhost:18080");
-
- VersionedProcessGroup processGroupA = new VersionedProcessGroup();
- processGroupA.setVersionedFlowCoordinates(coordinatesA);
-
- VersionedFlowCoordinates coordinatesB = new VersionedFlowCoordinates();
- coordinatesB.setRegistryUrl("http://localhost:18080/");
-
- VersionedProcessGroup processGroupB = new VersionedProcessGroup();
- processGroupB.setVersionedFlowCoordinates(coordinatesB);
-
- StandardFlowDifference flowDifference = new StandardFlowDifference(
- DifferenceType.VERSIONED_FLOW_COORDINATES_CHANGED,
- processGroupA, processGroupB,
- coordinatesA.getRegistryUrl(), coordinatesB.getRegistryUrl(),
- "");
-
- assertFalse(FlowDifferenceFilters.FILTER_IGNORABLE_VERSIONED_FLOW_COORDINATE_CHANGES.test(flowDifference));
- }
-
- @Test
- public void testFilterIgnorableVersionedCoordinateDifferencesWithNonIgnorableDifference() {
- VersionedFlowCoordinates coordinatesA = new VersionedFlowCoordinates();
- coordinatesA.setRegistryUrl("http://localhost:18080");
-
- VersionedProcessGroup processGroupA = new VersionedProcessGroup();
- processGroupA.setVersionedFlowCoordinates(coordinatesA);
-
- VersionedFlowCoordinates coordinatesB = new VersionedFlowCoordinates();
- coordinatesB.setRegistryUrl("http://localhost:18080");
-
- VersionedProcessGroup processGroupB = new VersionedProcessGroup();
- processGroupB.setVersionedFlowCoordinates(coordinatesB);
-
- StandardFlowDifference flowDifference = new StandardFlowDifference(
- DifferenceType.VERSIONED_FLOW_COORDINATES_CHANGED,
- processGroupA, processGroupB,
- coordinatesA.getRegistryUrl(), coordinatesB.getRegistryUrl(),
- "");
-
- assertTrue(FlowDifferenceFilters.FILTER_IGNORABLE_VERSIONED_FLOW_COORDINATE_CHANGES.test(flowDifference));
- }
@Test
public void testFilterPublicPortNameChangeWhenNotNameChange() {
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/resources/snapshots/versioned-child-services-from-parent/parent.json b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/resources/snapshots/versioned-child-services-from-parent/parent.json
index a2f458cf9f..03f1017df9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/resources/snapshots/versioned-child-services-from-parent/parent.json
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/test/resources/snapshots/versioned-child-services-from-parent/parent.json
@@ -237,7 +237,6 @@
"versionedFlowCoordinates": {
"bucketId": "6e767fa6-5113-46f0-9b7e-8024b75a40c1",
"flowId": "2192834e-60d7-413b-9ea9-dc4187824a41",
- "registryUrl": "http://localhost:18080",
"storageLocation": "http://localhost:18080/nifi-registry-api/buckets/6e767fa6-5113-46f0-9b7e-8024b75a40c1/flows/2192834e-60d7-413b-9ea9-dc4187824a41/versions/2",
"version": 2
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index a7ef827fa7..e811a0f711 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -269,9 +269,6 @@ public class FlowController implements ReportingTaskProvider, FlowAnalysisRulePr
public static final String GRACEFUL_SHUTDOWN_PERIOD = "nifi.flowcontroller.graceful.shutdown.seconds";
public static final long DEFAULT_GRACEFUL_SHUTDOWN_SECONDS = 10;
- // default properties for scaling the positions of components from pre-1.0 flow encoding versions.
- public static final double DEFAULT_POSITION_SCALE_FACTOR_X = 1.5;
- public static final double DEFAULT_POSITION_SCALE_FACTOR_Y = 1.34;
private final AtomicInteger maxTimerDrivenThreads;
private final AtomicReference<FlowEngine> timerDrivenEngineRef;
@@ -465,7 +462,6 @@ public class FlowController implements ReportingTaskProvider, FlowAnalysisRulePr
return flowController;
}
- @SuppressWarnings("deprecation")
private FlowController(
final FlowFileEventRepository flowFileEventRepo,
final NiFiProperties nifiProperties,
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizer.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizer.java
index 18341d5317..b667519a8b 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizer.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/serialization/VersionedFlowSynchronizer.java
@@ -130,11 +130,6 @@ import static org.apache.nifi.controller.serialization.FlowSynchronizationUtils.
public class VersionedFlowSynchronizer implements FlowSynchronizer {
private static final Logger logger = LoggerFactory.getLogger(VersionedFlowSynchronizer.class);
- /**
- * The Registry Client Type to use for registry clients that are configured using the deprecated style
- */
- private static final String DEPRECATED_FLOW_REGISTRY_CLIENT_TYPE = "org.apache.nifi.registry.flow.NifiRegistryFlowRegistryClient";
- private static final BundleCoordinate DEPRECATED_FLOW_REGISTRY_BUNDLE = new BundleCoordinate("org.apache.nifi", "nifi-flow-registry-client-nar", "1.18.0");
private final ExtensionManager extensionManager;
private final File flowStorageFile;
@@ -293,10 +288,8 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
if (dataflow.getRegistries() == null) {
dataflow.setRegistries(new ArrayList<>());
}
+
for (final VersionedFlowRegistryClient registry : dataflow.getRegistries()) {
- if (isOldStyleRegistryClient(registry)) {
- continue;
- }
if (missingComponentIds.contains(registry.getInstanceIdentifier())) {
continue;
}
@@ -570,11 +563,6 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
}
private void addFlowRegistryClient(final FlowController flowController, final VersionedFlowRegistryClient versionedFlowRegistryClient) {
- if (isOldStyleRegistryClient(versionedFlowRegistryClient)) {
- addOldStyleRegistryClient(flowController.getFlowManager(), versionedFlowRegistryClient);
- return;
- }
-
final BundleCoordinate coordinate = createBundleCoordinate(extensionManager, versionedFlowRegistryClient.getBundle(), versionedFlowRegistryClient.getType());
final FlowRegistryClientNode flowRegistryClient = flowController.getFlowManager().createFlowRegistryClient(
@@ -582,40 +570,6 @@ public class VersionedFlowSynchronizer implements FlowSynchronizer {
updateRegistry(flowRegistryClient, versionedFlowRegistryClient, flowController);
}
- /**
- * Checks if hte given VersionedFlowRegistryClient matches the old configuration style, which was used before Registry Clients
- * were made into an extension point
- * @param client the client to check
- * @return <code>true</code> if the client is from an older configuration, <code>false</code> otherwise.
- */
- private boolean isOldStyleRegistryClient(final VersionedFlowRegistryClient client) {
- return client.getIdentifier() != null && client.getIdentifier() == null && client.getBundle() == null;
- }
-
- /**
- * Creates and adds to the flow a Flow Registry Client using the old style configuration for the VersionedFlowRegistryClient
- * @param flowManager the flow manager
- * @param client the versioned client
- */
- private void addOldStyleRegistryClient(final FlowManager flowManager, final VersionedFlowRegistryClient client) {
- BundleCoordinate chosenCoordinate = getCompatibleBundle(DEPRECATED_FLOW_REGISTRY_BUNDLE, extensionManager, DEPRECATED_FLOW_REGISTRY_CLIENT_TYPE);
- if (chosenCoordinate == null) {
- // If unable to find a compatible bundle coordinate just use the deprecated coordinates, which can create a Ghosted component
- chosenCoordinate = DEPRECATED_FLOW_REGISTRY_BUNDLE;
- }
-
- final FlowRegistryClientNode flowRegistryClient = flowManager.createFlowRegistryClient(DEPRECATED_FLOW_REGISTRY_CLIENT_TYPE, client.getIdentifier(),
- chosenCoordinate, Collections.emptySet(), false,true, null);
-
- flowRegistryClient.setName(client.getName());
- flowRegistryClient.setDescription(client.getDescription());
- flowRegistryClient.setAnnotationData(null);
-
- final Map<String, String> properties = new HashMap<>();
- properties.put("url", client.getUrl());
- flowRegistryClient.setProperties(properties, false, Collections.emptySet());
- }
-
private void updateRegistry(final FlowRegistryClientNode flowRegistryClient, final VersionedFlowRegistryClient versionedFlowRegistryClient, final FlowController flowController) {
flowRegistryClient.setName(versionedFlowRegistryClient.getName());
flowRegistryClient.setDescription(versionedFlowRegistryClient.getDescription());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java
index e7cc90b187..7f7347a77e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestWriteAheadFlowFileRepository.java
@@ -19,7 +19,6 @@ package org.apache.nifi.controller.repository;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.controller.queue.DropFlowFileStatus;
-import org.apache.nifi.controller.status.FlowFileAvailability;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.apache.nifi.controller.queue.FlowFileQueueSize;
import org.apache.nifi.controller.queue.ListFlowFileStatus;
@@ -36,6 +35,7 @@ import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
import org.apache.nifi.controller.repository.claim.StandardContentClaim;
import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager;
+import org.apache.nifi.controller.status.FlowFileAvailability;
import org.apache.nifi.controller.swap.StandardSwapContents;
import org.apache.nifi.controller.swap.StandardSwapSummary;
import org.apache.nifi.flowfile.FlowFilePrioritizer;
@@ -77,7 +77,6 @@ import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.when;
-@SuppressWarnings("deprecation")
public class TestWriteAheadFlowFileRepository {
private static NiFiProperties niFiProperties;
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapperTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapperTest.java
index 06f2b6e00b..a4d1dda842 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapperTest.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/registry/flow/mapping/NiFiRegistryFlowMapperTest.java
@@ -105,6 +105,7 @@ import java.util.Set;
import java.util.UUID;
import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -225,11 +226,11 @@ public class NiFiRegistryFlowMapperTest {
// prepare a versioned process group with a nested versioned process group, each with 1 processor
final ProcessGroup innerProcessGroup =
prepareProcessGroup(1,false, false, false,
- false, false,null,
+ false, null,
false, true, Collections.emptyList());
final ProcessGroup processGroup =
prepareProcessGroup(1,false,false, false,
- false, false, null,
+ false, null,
false, true, Collections.singletonList(innerProcessGroup));
final List<ProcessGroup> allProcessGroups = Collections.singletonList(innerProcessGroup);
@@ -237,13 +238,11 @@ public class NiFiRegistryFlowMapperTest {
// perform the mapping, excluding descendant versioned flows
final InstantiatedVersionedProcessGroup versionedProcessGroup =
- flowMapper.mapProcessGroup(processGroup, controllerServiceProvider, flowManager,
- false);
- final VersionedProcessGroup innerVersionedProcessGroup =
- versionedProcessGroup.getProcessGroups().iterator().next();
+ flowMapper.mapProcessGroup(processGroup, controllerServiceProvider, flowManager, false);
+ final VersionedProcessGroup innerVersionedProcessGroup = versionedProcessGroup.getProcessGroups().iterator().next();
// ensure the Registry URL has been set correctly in the flowManager
- assert(StringUtils.isNotEmpty(innerVersionedProcessGroup.getVersionedFlowCoordinates().getStorageLocation()));
+ assertFalse(StringUtils.isEmpty(innerVersionedProcessGroup.getVersionedFlowCoordinates().getStorageLocation()));
// verify root versioned process group contents only
verifyVersionedProcessGroup(processGroup, versionedProcessGroup,false,false);
@@ -265,15 +264,15 @@ public class NiFiRegistryFlowMapperTest {
// prepare a process group with nested process groups
final ProcessGroup innerInnerProcessGroup =
prepareProcessGroup(0,false, true, false,
- true, false,null,
+ false, null,
true, false, Collections.emptyList());
final ProcessGroup innerProcessGroup =
prepareProcessGroup(1,true, false, false,
- true, true, externalControllerServiceNode,
+ true, externalControllerServiceNode,
true, true, Collections.singletonList(innerInnerProcessGroup));
final ProcessGroup processGroup =
prepareProcessGroup(2,false,false, true,
- false, true, null,
+ true, null,
false, true, Collections.singletonList(innerProcessGroup));
final List<ProcessGroup> allProcessGroups = Arrays.asList(innerProcessGroup, innerInnerProcessGroup);
@@ -390,8 +389,7 @@ public class NiFiRegistryFlowMapperTest {
}
private ProcessGroup prepareProcessGroup(final int numProcessors, final boolean includeFunnel,final boolean includePorts,
- final boolean includeLabels, final boolean includeVariableRegistry,
- final boolean includeControllerService,
+ final boolean includeLabels, final boolean includeControllerService,
final ControllerServiceNode externalControllerServiceNode,
final boolean includeRemoteProcessGroup, final boolean includeVersionControlInfo,
final List<ProcessGroup> childProcessGroups) {
@@ -598,6 +596,7 @@ public class NiFiRegistryFlowMapperTest {
when(versionControlInformation.getBucketIdentifier()).thenReturn(UUID.randomUUID().toString());
when(versionControlInformation.getFlowIdentifier()).thenReturn(UUID.randomUUID().toString());
when(versionControlInformation.getVersion()).thenReturn(counter++);
+ when(versionControlInformation.getStorageLocation()).thenReturn("http://localhost:18080");
return versionControlInformation;
}
@@ -642,7 +641,7 @@ public class NiFiRegistryFlowMapperTest {
if (expectVersionControlInfo) {
final VersionControlInformation versionControlInfo = processGroup.getVersionControlInformation();
final VersionedFlowCoordinates versionedFlowCoordinates = versionedProcessGroup.getVersionedFlowCoordinates();
- assertNotNull(versionedFlowCoordinates.getRegistryUrl());
+ assertNotNull(versionedFlowCoordinates.getStorageLocation());
assertEquals(versionControlInfo.getBucketIdentifier(), versionedFlowCoordinates.getBucketId());
assertEquals(versionControlInfo.getFlowIdentifier(), versionedFlowCoordinates.getFlowId());
assertEquals(versionControlInfo.getVersion(), versionedFlowCoordinates.getVersion());
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
index 6ac784df7f..e0382badc2 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -5442,7 +5442,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
final Set<AffectedComponentEntity> affectedComponents = comparison.getDifferences().stream()
.filter(difference -> difference.getDifferenceType() != DifferenceType.COMPONENT_ADDED) // components that are added are not components that will be affected in the local flow.
.filter(FlowDifferenceFilters.FILTER_ADDED_REMOVED_REMOTE_PORTS)
- .filter(FlowDifferenceFilters.FILTER_IGNORABLE_VERSIONED_FLOW_COORDINATE_CHANGES)
.filter(diff -> !FlowDifferenceFilters.isNewPropertyWithDefaultValue(diff, flowManager))
.filter(diff -> !FlowDifferenceFilters.isNewRelationshipAutoTerminatedAndDefaulted(diff, proposedFlow.getContents(), flowManager))
.filter(diff -> !FlowDifferenceFilters.isScheduledStateNew(diff))
@@ -5526,10 +5525,6 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
continue;
}
- if (FlowDifferenceFilters.isIgnorableVersionedFlowCoordinateChange(difference)) {
- continue;
- }
-
if (FlowDifferenceFilters.isNewPropertyWithDefaultValue(difference, controllerFacade.getFlowManager())) {
continue;
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
index 73846c8e9c..afb8fe7b7c 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerResource.java
@@ -52,7 +52,6 @@ import org.apache.nifi.web.api.dto.ComponentStateDTO;
import org.apache.nifi.web.api.dto.ConfigVerificationResultDTO;
import org.apache.nifi.web.api.dto.ConfigurationAnalysisDTO;
import org.apache.nifi.web.api.dto.ControllerServiceDTO;
-import org.apache.nifi.web.api.dto.DocumentedTypeDTO;
import org.apache.nifi.web.api.dto.FlowAnalysisRuleDTO;
import org.apache.nifi.web.api.dto.FlowRegistryClientDTO;
import org.apache.nifi.web.api.dto.NodeDTO;
@@ -106,9 +105,7 @@ import javax.ws.rs.core.Response;
import java.net.URI;
import java.util.Collections;
import java.util.Date;
-import java.util.HashMap;
import java.util.List;
-import java.util.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
@@ -1526,8 +1523,6 @@ public class ControllerResource extends ApplicationResource {
// authorize access
authorizeController(RequestAction.READ);
- preprocessObsoleteRequest(requestFlowRegistryClientEntity);
-
if (requestFlowRegistryClientEntity == null || requestFlowRegistryClientEntity.getComponent() == null) {
throw new IllegalArgumentException("Flow Registry client details must be specified.");
}
@@ -1547,6 +1542,10 @@ public class ControllerResource extends ApplicationResource {
throw new IllegalArgumentException("Flow Registry name must be specified.");
}
+ if (requestRegistryClient.getType() == null) {
+ throw new IllegalArgumentException("The flow registry client type must be specified.");
+ }
+
if (serviceFacade.getRegistryClients().stream().anyMatch(rce -> requestRegistryClient.getName().equals(rce.getComponent().getName()))) {
throw new IllegalArgumentException("A Flow Registry already exists with the name " + requestRegistryClient.getName());
}
@@ -1666,8 +1665,6 @@ public class ControllerResource extends ApplicationResource {
required = true
) final FlowRegistryClientEntity requestFlowRegistryClientEntity) {
- preprocessObsoleteRequest(requestFlowRegistryClientEntity);
-
if (requestFlowRegistryClientEntity == null || requestFlowRegistryClientEntity.getComponent() == null) {
throw new IllegalArgumentException("Flow registry client details must be specified.");
}
@@ -1874,7 +1871,6 @@ public class ControllerResource extends ApplicationResource {
* Retrieves the types of flow registry clients that this NiFi supports.
*
* @return A flowRegistryTypesEntity.
- * @throws InterruptedException if interrupted
*/
@GET
@Consumes(MediaType.WILDCARD)
@@ -1911,25 +1907,6 @@ public class ControllerResource extends ApplicationResource {
return generateOkResponse(entity).build();
}
- private void preprocessObsoleteRequest(final FlowRegistryClientEntity requestFlowRegistryClientEntity) {
- final FlowRegistryClientDTO dto = requestFlowRegistryClientEntity.getComponent();
-
- if (dto.getType() == null && dto.getBundle() == null && dto.getUri() != null) {
- LOGGER.warn("The flow registry client operation request is considered legacy, will be populated using defaults!");
-
- final Optional<DocumentedTypeDTO> nifiRegistryBundle = serviceFacade.getFlowRegistryTypes().stream().filter(b -> NIFI_REGISTRY_TYPE.equals(b.getType())).findFirst();
-
- if (nifiRegistryBundle.isEmpty()) {
- throw new IllegalStateException("NiFi instance cannot find a NifiRegistryFlowRegistryClient implementation!");
- }
-
- dto.setType(NIFI_REGISTRY_TYPE);
- dto.setBundle(nifiRegistryBundle.get().getBundle());
- dto.setProperties(new HashMap<>(Collections.singletonMap("url", dto.getUri())));
- dto.setUri(null);
- }
- }
-
/**
* Populate the uri's for the specified flow registry client and also extend the result to make it backward compatible.
*
@@ -1938,11 +1915,6 @@ public class ControllerResource extends ApplicationResource {
*/
private FlowRegistryClientEntity populateRemainingRegistryClientEntityContent(final FlowRegistryClientEntity flowRegistryClientEntity) {
flowRegistryClientEntity.setUri(generateResourceUri("controller", "registry-clients", flowRegistryClientEntity.getId()));
-
- if (flowRegistryClientEntity.getComponent().getType().equals(NIFI_REGISTRY_TYPE)) {
- flowRegistryClientEntity.getComponent().setUri(flowRegistryClientEntity.getComponent().getProperties().get("url"));
- }
-
return flowRegistryClientEntity;
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
index 916d47270b..7bdf7e4dd1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/FlowResource.java
@@ -1812,13 +1812,6 @@ public class FlowResource extends ApplicationResource {
*/
private FlowRegistryClientEntity populateRemainingRegistryClientEntityContent(final FlowRegistryClientEntity flowRegistryClientEntity) {
flowRegistryClientEntity.setUri(generateResourceUri("controller", "registry-clients", flowRegistryClientEntity.getId()));
-
- if (flowRegistryClientEntity.getComponent() != null) {
- if (flowRegistryClientEntity.getComponent().getType().equals(NIFI_REGISTRY_TYPE)) {
- flowRegistryClientEntity.getComponent().setUri(flowRegistryClientEntity.getComponent().getProperties().get("url"));
- }
- }
-
return flowRegistryClientEntity;
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardFlowRegistryDAO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardFlowRegistryDAO.java
index 3e30e31ca0..00671acd51 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardFlowRegistryDAO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardFlowRegistryDAO.java
@@ -20,8 +20,8 @@ package org.apache.nifi.web.dao.impl;
import org.apache.nifi.bundle.BundleCoordinate;
import org.apache.nifi.controller.FlowController;
import org.apache.nifi.registry.flow.FlowRegistryBucket;
-import org.apache.nifi.registry.flow.FlowRegistryClientUserContext;
import org.apache.nifi.registry.flow.FlowRegistryClientNode;
+import org.apache.nifi.registry.flow.FlowRegistryClientUserContext;
import org.apache.nifi.registry.flow.FlowRegistryException;
import org.apache.nifi.registry.flow.RegisteredFlow;
import org.apache.nifi.registry.flow.RegisteredFlowSnapshotMetadata;
@@ -43,7 +43,6 @@ public class StandardFlowRegistryDAO extends ComponentDAO implements FlowRegistr
@Override
public FlowRegistryClientNode createFlowRegistryClient(final FlowRegistryClientDTO flowRegistryClientDto) {
- // ensure the type is specified
if (flowRegistryClientDto.getType() == null) {
throw new IllegalArgumentException("The flow registry client type must be specified.");
}
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-flow-version.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-flow-version.js
index 80365699d8..e003306bcd 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-flow-version.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-flow-version.js
@@ -204,7 +204,7 @@
});
$.each(registriesResponse.registries, function (_, registryEntity) {
- var registry = registryEntity.registry;
+ var registry = registryEntity.component;
registries.push({
text: registry.name,
value: registry.id,
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/test/java/org/apache/nifi/ssl/SSLContextServiceTest.java b/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/test/java/org/apache/nifi/ssl/SSLContextServiceTest.java
index 121991da84..4a0b5c65c7 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/test/java/org/apache/nifi/ssl/SSLContextServiceTest.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/test/java/org/apache/nifi/ssl/SSLContextServiceTest.java
@@ -253,22 +253,4 @@ public class SSLContextServiceTest {
assertNotNull(service);
service.createContext();
}
-
- @Test
- @Deprecated
- public void testGoodKeyOnly() throws Exception {
- TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
- SSLContextService service = new StandardSSLContextService();
- HashMap<String, String> properties = new HashMap<>();
- properties.put(StandardSSLContextService.KEYSTORE.getName(), tlsConfiguration.getKeystorePath());
- properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), tlsConfiguration.getKeystorePassword());
- properties.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), tlsConfiguration.getKeystoreType().getType());
- runner.addControllerService("test-good3", service, properties);
- runner.enableControllerService(service);
-
- runner.setProperty("SSL Context Svc ID", "test-good3");
- runner.assertValid();
- assertNotNull(service);
- service.createContext();
- }
}
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/ConciseEvolvingDifferenceDescriptor.java b/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/ConciseEvolvingDifferenceDescriptor.java
index e1e71b49b7..57d4fe1e36 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/ConciseEvolvingDifferenceDescriptor.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/ConciseEvolvingDifferenceDescriptor.java
@@ -77,7 +77,7 @@ public class ConciseEvolvingDifferenceDescriptor implements DifferenceDescriptor
final VersionedFlowCoordinates coordinatesB = (VersionedFlowCoordinates) valueB;
// If the two vary only by version, then use a more concise message. If anything else is different, then use a fully explanation.
- if (FlowDifferenceUtil.areRegistryUrlsEqual(coordinatesA, coordinatesB) && Objects.equals(coordinatesA.getBucketId(), coordinatesB.getBucketId())
+ if (Objects.equals(coordinatesA.getStorageLocation(), coordinatesB.getStorageLocation()) && Objects.equals(coordinatesA.getBucketId(), coordinatesB.getBucketId())
&& Objects.equals(coordinatesA.getFlowId(), coordinatesB.getFlowId()) && coordinatesA.getVersion() != coordinatesB.getVersion()) {
description = String.format("Flow Version changed from %s to %s", coordinatesA.getVersion(), coordinatesB.getVersion());
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/FlowDifferenceUtil.java b/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/FlowDifferenceUtil.java
deleted file mode 100644
index 1d69b06bfb..0000000000
--- a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/FlowDifferenceUtil.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.flow.diff;
-
-import org.apache.nifi.flow.VersionedFlowCoordinates;
-
-public final class FlowDifferenceUtil {
-
- private FlowDifferenceUtil() {
- // Not to be instantiated
- }
-
- public static boolean areRegistryStrictlyEqual(final VersionedFlowCoordinates coordinatesA, final VersionedFlowCoordinates coordinatesB) {
- final String registryUrlA = coordinatesA.getRegistryUrl();
- final String registryUrlB = coordinatesB.getRegistryUrl();
- return registryUrlA != null && registryUrlB != null && registryUrlA.equals(registryUrlB);
- }
-
- public static boolean areRegistryUrlsEqual(final VersionedFlowCoordinates coordinatesA, final VersionedFlowCoordinates coordinatesB) {
- final String registryUrlA = coordinatesA.getRegistryUrl();
- final String registryUrlB = coordinatesB.getRegistryUrl();
-
- if (registryUrlA != null && registryUrlB != null) {
- if (registryUrlA.equals(registryUrlB)) {
- return true;
- }
-
- final String normalizedRegistryUrlA = registryUrlA.endsWith("/") ? registryUrlA.substring(0, registryUrlA.length() - 1) : registryUrlA;
- final String normalizedRegistryUrlB = registryUrlB.endsWith("/") ? registryUrlB.substring(0, registryUrlB.length() - 1) : registryUrlB;
-
- return normalizedRegistryUrlA.equals(normalizedRegistryUrlB);
- }
-
- return false;
- }
-}
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardFlowComparator.java b/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardFlowComparator.java
index e306765771..cc8f4340e1 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardFlowComparator.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StandardFlowComparator.java
@@ -599,14 +599,6 @@ public class StandardFlowComparator implements FlowComparator {
return;
}
- // If Registry URL is specified for both coordinates, compare them
- final String registryUrlA = coordinatesA.getRegistryUrl();
- final String registryUrlB = coordinatesB.getRegistryUrl();
- if (registryUrlA != null && registryUrlB != null && !registryUrlA.equals(registryUrlB)) {
- differences.add(difference(DifferenceType.VERSIONED_FLOW_COORDINATES_CHANGED, groupA, groupB, coordinatesA, coordinatesB));
- return;
- }
-
// If Storage Location is specified for both coordinates, compare them
final String storageLocationA = coordinatesA.getStorageLocation();
final String storageLocationB = coordinatesB.getStorageLocation();
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StaticDifferenceDescriptor.java b/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StaticDifferenceDescriptor.java
index 17f43291ab..c37519a638 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StaticDifferenceDescriptor.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-flow-diff/src/main/java/org/apache/nifi/registry/flow/diff/StaticDifferenceDescriptor.java
@@ -71,7 +71,7 @@ public class StaticDifferenceDescriptor implements DifferenceDescriptor {
final VersionedFlowCoordinates coordinatesB = (VersionedFlowCoordinates) valueB;
// If the two vary only by version, then use a more concise message. If anything else is different, then use a fully explanation.
- if (Objects.equals(coordinatesA.getRegistryUrl(), coordinatesB.getRegistryUrl()) && Objects.equals(coordinatesA.getBucketId(), coordinatesB.getBucketId())
+ if (Objects.equals(coordinatesA.getStorageLocation(), coordinatesB.getStorageLocation()) && Objects.equals(coordinatesA.getBucketId(), coordinatesB.getBucketId())
&& Objects.equals(coordinatesA.getFlowId(), coordinatesB.getFlowId()) && coordinatesA.getVersion() != coordinatesB.getVersion()) {
description = String.format("Flow Version is %s in %s but %s in %s", coordinatesA.getVersion(), flowAName, coordinatesB.getVersion(), flowBName);
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/RegistryService.java b/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/RegistryService.java
index 247ecd0c7f..fcdff15030 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/RegistryService.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/RegistryService.java
@@ -18,6 +18,8 @@ package org.apache.nifi.registry.service;
import org.apache.commons.lang3.ObjectUtils;
import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.flow.VersionedComponent;
+import org.apache.nifi.flow.VersionedProcessGroup;
import org.apache.nifi.registry.bucket.Bucket;
import org.apache.nifi.registry.bucket.BucketItem;
import org.apache.nifi.registry.db.entity.BucketEntity;
@@ -32,11 +34,9 @@ import org.apache.nifi.registry.extension.BundleCoordinate;
import org.apache.nifi.registry.extension.BundlePersistenceProvider;
import org.apache.nifi.registry.flow.FlowPersistenceProvider;
import org.apache.nifi.registry.flow.FlowSnapshotContext;
-import org.apache.nifi.flow.VersionedComponent;
import org.apache.nifi.registry.flow.VersionedFlow;
import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
-import org.apache.nifi.flow.VersionedProcessGroup;
import org.apache.nifi.registry.flow.diff.ComparableDataFlow;
import org.apache.nifi.registry.flow.diff.ConciseEvolvingDifferenceDescriptor;
import org.apache.nifi.registry.flow.diff.FlowComparator;
@@ -49,7 +49,6 @@ import org.apache.nifi.registry.provider.extension.StandardBundleCoordinate;
import org.apache.nifi.registry.provider.flow.StandardFlowSnapshotContext;
import org.apache.nifi.registry.serialization.FlowContent;
import org.apache.nifi.registry.serialization.FlowContentSerializer;
-import org.apache.nifi.registry.service.alias.RegistryUrlAliasService;
import org.apache.nifi.registry.service.mapper.BucketMappings;
import org.apache.nifi.registry.service.mapper.ExtensionMappings;
import org.apache.nifi.registry.service.mapper.FlowMappings;
@@ -89,21 +88,18 @@ public class RegistryService {
private final BundlePersistenceProvider bundlePersistenceProvider;
private final FlowContentSerializer flowContentSerializer;
private final Validator validator;
- private final RegistryUrlAliasService registryUrlAliasService;
@Autowired
public RegistryService(final MetadataService metadataService,
final FlowPersistenceProvider flowPersistenceProvider,
final BundlePersistenceProvider bundlePersistenceProvider,
final FlowContentSerializer flowContentSerializer,
- final Validator validator,
- final RegistryUrlAliasService registryUrlAliasService) {
+ final Validator validator) {
this.metadataService = Objects.requireNonNull(metadataService);
this.flowPersistenceProvider = Objects.requireNonNull(flowPersistenceProvider);
this.bundlePersistenceProvider = Objects.requireNonNull(bundlePersistenceProvider);
this.flowContentSerializer = Objects.requireNonNull(flowContentSerializer);
this.validator = Objects.requireNonNull(validator);
- this.registryUrlAliasService = Objects.requireNonNull(registryUrlAliasService);
}
private <T> void validate(T t, String invalidMessage) {
@@ -608,7 +604,6 @@ public class RegistryService {
// serialize the snapshot
final ByteArrayOutputStream out = new ByteArrayOutputStream();
- registryUrlAliasService.setInternal(flowSnapshot.getFlowContents());
final FlowContent flowContent = new FlowContent();
flowContent.setFlowSnapshot(flowSnapshot);
@@ -639,7 +634,6 @@ public class RegistryService {
flowSnapshot.setBucket(bucket);
flowSnapshot.setFlow(updatedVersionedFlow);
- registryUrlAliasService.setExternal(flowSnapshot.getFlowContents());
return flowSnapshot;
}
@@ -702,7 +696,6 @@ public class RegistryService {
final VersionedFlowSnapshotMetadata snapshotMetadata = FlowMappings.map(bucketEntity, snapshotEntity);
// create the snapshot to return
- registryUrlAliasService.setExternal(snapshot.getFlowContents());
snapshot.setSnapshotMetadata(snapshotMetadata);
snapshot.setFlow(versionedFlow);
snapshot.setBucket(bucket);
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/alias/RegistryUrlAliasService.java b/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/alias/RegistryUrlAliasService.java
deleted file mode 100644
index 650fb01366..0000000000
--- a/nifi-registry/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/alias/RegistryUrlAliasService.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.service.alias;
-
-import org.apache.nifi.flow.VersionedFlowCoordinates;
-import org.apache.nifi.flow.VersionedProcessGroup;
-import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.apache.nifi.registry.provider.ProviderFactoryException;
-import org.apache.nifi.registry.provider.StandardProviderFactory;
-import org.apache.nifi.registry.url.aliaser.generated.Alias;
-import org.apache.nifi.registry.url.aliaser.generated.Aliases;
-import org.apache.nifi.xml.processing.ProcessingException;
-import org.apache.nifi.xml.processing.stream.StandardXMLStreamReaderProvider;
-import org.apache.nifi.xml.processing.stream.XMLStreamReaderProvider;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Service;
-import org.xml.sax.SAXException;
-
-import javax.xml.XMLConstants;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.stream.XMLStreamReader;
-import javax.xml.transform.stream.StreamSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.File;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Pattern;
-
-/**
- * Allows aliasing of registry url(s) without modifying the flows on disk.
- */
-@Service
-public class RegistryUrlAliasService {
- private static final String ALIASES_XSD = "/aliases.xsd";
- private static final String JAXB_GENERATED_PATH = "org.apache.nifi.registry.url.aliaser.generated";
- private static final JAXBContext JAXB_CONTEXT = initializeJaxbContext();
-
- /**
- * Load the JAXBContext.
- */
- private static JAXBContext initializeJaxbContext() {
- try {
- return JAXBContext.newInstance(JAXB_GENERATED_PATH, RegistryUrlAliasService.class.getClassLoader());
- } catch (JAXBException e) {
- throw new RuntimeException("Unable to create JAXBContext.", e);
- }
- }
-
- // Will be LinkedHashMap to preserve insertion order.
- private final Map<String, String> aliases;
-
- @Autowired
- public RegistryUrlAliasService(NiFiRegistryProperties niFiRegistryProperties) {
- this(createAliases(niFiRegistryProperties));
- }
-
- private static List<Alias> createAliases(NiFiRegistryProperties niFiRegistryProperties) {
- File configurationFile = niFiRegistryProperties.getRegistryAliasConfigurationFile();
- if (configurationFile.exists()) {
- try {
- // find the schema
- final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
- final Schema schema = schemaFactory.newSchema(StandardProviderFactory.class.getResource(ALIASES_XSD));
-
- // attempt to unmarshal
- final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
- unmarshaller.setSchema(schema);
-
- final XMLStreamReaderProvider provider = new StandardXMLStreamReaderProvider();
- final XMLStreamReader reader = provider.getStreamReader(new StreamSource(configurationFile));
- final JAXBElement<Aliases> element = unmarshaller.unmarshal(reader, Aliases.class);
- return element.getValue().getAlias();
- } catch (final SAXException | JAXBException | ProcessingException e) {
- throw new ProviderFactoryException("Unable to load the registry alias configuration file at: " + configurationFile.getAbsolutePath(), e);
- }
- } else {
- return Collections.emptyList();
- }
- }
-
- protected RegistryUrlAliasService(List<Alias> aliases) {
- Pattern urlStart = Pattern.compile("^https?://");
-
- this.aliases = new LinkedHashMap<>();
-
- for (Alias alias : aliases) {
- String internal = alias.getInternal();
- String external = alias.getExternal();
-
- if (!urlStart.matcher(external).find()) {
- throw new IllegalArgumentException("Expected " + external + " to start with http:// or https://");
- }
-
- if (this.aliases.put(internal, external) != null) {
- throw new IllegalArgumentException("Duplicate internal token " + internal);
- }
- }
- }
-
- /**
- * Recursively replaces the aliases with the external url for a process group and children.
- */
- public void setExternal(VersionedProcessGroup processGroup) {
- processGroup.getProcessGroups().forEach(this::setExternal);
-
- VersionedFlowCoordinates coordinates = processGroup.getVersionedFlowCoordinates();
- if (coordinates != null) {
- coordinates.setRegistryUrl(getExternal(coordinates.getRegistryUrl()));
- }
- }
-
- /**
- * Recursively replaces the external url with the aliases for a process group and children.
- */
- public void setInternal(VersionedProcessGroup processGroup) {
- processGroup.getProcessGroups().forEach(this::setInternal);
-
- VersionedFlowCoordinates coordinates = processGroup.getVersionedFlowCoordinates();
- if (coordinates != null) {
- coordinates.setRegistryUrl(getInternal(coordinates.getRegistryUrl()));
- }
- }
-
- protected String getExternal(String url) {
- for (Map.Entry<String, String> alias : aliases.entrySet()) {
- String internal = alias.getKey();
- String external = alias.getValue();
-
- if (url.startsWith(internal)) {
- int internalLength = internal.length();
- if (url.length() == internalLength) {
- return external;
- }
- return external + url.substring(internalLength);
- }
- }
- return url;
- }
-
- protected String getInternal(String url) {
- for (Map.Entry<String, String> alias : aliases.entrySet()) {
- String internal = alias.getKey();
- String external = alias.getValue();
-
- if (url.startsWith(external)) {
- int externalLength = external.length();
- if (url.length() == externalLength) {
- return internal;
- }
- return internal + url.substring(externalLength);
- }
- }
- return url;
- }
-}
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/TestRegistryService.java b/nifi-registry/nifi-registry-core/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/TestRegistryService.java
index 1964687b38..9ae65da0ae 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/TestRegistryService.java
+++ b/nifi-registry/nifi-registry-core/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/TestRegistryService.java
@@ -33,7 +33,6 @@ import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
import org.apache.nifi.registry.serialization.FlowContent;
import org.apache.nifi.registry.serialization.FlowContentSerializer;
-import org.apache.nifi.registry.service.alias.RegistryUrlAliasService;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.invocation.InvocationOnMock;
@@ -78,7 +77,6 @@ public class TestRegistryService {
private BundlePersistenceProvider bundlePersistenceProvider;
private FlowContentSerializer flowContentSerializer;
private Validator validator;
- private RegistryUrlAliasService registryUrlAliasService;
private RegistryService registryService;
@@ -88,13 +86,12 @@ public class TestRegistryService {
flowPersistenceProvider = mock(FlowPersistenceProvider.class);
bundlePersistenceProvider = mock(BundlePersistenceProvider.class);
flowContentSerializer = mock(FlowContentSerializer.class);
- registryUrlAliasService = mock(RegistryUrlAliasService.class);
final ValidatorFactory validatorFactory = Validation.buildDefaultValidatorFactory();
validator = validatorFactory.getValidator();
registryService = new RegistryService(metadataService, flowPersistenceProvider, bundlePersistenceProvider,
- flowContentSerializer, validator, registryUrlAliasService);
+ flowContentSerializer, validator);
}
// ---------------------- Test Bucket methods ---------------------------------------------
diff --git a/nifi-registry/nifi-registry-core/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/alias/RegistryUrlAliasServiceTest.java b/nifi-registry/nifi-registry-core/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/alias/RegistryUrlAliasServiceTest.java
deleted file mode 100644
index 414dab3b73..0000000000
--- a/nifi-registry/nifi-registry-core/nifi-registry-framework/src/test/java/org/apache/nifi/registry/service/alias/RegistryUrlAliasServiceTest.java
+++ /dev/null
@@ -1,173 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.service.alias;
-
-import org.apache.nifi.registry.url.aliaser.generated.Alias;
-import org.junit.jupiter.api.Test;
-
-import java.util.Arrays;
-import java.util.Collections;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-
-
-public class RegistryUrlAliasServiceTest {
- private static Alias createAlias(String internal, String external) {
- Alias result = new Alias();
- result.setInternal(internal);
- result.setExternal(external);
- return result;
- }
-
- @Test
- public void testNoAliases() {
- RegistryUrlAliasService aliaser = new RegistryUrlAliasService(Collections.emptyList());
-
- String url = "https://registry.com:18080";
-
- assertEquals(url, aliaser.getExternal(url));
- assertEquals(url, aliaser.getInternal(url));
- }
-
- @Test
- public void testMalformedExternal() {
- assertThrows(IllegalArgumentException.class, () -> new RegistryUrlAliasService(Collections.singletonList(createAlias("https://registry.com:18080", "registry.com:18080"))));
- }
-
- @Test
- public void testSingleAliasUrl() {
- String internal = "https://registry-1.com:18443";
- String external = "http://localhost:18080";
- String unchanged = "https://registry-2.com:18443";
-
- RegistryUrlAliasService aliaser = new RegistryUrlAliasService(Collections.singletonList(createAlias(internal, external)));
-
- assertEquals(external, aliaser.getExternal(internal));
- assertEquals(internal, aliaser.getInternal(external));
-
- assertEquals(unchanged, aliaser.getExternal(unchanged));
- assertEquals(unchanged, aliaser.getInternal(unchanged));
-
- // Ensure replacement is only the prefix
- internal += "/nifi-registry/";
- external += "/nifi-registry/";
- unchanged += "/nifi-registry/";
-
- assertEquals(external, aliaser.getExternal(internal));
- assertEquals(internal, aliaser.getInternal(external));
-
- assertEquals(unchanged, aliaser.getExternal(unchanged));
- assertEquals(unchanged, aliaser.getInternal(unchanged));
- }
-
- @Test
- public void testSingleAliasToken() {
- String internal = "THIS_NIFI_REGISTRY";
- String external = "http://localhost:18080";
- String unchanged = "https://registry-2.com:18443";
-
- RegistryUrlAliasService aliaser = new RegistryUrlAliasService(Collections.singletonList(createAlias(internal, external)));
-
- assertEquals(external, aliaser.getExternal(internal));
- assertEquals(internal, aliaser.getInternal(external));
-
- assertEquals(unchanged, aliaser.getExternal(unchanged));
- assertEquals(unchanged, aliaser.getInternal(unchanged));
-
- // Ensure replacement is only the prefix
- internal += "/nifi-registry/";
- external += "/nifi-registry/";
- unchanged += "/nifi-registry/";
-
- assertEquals(external, aliaser.getExternal(internal));
- assertEquals(internal, aliaser.getInternal(external));
-
- assertEquals(unchanged, aliaser.getExternal(unchanged));
- assertEquals(unchanged, aliaser.getInternal(unchanged));
- }
-
- @Test
- public void testMultipleAliases() {
- String internal1 = "https://registry-1.com:18443";
- String external1 = "http://localhost:18080";
- String internal2 = "https://registry-2.com:18443";
- String external2 = "http://localhost:18081";
- String internal3 = "THIS_NIFI_REGISTRY";
- String external3 = "http://localhost:18082";
-
- String unchanged = "https://registry-3.com:18443";
-
- RegistryUrlAliasService aliaser = new RegistryUrlAliasService(Arrays.asList(createAlias(internal1, external1), createAlias(internal2, external2), createAlias(internal3, external3)));
-
- assertEquals(external1, aliaser.getExternal(internal1));
- assertEquals(external2, aliaser.getExternal(internal2));
- assertEquals(external3, aliaser.getExternal(internal3));
-
- assertEquals(internal1, aliaser.getInternal(external1));
- assertEquals(internal2, aliaser.getInternal(external2));
- assertEquals(internal3, aliaser.getInternal(external3));
-
- assertEquals(unchanged, aliaser.getExternal(unchanged));
- assertEquals(unchanged, aliaser.getInternal(unchanged));
-
- // Ensure replacement is only the prefix
- internal1 += "/nifi-registry/";
- internal2 += "/nifi-registry/";
- internal3 += "/nifi-registry/";
-
- external1 += "/nifi-registry/";
- external2 += "/nifi-registry/";
- external3 += "/nifi-registry/";
-
- unchanged += "/nifi-registry/";
-
- assertEquals(external1, aliaser.getExternal(internal1));
- assertEquals(external2, aliaser.getExternal(internal2));
- assertEquals(external3, aliaser.getExternal(internal3));
-
- assertEquals(internal1, aliaser.getInternal(external1));
- assertEquals(internal2, aliaser.getInternal(external2));
- assertEquals(internal3, aliaser.getInternal(external3));
-
- assertEquals(unchanged, aliaser.getExternal(unchanged));
- assertEquals(unchanged, aliaser.getInternal(unchanged));
- }
-
- @Test
- public void testMigrationPath() {
- String internal1 = "INTERNAL_TOKEN";
- String internal2 = "http://old.registry.url";
- String external = "https://new.registry.url";
-
- RegistryUrlAliasService aliaser = new RegistryUrlAliasService(Arrays.asList(createAlias(internal1, external), createAlias(internal2, external)));
-
- assertEquals(internal1, aliaser.getInternal(external));
-
- assertEquals(external, aliaser.getExternal(internal1));
- assertEquals(external, aliaser.getExternal(internal2));
- }
-
- @Test
- public void testDuplicateInternalTokens() {
- String internal = "THIS_NIFI_REGISTRY";
- String external1 = "http://localhost:18080";
- String external2 = "http://localhost:18081";
-
- assertThrows(IllegalArgumentException.class, () -> new RegistryUrlAliasService(Arrays.asList(createAlias(internal, external1), createAlias(internal, external2))));
- }
-}
diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/core/RegistryUtil.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/core/RegistryUtil.java
index 75ecd3b2cc..949397b5b1 100644
--- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/core/RegistryUtil.java
+++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/core/RegistryUtil.java
@@ -17,6 +17,8 @@
package org.apache.nifi.stateless.core;
import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.flow.VersionedFlowCoordinates;
+import org.apache.nifi.flow.VersionedProcessGroup;
import org.apache.nifi.registry.client.FlowClient;
import org.apache.nifi.registry.client.FlowSnapshotClient;
import org.apache.nifi.registry.client.NiFiRegistryClient;
@@ -24,9 +26,7 @@ import org.apache.nifi.registry.client.NiFiRegistryClientConfig;
import org.apache.nifi.registry.client.NiFiRegistryException;
import org.apache.nifi.registry.client.impl.JerseyNiFiRegistryClient;
import org.apache.nifi.registry.client.impl.request.ProxiedEntityRequestConfig;
-import org.apache.nifi.flow.VersionedFlowCoordinates;
import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
-import org.apache.nifi.flow.VersionedProcessGroup;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -47,10 +47,6 @@ public class RegistryUtil {
}
- public VersionedFlowSnapshot getFlowByID(String bucketID, String flowID) throws IOException, NiFiRegistryException {
- return getFlowByID(bucketID, flowID, -1);
- }
-
public VersionedFlowSnapshot getFlowByID(String bucketID, String flowID, int versionID) throws IOException, NiFiRegistryException {
if (versionID == -1) {
// TODO: Have to support providing some sort of user
@@ -116,7 +112,7 @@ public class RegistryUtil {
if (fetchRemoteFlows) {
final VersionedProcessGroup contents = flowSnapshot.getFlowContents();
for (final VersionedProcessGroup child : contents.getProcessGroups()) {
- populateVersionedContentsRecursively(child, user);
+ populateVersionedContentsRecursively(child);
}
}
@@ -124,14 +120,14 @@ public class RegistryUtil {
}
- private void populateVersionedContentsRecursively(final VersionedProcessGroup group, final NiFiUser user) throws NiFiRegistryException, IOException {
+ private void populateVersionedContentsRecursively(final VersionedProcessGroup group) throws NiFiRegistryException, IOException {
if (group == null) {
return;
}
final VersionedFlowCoordinates coordinates = group.getVersionedFlowCoordinates();
if (coordinates != null) {
- final String registryUrl = coordinates.getRegistryUrl();
+ final String registryUrl = coordinates.getStorageLocation();
final String bucketId = coordinates.getBucketId();
final String flowId = coordinates.getFlowId();
final int version = coordinates.getVersion();
@@ -160,7 +156,7 @@ public class RegistryUtil {
}
for (final VersionedProcessGroup child : group.getProcessGroups()) {
- populateVersionedContentsRecursively(child, user);
+ populateVersionedContentsRecursively(child);
}
}
}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/CommandOption.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/CommandOption.java
index de5d6cfc95..568d1e95c0 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/CommandOption.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/CommandOption.java
@@ -79,6 +79,7 @@ public enum CommandOption {
REGISTRY_CLIENT_NAME("rcn", "registryClientName", "The name of the registry client", true),
REGISTRY_CLIENT_URL("rcu", "registryClientUrl", "The url of the registry client", true),
REGISTRY_CLIENT_DESC("rcd", "registryClientDesc", "The description of the registry client", true),
+ REGISTRY_CLIENT_TYPE("rct", "registryClientType", "The type of the registry client", true),
// NiFi - PGs
PG_ID("pgid", "processGroupId", "The id of a process group", true),
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/registry/CreateRegistryClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/registry/CreateRegistryClient.java
index a43a041e5e..6e6ce409d2 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/registry/CreateRegistryClient.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/nifi/registry/CreateRegistryClient.java
@@ -27,6 +27,7 @@ import org.apache.nifi.web.api.dto.FlowRegistryClientDTO;
import org.apache.nifi.web.api.entity.FlowRegistryClientEntity;
import java.io.IOException;
+import java.util.Map;
import java.util.Properties;
/**
@@ -35,6 +36,7 @@ import java.util.Properties;
* Note: currently the NiFi Registry backed legacy creation is supported.
*/
public class CreateRegistryClient extends AbstractNiFiCommand<StringResult> {
+ public static final String DEFAULT_REGISTRY_CLIENT_TYPE = "org.apache.nifi.registry.flow.NifiRegistryFlowRegistryClient";
public CreateRegistryClient() {
super("create-reg-client", StringResult.class);
@@ -59,11 +61,16 @@ public class CreateRegistryClient extends AbstractNiFiCommand<StringResult> {
final String name = getRequiredArg(properties, CommandOption.REGISTRY_CLIENT_NAME);
final String url = getRequiredArg(properties, CommandOption.REGISTRY_CLIENT_URL);
final String desc = getArg(properties, CommandOption.REGISTRY_CLIENT_DESC);
+ String type = getArg(properties, CommandOption.REGISTRY_CLIENT_TYPE);
+ if (type == null) {
+ type = DEFAULT_REGISTRY_CLIENT_TYPE;
+ }
final FlowRegistryClientDTO flowRegistryClientDTO = new FlowRegistryClientDTO();
flowRegistryClientDTO.setName(name);
- flowRegistryClientDTO.setUri(url);
+ flowRegistryClientDTO.setProperties(Map.of("url", url));
flowRegistryClientDTO.setDescription(desc);
+ flowRegistryClientDTO.setType(type);
final FlowRegistryClientEntity clientEntity = new FlowRegistryClientEntity();
clientEntity.setComponent(flowRegistryClientDTO);
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/flow/ImportAllFlows.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/flow/ImportAllFlows.java
index d1d031a5ee..4f5808491e 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/flow/ImportAllFlows.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/flow/ImportAllFlows.java
@@ -272,8 +272,8 @@ public class ImportAllFlows extends AbstractNiFiRegistryCommand<StringResult> {
flowCoordinates.getFlowId(), flowCoordinates.getVersion());
flowCoordinates.setStorageLocation(updatedStorageLocation);
- flowCoordinates.setRegistryUrl(registryUrl);
}
+
for (VersionedProcessGroup processGroup : group.getProcessGroups()) {
updateStorageLocation(processGroup, registryUrl);
}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/result/nifi/RegistryClientsResult.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/result/nifi/RegistryClientsResult.java
index 339fbee9fd..5f440c7e0c 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/result/nifi/RegistryClientsResult.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/result/nifi/RegistryClientsResult.java
@@ -28,9 +28,9 @@ import org.apache.nifi.web.api.entity.FlowRegistryClientsEntity;
import java.io.PrintStream;
import java.util.Comparator;
import java.util.List;
+import java.util.Map;
import java.util.Objects;
import java.util.Set;
-import java.util.stream.Collectors;
/**
* Result for a RegistryClientsEntity.
@@ -56,20 +56,22 @@ public class RegistryClientsResult extends AbstractWritableResult<FlowRegistryCl
return;
}
- final List<FlowRegistryClientDTO> registries = clients.stream().map(FlowRegistryClientEntity::getComponent)
- .sorted(Comparator.comparing(FlowRegistryClientDTO::getName))
- .collect(Collectors.toList());
+ final List<FlowRegistryClientDTO> registries = clients.stream()
+ .map(FlowRegistryClientEntity::getComponent)
+ .sorted(Comparator.comparing(FlowRegistryClientDTO::getName))
+ .toList();
final Table table = new Table.Builder()
.column("#", 3, 3, false)
.column("Name", 20, 36, true)
.column("Id", 36, 36, false)
- .column("Uri", 3, Integer.MAX_VALUE, false)
+ .column("Properties", 3, Integer.MAX_VALUE, false)
.build();
for (int i = 0; i < registries.size(); i++) {
- FlowRegistryClientDTO r = registries.get(i);
- table.addRow("" + (i+1), r.getName(), r.getId(), r.getUri());
+ FlowRegistryClientDTO clientDto = registries.get(i);
+ final Map<String, String> properties = clientDto.getProperties();
+ table.addRow("" + (i + 1), clientDto.getName(), clientDto.getId(), properties == null ? "" : properties.toString());
}
final TableWriter tableWriter = new DynamicTableWriter();
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestRegistryClientResult.java b/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestRegistryClientResult.java
index 209628574b..adb5e0dd8c 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestRegistryClientResult.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/test/java/org/apache/nifi/toolkit/cli/impl/result/TestRegistryClientResult.java
@@ -31,6 +31,7 @@ import java.io.IOException;
import java.io.PrintStream;
import java.nio.charset.StandardCharsets;
import java.util.HashSet;
+import java.util.Map;
import java.util.Set;
import java.util.UUID;
@@ -48,17 +49,16 @@ public class TestRegistryClientResult {
this.printStream = new PrintStream(outputStream, true);
}
- @SuppressWarnings("deprecation")
@Test
public void testWriteSimpleRegistryClientsResult() throws IOException {
final FlowRegistryClientDTO r1 = new FlowRegistryClientDTO();
r1.setName("Registry 1");
- r1.setUri("http://thisisalonglonglonglonglonglonglonglonglonguri.com:18080");
+ r1.setProperties(Map.of("url", "http://thisisalonglonglonglonglonglonglonglonglonguri.com:18080"));
r1.setId(UUID.fromString("ea752054-22c6-4fc0-b851-967d9a3837cb").toString());
final FlowRegistryClientDTO r2 = new FlowRegistryClientDTO();
r2.setName("Registry 2 with a longer than usual name");
- r2.setUri("http://localhost:18080");
+ r2.setProperties(Map.of("url", "http://localhost:18080"));
r2.setId(UUID.fromString("ddf5f289-7502-46df-9798-4b0457c1816b").toString());
final FlowRegistryClientEntity clientEntity1 = new FlowRegistryClientEntity();
@@ -79,14 +79,16 @@ public class TestRegistryClientResult {
final RegistryClientsResult result = new RegistryClientsResult(ResultType.SIMPLE, flowRegistryClientsEntity);
result.write(printStream);
- final String resultOut = new String(outputStream.toByteArray(), StandardCharsets.UTF_8);
+ final String resultOut = outputStream.toString(StandardCharsets.UTF_8);
- final String expected = "\n" +
- "# Name Id Uri \n" +
- "- ------------------------------------ ------------------------------------ --------------------------------------------------------------- \n" +
- "1 Registry 1 ea752054-22c6-4fc0-b851-967d9a3837cb http://thisisalonglonglonglonglonglonglonglonglonguri.com:18080 \n" +
- "2 Registry 2 with a longer than usu... ddf5f289-7502-46df-9798-4b0457c1816b http://localhost:18080 \n" +
- "\n";
+ final String expected = """
+
+ # Name Id Properties \s
+ - ------------------------------------ ------------------------------------ --------------------------------------------------------------------- \s
+ 1 Registry 1 ea752054-22c6-4fc0-b851-967d9a3837cb {url=http://thisisalonglonglonglonglonglonglonglonglonguri.com:18080} \s
+ 2 Registry 2 with a longer than usu... ddf5f289-7502-46df-9798-4b0457c1816b {url=http://localhost:18080} \s
+
+ """;
assertEquals(expected, resultOut);
}