You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2019/03/13 15:49:39 UTC

[GitHub] [nifi-registry] kevdoran commented on a change in pull request #160: NIFIREG-213 Implementing parser for extension-manifest.xml file and integ…

kevdoran commented on a change in pull request #160: NIFIREG-213 Implementing parser for extension-manifest.xml file and integ…
URL: https://github.com/apache/nifi-registry/pull/160#discussion_r264818754
 
 

 ##########
 File path: nifi-registry-core/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ExtensionResource.java
 ##########
 @@ -22,401 +22,172 @@
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
 import io.swagger.annotations.Authorization;
-import io.swagger.annotations.Extension;
-import io.swagger.annotations.ExtensionProperty;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.event.EventFactory;
 import org.apache.nifi.registry.event.EventService;
-import org.apache.nifi.registry.extension.ExtensionBundle;
-import org.apache.nifi.registry.extension.ExtensionBundleVersion;
-import org.apache.nifi.registry.extension.ExtensionBundleVersionMetadata;
-import org.apache.nifi.registry.extension.filter.ExtensionBundleFilterParams;
-import org.apache.nifi.registry.extension.filter.ExtensionBundleVersionFilterParams;
+import org.apache.nifi.registry.extension.bundle.BundleType;
+import org.apache.nifi.registry.extension.component.ExtensionFilterParams;
+import org.apache.nifi.registry.extension.component.ExtensionMetadata;
+import org.apache.nifi.registry.extension.component.ExtensionMetadataContainer;
+import org.apache.nifi.registry.extension.component.TagCount;
+import org.apache.nifi.registry.extension.component.manifest.ExtensionType;
+import org.apache.nifi.registry.extension.component.manifest.ProvidedServiceAPI;
 import org.apache.nifi.registry.security.authorization.RequestAction;
 import org.apache.nifi.registry.service.AuthorizationService;
 import org.apache.nifi.registry.service.RegistryService;
-import org.apache.nifi.registry.service.extension.ExtensionBundleVersionCoordinate;
 import org.apache.nifi.registry.web.link.LinkService;
-import org.apache.nifi.registry.web.security.PermissionsService;
-import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Component;
 
 import javax.ws.rs.Consumes;
