You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by bbende <gi...@git.apache.org> on 2018/11/20 16:11:57 UTC

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

GitHub user bbende opened a pull request:

    https://github.com/apache/nifi-registry/pull/148

    NIFIREG-211 Initial work for adding extenion bundles to NiFi Registry

    This PR includes the foundational work for adding extension bundles as a new type of versioned item. With this PR it is possible to upload NARs via the REST API with them being stored in the local filesystem persistence provider, and then interact with them through various REST APIs.
    
    In the current state of this PR, registry does not have a way to know anything about the actual extensions contained in a bundle. This will need to come from a component descriptor provided within the NAR which will then be parsed and extracted on registry side.
    
    We can leave NIFIREG-211 open after merging this so that we can continue to work against it and build out the additional functionality.
    
    A NAR can be uploaded by making a multi-part form POST to the end-point:
    
    _buckets/{bucketId}/extensions/bundles/{bundleType}_
    
    An example using curl would be:
    
    `curl -v -F file=@./nifi-example-processors-nar-1.0.0.nar http://localhost:18080/nifi-registry-api/buckets/65d6d70d-ecb7-478f-bbc4-1e378e52827c/extensions/bundles/nifi-nar`
    
    From there you can interact with the REST API from the swagger docs.
    
    I've also started a branch of NiFi with CLI commands related to extension bundles:
    https://github.com/bbende/nifi/tree/extensions
    
    If you run a full build of this PR first with mvn clean install, and then a build of the NiFi branch, you can launch the CLI in nifi-toolkit/nifi-toolkit-assembly/target/nifi-toolkit-1.9.0-SNAPSHOT-bin/nifi-toolkit-1.9.0-SNAPSHOT/bin/cli.sh and run the "upload-nars" command:
    
    `registry upload-nars -b 024dba45-95c2-4a54-b79d-0abe1bb1122a -ebd /path/to/nifi-releases/nifi-1.8.0/lib -u http://localhost:18080 -verbose`
    
    This should upload all of the NARs from the 1.8.0 release to your local registry.
    
    Here is the commit history for all the changes made in this PR:
    
    - Setting up DB tables and entities for extensions
    - Updated MetadataService and DatabaseMetadataService with new methods for extension entities
    - Added ON DELETE CASCADE to existing tables and simplified delete logic for buckets and flows
    - Created data model for extension bundles and mapping to/from DB entities
    - Created ExtensionBundleExtractor with an implemenetation for NARs
    - Setup LinkService and LinkBuilder for extension bundles
    - Setup pluggable persistence provider for extension bundles and implemented a local file-system provider.
    - Refactored LinkService and add links for all extension related items
    - Changed extension service to write out bundles to a temp directory before extracting and passing to persistence provider
    - Implemented multi-part form upload for extensions bundles
    - Upgraded to spring-boot 2.1.0
    - Added SHA-256 checksums for bundle versions
    - Initial client methods for uploading and retrieving bundles
    - Configuring NiFi Registry Jersey client to use chunked entity processing so we don't load the entire bundle content into memory during an upload

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/bbende/nifi-registry extensions

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi-registry/pull/148.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #148
    
----
commit 7744072b5f059fe5240746e968493eddefa719a6
Author: Bryan Bende <bb...@...>
Date:   2018-11-20T15:57:08Z

    NIFIREG-211 Initial work for adding extenion bundles to NiFi Registry
    - Setting up DB tables and entities for extensions
    - Updated MetadataService and DatabaseMetadataService with new methods for extension entities
    - Added ON DELETE CASCADE to existing tables and simplified delete logic for buckets and flows
    - Created data model for extension bundles and mapping to/from DB entities
    - Created ExtensionBundleExtractor with an implemenetation for NARs
    - Setup LinkService and LinkBuilder for extension bundles
    - Setup pluggable persistence provider for extension bundles and implemented a local file-system provider.
    - Refactored LinkService and add links for all extension related items
    - Changed extension service to write out bundles to a temp directory before extracting and passing to persistence provider
    - Implemented multi-part form upload for extensions bundles
    - Upgraded to spring-boot 2.1.0
    - Added SHA-256 checksums for bundle versions
    - Initial client methods for uploading and retrieving bundles
    - Configuring NiFi Registry Jersey client to use chunked entity processing so we don't load the entire bundle content into memory during an upload