-import javax.ws.rs.DELETE;
 import javax.ws.rs.GET;
 import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import javax.ws.rs.core.StreamingOutput;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.List;
 import java.util.Set;
 import java.util.SortedSet;
 
 @Component
 @Path("/extensions")
 @Api(
         value = "extensions",
-        description = "Gets metadata about extension bundles and extensions.",
+        description = "Find and retrieve extensions.",
         authorizations = { @Authorization("Authorization") }
 )
 public class ExtensionResource extends AuthorizableApplicationResource {
 
-    public static final String CONTENT_DISPOSITION_HEADER = "content-disposition";
     private final RegistryService registryService;
     private final LinkService linkService;
-    private final PermissionsService permissionsService;
 
-    @Autowired
-    public ExtensionResource(final RegistryService registryService,
-                             final LinkService linkService,
-                             final PermissionsService permissionsService,
-                             final AuthorizationService authorizationService,
-                             final EventService eventService) {
+    public ExtensionResource(final AuthorizationService authorizationService,
+                             final EventService eventService,
+                             final RegistryService registryService,
+                             final LinkService linkService) {
         super(authorizationService, eventService);
         this.registryService = registryService;
         this.linkService = linkService;
-        this.permissionsService = permissionsService;
     }
 
-    // ---------- Extension Bundles ----------
-
     @GET
-    @Path("bundles")
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
     @ApiOperation(
-            value = "Get extension bundles across all authorized buckets",
-            notes = "The returned items will include only items from buckets for which the user is authorized. " +
-                    "If the user is not authorized to any buckets, an empty list will be returned.",
-            response = ExtensionBundle.class,
+            value = "Gets all extensions that match the filter params and are part of bundles located in buckets the current user is authorized for.",
+            response = ExtensionMetadataContainer.class,
             responseContainer = "List"
     )
-    @ApiResponses({ @ApiResponse(code = 401, message = HttpStatusMessages.MESSAGE_401) })
-    public Response getExtensionBundles(
-            @QueryParam("groupId")
-            @ApiParam("Optional groupId to filter results. The value may be an exact match, or a wildcard, " +
-                    "such as 'com.%' to select all bundles where the groupId starts with 'com.'.")
-                final String groupId,
-            @QueryParam("artifactId")
-            @ApiParam("Optional artifactId to filter results. The value may be an exact match, or a wildcard, " +
-                    "such as 'nifi-%' to select all bundles where the artifactId starts with 'nifi-'.")
-                final String artifactId) {
+    @ApiResponses({
+            @ApiResponse(code = 400, message = HttpStatusMessages.MESSAGE_400),
+            @ApiResponse(code = 401, message = HttpStatusMessages.MESSAGE_401),
+            @ApiResponse(code = 403, message = HttpStatusMessages.MESSAGE_403),
+            @ApiResponse(code = 404, message = HttpStatusMessages.MESSAGE_404),
+            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409) })
+    public Response getExtensions(
+            @QueryParam("bundleType")
+            @ApiParam(value = "The type of bundles to return (i.e. NAR, CPP, etc.)")
+                final BundleType bundleType,
+            @QueryParam("extensionType")
+            @ApiParam(value = "The type of extensions to return (i.e. processor, service, reporting task)")
+                final ExtensionType extensionType,
+            @QueryParam("tag")
+            @ApiParam(value = "The tags to filter on, will be used in an OR statement")
+                final Set<String> tags
+            ) {
 
         final Set<String> authorizedBucketIds = getAuthorizedBucketIds(RequestAction.READ);
         if (authorizedBucketIds == null || authorizedBucketIds.isEmpty()) {
             // not authorized for any bucket, return empty list of items
             return Response.status(Response.Status.OK).entity(new ArrayList<>()).build();
         }
 
-        final ExtensionBundleFilterParams filterParams = ExtensionBundleFilterParams.of(groupId, artifactId);
-
-        List<ExtensionBundle> bundles = registryService.getExtensionBundles(authorizedBucketIds, filterParams);
-        if (bundles == null) {
-            bundles = Collections.emptyList();
-        }
-        permissionsService.populateItemPermissions(bundles);
-        linkService.populateLinks(bundles);
-
-        return Response.status(Response.Status.OK).entity(bundles).build();
-    }
-
-    @GET
-    @Path("bundles/{bundleId}")
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.APPLICATION_JSON)
-    @ApiOperation(
-            value = "Gets the metadata about an extension bundle",
-            nickname = "globalGetExtensionBundle",
-            response = ExtensionBundle.class,
-            extensions = {
-                    @Extension(name = "access-policy", properties = {
-                            @ExtensionProperty(name = "action", value = "read"),
-                            @ExtensionProperty(name = "resource", value = "/buckets/{bucketId}") })
-            }
-    )
-    @ApiResponses({
-            @ApiResponse(code = 400, message = HttpStatusMessages.MESSAGE_400),
-            @ApiResponse(code = 401, message = HttpStatusMessages.MESSAGE_401),
-            @ApiResponse(code = 403, message = HttpStatusMessages.MESSAGE_403),
-            @ApiResponse(code = 404, message = HttpStatusMessages.MESSAGE_404),
-            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409) })
-    public Response getExtensionBundle(
-            @PathParam("bundleId")
-            @ApiParam("The extension bundle identifier")
-                final String bundleId) {
+        final ExtensionFilterParams filterParams = new ExtensionFilterParams.Builder()
+                .bundleType(bundleType)
+                .extensionType(extensionType)
+                .addTags(tags == null ? Collections.emptyList() : tags)
+                .build();
 
-        final ExtensionBundle extensionBundle = getExtensionBundleWithBucketReadAuthorization(bundleId);
+        final SortedSet<ExtensionMetadata> extensionMetadata = registryService.getExtensionMetadata(authorizedBucketIds, filterParams);
+        linkService.populateLinks(extensionMetadata);
 
-        permissionsService.populateItemPermissions(extensionBundle);
-        linkService.populateLinks(extensionBundle);
+        final ExtensionMetadataContainer container = new ExtensionMetadataContainer();
+        container.setExtensions(extensionMetadata);
+        container.setNumResults(extensionMetadata.size());
+        container.setFilterParams(filterParams);
 
-        return Response.status(Response.Status.OK).entity(extensionBundle).build();
+        return Response.status(Response.Status.OK).entity(container).build();
     }
 