----


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236850477
  
    --- Diff: nifi-registry-core/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ExtensionRepositoryResource.java ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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.web.api;
    +
    +import io.swagger.annotations.Api;
    +import io.swagger.annotations.ApiOperation;
    +import io.swagger.annotations.ApiParam;
    +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.nifi.registry.bucket.Bucket;
    +import org.apache.nifi.registry.bucket.BucketItem;
    +import org.apache.nifi.registry.event.EventService;
    +import org.apache.nifi.registry.extension.ExtensionBundleVersion;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoArtifact;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoBucket;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoGroup;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoVersion;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoVersionSummary;
    +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.springframework.beans.factory.annotation.Autowired;
    +import org.springframework.stereotype.Component;
    +
    +import javax.ws.rs.Consumes;
    +import javax.ws.rs.GET;
    +import javax.ws.rs.Path;
    +import javax.ws.rs.PathParam;
    +import javax.ws.rs.Produces;
    +import javax.ws.rs.core.Link;
    +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.Set;
    +import java.util.SortedSet;
    +
    +@Component
    +@Path("/extensions/repo")
    +@Api(
    +        value = "extension_repository",
    +        description = "Interact with extension bundles via the hierarchy of bucket/group/artifact/version.",
    +        authorizations = { @Authorization("Authorization") }
    +)
    +public class ExtensionRepositoryResource extends AuthorizableApplicationResource {
    +
    +    public static final String CONTENT_DISPOSITION_HEADER = "content-disposition";
    +    private final RegistryService registryService;
    +    private final LinkService linkService;
    +
    +    @Autowired
    +    public ExtensionRepositoryResource(
    +            final RegistryService registryService,
    +            final LinkService linkService,
    +            final AuthorizationService authorizationService,
    +            final EventService eventService) {
    +        super(authorizationService, eventService);
    +        this.registryService = registryService;
    +        this.linkService = linkService;
    +    }
    +
    +    @GET
    +    @Consumes(MediaType.WILDCARD)
    +    @Produces(MediaType.APPLICATION_JSON)
    +    @ApiOperation(
    +            value = "Gets the names of the buckets the current user is authorized for in order to browse the repo by bucket",
    +            response = ExtensionRepoBucket.class,
    +            responseContainer = "List",
    +            extensions = {
    +                    @Extension(name = "access-policy", properties = {
    +                            @ExtensionProperty(name = "action", value = "read"),
    +                            @ExtensionProperty(name = "resource", value = "/buckets/{bucketId}") })
    +            }
    --- End diff --
    
    That sounds consistent with the getBuckets end-point, so I'll remove that.


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236849819
  
    --- Diff: nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/extension/StandardExtensionService.java ---
    @@ -0,0 +1,589 @@
    +/*
    + * 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.extension;
    +
    +import org.apache.commons.codec.binary.Hex;
    +import org.apache.commons.codec.digest.DigestUtils;
    +import org.apache.commons.io.IOUtils;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.commons.lang3.Validate;
    +import org.apache.nifi.registry.bucket.Bucket;
    +import org.apache.nifi.registry.db.entity.BucketEntity;
    +import org.apache.nifi.registry.db.entity.ExtensionBundleEntity;
    +import org.apache.nifi.registry.db.entity.ExtensionBundleEntityType;
    +import org.apache.nifi.registry.db.entity.ExtensionBundleVersionEntity;
    +import org.apache.nifi.registry.exception.ResourceNotFoundException;
    +import org.apache.nifi.registry.extension.BundleCoordinate;
    +import org.apache.nifi.registry.extension.BundleDetails;
    +import org.apache.nifi.registry.extension.BundleExtractor;
    +import org.apache.nifi.registry.extension.ExtensionBundle;
    +import org.apache.nifi.registry.extension.ExtensionBundleContext;
    +import org.apache.nifi.registry.extension.ExtensionBundlePersistenceProvider;
    +import org.apache.nifi.registry.extension.ExtensionBundleType;
    +import org.apache.nifi.registry.extension.ExtensionBundleVersion;
    +import org.apache.nifi.registry.extension.ExtensionBundleVersionDependency;
    +import org.apache.nifi.registry.extension.ExtensionBundleVersionMetadata;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoArtifact;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoBucket;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoGroup;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoVersionSummary;
    +import org.apache.nifi.registry.properties.NiFiRegistryProperties;
    +import org.apache.nifi.registry.provider.extension.StandardExtensionBundleContext;
    +import org.apache.nifi.registry.security.authorization.user.NiFiUserUtils;
    +import org.apache.nifi.registry.service.DataModelMapper;
    +import org.apache.nifi.registry.service.MetadataService;
    +import org.apache.nifi.registry.util.FileUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.springframework.beans.factory.annotation.Autowired;
    +import org.springframework.stereotype.Service;
    +
    +import javax.validation.ConstraintViolation;
    +import javax.validation.ConstraintViolationException;
    +import javax.validation.Validator;
    +import java.io.BufferedInputStream;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.security.DigestInputStream;
    +import java.security.MessageDigest;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.SortedSet;
    +import java.util.TreeSet;
    +import java.util.UUID;
    +import java.util.stream.Collectors;
    +
    +@Service
    +public class StandardExtensionService implements ExtensionService {
    +
    +    private static final Logger LOGGER = LoggerFactory.getLogger(StandardExtensionService.class);
    +
    +    private final MetadataService metadataService;
    +    private final Map<ExtensionBundleType, BundleExtractor> extractors;
    +    private final ExtensionBundlePersistenceProvider bundlePersistenceProvider;
    +    private final Validator validator;
    +    private final File extensionsWorkingDir;
    +
    +    @Autowired
    +    public StandardExtensionService(final MetadataService metadataService,
    +                                    final Map<ExtensionBundleType, BundleExtractor> extractors,
    +                                    final ExtensionBundlePersistenceProvider bundlePersistenceProvider,
    +                                    final Validator validator,
    +                                    final NiFiRegistryProperties properties) {
    +        this.metadataService = metadataService;
    +        this.extractors = extractors;
    +        this.bundlePersistenceProvider = bundlePersistenceProvider;
    +        this.validator = validator;
    +        this.extensionsWorkingDir = properties.getExtensionsWorkingDirectory();
    +        Validate.notNull(this.metadataService);
    +        Validate.notNull(this.extractors);
    +        Validate.notNull(this.bundlePersistenceProvider);
    +        Validate.notNull(this.validator);
    +        Validate.notNull(this.extensionsWorkingDir);
    +    }
    +
    +    private <T>  void validate(T t, String invalidMessage) {
    +        final Set<ConstraintViolation<T>> violations = validator.validate(t);
    +        if (violations.size() > 0) {
    +            throw new ConstraintViolationException(invalidMessage, violations);
    +        }
    +    }
    +
    +    @Override
    +    public ExtensionBundleVersion createExtensionBundleVersion(final String bucketIdentifier, final ExtensionBundleType bundleType,
    +                                                               final InputStream inputStream) throws IOException {
    +        if (StringUtils.isBlank(bucketIdentifier)) {
    +            throw new IllegalArgumentException("Bucket identifier cannot be null or blank");
    +        }
    +
    +        if (bundleType == null) {
    +            throw new IllegalArgumentException("Bundle type cannot be null");
    +        }
    +
    +        if (inputStream == null) {
    +            throw new IllegalArgumentException("Extension bundle input stream cannot be null");
    +        }
    +
    +        if (!extractors.containsKey(bundleType)) {
    +            throw new IllegalArgumentException("No metadata extractor is registered for bundle-type: " + bundleType);
    +        }
    +
    +        // ensure the bucket exists
    +        final BucketEntity existingBucket = metadataService.getBucketById(bucketIdentifier);
    +        if (existingBucket == null) {
    +            LOGGER.warn("The specified bucket id [{}] does not exist.", bucketIdentifier);
    +            throw new ResourceNotFoundException("The specified bucket ID does not exist in this registry.");
    +        }
    +
    +        // ensure the extensions directory exists and we can read and write to it
    +        FileUtils.ensureDirectoryExistAndCanReadAndWrite(extensionsWorkingDir);
    +
    +        final String extensionWorkingFilename = UUID.randomUUID().toString();
    +        final File extensionWorkingFile = new File(extensionsWorkingDir, extensionWorkingFilename);
    +        LOGGER.debug("Writing bundle contents to working directory at {}", new Object[]{extensionWorkingFile.getAbsolutePath()});
    +
    +        try {
    +            // write the contents of the input stream to a temporary file in the extensions working directory
    +            final MessageDigest sha256Digest = DigestUtils.getSha256Digest();
    +            try (final DigestInputStream digestInputStream = new DigestInputStream(inputStream, sha256Digest);
    +                 final OutputStream out = new FileOutputStream(extensionWorkingFile)) {
    +                IOUtils.copy(digestInputStream, out);
    +            }
    +
    +            final String sha256Hex = Hex.encodeHexString(sha256Digest.digest());
    +
    +            // extract the details of the bundle from the temp file in the working directory
    +            final BundleDetails bundleDetails;
    +            try (final InputStream in = new FileInputStream(extensionWorkingFile)) {
    +                final BundleExtractor extractor = extractors.get(bundleType);
    +                bundleDetails = extractor.extract(in);
    +            }
    +
    +            final BundleCoordinate bundleCoordinate = bundleDetails.getBundleCoordinate();
    +            final BundleCoordinate dependencyCoordinate = bundleDetails.getDependencyBundleCoordinate();
    +
    +            final String groupId = bundleCoordinate.getGroupId();
    +            final String artifactId = bundleCoordinate.getArtifactId();
    +            final String version = bundleCoordinate.getVersion();
    +            LOGGER.debug("Extracted bundle details - '{}' - '{}' - '{}'", new Object[]{groupId, artifactId, version});
    +
    +            // a bundle with the same group, artifact, and version can exist in multiple buckets, but only if it contains the same binary content,
    +            // we can determine that by comparing the SHA-256 digest of the incoming bundle against existing bundles with the same group, artifact, version
    +            final List<ExtensionBundleVersionEntity> allExistingVersions = metadataService.getExtensionBundleVersionsGlobal(groupId, artifactId, version);
    +            for (final ExtensionBundleVersionEntity existingVersionEntity : allExistingVersions) {
    +                if (!existingVersionEntity.getSha256Hex().equals(sha256Hex)) {
    +                    throw new IllegalStateException("Found existing extension bundle with same group, artifact, and version, but different SHA-256 check-sum");
    +                }
    +            }
    +
    +            // get the existing extension bundle entity, or create a new one if one does not exist in the bucket with the group + artifact
    +            final ExtensionBundleEntity extensionBundle = getOrCreateExtensionBundle(bucketIdentifier, groupId, artifactId, bundleType);
    +
    +            // ensure there isn't already a version of the bundle with the same version
    +            final ExtensionBundleVersionEntity existingVersion = metadataService.getExtensionBundleVersion(bucketIdentifier, groupId, artifactId, version);
    +            if (existingVersion != null) {
    +                LOGGER.warn("The specified version [{}] already exists for extension bundle [{}].", new Object[]{version, extensionBundle.getId()});
    +                throw new IllegalStateException("The specified version already exists for the given extension bundle");
    +            }
    +
    +            // create the bundle version in the metadata db
    +            final String userIdentity = NiFiUserUtils.getNiFiUserIdentity();
    +            final long bundleCreatedTime = extensionBundle.getCreated().getTime();
    +
    +            final ExtensionBundleVersionMetadata versionMetadata = new ExtensionBundleVersionMetadata();
    +            versionMetadata.setId(UUID.randomUUID().toString());
    +            versionMetadata.setExtensionBundleId(extensionBundle.getId());
    +            versionMetadata.setBucketId(bucketIdentifier);
    +            versionMetadata.setVersion(version);
    +            versionMetadata.setTimestamp(bundleCreatedTime);
    +            versionMetadata.setAuthor(userIdentity);
    +            versionMetadata.setSha256Hex(sha256Hex);
    +
    +            if (dependencyCoordinate != null) {
    +                final ExtensionBundleVersionDependency versionDependency = new ExtensionBundleVersionDependency();
    +                versionDependency.setGroupId(dependencyCoordinate.getGroupId());
    +                versionDependency.setArtifactId(dependencyCoordinate.getArtifactId());
    +                versionDependency.setVersion(dependencyCoordinate.getVersion());
    +                versionMetadata.setDependency(versionDependency);
    +            }
    +
    +            validate(versionMetadata, "Cannot create extension bundle version");
    +
    +            final ExtensionBundleVersionEntity versionEntity = DataModelMapper.map(versionMetadata);
    +            metadataService.createExtensionBundleVersion(versionEntity);
    +
    +            // persist the content of the bundle to the persistence provider
    +            final ExtensionBundleContext context = new StandardExtensionBundleContext.Builder()
    +                    .bundleType(getProviderBundleType(bundleType))
    +                    .bucketId(existingBucket.getId())
    +                    .bucketName(existingBucket.getName())
    +                    .bundleId(extensionBundle.getId())
    +                    .bundleGroupId(extensionBundle.getGroupId())
    +                    .bundleArtifactId(extensionBundle.getArtifactId())
    +                    .bundleVersion(versionMetadata.getVersion())
    +                    .author(versionMetadata.getAuthor())
    +                    .timestamp(versionMetadata.getTimestamp())
    +                    .build();
    +
    +            try (final InputStream in = new FileInputStream(extensionWorkingFile);
    +                 final InputStream bufIn = new BufferedInputStream(in)) {
    +                bundlePersistenceProvider.saveBundleVersion(context, bufIn);
    +                LOGGER.debug("Bundle saved to persistence provider - '{}' - '{}' - '{}'",
    +                        new Object[]{groupId, artifactId, version});
    +            }
    +
    +            // get the updated extension bundle so it contains the correct version count
    +            final ExtensionBundleEntity updatedBundle = metadataService.getExtensionBundle(bucketIdentifier, groupId, artifactId);
    +
    +            // create the full ExtensionBundleVersion instance to return
    +            final ExtensionBundleVersion extensionBundleVersion = new ExtensionBundleVersion();
    +            extensionBundleVersion.setVersionMetadata(versionMetadata);
    +            extensionBundleVersion.setExtensionBundle(DataModelMapper.map(existingBucket, updatedBundle));
    +            extensionBundleVersion.setBucket(DataModelMapper.map(existingBucket));
    +            return extensionBundleVersion;
    +
    +        } finally {
    +            if (extensionWorkingFile.exists()) {
    +                try {
    +                    extensionWorkingFile.delete();
    +                } catch (Exception e) {
    +                    LOGGER.warn("Error removing temporary extension bundle file at {}",
    +                            new Object[]{extensionWorkingFile.getAbsolutePath()});
    +                }
    +            }
    +        }
    +    }
    +
    +    private ExtensionBundleEntity getOrCreateExtensionBundle(final String bucketId, final String groupId,
    +                                                             final String artifactId, final ExtensionBundleType bundleType) {
    +        ExtensionBundleEntity existingBundleEntity = metadataService.getExtensionBundle(bucketId, groupId, artifactId);
    +        if (existingBundleEntity == null) {
    +            final ExtensionBundle bundle = new ExtensionBundle();
    +            bundle.setIdentifier(UUID.randomUUID().toString());
    +            bundle.setBucketIdentifier(bucketId);
    +            bundle.setName(groupId + " - " + artifactId);
    --- End diff --
    
    Makes sense, I'm on-board with that


---

[GitHub] nifi-registry issue #148: NIFIREG-211 Initial work for adding extenion bundl...

Posted by ottobackwards <gi...@git.apache.org>.
Github user ottobackwards commented on the issue:

    https://github.com/apache/nifi-registry/pull/148
  
    What are some of the rules that can be tested?   Like uploading with duplicate names etc?


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236853819
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.extension;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkableEntity;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotBlank;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import java.util.Objects;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
    +
    +    @NotBlank
    +    private String id;
    +
    +    @NotBlank
    +    private String extensionBundleId;
    +
    +    @NotBlank
    +    private String bucketId;
    +
    +    @NotBlank
    +    private String version;
    +
    +    private ExtensionBundleVersionDependency dependency;
    +
    +    @Min(1)
    +    private long timestamp;
    +
    +    @NotBlank
    +    private String author;
    +
    +    private String description;
    +
    +    @NotBlank
    +    private String sha256Hex;
    +
    +
    +    @ApiModelProperty(value = "The id of this version of the extension bundle")
    +    public String getId() {
    +        return id;
    +    }
    +
    +    public void setId(String id) {
    +        this.id = id;
    +    }
    +
    +    @ApiModelProperty(value = "The id of the extension bundle this version is for")
    +    public String getExtensionBundleId() {
    +        return extensionBundleId;
    +    }
    +
    +    public void setExtensionBundleId(String extensionBundleId) {
    +        this.extensionBundleId = extensionBundleId;
    +    }
    +
    +    @ApiModelProperty(value = "The id of the bucket the extension bundle belongs to", required = true)
    +    public String getBucketId() {
    +        return bucketId;
    +    }
    +
    +    public void setBucketId(String bucketId) {
    +        this.bucketId = bucketId;
    +    }
    +
    +    @ApiModelProperty(value = "The version of the extension bundle")
    +    public String getVersion() {
    +        return version;
    +    }
    +
    +    public void setVersion(String version) {
    +        this.version = version;
    +    }
    +
    +    @ApiModelProperty(value = "The optional bundle dependency (i.e. another bundle this bundle is dependent on)")
    +    public ExtensionBundleVersionDependency getDependency() {
    +        return dependency;
    +    }
    +
    +    public void setDependency(ExtensionBundleVersionDependency dependency) {
    +        this.dependency = dependency;
    +    }
    +
    +    @ApiModelProperty(value = "The timestamp of the create date of this version")
    +    public long getTimestamp() {
    +        return timestamp;
    +    }
    +
    +    public void setTimestamp(long timestamp) {
    +        this.timestamp = timestamp;
    +    }
    +
    +    @ApiModelProperty(value = "The identity that created this version")
    +    public String getAuthor() {
    +        return author;
    +    }
    +
    +    public void setAuthor(String author) {
    +        this.author = author;
    +    }
    +
    +    @ApiModelProperty(value = "The description for this version")
    +    public String getDescription() {
    +        return description;
    +    }
    +
    +    public void setDescription(String description) {
    +        this.description = description;
    +    }
    +
    +    @ApiModelProperty(value = "The hex representation of the SHA-256 digest of the binary content for this version")
    +    public String getSha256Hex() {
    +        return sha256Hex;
    +    }
    +
    +    public void setSha256Hex(String sha256Hex) {
    +        this.sha256Hex = sha256Hex;
    +    }
    +
    +    @Override
    +    public int compareTo(final ExtensionBundleVersionMetadata o) {
    +        return o == null ? -1 : version.compareTo(o.version);
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        return Objects.hashCode(this.id);
    +    }
    +
    +    @Override
    +    public boolean equals(Object obj) {
    +        if (obj == null) {
    +            return false;
    +        }
    +        if (getClass() != obj.getClass()) {
    +            return false;
    +        }
    +
    +        final ExtensionBundleVersionMetadata other = (ExtensionBundleVersionMetadata) obj;
    +        return Objects.equals(this.id, other.id);
    --- End diff --
    
    Also, unfortunately we have no ORM here and are writing our own JDBC :(


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by kevdoran <gi...@git.apache.org>.
Github user kevdoran commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236827312
  
    --- Diff: nifi-registry-core/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ExtensionRepositoryResource.java ---
    @@ -0,0 +1,378 @@
    +/*
    + * 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.web.api;
    +
    +import io.swagger.annotations.Api;
    +import io.swagger.annotations.ApiOperation;
    +import io.swagger.annotations.ApiParam;
    +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.nifi.registry.bucket.Bucket;
    +import org.apache.nifi.registry.bucket.BucketItem;
    +import org.apache.nifi.registry.event.EventService;
    +import org.apache.nifi.registry.extension.ExtensionBundleVersion;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoArtifact;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoBucket;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoGroup;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoVersion;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoVersionSummary;
    +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.springframework.beans.factory.annotation.Autowired;
    +import org.springframework.stereotype.Component;
    +
    +import javax.ws.rs.Consumes;
    +import javax.ws.rs.GET;
    +import javax.ws.rs.Path;
    +import javax.ws.rs.PathParam;
    +import javax.ws.rs.Produces;
    +import javax.ws.rs.core.Link;
    +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.Set;
    +import java.util.SortedSet;
    +
    +@Component
    +@Path("/extensions/repo")
    +@Api(
    +        value = "extension_repository",
    +        description = "Interact with extension bundles via the hierarchy of bucket/group/artifact/version.",
    +        authorizations = { @Authorization("Authorization") }
    +)
    +public class ExtensionRepositoryResource extends AuthorizableApplicationResource {
    +
    +    public static final String CONTENT_DISPOSITION_HEADER = "content-disposition";
    +    private final RegistryService registryService;
    +    private final LinkService linkService;
    +
    +    @Autowired
    +    public ExtensionRepositoryResource(
    +            final RegistryService registryService,
    +            final LinkService linkService,
    +            final AuthorizationService authorizationService,
    +            final EventService eventService) {
    +        super(authorizationService, eventService);
    +        this.registryService = registryService;
    +        this.linkService = linkService;
    +    }
    +
    +    @GET
    +    @Consumes(MediaType.WILDCARD)
    +    @Produces(MediaType.APPLICATION_JSON)
    +    @ApiOperation(
    +            value = "Gets the names of the buckets the current user is authorized for in order to browse the repo by bucket",
    +            response = ExtensionRepoBucket.class,
    +            responseContainer = "List",
    +            extensions = {
    +                    @Extension(name = "access-policy", properties = {
    +                            @ExtensionProperty(name = "action", value = "read"),
    +                            @ExtensionProperty(name = "resource", value = "/buckets/{bucketId}") })
    +            }
    --- End diff --
    
    I don't think we want an access-policy documented here as read only access is allowed for all authenticated users.


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by kevdoran <gi...@git.apache.org>.
Github user kevdoran commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236810679
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/repo/ExtensionRepoVersion.java ---
    @@ -0,0 +1,58 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.nifi.registry.extension.repo;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkAdapter;
    +
    +import javax.ws.rs.core.Link;
    +import javax.xml.bind.annotation.XmlElement;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionRepoVersion {
    --- End diff --
    
    these dynamically generated links worked well in my testing


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by kevdoran <gi...@git.apache.org>.
Github user kevdoran commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236837704
  
    --- Diff: nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/extension/StandardExtensionService.java ---
    @@ -0,0 +1,589 @@
    +/*
    + * 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.extension;
    +
    +import org.apache.commons.codec.binary.Hex;
    +import org.apache.commons.codec.digest.DigestUtils;
    +import org.apache.commons.io.IOUtils;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.commons.lang3.Validate;
    +import org.apache.nifi.registry.bucket.Bucket;
    +import org.apache.nifi.registry.db.entity.BucketEntity;
    +import org.apache.nifi.registry.db.entity.ExtensionBundleEntity;
    +import org.apache.nifi.registry.db.entity.ExtensionBundleEntityType;
    +import org.apache.nifi.registry.db.entity.ExtensionBundleVersionEntity;
    +import org.apache.nifi.registry.exception.ResourceNotFoundException;
    +import org.apache.nifi.registry.extension.BundleCoordinate;
    +import org.apache.nifi.registry.extension.BundleDetails;
    +import org.apache.nifi.registry.extension.BundleExtractor;
    +import org.apache.nifi.registry.extension.ExtensionBundle;
    +import org.apache.nifi.registry.extension.ExtensionBundleContext;
    +import org.apache.nifi.registry.extension.ExtensionBundlePersistenceProvider;
    +import org.apache.nifi.registry.extension.ExtensionBundleType;
    +import org.apache.nifi.registry.extension.ExtensionBundleVersion;
    +import org.apache.nifi.registry.extension.ExtensionBundleVersionDependency;
    +import org.apache.nifi.registry.extension.ExtensionBundleVersionMetadata;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoArtifact;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoBucket;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoGroup;
    +import org.apache.nifi.registry.extension.repo.ExtensionRepoVersionSummary;
    +import org.apache.nifi.registry.properties.NiFiRegistryProperties;
    +import org.apache.nifi.registry.provider.extension.StandardExtensionBundleContext;
    +import org.apache.nifi.registry.security.authorization.user.NiFiUserUtils;
    +import org.apache.nifi.registry.service.DataModelMapper;
    +import org.apache.nifi.registry.service.MetadataService;
    +import org.apache.nifi.registry.util.FileUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.springframework.beans.factory.annotation.Autowired;
    +import org.springframework.stereotype.Service;
    +
    +import javax.validation.ConstraintViolation;
    +import javax.validation.ConstraintViolationException;
    +import javax.validation.Validator;
    +import java.io.BufferedInputStream;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.security.DigestInputStream;
    +import java.security.MessageDigest;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.SortedSet;
    +import java.util.TreeSet;
    +import java.util.UUID;
    +import java.util.stream.Collectors;
    +
    +@Service
    +public class StandardExtensionService implements ExtensionService {
    +
    +    private static final Logger LOGGER = LoggerFactory.getLogger(StandardExtensionService.class);
    +
    +    private final MetadataService metadataService;
    +    private final Map<ExtensionBundleType, BundleExtractor> extractors;
    +    private final ExtensionBundlePersistenceProvider bundlePersistenceProvider;
    +    private final Validator validator;
    +    private final File extensionsWorkingDir;
    +
    +    @Autowired
    +    public StandardExtensionService(final MetadataService metadataService,
    +                                    final Map<ExtensionBundleType, BundleExtractor> extractors,
    +                                    final ExtensionBundlePersistenceProvider bundlePersistenceProvider,
    +                                    final Validator validator,
    +                                    final NiFiRegistryProperties properties) {
    +        this.metadataService = metadataService;
    +        this.extractors = extractors;
    +        this.bundlePersistenceProvider = bundlePersistenceProvider;
    +        this.validator = validator;
    +        this.extensionsWorkingDir = properties.getExtensionsWorkingDirectory();
    +        Validate.notNull(this.metadataService);
    +        Validate.notNull(this.extractors);
    +        Validate.notNull(this.bundlePersistenceProvider);
    +        Validate.notNull(this.validator);
    +        Validate.notNull(this.extensionsWorkingDir);
    +    }
    +
    +    private <T>  void validate(T t, String invalidMessage) {
    +        final Set<ConstraintViolation<T>> violations = validator.validate(t);
    +        if (violations.size() > 0) {
    +            throw new ConstraintViolationException(invalidMessage, violations);
    +        }
    +    }
    +
    +    @Override
    +    public ExtensionBundleVersion createExtensionBundleVersion(final String bucketIdentifier, final ExtensionBundleType bundleType,
    +                                                               final InputStream inputStream) throws IOException {
    +        if (StringUtils.isBlank(bucketIdentifier)) {
    +            throw new IllegalArgumentException("Bucket identifier cannot be null or blank");
    +        }
    +
    +        if (bundleType == null) {
    +            throw new IllegalArgumentException("Bundle type cannot be null");
    +        }
    +
    +        if (inputStream == null) {
    +            throw new IllegalArgumentException("Extension bundle input stream cannot be null");
    +        }
    +
    +        if (!extractors.containsKey(bundleType)) {
    +            throw new IllegalArgumentException("No metadata extractor is registered for bundle-type: " + bundleType);
    +        }
    +
    +        // ensure the bucket exists
    +        final BucketEntity existingBucket = metadataService.getBucketById(bucketIdentifier);
    +        if (existingBucket == null) {
    +            LOGGER.warn("The specified bucket id [{}] does not exist.", bucketIdentifier);
    +            throw new ResourceNotFoundException("The specified bucket ID does not exist in this registry.");
    +        }
    +
    +        // ensure the extensions directory exists and we can read and write to it
    +        FileUtils.ensureDirectoryExistAndCanReadAndWrite(extensionsWorkingDir);
    +
    +        final String extensionWorkingFilename = UUID.randomUUID().toString();
    +        final File extensionWorkingFile = new File(extensionsWorkingDir, extensionWorkingFilename);
    +        LOGGER.debug("Writing bundle contents to working directory at {}", new Object[]{extensionWorkingFile.getAbsolutePath()});
    +
    +        try {
    +            // write the contents of the input stream to a temporary file in the extensions working directory
    +            final MessageDigest sha256Digest = DigestUtils.getSha256Digest();
    +            try (final DigestInputStream digestInputStream = new DigestInputStream(inputStream, sha256Digest);
    +                 final OutputStream out = new FileOutputStream(extensionWorkingFile)) {
    +                IOUtils.copy(digestInputStream, out);
    +            }
    +
    +            final String sha256Hex = Hex.encodeHexString(sha256Digest.digest());
    +
    +            // extract the details of the bundle from the temp file in the working directory
    +            final BundleDetails bundleDetails;
    +            try (final InputStream in = new FileInputStream(extensionWorkingFile)) {
    +                final BundleExtractor extractor = extractors.get(bundleType);
    +                bundleDetails = extractor.extract(in);
    +            }
    +
    +            final BundleCoordinate bundleCoordinate = bundleDetails.getBundleCoordinate();
    +            final BundleCoordinate dependencyCoordinate = bundleDetails.getDependencyBundleCoordinate();
    +
    +            final String groupId = bundleCoordinate.getGroupId();
    +            final String artifactId = bundleCoordinate.getArtifactId();
    +            final String version = bundleCoordinate.getVersion();
    +            LOGGER.debug("Extracted bundle details - '{}' - '{}' - '{}'", new Object[]{groupId, artifactId, version});
    +
    +            // a bundle with the same group, artifact, and version can exist in multiple buckets, but only if it contains the same binary content,
    +            // we can determine that by comparing the SHA-256 digest of the incoming bundle against existing bundles with the same group, artifact, version
    +            final List<ExtensionBundleVersionEntity> allExistingVersions = metadataService.getExtensionBundleVersionsGlobal(groupId, artifactId, version);
    +            for (final ExtensionBundleVersionEntity existingVersionEntity : allExistingVersions) {
    +                if (!existingVersionEntity.getSha256Hex().equals(sha256Hex)) {
    +                    throw new IllegalStateException("Found existing extension bundle with same group, artifact, and version, but different SHA-256 check-sum");
    +                }
    +            }
    +
    +            // get the existing extension bundle entity, or create a new one if one does not exist in the bucket with the group + artifact
    +            final ExtensionBundleEntity extensionBundle = getOrCreateExtensionBundle(bucketIdentifier, groupId, artifactId, bundleType);
    +
    +            // ensure there isn't already a version of the bundle with the same version
    +            final ExtensionBundleVersionEntity existingVersion = metadataService.getExtensionBundleVersion(bucketIdentifier, groupId, artifactId, version);
    +            if (existingVersion != null) {
    +                LOGGER.warn("The specified version [{}] already exists for extension bundle [{}].", new Object[]{version, extensionBundle.getId()});
    +                throw new IllegalStateException("The specified version already exists for the given extension bundle");
    +            }
    +
    +            // create the bundle version in the metadata db
    +            final String userIdentity = NiFiUserUtils.getNiFiUserIdentity();
    +            final long bundleCreatedTime = extensionBundle.getCreated().getTime();
    +
    +            final ExtensionBundleVersionMetadata versionMetadata = new ExtensionBundleVersionMetadata();
    +            versionMetadata.setId(UUID.randomUUID().toString());
    +            versionMetadata.setExtensionBundleId(extensionBundle.getId());
    +            versionMetadata.setBucketId(bucketIdentifier);
    +            versionMetadata.setVersion(version);
    +            versionMetadata.setTimestamp(bundleCreatedTime);
    +            versionMetadata.setAuthor(userIdentity);
    +            versionMetadata.setSha256Hex(sha256Hex);
    +
    +            if (dependencyCoordinate != null) {
    +                final ExtensionBundleVersionDependency versionDependency = new ExtensionBundleVersionDependency();
    +                versionDependency.setGroupId(dependencyCoordinate.getGroupId());
    +                versionDependency.setArtifactId(dependencyCoordinate.getArtifactId());
    +                versionDependency.setVersion(dependencyCoordinate.getVersion());
    +                versionMetadata.setDependency(versionDependency);
    +            }
    +
    +            validate(versionMetadata, "Cannot create extension bundle version");
    +
    +            final ExtensionBundleVersionEntity versionEntity = DataModelMapper.map(versionMetadata);
    +            metadataService.createExtensionBundleVersion(versionEntity);
    +
    +            // persist the content of the bundle to the persistence provider
    +            final ExtensionBundleContext context = new StandardExtensionBundleContext.Builder()
    +                    .bundleType(getProviderBundleType(bundleType))
    +                    .bucketId(existingBucket.getId())
    +                    .bucketName(existingBucket.getName())
    +                    .bundleId(extensionBundle.getId())
    +                    .bundleGroupId(extensionBundle.getGroupId())
    +                    .bundleArtifactId(extensionBundle.getArtifactId())
    +                    .bundleVersion(versionMetadata.getVersion())
    +                    .author(versionMetadata.getAuthor())
    +                    .timestamp(versionMetadata.getTimestamp())
    +                    .build();
    +
    +            try (final InputStream in = new FileInputStream(extensionWorkingFile);
    +                 final InputStream bufIn = new BufferedInputStream(in)) {
    +                bundlePersistenceProvider.saveBundleVersion(context, bufIn);
    +                LOGGER.debug("Bundle saved to persistence provider - '{}' - '{}' - '{}'",
    +                        new Object[]{groupId, artifactId, version});
    +            }
    +
    +            // get the updated extension bundle so it contains the correct version count
    +            final ExtensionBundleEntity updatedBundle = metadataService.getExtensionBundle(bucketIdentifier, groupId, artifactId);
    +
    +            // create the full ExtensionBundleVersion instance to return
    +            final ExtensionBundleVersion extensionBundleVersion = new ExtensionBundleVersion();
    +            extensionBundleVersion.setVersionMetadata(versionMetadata);
    +            extensionBundleVersion.setExtensionBundle(DataModelMapper.map(existingBucket, updatedBundle));
    +            extensionBundleVersion.setBucket(DataModelMapper.map(existingBucket));
    +            return extensionBundleVersion;
    +
    +        } finally {
    +            if (extensionWorkingFile.exists()) {
    +                try {
    +                    extensionWorkingFile.delete();
    +                } catch (Exception e) {
    +                    LOGGER.warn("Error removing temporary extension bundle file at {}",
    +                            new Object[]{extensionWorkingFile.getAbsolutePath()});
    +                }
    +            }
    +        }
    +    }
    +
    +    private ExtensionBundleEntity getOrCreateExtensionBundle(final String bucketId, final String groupId,
    +                                                             final String artifactId, final ExtensionBundleType bundleType) {
    +        ExtensionBundleEntity existingBundleEntity = metadataService.getExtensionBundle(bucketId, groupId, artifactId);
    +        if (existingBundleEntity == null) {
    +            final ExtensionBundle bundle = new ExtensionBundle();
    +            bundle.setIdentifier(UUID.randomUUID().toString());
    +            bundle.setBucketIdentifier(bucketId);
    +            bundle.setName(groupId + " - " + artifactId);
    --- End diff --
    
    Don't know how others feel about this, but personally I would have a slight preference for using the Maven coordinate style here for the inferred bundle name, i.e.:
    
        bundle.setName(groupId + ":" + artifactId);


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r237093889
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.extension;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkableEntity;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotBlank;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import java.util.Objects;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
    +
    +    @NotBlank
    +    private String id;
    +
    +    @NotBlank
    +    private String extensionBundleId;
    +
    +    @NotBlank
    +    private String bucketId;
    +
    +    @NotBlank
    +    private String version;
    +
    +    private ExtensionBundleVersionDependency dependency;
    +
    +    @Min(1)
    +    private long timestamp;
    +
    +    @NotBlank
    +    private String author;
    +
    +    private String description;
    +
    +    @NotBlank
    +    private String sha256Hex;
    --- End diff --
    
    Right now it is only calculated on the NiFi Registry side and would be used by consumers, such as NiFi, to verify they have downloaded the correct bundle:
    
    https://github.com/bbende/nifi-registry/blob/extensions/nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/extension/StandardExtensionService.java#L146-L153
    
    The InputStream there is from a REST end-point where you make a multipart form POST, which is here:
    
    https://github.com/bbende/nifi-registry/blob/extensions/nifi-registry-core/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketExtensionResource.java#L107-L125
    
    Do you think we should be requiring the SHA-256 to be submitted along with the file? This way we can verify that what we received was what was expected to be sent? 
    
    We don't have anything like that when saving flows so I hadn't thought of it that way.


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236840513
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.extension;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkableEntity;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotBlank;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import java.util.Objects;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
    +
    +    @NotBlank
    +    private String id;
    +
    +    @NotBlank
    +    private String extensionBundleId;
    +
    +    @NotBlank
    +    private String bucketId;
    +
    +    @NotBlank
    +    private String version;
    +
    +    private ExtensionBundleVersionDependency dependency;
    +
    +    @Min(1)
    +    private long timestamp;
    +
    +    @NotBlank
    +    private String author;
    +
    +    private String description;
    +
    +    @NotBlank
    +    private String sha256Hex;
    --- End diff --
    
    How easy is it to add a new field here? I'd like to propose storage of a cryptographic signature of some sort, in addition to the SHA-256 checksum, which can be modified (generated over new contents) by a malicious user. What inputs (data, fields, encoding, etc.) does the SHA-256 cover?


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by kevdoran <gi...@git.apache.org>.
Github user kevdoran commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r237568373
  
    --- Diff: nifi-registry-core/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/extension/StandardExtensionService.java ---
    @@ -188,13 +188,14 @@ public ExtensionBundleVersion createExtensionBundleVersion(final String bucketId
     
                 // create the bundle version in the metadata db
                 final String userIdentity = NiFiUserUtils.getNiFiUserIdentity();
    +            final long bundleCreatedTime = extensionBundle.getCreated().getTime();
     
                 final ExtensionBundleVersionMetadata versionMetadata = new ExtensionBundleVersionMetadata();
                 versionMetadata.setId(UUID.randomUUID().toString());
                 versionMetadata.setExtensionBundleId(extensionBundle.getId());
                 versionMetadata.setBucketId(bucketIdentifier);
                 versionMetadata.setVersion(version);
    -            versionMetadata.setTimestamp(System.currentTimeMillis());
    +            versionMetadata.setTimestamp(bundleCreatedTime);
    --- End diff --
    
    This works great when the bundle is being created with the first version uploaded. But I noticed when subsequent versions are uploaded they are also getting tagged with the same creation time. Given that we don't know what was done in getOrCreateExtensionBundle, we might need to refactor the code above, or... it might be better to just revert this change to always use currentTime for the bundle version. I'm good with either


---

[GitHub] nifi-registry issue #148: NIFIREG-211 Initial work for adding extenion bundl...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on the issue:

    https://github.com/apache/nifi-registry/pull/148
  
    All versioned items (flows and now bundles) live in a bucket, and a bucket is where security policies are applied. So each bucket can be kind of like a mini extension repo, there could be a bucket for project nars and another bucket for other nars. 
    
    The main rules are the following...
    
    - Within a bucket, the bundle coordinate is unique, so you can't upload a NAR with the same group+artifact+version to the same bucket twice
    
    - Across buckets you CAN upload NARs with the same group+artifact+version, BUT they must have the same checksum (currently SHA-256) which ensures they are actually the same bundle.


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236840020
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.extension;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkableEntity;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotBlank;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import java.util.Objects;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
    +
    +    @NotBlank
    +    private String id;
    +
    +    @NotBlank
    +    private String extensionBundleId;
    +
    +    @NotBlank
    +    private String bucketId;
    +
    +    @NotBlank
    +    private String version;
    +
    +    private ExtensionBundleVersionDependency dependency;
    +
    +    @Min(1)
    +    private long timestamp;
    +
    +    @NotBlank
    +    private String author;
    +
    +    private String description;
    +
    +    @NotBlank
    +    private String sha256Hex;
    +
    +
    +    @ApiModelProperty(value = "The id of this version of the extension bundle")
    +    public String getId() {
    +        return id;
    +    }
    +
    +    public void setId(String id) {
    +        this.id = id;
    +    }
    +
    +    @ApiModelProperty(value = "The id of the extension bundle this version is for")
    +    public String getExtensionBundleId() {
    +        return extensionBundleId;
    +    }
    +
    +    public void setExtensionBundleId(String extensionBundleId) {
    +        this.extensionBundleId = extensionBundleId;
    +    }
    +
    +    @ApiModelProperty(value = "The id of the bucket the extension bundle belongs to", required = true)
    +    public String getBucketId() {
    +        return bucketId;
    +    }
    +
    +    public void setBucketId(String bucketId) {
    +        this.bucketId = bucketId;
    +    }
    +
    +    @ApiModelProperty(value = "The version of the extension bundle")
    +    public String getVersion() {
    +        return version;
    +    }
    +
    +    public void setVersion(String version) {
    +        this.version = version;
    +    }
    +
    +    @ApiModelProperty(value = "The optional bundle dependency (i.e. another bundle this bundle is dependent on)")
    +    public ExtensionBundleVersionDependency getDependency() {
    +        return dependency;
    +    }
    +
    +    public void setDependency(ExtensionBundleVersionDependency dependency) {
    +        this.dependency = dependency;
    +    }
    +
    +    @ApiModelProperty(value = "The timestamp of the create date of this version")
    +    public long getTimestamp() {
    +        return timestamp;
    +    }
    +
    +    public void setTimestamp(long timestamp) {
    +        this.timestamp = timestamp;
    +    }
    +
    +    @ApiModelProperty(value = "The identity that created this version")
    +    public String getAuthor() {
    +        return author;
    +    }
    +
    +    public void setAuthor(String author) {
    +        this.author = author;
    +    }
    +
    +    @ApiModelProperty(value = "The description for this version")
    +    public String getDescription() {
    +        return description;
    +    }
    +
    +    public void setDescription(String description) {
    +        this.description = description;
    +    }
    +
    +    @ApiModelProperty(value = "The hex representation of the SHA-256 digest of the binary content for this version")
    +    public String getSha256Hex() {
    +        return sha256Hex;
    +    }
    +
    +    public void setSha256Hex(String sha256Hex) {
    +        this.sha256Hex = sha256Hex;
    +    }
    +
    +    @Override
    +    public int compareTo(final ExtensionBundleVersionMetadata o) {
    +        return o == null ? -1 : version.compareTo(o.version);
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        return Objects.hashCode(this.id);
    +    }
    +
    +    @Override
    +    public boolean equals(Object obj) {
    +        if (obj == null) {
    +            return false;
    +        }
    +        if (getClass() != obj.getClass()) {
    +            return false;
    +        }
    +
    +        final ExtensionBundleVersionMetadata other = (ExtensionBundleVersionMetadata) obj;
    +        return Objects.equals(this.id, other.id);
    --- End diff --
    
    Will this cause issues with `Set`/`List` assignments/`#contains()` because the IDs can be equal but the data within them different? I remember this was always an issue with Java O/RM. 


---

[GitHub] nifi-registry issue #148: NIFIREG-211 Initial work for adding extenion bundl...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on the issue:

    https://github.com/apache/nifi-registry/pull/148
  
    @kevdoran just pushed a commit that addresses the review comments, let me know if you find anything else. As an FYI, I did try to change that swagger annotation for LinkableEntity, but it doesn't seem to like Link.JaxbLink or the fully qualified classname, so will leave that for another time.
    
    The only outstanding question from the conversation yesterday was whether the current SHA-256 approach is acceptable, or if we clients she be providing that when uploading a bundle. Need @alopresto 's thoughts on that.


---

[GitHub] nifi-registry issue #148: NIFIREG-211 Initial work for adding extenion bundl...

Posted by ottobackwards <gi...@git.apache.org>.
Github user ottobackwards commented on the issue:

    https://github.com/apache/nifi-registry/pull/148
  
    Would it make sense to have separate areas to store the project nars vs other nars?


---

[GitHub] nifi-registry issue #148: NIFIREG-211 Initial work for adding extenion bundl...

Posted by ottobackwards <gi...@git.apache.org>.
Github user ottobackwards commented on the issue:

    https://github.com/apache/nifi-registry/pull/148
  
    I don't want to pepper you with questions on this, is there some documentation that goes along with this? 


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by kevdoran <gi...@git.apache.org>.
Github user kevdoran commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236808042
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.extension;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkableEntity;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotBlank;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import java.util.Objects;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
    +
    +    @NotBlank
    +    private String id;
    +
    +    @NotBlank
    +    private String extensionBundleId;
    +
    +    @NotBlank
    +    private String bucketId;
    +
    +    @NotBlank
    +    private String version;
    +
    +    private ExtensionBundleVersionDependency dependency;
    +
    +    @Min(1)
    +    private long timestamp;
    +
    +    @NotBlank
    +    private String author;
    +
    +    private String description;
    +
    +    @NotBlank
    +    private String sha256Hex;
    --- End diff --
    
    This is a minor nitpick, but for the JSON field name here, I would have a slight preference for just using `sha256` as the field name as that is consistent with the `/extensions/repo/{bucket}/{group}/{artifact}/{version}/sha256` endpoint (and the field description in the REST API documentation clarifies that it is the hex digest).


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236839456
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.extension;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkableEntity;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotBlank;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import java.util.Objects;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
    +
    +    @NotBlank
    +    private String id;
    +
    +    @NotBlank
    +    private String extensionBundleId;
    +
    +    @NotBlank
    +    private String bucketId;
    +
    +    @NotBlank
    +    private String version;
    +
    +    private ExtensionBundleVersionDependency dependency;
    +
    +    @Min(1)
    +    private long timestamp;
    +
    +    @NotBlank
    +    private String author;
    +
    +    private String description;
    +
    +    @NotBlank
    +    private String sha256Hex;
    +
    +
    +    @ApiModelProperty(value = "The id of this version of the extension bundle")
    +    public String getId() {
    +        return id;
    +    }
    +
    +    public void setId(String id) {
    +        this.id = id;
    +    }
    +
    +    @ApiModelProperty(value = "The id of the extension bundle this version is for")
    +    public String getExtensionBundleId() {
    +        return extensionBundleId;
    +    }
    +
    +    public void setExtensionBundleId(String extensionBundleId) {
    +        this.extensionBundleId = extensionBundleId;
    +    }
    +
    +    @ApiModelProperty(value = "The id of the bucket the extension bundle belongs to", required = true)
    +    public String getBucketId() {
    +        return bucketId;
    +    }
    +
    +    public void setBucketId(String bucketId) {
    +        this.bucketId = bucketId;
    +    }
    +
    +    @ApiModelProperty(value = "The version of the extension bundle")
    +    public String getVersion() {
    +        return version;
    +    }
    +
    +    public void setVersion(String version) {
    +        this.version = version;
    +    }
    +
    +    @ApiModelProperty(value = "The optional bundle dependency (i.e. another bundle this bundle is dependent on)")
    +    public ExtensionBundleVersionDependency getDependency() {
    +        return dependency;
    +    }
    +
    +    public void setDependency(ExtensionBundleVersionDependency dependency) {
    +        this.dependency = dependency;
    +    }
    +
    +    @ApiModelProperty(value = "The timestamp of the create date of this version")
    +    public long getTimestamp() {
    +        return timestamp;
    +    }
    +
    +    public void setTimestamp(long timestamp) {
    +        this.timestamp = timestamp;
    +    }
    +
    +    @ApiModelProperty(value = "The identity that created this version")
    +    public String getAuthor() {
    --- End diff --
    
    Understanding that "authors" can take many forms, is this a standardized String representation, or could it be "alopresto", "Andy LoPresto", and "Andy LoPresto <al...@apache.org>" are all valid and used simultaneously, causing lookup/resolution issues later? Would using a custom datatype here make sense?


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by kevdoran <gi...@git.apache.org>.
Github user kevdoran commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236826377
  
    --- Diff: nifi-registry-core/nifi-registry-framework/src/main/resources/db/migration/V3__AddExtensions.sql ---
    @@ -0,0 +1,62 @@
    +-- 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.
    +
    +CREATE TABLE EXTENSION_BUNDLE (
    +    ID VARCHAR(50) NOT NULL,
    +    BUCKET_ID VARCHAR(50) NOT NULL,
    +    BUNDLE_TYPE VARCHAR(200) NOT NULL,
    +    GROUP_ID VARCHAR(500) NOT NULL,
    +    ARTIFACT_ID VARCHAR(500) NOT NULL,
    +    CONSTRAINT PK__EXTENSION_BUNDLE_ID PRIMARY KEY (ID),
    +    CONSTRAINT FK__EXTENSION_BUNDLE_BUCKET_ITEM_ID FOREIGN KEY (ID) REFERENCES BUCKET_ITEM(ID) ON DELETE CASCADE,
    +    CONSTRAINT FK__EXTENSION_BUNDLE_BUCKET_ID FOREIGN KEY(BUCKET_ID) REFERENCES BUCKET(ID) ON DELETE CASCADE,
    +    CONSTRAINT UNIQUE__EXTENSION_BUNDLE_BUCKET_GROUP_ARTIFACT UNIQUE (BUCKET_ID, GROUP_ID, ARTIFACT_ID)
    +);
    +
    +CREATE TABLE EXTENSION_BUNDLE_VERSION (
    +    ID VARCHAR(50) NOT NULL,
    +    EXTENSION_BUNDLE_ID VARCHAR(50) NOT NULL,
    +    VERSION VARCHAR(100) NOT NULL,
    +    DEPENDENCY_GROUP_ID VARCHAR(500),
    --- End diff --
    
    CPP extensions may have more than one dependency, in which case having these in a separate table may be worthwhile.


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236849686
  
    --- Diff: nifi-registry-core/nifi-registry-framework/src/main/resources/db/migration/V3__AddExtensions.sql ---
    @@ -0,0 +1,62 @@
    +-- 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.
    +
    +CREATE TABLE EXTENSION_BUNDLE (
    +    ID VARCHAR(50) NOT NULL,
    +    BUCKET_ID VARCHAR(50) NOT NULL,
    +    BUNDLE_TYPE VARCHAR(200) NOT NULL,
    +    GROUP_ID VARCHAR(500) NOT NULL,
    +    ARTIFACT_ID VARCHAR(500) NOT NULL,
    +    CONSTRAINT PK__EXTENSION_BUNDLE_ID PRIMARY KEY (ID),
    +    CONSTRAINT FK__EXTENSION_BUNDLE_BUCKET_ITEM_ID FOREIGN KEY (ID) REFERENCES BUCKET_ITEM(ID) ON DELETE CASCADE,
    +    CONSTRAINT FK__EXTENSION_BUNDLE_BUCKET_ID FOREIGN KEY(BUCKET_ID) REFERENCES BUCKET(ID) ON DELETE CASCADE,
    +    CONSTRAINT UNIQUE__EXTENSION_BUNDLE_BUCKET_GROUP_ARTIFACT UNIQUE (BUCKET_ID, GROUP_ID, ARTIFACT_ID)
    +);
    +
    +CREATE TABLE EXTENSION_BUNDLE_VERSION (
    +    ID VARCHAR(50) NOT NULL,
    +    EXTENSION_BUNDLE_ID VARCHAR(50) NOT NULL,
    +    VERSION VARCHAR(100) NOT NULL,
    +    DEPENDENCY_GROUP_ID VARCHAR(500),
    --- End diff --
    
    That is a good point, it probably makes sense to separate out a separate table where bundle version can have one-to-many dependencies, and NARs will just so happen to only ever have one. Will work on this update.


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236852286
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.extension;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkableEntity;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotBlank;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import java.util.Objects;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
    +
    +    @NotBlank
    +    private String id;
    +
    +    @NotBlank
    +    private String extensionBundleId;
    +
    +    @NotBlank
    +    private String bucketId;
    +
    +    @NotBlank
    +    private String version;
    +
    +    private ExtensionBundleVersionDependency dependency;
    +
    +    @Min(1)
    +    private long timestamp;
    +
    +    @NotBlank
    +    private String author;
    +
    +    private String description;
    +
    +    @NotBlank
    +    private String sha256Hex;
    +
    +
    +    @ApiModelProperty(value = "The id of this version of the extension bundle")
    +    public String getId() {
    +        return id;
    +    }
    +
    +    public void setId(String id) {
    +        this.id = id;
    +    }
    +
    +    @ApiModelProperty(value = "The id of the extension bundle this version is for")
    +    public String getExtensionBundleId() {
    +        return extensionBundleId;
    +    }
    +
    +    public void setExtensionBundleId(String extensionBundleId) {
    +        this.extensionBundleId = extensionBundleId;
    +    }
    +
    +    @ApiModelProperty(value = "The id of the bucket the extension bundle belongs to", required = true)
    +    public String getBucketId() {
    +        return bucketId;
    +    }
    +
    +    public void setBucketId(String bucketId) {
    +        this.bucketId = bucketId;
    +    }
    +
    +    @ApiModelProperty(value = "The version of the extension bundle")
    +    public String getVersion() {
    +        return version;
    +    }
    +
    +    public void setVersion(String version) {
    +        this.version = version;
    +    }
    +
    +    @ApiModelProperty(value = "The optional bundle dependency (i.e. another bundle this bundle is dependent on)")
    +    public ExtensionBundleVersionDependency getDependency() {
    +        return dependency;
    +    }
    +
    +    public void setDependency(ExtensionBundleVersionDependency dependency) {
    +        this.dependency = dependency;
    +    }
    +
    +    @ApiModelProperty(value = "The timestamp of the create date of this version")
    +    public long getTimestamp() {
    +        return timestamp;
    +    }
    +
    +    public void setTimestamp(long timestamp) {
    +        this.timestamp = timestamp;
    +    }
    +
    +    @ApiModelProperty(value = "The identity that created this version")
    +    public String getAuthor() {
    --- End diff --
    
    Currently the author for extension bundles works the same as the author for versioned flows, it is the value of NiFiUserUtils.getNiFiUserIdentity() during the request that created the entity.
    
    Personally I view this as ok since we have no way of knowing that the client cert CN=alopresto is the same as the kerberos principal alopresto@nifi.com, unless identity mappings are configured that map these to the same value. 


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236865785
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.extension;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkableEntity;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotBlank;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import java.util.Objects;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
    +
    +    @NotBlank
    +    private String id;
    +
    +    @NotBlank
    +    private String extensionBundleId;
    +
    +    @NotBlank
    +    private String bucketId;
    +
    +    @NotBlank
    +    private String version;
    +
    +    private ExtensionBundleVersionDependency dependency;
    +
    +    @Min(1)
    +    private long timestamp;
    +
    +    @NotBlank
    +    private String author;
    +
    +    private String description;
    +
    +    @NotBlank
    +    private String sha256Hex;
    --- End diff --
    
    In this case, all of the metadata comes from extracting information from within the binary (i.e. reading the MANIFEST file inside the NAR, or reading the symbol table of a C++ object), so wouldn't a checksum of only the binary suffice then? 


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236898636
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.extension;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkableEntity;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotBlank;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import java.util.Objects;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
    +
    +    @NotBlank
    +    private String id;
    +
    +    @NotBlank
    +    private String extensionBundleId;
    +
    +    @NotBlank
    +    private String bucketId;
    +
    +    @NotBlank
    +    private String version;
    +
    +    private ExtensionBundleVersionDependency dependency;
    +
    +    @Min(1)
    +    private long timestamp;
    +
    +    @NotBlank
    +    private String author;
    +
    +    private String description;
    +
    +    @NotBlank
    +    private String sha256Hex;
    --- End diff --
    
    Ok. Is there any way to craft an instance of a binary with different metadata that generates the same checksum? Can you point me to the code which performs this inspection and calculation? Where does the checksum reside in transmission (i.e. the binary and checksum are sent separately, or it's just calculated on one end and there is nothing to compare it to?)?


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/nifi-registry/pull/148


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236859763
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.extension;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkableEntity;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotBlank;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import java.util.Objects;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
    +
    +    @NotBlank
    +    private String id;
    +
    +    @NotBlank
    +    private String extensionBundleId;
    +
    +    @NotBlank
    +    private String bucketId;
    +
    +    @NotBlank
    +    private String version;
    +
    +    private ExtensionBundleVersionDependency dependency;
    +
    +    @Min(1)
    +    private long timestamp;
    +
    +    @NotBlank
    +    private String author;
    +
    +    private String description;
    +
    +    @NotBlank
    +    private String sha256Hex;
    --- End diff --
    
    I would like to have a conversation around the cryptographic signatures so if it's possible to add a new field fairly easily, I'm ok waiting for this to be in. I am curious if people think there should be any checksum coverage on the metadata for the extension as well. I like having a calculation over the binary, but I also think there is value in ensuring the metadata values are checked as well. Looking for feedback, but this should be independent from this PR. 


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236860247
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.extension;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkableEntity;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotBlank;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import java.util.Objects;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
    +
    +    @NotBlank
    +    private String id;
    +
    +    @NotBlank
    +    private String extensionBundleId;
    +
    +    @NotBlank
    +    private String bucketId;
    +
    +    @NotBlank
    +    private String version;
    +
    +    private ExtensionBundleVersionDependency dependency;
    +
    +    @Min(1)
    +    private long timestamp;
    +
    +    @NotBlank
    +    private String author;
    +
    +    private String description;
    +
    +    @NotBlank
    +    private String sha256Hex;
    +
    +
    +    @ApiModelProperty(value = "The id of this version of the extension bundle")
    +    public String getId() {
    +        return id;
    +    }
    +
    +    public void setId(String id) {
    +        this.id = id;
    +    }
    +
    +    @ApiModelProperty(value = "The id of the extension bundle this version is for")
    +    public String getExtensionBundleId() {
    +        return extensionBundleId;
    +    }
    +
    +    public void setExtensionBundleId(String extensionBundleId) {
    +        this.extensionBundleId = extensionBundleId;
    +    }
    +
    +    @ApiModelProperty(value = "The id of the bucket the extension bundle belongs to", required = true)
    +    public String getBucketId() {
    +        return bucketId;
    +    }
    +
    +    public void setBucketId(String bucketId) {
    +        this.bucketId = bucketId;
    +    }
    +
    +    @ApiModelProperty(value = "The version of the extension bundle")
    +    public String getVersion() {
    +        return version;
    +    }
    +
    +    public void setVersion(String version) {
    +        this.version = version;
    +    }
    +
    +    @ApiModelProperty(value = "The optional bundle dependency (i.e. another bundle this bundle is dependent on)")
    +    public ExtensionBundleVersionDependency getDependency() {
    +        return dependency;
    +    }
    +
    +    public void setDependency(ExtensionBundleVersionDependency dependency) {
    +        this.dependency = dependency;
    +    }
    +
    +    @ApiModelProperty(value = "The timestamp of the create date of this version")
    +    public long getTimestamp() {
    +        return timestamp;
    +    }
    +
    +    public void setTimestamp(long timestamp) {
    +        this.timestamp = timestamp;
    +    }
    +
    +    @ApiModelProperty(value = "The identity that created this version")
    +    public String getAuthor() {
    +        return author;
    +    }
    +
    +    public void setAuthor(String author) {
    +        this.author = author;
    +    }
    +
    +    @ApiModelProperty(value = "The description for this version")
    +    public String getDescription() {
    +        return description;
    +    }
    +
    +    public void setDescription(String description) {
    +        this.description = description;
    +    }
    +
    +    @ApiModelProperty(value = "The hex representation of the SHA-256 digest of the binary content for this version")
    +    public String getSha256Hex() {
    +        return sha256Hex;
    +    }
    +
    +    public void setSha256Hex(String sha256Hex) {
    +        this.sha256Hex = sha256Hex;
    +    }
    +
    +    @Override
    +    public int compareTo(final ExtensionBundleVersionMetadata o) {
    +        return o == null ? -1 : version.compareTo(o.version);
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        return Objects.hashCode(this.id);
    +    }
    +
    +    @Override
    +    public boolean equals(Object obj) {
    +        if (obj == null) {
    +            return false;
    +        }
    +        if (getClass() != obj.getClass()) {
    +            return false;
    +        }
    +
    +        final ExtensionBundleVersionMetadata other = (ExtensionBundleVersionMetadata) obj;
    +        return Objects.equals(this.id, other.id);
    --- End diff --
    
    Works as expected. 


---

[GitHub] nifi-registry issue #148: NIFIREG-211 Initial work for adding extenion bundl...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on the issue:

    https://github.com/apache/nifi-registry/pull/148
  
    Just pushed a new commit that allows the client to specify a SHA-256 when uploading the bundle. If they specify one and it doesn't match what the server calculates, then it is rejected. If they don't specify one then we accept the bundle, but we mark a flag called "sha256Supplied" as false, so consumers later will know if the bundle was uploaded with or without a client provided checksum.
    
    Since the upload is already a multi-part request, I thought it made the most sense for the SHA-256 to be another field in the multi-part form data. As an example of a curl command to upload with the checksum:
    
    `curl -v -F file=@./nifi-example-processors-nar-1.0.0.nar -Fsha256=93d2c6537142497ca3f908452d3c3c2bcae928fdc07e1b48fd3ba95ec22bc639 "http://localhost:18080/nifi-registry-api/buckets/72f897d9-5645-4168-852f-4885cc7c9cfe/extensions/bundles/nifi-nar"
    `


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by alopresto <gi...@git.apache.org>.
Github user alopresto commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236860010
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.extension;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkableEntity;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotBlank;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import java.util.Objects;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
    +
    +    @NotBlank
    +    private String id;
    +
    +    @NotBlank
    +    private String extensionBundleId;
    +
    +    @NotBlank
    +    private String bucketId;
    +
    +    @NotBlank
    +    private String version;
    +
    +    private ExtensionBundleVersionDependency dependency;
    +
    +    @Min(1)
    +    private long timestamp;
    +
    +    @NotBlank
    +    private String author;
    +
    +    private String description;
    +
    +    @NotBlank
    +    private String sha256Hex;
    +
    +
    +    @ApiModelProperty(value = "The id of this version of the extension bundle")
    +    public String getId() {
    +        return id;
    +    }
    +
    +    public void setId(String id) {
    +        this.id = id;
    +    }
    +
    +    @ApiModelProperty(value = "The id of the extension bundle this version is for")
    +    public String getExtensionBundleId() {
    +        return extensionBundleId;
    +    }
    +
    +    public void setExtensionBundleId(String extensionBundleId) {
    +        this.extensionBundleId = extensionBundleId;
    +    }
    +
    +    @ApiModelProperty(value = "The id of the bucket the extension bundle belongs to", required = true)
    +    public String getBucketId() {
    +        return bucketId;
    +    }
    +
    +    public void setBucketId(String bucketId) {
    +        this.bucketId = bucketId;
    +    }
    +
    +    @ApiModelProperty(value = "The version of the extension bundle")
    +    public String getVersion() {
    +        return version;
    +    }
    +
    +    public void setVersion(String version) {
    +        this.version = version;
    +    }
    +
    +    @ApiModelProperty(value = "The optional bundle dependency (i.e. another bundle this bundle is dependent on)")
    +    public ExtensionBundleVersionDependency getDependency() {
    +        return dependency;
    +    }
    +
    +    public void setDependency(ExtensionBundleVersionDependency dependency) {
    +        this.dependency = dependency;
    +    }
    +
    +    @ApiModelProperty(value = "The timestamp of the create date of this version")
    +    public long getTimestamp() {
    +        return timestamp;
    +    }
    +
    +    public void setTimestamp(long timestamp) {
    +        this.timestamp = timestamp;
    +    }
    +
    +    @ApiModelProperty(value = "The identity that created this version")
    +    public String getAuthor() {
    --- End diff --
    
    Understood. I think that long-term, we probably need to re-evaluate how that is being done, as the identities between NiFi and NiFi Registry may not always match. 


---

[GitHub] nifi-registry issue #148: NIFIREG-211 Initial work for adding extenion bundl...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on the issue:

    https://github.com/apache/nifi-registry/pull/148
  
    Before releasing any of this we would definitely have sections in the user guide and admin guide related to extension bundles, but we can't really write all that until the work is done, and this is only a starting point for the work.
    
    Most of this is just integrating a new type of versioned item into the existing registry framework, so if you aren't already familiar with how registry works, then a starting point would probably be to play around with version controlling flows, and go through all the existing documentation, and then think of extension bundles as just another versioned thing like flows.
    
    You can access the swagger documentation for the REST API from the running application:
    
    http://localhost:18080/nifi-registry-api/swagger/ui.html
    
    You can also get a feel for how the current API works from looking at this integration test:
    
    https://github.com/apache/nifi-registry/blob/7744072b5f059fe5240746e968493eddefa719a6/nifi-registry-core/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/UnsecuredNiFiRegistryClientIT.java#L320-L469


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236849392
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.extension;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkableEntity;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotBlank;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import java.util.Objects;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
    +
    +    @NotBlank
    +    private String id;
    +
    +    @NotBlank
    +    private String extensionBundleId;
    +
    +    @NotBlank
    +    private String bucketId;
    +
    +    @NotBlank
    +    private String version;
    +
    +    private ExtensionBundleVersionDependency dependency;
    +
    +    @Min(1)
    +    private long timestamp;
    +
    +    @NotBlank
    +    private String author;
    +
    +    private String description;
    +
    +    @NotBlank
    +    private String sha256Hex;
    --- End diff --
    
    @kevdoran I'm good with calling it sha256 for consistency
    
    @alopresto It is fairly straight forward to add another field here, just typical changes like DB schema update, new member field, modify mapping code (no ORM here :( ). Right now the input of the SHA-256 is the binary content of the bundle (i.e. the input stream of actual NAR file). 
    
    Did you want me to add in the cryptographic signature in this PR (assuming you point at how to calculate it), or do you want to wait til this is merged and add as follow on work? 
    
    It can be done either way, there is still a bit more work to do before this is ready anyway.


---

[GitHub] nifi-registry pull request #148: NIFIREG-211 Initial work for adding extenio...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on a diff in the pull request:

    https://github.com/apache/nifi-registry/pull/148#discussion_r236853713
  
    --- Diff: nifi-registry-core/nifi-registry-data-model/src/main/java/org/apache/nifi/registry/extension/ExtensionBundleVersionMetadata.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * 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.extension;
    +
    +import io.swagger.annotations.ApiModel;
    +import io.swagger.annotations.ApiModelProperty;
    +import org.apache.nifi.registry.link.LinkableEntity;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotBlank;
    +import javax.xml.bind.annotation.XmlRootElement;
    +import java.util.Objects;
    +
    +@ApiModel
    +@XmlRootElement
    +public class ExtensionBundleVersionMetadata extends LinkableEntity implements Comparable<ExtensionBundleVersionMetadata> {
    +
    +    @NotBlank
    +    private String id;
    +
    +    @NotBlank
    +    private String extensionBundleId;
    +
    +    @NotBlank
    +    private String bucketId;
    +
    +    @NotBlank
    +    private String version;
    +
    +    private ExtensionBundleVersionDependency dependency;
    +
    +    @Min(1)
    +    private long timestamp;
    +
    +    @NotBlank
    +    private String author;
    +
    +    private String description;
    +
    +    @NotBlank
    +    private String sha256Hex;
    +
    +
    +    @ApiModelProperty(value = "The id of this version of the extension bundle")
    +    public String getId() {
    +        return id;
    +    }
    +
    +    public void setId(String id) {
    +        this.id = id;
    +    }
    +
    +    @ApiModelProperty(value = "The id of the extension bundle this version is for")
    +    public String getExtensionBundleId() {
    +        return extensionBundleId;
    +    }
    +
    +    public void setExtensionBundleId(String extensionBundleId) {
    +        this.extensionBundleId = extensionBundleId;
    +    }
    +
    +    @ApiModelProperty(value = "The id of the bucket the extension bundle belongs to", required = true)
    +    public String getBucketId() {
    +        return bucketId;
    +    }
    +
    +    public void setBucketId(String bucketId) {
    +        this.bucketId = bucketId;
    +    }
    +
    +    @ApiModelProperty(value = "The version of the extension bundle")
    +    public String getVersion() {
    +        return version;
    +    }
    +
    +    public void setVersion(String version) {
    +        this.version = version;
    +    }
    +
    +    @ApiModelProperty(value = "The optional bundle dependency (i.e. another bundle this bundle is dependent on)")
    +    public ExtensionBundleVersionDependency getDependency() {
    +        return dependency;
    +    }
    +
    +    public void setDependency(ExtensionBundleVersionDependency dependency) {
    +        this.dependency = dependency;
    +    }
    +
    +    @ApiModelProperty(value = "The timestamp of the create date of this version")
    +    public long getTimestamp() {
    +        return timestamp;
    +    }
    +
    +    public void setTimestamp(long timestamp) {
    +        this.timestamp = timestamp;
    +    }
    +
    +    @ApiModelProperty(value = "The identity that created this version")
    +    public String getAuthor() {
    +        return author;
    +    }
    +
    +    public void setAuthor(String author) {
    +        this.author = author;
    +    }
    +
    +    @ApiModelProperty(value = "The description for this version")
    +    public String getDescription() {
    +        return description;
    +    }
    +
    +    public void setDescription(String description) {
    +        this.description = description;
    +    }
    +
    +    @ApiModelProperty(value = "The hex representation of the SHA-256 digest of the binary content for this version")
    +    public String getSha256Hex() {
    +        return sha256Hex;
    +    }
    +
    +    public void setSha256Hex(String sha256Hex) {
    +        this.sha256Hex = sha256Hex;
    +    }
    +
    +    @Override
    +    public int compareTo(final ExtensionBundleVersionMetadata o) {
    +        return o == null ? -1 : version.compareTo(o.version);
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        return Objects.hashCode(this.id);
    +    }
    +
    +    @Override
    +    public boolean equals(Object obj) {
    +        if (obj == null) {
    +            return false;
    +        }
    +        if (getClass() != obj.getClass()) {
    +            return false;
    +        }
    +
    +        final ExtensionBundleVersionMetadata other = (ExtensionBundleVersionMetadata) obj;
    +        return Objects.equals(this.id, other.id);
    --- End diff --
    
    It is true that from pure java perspective someone could write code and construct two instances of this object with the same id but different data in the other fields and it would return as equal, but this can't really happen in practice... 
    
    We are either constructing a new instance of the bundle metadata from the info in the bundle and assigning it a new UUID, or we are returning a result from the database which already has a unique id and other constraints.


---