-    @DELETE
-    @Path("bundles/{bundleId}")
+    @GET
+    @Path("provided-service-api")
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
     @ApiOperation(
-            value = "Deletes the given extension bundle and all of it's versions",
-            nickname = "globalDeleteExtensionBundle",
-            response = ExtensionBundle.class,
-            extensions = {
-                    @Extension(name = "access-policy", properties = {
-                            @ExtensionProperty(name = "action", value = "write"),
-                            @ExtensionProperty(name = "resource", value = "/buckets/{bucketId}") })
-            }
+            value = "Gets all extensions that provide the specified API and are part of bundles located in buckets the current user is authorized for.",
+            response = ExtensionMetadataContainer.class,
+            responseContainer = "List"
     )
     @ApiResponses({
             @ApiResponse(code = 400, message = HttpStatusMessages.MESSAGE_400),
             @ApiResponse(code = 401, message = HttpStatusMessages.MESSAGE_401),
             @ApiResponse(code = 403, message = HttpStatusMessages.MESSAGE_403),
             @ApiResponse(code = 404, message = HttpStatusMessages.MESSAGE_404),
             @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409) })
-    public Response deleteExtensionBundle(
-            @PathParam("bundleId")
-            @ApiParam("The extension bundle identifier")
-                final String bundleId) {
-
-        final ExtensionBundle extensionBundle = getExtensionBundleWithBucketReadAuthorization(bundleId);
-
-        final ExtensionBundle deletedExtensionBundle = registryService.deleteExtensionBundle(extensionBundle);
-        publish(EventFactory.extensionBundleDeleted(deletedExtensionBundle));
-
-        permissionsService.populateItemPermissions(deletedExtensionBundle);
-        linkService.populateLinks(deletedExtensionBundle);
-
-        return Response.status(Response.Status.OK).entity(deletedExtensionBundle).build();
-    }
-
-    // ---------- Extension Bundle Versions ----------
-
-    @GET
-    @Path("bundles/versions")
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.APPLICATION_JSON)
-    @ApiOperation(
-            value = "Get extension bundles versions across all authorized buckets",
-            notes = "The returned items will include only items from buckets for which the user is authorized. " +
-                    "If the user is not authorized to any buckets, an empty list will be returned.",
-            response = ExtensionBundleVersionMetadata.class,
-            responseContainer = "List"
-    )
-    @ApiResponses({ @ApiResponse(code = 401, message = HttpStatusMessages.MESSAGE_401) })
-    public Response getExtensionBundleVersions(
+    public Response getExtensionsProvidingServiceAPI(
+            @QueryParam("className")
+            @ApiParam(value = "The name of the service API class")
+                final String className,
             @QueryParam("groupId")
-            @ApiParam("Optional groupId to filter results. The value may be an exact match, or a wildcard, " +
-                    "such as 'com.%' to select all bundle versions where the groupId starts with 'com.'.")
+            @ApiParam(value = "The groupId of the bundle containing the service API class")
                 final String groupId,
             @QueryParam("artifactId")
-            @ApiParam("Optional artifactId to filter results. The value may be an exact match, or a wildcard, " +
-                    "such as 'nifi-%' to select all bundle versions where the artifactId starts with 'nifi-'.")
+            @ApiParam(value = "The artifactId of the bundle containing the service API class")
                 final String artifactId,
             @QueryParam("version")
-            @ApiParam("Optional version to filter results. The value maye be an exact match, or a wildcard, " +
-                    "such as '1.0.%' to select all bundle versions where the version starts with '1.0.'.")
+            @ApiParam(value = "The version of the bundle containing the service API class")
                 final String version
-            ) {
+    ) {
 
         final Set<String> authorizedBucketIds = getAuthorizedBucketIds(RequestAction.READ);
         if (authorizedBucketIds == null || authorizedBucketIds.isEmpty()) {
             // not authorized for any bucket, return empty list of items
             return Response.status(Response.Status.OK).entity(new ArrayList<>()).build();
         }
 
-        final ExtensionBundleVersionFilterParams filterParams = ExtensionBundleVersionFilterParams.of(groupId, artifactId, version);
-        final SortedSet<ExtensionBundleVersionMetadata> bundleVersions = registryService.getExtensionBundleVersions(authorizedBucketIds, filterParams);
-        linkService.populateLinks(bundleVersions);
-
-        return Response.status(Response.Status.OK).entity(bundleVersions).build();
-    }
-
-    @GET
-    @Path("bundles/{bundleId}/versions")
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.APPLICATION_JSON)
-    @ApiOperation(
-            value = "Gets the metadata about the versions of an extension bundle",
-            nickname = "globalGetExtensionBundleVersions",
-            response = ExtensionBundleVersionMetadata.class,
-            responseContainer = "List",
-            extensions = {
-                    @Extension(name = "access-policy", properties = {
-                            @ExtensionProperty(name = "action", value = "read"),
-                            @ExtensionProperty(name = "resource", value = "/buckets/{bucketId}") })
-            }
-    )
-    @ApiResponses({
-            @ApiResponse(code = 400, message = HttpStatusMessages.MESSAGE_400),
-            @ApiResponse(code = 401, message = HttpStatusMessages.MESSAGE_401),
-            @ApiResponse(code = 403, message = HttpStatusMessages.MESSAGE_403),
-            @ApiResponse(code = 404, message = HttpStatusMessages.MESSAGE_404),
-            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409) })
-    public Response getExtensionBundleVersions(
-            @PathParam("bundleId")
-            @ApiParam("The extension bundle identifier")
-                final String bundleId) {
-
-        final ExtensionBundle extensionBundle = getExtensionBundleWithBucketReadAuthorization(bundleId);
-
-        final SortedSet<ExtensionBundleVersionMetadata> bundleVersions = registryService.getExtensionBundleVersions(extensionBundle.getIdentifier());
-        linkService.populateLinks(bundleVersions);
-
-        return Response.status(Response.Status.OK).entity(bundleVersions).build();
-    }
-
-    @GET
-    @Path("bundles/{bundleId}/versions/{version}")
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.APPLICATION_JSON)
-    @ApiOperation(
-            value = "Gets the descriptor for the specified version of the extension bundle",
-            nickname = "globalGetExtensionBundleVersionDescriptor",
-            response = ExtensionBundleVersion.class,
-            extensions = {
-                    @Extension(name = "access-policy", properties = {
-                            @ExtensionProperty(name = "action", value = "read"),
-                            @ExtensionProperty(name = "resource", value = "/buckets/{bucketId}") })
-            }
-    )
-    @ApiResponses({
-            @ApiResponse(code = 400, message = HttpStatusMessages.MESSAGE_400),
-            @ApiResponse(code = 401, message = HttpStatusMessages.MESSAGE_401),
-            @ApiResponse(code = 403, message = HttpStatusMessages.MESSAGE_403),
-            @ApiResponse(code = 404, message = HttpStatusMessages.MESSAGE_404),
-            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409) })
-    public Response getExtensionBundleVersion(
-            @PathParam("bundleId")
-            @ApiParam("The extension bundle identifier")
-                final String bundleId,
-            @PathParam("version")
-            @ApiParam("The version of the bundle")
-                final String version) {
-
-        final ExtensionBundle extensionBundle = getExtensionBundleWithBucketReadAuthorization(bundleId);
+        final ProvidedServiceAPI serviceAPI = new ProvidedServiceAPI();
+        serviceAPI.setClassName(className);
+        serviceAPI.setGroupId(groupId);
+        serviceAPI.setArtifactId(artifactId);
+        serviceAPI.setVersion(version);
 
-        final ExtensionBundleVersionCoordinate versionCoordinate = new ExtensionBundleVersionCoordinate(
-                extensionBundle.getBucketIdentifier(),
-                extensionBundle.getGroupId(),
-                extensionBundle.getArtifactId(),
-                version);
+        final SortedSet<ExtensionMetadata> extensionMetadata = registryService.getExtensionMetadata(authorizedBucketIds, serviceAPI);
+        linkService.populateLinks(extensionMetadata);
 
-        final ExtensionBundleVersion bundleVersion = registryService.getExtensionBundleVersion(versionCoordinate);
-        linkService.populateLinks(bundleVersion);
+        final ExtensionMetadataContainer container = new ExtensionMetadataContainer();
+        container.setExtensions(extensionMetadata);
+        container.setNumResults(extensionMetadata.size());
 
-        return Response.ok(bundleVersion).build();
+        return Response.status(Response.Status.OK).entity(container).build();
     }
 
     @GET
-    @Path("bundles/{bundleId}/versions/{version}/content")
-    @Consumes(MediaType.WILDCARD)
-    @Produces(MediaType.APPLICATION_OCTET_STREAM)
-    @ApiOperation(
-            value = "Gets the binary content for the specified version of the extension bundle",
-            nickname = "globalGetExtensionBundleVersion",
-            response = byte[].class,
-            extensions = {
-                    @Extension(name = "access-policy", properties = {
-                            @ExtensionProperty(name = "action", value = "read"),
-                            @ExtensionProperty(name = "resource", value = "/buckets/{bucketId}") })
-            }
-    )
-    @ApiResponses({
-            @ApiResponse(code = 400, message = HttpStatusMessages.MESSAGE_400),
-            @ApiResponse(code = 401, message = HttpStatusMessages.MESSAGE_401),
-            @ApiResponse(code = 403, message = HttpStatusMessages.MESSAGE_403),
-            @ApiResponse(code = 404, message = HttpStatusMessages.MESSAGE_404),
-            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409) })
-    public Response getExtensionBundleVersionContent(
-            @PathParam("bundleId")
-            @ApiParam("The extension bundle identifier")
-                final String bundleId,
-            @PathParam("version")
-            @ApiParam("The version of the bundle")
-                final String version) {
-
-        final ExtensionBundle extensionBundle = getExtensionBundleWithBucketReadAuthorization(bundleId);
-
-        final ExtensionBundleVersionCoordinate versionCoordinate = new ExtensionBundleVersionCoordinate(
-                extensionBundle.getBucketIdentifier(),
-                extensionBundle.getGroupId(),
-                extensionBundle.getArtifactId(),
-                version);
-
-        final ExtensionBundleVersion bundleVersion = registryService.getExtensionBundleVersion(versionCoordinate);
-        final StreamingOutput streamingOutput = (output) -> registryService.writeExtensionBundleVersionContent(bundleVersion, output);
-
-        return Response.ok(streamingOutput)
-                .header(CONTENT_DISPOSITION_HEADER,"attachment; filename = " + bundleVersion.getFilename())
-                .build();
-    }
-
-    @DELETE
-    @Path("bundles/{bundleId}/versions/{version}")
+    @Path("/tags")
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
     @ApiOperation(
-            value = "Deletes the given extension bundle version",
-            nickname = "globalDeleteExtensionBundleVersion",
-            response = ExtensionBundleVersion.class,
-            extensions = {
-                    @Extension(name = "access-policy", properties = {
-                            @ExtensionProperty(name = "action", value = "write"),
-                            @ExtensionProperty(name = "resource", value = "/buckets/{bucketId}") })
-            }
+            value = "Gets all the extension tags known to this NiFi Registry instance.",
+            response = String.class,
 
 Review comment:
   This response type needs to be updated to `TagCount.class`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services