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

[GitHub] nifi-registry pull request #112: NIFIREG-162: Support Git backed Persistence...

GitHub user ijokarumawak opened a pull request:

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

    NIFIREG-162: Support Git backed PersistenceProvider

    - Added GitPersistenceProvider which manages Flow snapshot versions
    by Git commits
    - Updated FlowSnapshotContext to provide author
    - Added new serialization data model version 2 which stores Flow
    snapshots as JSON files

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

    $ git pull https://github.com/ijokarumawak/nifi-registry nifireg-162

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

    https://github.com/apache/nifi-registry/pull/112.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 #112
    
----
commit 4e70373c410eda83bf3b2f0ef2b4add7512d7b44
Author: Koji Kawamura <ij...@...>
Date:   2018-03-30T04:17:19Z

    NIFIREG-162: Support Git backed PersistenceProvider
    
    - Added GitPersistenceProvider which manages Flow snapshot versions
    by Git commits
    - Updated FlowSnapshotContext to provide author
    - Added new serialization data model version 2 which stores Flow
    snapshots as JSON files

----


---

[GitHub] nifi-registry issue #112: NIFIREG-162: Support Git backed PersistenceProvide...

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

    https://github.com/apache/nifi-registry/pull/112
  
    Nice work @ijokarumawak and everyone who contributed to this. This is a major feature to have as part of NiFi Registry and will certainly be useful.


---

[GitHub] nifi-registry pull request #112: NIFIREG-162: Support Git backed Persistence...

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/112#discussion_r184458388
  
    --- Diff: nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/flow/git/GitFlowMetaData.java ---
    @@ -0,0 +1,384 @@
    +/*
    + * 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.provider.flow.git;
    +
    +import org.eclipse.jgit.api.Git;
    +import org.eclipse.jgit.api.PushCommand;
    +import org.eclipse.jgit.api.Status;
    +import org.eclipse.jgit.api.errors.GitAPIException;
    +import org.eclipse.jgit.api.errors.NoHeadException;
    +import org.eclipse.jgit.lib.ObjectId;
    +import org.eclipse.jgit.lib.Repository;
    +import org.eclipse.jgit.lib.UserConfig;
    +import org.eclipse.jgit.revwalk.RevCommit;
    +import org.eclipse.jgit.revwalk.RevTree;
    +import org.eclipse.jgit.storage.file.FileRepositoryBuilder;
    +import org.eclipse.jgit.transport.CredentialsProvider;
    +import org.eclipse.jgit.transport.PushResult;
    +import org.eclipse.jgit.transport.RemoteConfig;
    +import org.eclipse.jgit.transport.UsernamePasswordCredentialsProvider;
    +import org.eclipse.jgit.treewalk.TreeWalk;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStreamWriter;
    +import java.io.Writer;
    +import java.nio.charset.StandardCharsets;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import static java.lang.String.format;
    +import static org.apache.commons.lang3.StringUtils.isEmpty;
    +
    +class GitFlowMetaData {
    +
    +    static final int CURRENT_LAYOUT_VERSION = 1;
    +
    +    static final String LAYOUT_VERSION = "layoutVer";
    +    static final String BUCKET_ID = "bucketId";
    +    static final String FLOWS = "flows";
    +    static final String VER = "ver";
    +    static final String FILE = "file";
    +    static final String BUCKET_FILENAME = "bucket.yml";
    +
    +    private static final Logger logger = LoggerFactory.getLogger(GitFlowMetaData.class);
    +
    +    private Repository gitRepo;
    +    private String remoteToPush;
    +    private CredentialsProvider credentialsProvider;
    +
    +    /**
    +     * Bucket ID to Bucket.
    +     */
    +    private Map<String, Bucket> buckets = new HashMap<>();
    +
    +    public void setRemoteToPush(String remoteToPush) {
    +        this.remoteToPush = remoteToPush;
    +    }
    +
    +    public void setRemoteCredential(String userName, String password) {
    +        this.credentialsProvider = new UsernamePasswordCredentialsProvider(userName, password);
    +    }
    +
    +    /**
    +     * Open a Git repository using the specified directory.
    +     * @param gitProjectRootDir a root directory of a Git project
    +     * @return created Repository
    +     * @throws IOException thrown when the specified directory does not exist,
    +     * does not have read/write privilege or not containing .git directory
    +     */
    +    private Repository openRepository(final File gitProjectRootDir) throws IOException {
    +
    +        // Instead of using FileUtils.ensureDirectoryExistAndCanReadAndWrite, check availability manually here.
    +        // Because the util will try to create a dir if not exist.
    +        // The git dir should be initialized and configured by users.
    +        if (!gitProjectRootDir.isDirectory()) {
    +            throw new IOException(format("'%s' is not a directory or does not exist.", gitProjectRootDir));
    +        }
    +
    +        if (!(gitProjectRootDir.canRead() && gitProjectRootDir.canWrite())) {
    +            throw new IOException(format("Directory '%s' does not have read/write privilege.", gitProjectRootDir));
    +        }
    +
    +        // Search .git dir but avoid searching parent directories.
    +        final FileRepositoryBuilder builder = new FileRepositoryBuilder()
    +                .readEnvironment()
    +                .setMustExist(true)
    +                .addCeilingDirectory(gitProjectRootDir)
    +                .findGitDir(gitProjectRootDir);
    +
    +        if (builder.getGitDir() == null) {
    +            throw new IOException(format("Directory '%s' does not contain a .git directory." +
    +                    " Please init and configure the directory with 'git init' command before using it from NiFi Registry.",
    +                    gitProjectRootDir));
    +        }
    +
    +        return builder.build();
    +    }
    +
    +    @SuppressWarnings("unchecked")
    +    public void loadGitRepository(File gitProjectRootDir) throws IOException, GitAPIException {
    +        gitRepo = openRepository(gitProjectRootDir);
    +
    +        try (final Git git = new Git(gitRepo)) {
    +
    +            // Check if remote exists.
    +            if (!isEmpty(remoteToPush)) {
    +                final List<RemoteConfig> remotes = git.remoteList().call();
    +                final boolean isRemoteExist = remotes.stream().anyMatch(remote -> remote.getName().equals(remoteToPush));
    +                if (!isRemoteExist) {
    +                    final List<String> remoteNames = remotes.stream().map(RemoteConfig::getName).collect(Collectors.toList());
    +                    throw new IllegalArgumentException(
    +                            format("The configured remote '%s' to push does not exist. Available remotes are %s", remoteToPush, remoteNames));
    +                }
    +            }
    +
    +            boolean isLatestCommit = true;
    +            try {
    +                for (RevCommit commit : git.log().call()) {
    +                    final String shortCommitId = commit.getId().abbreviate(7).name();
    +                    logger.debug("Processing a commit: {}", shortCommitId);
    +                    final RevTree tree = commit.getTree();
    +
    +                    try (final TreeWalk treeWalk = new TreeWalk(gitRepo)) {
    +                        treeWalk.addTree(tree);
    +
    +                        // Path -> ObjectId
    +                        final Map<String, ObjectId> bucketObjectIds = new HashMap<>();
    +                        final Map<String, ObjectId> flowSnapshotObjectIds = new HashMap<>();
    +                        while (treeWalk.next()) {
    +                            if (treeWalk.isSubtree()) {
    +                                treeWalk.enterSubtree();
    +                            } else {
    +                                final String pathString = treeWalk.getPathString();
    +                                // TODO: what is this nth?? When does it get grater than 0? Tree count seems to be always 1..
    +                                if (pathString.endsWith("/" + BUCKET_FILENAME)) {
    +                                    bucketObjectIds.put(pathString, treeWalk.getObjectId(0));
    +                                } else if (pathString.endsWith(GitFlowPersistenceProvider.SNAPSHOT_EXTENSION)) {
    +                                    flowSnapshotObjectIds.put(pathString, treeWalk.getObjectId(0));
    +                                }
    +                            }
    +                        }
    +
    +                        if (bucketObjectIds.isEmpty()) {
    +                            // No bucket.yml means at this point, all flows are deleted. No need to scan older commits because those are already deleted.
    +                            logger.debug("Tree at commit {} does not contain any " + BUCKET_FILENAME + ". Stop loading commits here.", shortCommitId);
    +                            return;
    +                        }
    +
    +                        loadBuckets(gitRepo, commit, isLatestCommit, bucketObjectIds, flowSnapshotObjectIds);
    +                        isLatestCommit = false;
    +                    }
    +                }
    +            } catch (NoHeadException e) {
    +                logger.debug("'{}' does not have any commit yet. Starting with empty buckets.", gitProjectRootDir);
    +            }
    +        }
    +    }
    +
    +    @SuppressWarnings("unchecked")
    +    private void loadBuckets(Repository gitRepo, RevCommit commit, boolean isLatestCommit, Map<String, ObjectId> bucketObjectIds, Map<String, ObjectId> flowSnapshotObjectIds) throws IOException {
    +        final Yaml yaml = new Yaml();
    +        for (String bucketFilePath : bucketObjectIds.keySet()) {
    +            final ObjectId bucketObjectId = bucketObjectIds.get(bucketFilePath);
    +            final Map<String, Object> bucketMeta;
    +            try (InputStream bucketIn = gitRepo.newObjectReader().open(bucketObjectId).openStream()) {
    +                bucketMeta = yaml.load(bucketIn);
    +            }
    +
    +            if (!validateRequiredValue(bucketMeta, bucketFilePath, LAYOUT_VERSION, BUCKET_ID, FLOWS)) {
    +                continue;
    +            }
    +
    +            int layoutVersion = (int) bucketMeta.get(LAYOUT_VERSION);
    +            if (layoutVersion > CURRENT_LAYOUT_VERSION) {
    +                logger.warn("{} has unsupported {} {}. This Registry can only support {} or lower. Skipping it.",
    +                        bucketFilePath, LAYOUT_VERSION, layoutVersion, CURRENT_LAYOUT_VERSION);
    +                continue;
    +            }
    +
    +            final String bucketId = (String) bucketMeta.get(BUCKET_ID);
    +
    +            final Bucket bucket;
    +            if (isLatestCommit) {
    +                // If this is the latest commit, then create one.
    +                bucket = getBucketOrCreate(bucketId);
    +            } else {
    +                // Otherwise non-existing bucket means it's already deleted.
    +                final Optional<Bucket> bucketOpt = getBucket(bucketId);
    +                if (bucketOpt.isPresent()) {
    +                    bucket = bucketOpt.get();
    +                } else {
    +                    logger.debug("Bucket {} does not exist any longer. It may have been deleted.", bucketId);
    +                    continue;
    +                }
    +            }
    +
    +            // E.g. DirA/DirB/DirC/bucket.yml -> DirC will be the bucket name.
    +            final String[] pathNames = bucketFilePath.split("/");
    --- End diff --
    
    This made me wonder... should we do some sanitizing/converting of bucket and flow names before using them to create directories and files?
    
    The Registry UI lets you enter anything so I created a bucket with the name "This / is / a test" which created three directories for the repo since it split on forward slashes. 
    
    Technically it works fine and did not produce any errors when saving/retrieving flows, but we could possibly convert slashes and spaces to underscores?
    
    Something like:
    `This_is_a_test/This_is_my_flow.snapshot`


---

[GitHub] nifi-registry pull request #112: NIFIREG-162: Support Git backed Persistence...

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

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


---

[GitHub] nifi-registry pull request #112: NIFIREG-162: Support Git backed Persistence...

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/112#discussion_r184694579
  
    --- Diff: nifi-registry-docs/src/main/asciidoc/administration-guide.adoc ---
    @@ -895,3 +895,167 @@ Providing 2 total locations, including `nifi.registry.extension.dir.1`.
       Example: `/etc/http-nifi-registry.keytab`
     |nifi.registry.kerberos.spengo.authentication.expiration|The expiration duration of a successful Kerberos user authentication, if used. The default value is `12 hours`.
     |====
    +
    +== Persistence Providers
    +
    +NiFi Registry uses a pluggable flow persistence provider to store the content of the flows saved to the registry. NiFi Registry provides `<<FileSystemFlowPersistenceProvider>>` and `<<GitFlowPersistenceProvider>>`.
    +
    +Each persistence provider has its own configuration parameters, those can be configured in a XML file specified in <<Providers Properties,nifi-registry.properties>>.
    +
    +The XML configuration file looks like below. It has a `flowPersistenceProvider` element in which qualified class name of a persistence provider implementation and its configuration properties are defined. See following sections for available configurations for each providers.
    +
    +.Example providers.xml
    +[source,xml]
    +....
    +<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
    +<providers>
    +
    +    <flowPersistenceProvider>
    +        <class>persistence-provider-qualified-class-name</class>
    +        <property name="property-1">property-value-1</property>
    +        <property name="property-2">property-value-2</property>
    +        <property name="property-n">property-value-n</property>
    +    </flowPersistenceProvider>
    +
    +</providers>
    +....
    +
    +
    +=== FileSystemFlowPersistenceProvider
    +
    +FileSystemFlowPersistenceProvider simply stores serialized Flow contents into `{bucket-id}/{flow-id}/{version}` directories.
    +
    +Example of persisted files:
    +....
    +Flow Storage Directory/
    +├── {bucket-id}/
    +│   └── {flow-id}/
    +│       ├── {version}/{version}.snapshot
    +└── d1beba88-32e9-45d1-bfe9-057cc41f7ce8/
    +    └── 219cf539-427f-43be-9294-0644fb07ca63/
    +        ├── 1/1.snapshot
    +        └── 2/2.snapshot
    +....
    +
    +Qualified class name: `org.apache.nifi.registry.provider.flow.FileSystemFlowPersistenceProvider`
    +
    +|====
    +|*Property*|*Description*
    +|Flow Storage Directory|REQUIRED: File system path for a directory where flow contents files are persisted to. If the directory does not exist when NiFi Registry starts, it will be created. If the directory exists, it must be readable and writable from NiFi Registry.
    +|====
    +
    +
    +=== GitFlowPersistenceProvider
    +
    +GitFlowPersistenceProvider stores flow contents under a Git directory.
    +
    +In contrast to FileSystemFlowPersistenceProvider, this provider uses human friendly Bucket and Flow names so that those files can be accessed by external tools. However, it is NOT supported to modify stored files outside of NiFi Registry. Persisted files are only read when NiFi Registry starts up.
    +
    +Buckets are represented as directories and Flow contents are stored as files in a Bucket directory they belong to. Flow snapshot histories are managed as Git commits, meaning only the latest version of Buckets and Flows exist in the Git directory. Old versions are retrieved from Git commit histories.
    +
    +.Example persisted files
    +....
    +Flow Storage Directory/
    +├── .git/
    +├── Bucket A/
    +│   ├── bucket.yml
    +│   ├── Flow 1.snapshot
    +│   └── Flow 2.snapshot
    +└── Bucket B/
    +    ├── bucket.yml
    +    └── Flow 4.snapshot
    +....
    +
    +Each Bucket directory contains a YAML file named `bucket.yml`. The file manages links from NiFi Registry Bucket and Flow IDs to actual directory and file names. When NiFi Registry starts, this provider reads through Git commit histories and lookup these `bucket.yml` files to restore Buckets and Flows for each snapshot version.
    +
    +.Example bucket.yml
    +[source,yml]
    +....
    +layoutVer: 1
    +bucketId: d1beba88-32e9-45d1-bfe9-057cc41f7ce8
    +flows:
    +  219cf539-427f-43be-9294-0644fb07ca63: {ver: 7, file: Flow 1.snapshot}
    +  22cccb6c-3011-4493-a996-611f8f112969: {ver: 3, file: Flow 2.snapshot}
    +....
    +
    +Qualified class name: `org.apache.nifi.registry.provider.flow.git.GitFlowPersistenceProvider`
    +
    +|====
    +|*Property*|*Description*
    +|Flow Storage Directory|REQUIRED: File system path for a directory where flow contents files are persisted to. The directory must exist when NiFi registry starts. Also must be initialized as a Git directory. See <<Initialize Git directory>> for detail.
    +|Remote To Push|When a new flow snapshot is created, this persistence provider updated files in the specified Git directory, then create a commit to the local repository. If `Remote To Push` is defined, it also pushes to the specified remote repository. E.g. 'origin'. To define more detailed remote spec such as branch names, use `Refspec`. See https://git-scm.com/book/en/v2/Git-Internals-The-Refspec
    +|Remote Access User|This user name is used to make push requests to the remote repository when `Remote To Push` is enabled, and the remote repository is accessed by HTTP protocol. If SSH is used, user authentication is done with SSH keys.
    +|Remote Access Password|Used with `Remote Access User`.
    +|====
    +
    +==== Initialize Git directory
    +
    +In order to use GitFlowPersistenceRepository, you need to prepare a Git directory on the local file system. You can do so by initializing a directory with `git init` command, or clone an existing Git project from a remote Git repository by `git clone` command.
    +
    +- Git init command
    +https://git-scm.com/docs/git-init
    +- Git clone command
    +https://git-scm.com/docs/git-clone
    +
    +
    +==== Git user configuration
    +
    +Git distinguishes a user by its username and email address. This persistence provider uses NiFi Registry username when it creates Git commits. However since NiFi Registry users do not provide email address, preconfigured Git user email address is used.
    +
    +You can configure Git user name and email address by `git config` command.
    +
    +- Git config command
    +https://git-scm.com/docs/git-config
    +
    +
    +==== Git user authentication
    +
    +By default, this persistence repository only create commits to local repository. No user authentication is needed to do so. However, if 'Commit To Push' is enabled, user authentication to the remote Git repository is required.
    +
    +If the remote repository is accessed by HTTP, then username and password for authentication can be configured in the providers XML configuration file.
    +
    +When SSH is used, SSH keys are used to identify a Git user. In order to pick the right key to a remote server, the SSH configuration file `${USER_HOME}/.ssh/config` is used. The SSH configuration file can contain multiple `Host` entries to specify a key file to login to a remote Git server. The `Host` must much with the target remote Git server hostname.
    +
    +.example SSH config file
    +....
    +Host git.example.com
    +  HostName git.example.com
    +  IdentityFile ~/.ssh/id_rsa
    +
    +Host github.com
    +  HostName github.com
    +  IdentityFile ~/.ssh/key-for-github
    +
    +Host bitbucket.org
    +  HostName bitbucket.org
    +  IdentityFile ~/.ssh/key-for-bitbucket
    +....
    +
    +=== Data model version of serialized Flow snapshots
    +
    +Serialized Flow snapshots saved by these persistence providers have versions, so that the data format and schema can evolve over time. Data model version update is done automatically by NiFi Registry when it reads and stores each Flow content.
    +
    +Here is the data model version histories:
    +
    +|====
    +|*Data model version*|*Since NiFi Registry*|*Description*
    +|2|0.2|JSON formatted text file. The root object contains header and Flow content object.
    +|1|0.1|Binary format having header bytes at the beginning followed by Flow content represented as XML.
    +|====
    +
    +=== Migrating stored files between different Persistence Provider
    --- End diff --
    
    I think instead of providing a tool we can just offer instructions for how to reset your registry to use the git provider, something like:
    ```
    - Stop version control on all PGs in NiFi
    - Stop registry
    - Move the H2 DB and file-based flow dir somewhere for back up
    - Configure git provider in providers.xml
    - Start registry
    - Recreate any buckets
    - Start version control on all PGs again
    ```
    This way the CLI doesn't need to depend on registry framework code since it is more of a client.
    
    What do you think?


---

[GitHub] nifi-registry issue #112: NIFIREG-162: Support Git backed PersistenceProvide...

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

    https://github.com/apache/nifi-registry/pull/112
  
    Thanks @ijokarumawak ! Will review this soon.


---

[GitHub] nifi-registry issue #112: NIFIREG-162: Support Git backed PersistenceProvide...

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

    https://github.com/apache/nifi-registry/pull/112
  
    Been testing this and has been looking good so far...
    
    I was doing a test where I configured to push to a remote, but I didn't supply a username/password because I honestly wasn't sure if the push would leverage my cached credentials at the OS level. 
    
    So the push failed with an exception like:
    ```
    
    Caused by: org.eclipse.jgit.errors.TransportException: https://github.com/bbende/nifi-versioned-flows.git: Authentication is required but no CredentialsProvider has been registered
    	at org.eclipse.jgit.transport.TransportHttp.connect(TransportHttp.java:522) ~[org.eclipse.jgit-4.11.0.201803080745-r.jar:4.11.0.201803080745-r]
    	at org.eclipse.jgit.transport.TransportHttp.openPush(TransportHttp.java:435) ~[org.eclipse.jgit-4.11.0.201803080745-r.jar:4.11.0.201803080745-r]
    	at org.eclipse.jgit.transport.PushProcess.execute(PushProcess.java:160) ~[org.eclipse.jgit-4.11.0.201803080745-r.jar:4.11.0.201803080745-r]
    	at org.eclipse.jgit.transport.Transport.push(Transport.java:1344) ~[org.eclipse.jgit-4.11.0.201803080745-r.jar:4.11.0.201803080745-r]
    	at org.eclipse.jgit.api.PushCommand.call(PushCommand.java:169) ~[org.eclipse.jgit-4.11.0.201803080745-r.jar:4.11.0.201803080745-r]
    	... 109 common frames omitted
    ```
    This makes sense, but the result was that the flow was committed to the local repo, but because of the error when pushing, the error was thrown out of the REST layer and the response to NiFi indicated that starting version control failed. So it was left in a weird state where NiFi now thinks the process group is not under version control, but the local repo does have the first version saved.
    
    Do you think we should catch any exceptions around the push inside the persistence provider and log them, but maybe not throw an error?
    
    That would leave things in a more consistent state, but I'm kind of torn because then it may not be obvious to users that the pushes are failing unless they look in the registry logs.


---

[GitHub] nifi-registry issue #112: NIFIREG-162: Support Git backed PersistenceProvide...

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

    https://github.com/apache/nifi-registry/pull/112
  
    @ijokarumawak I agree about the email and author approach, after reading what you wrote in the admin guide, it makes sense.
    
    Thanks for pushing those changes. Will try them out shortly.


---

[GitHub] nifi-registry pull request #112: NIFIREG-162: Support Git backed Persistence...

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/112#discussion_r184700563
  
    --- Diff: nifi-registry-docs/src/main/asciidoc/administration-guide.adoc ---
    @@ -895,3 +895,167 @@ Providing 2 total locations, including `nifi.registry.extension.dir.1`.
       Example: `/etc/http-nifi-registry.keytab`
     |nifi.registry.kerberos.spengo.authentication.expiration|The expiration duration of a successful Kerberos user authentication, if used. The default value is `12 hours`.
     |====
    +
    +== Persistence Providers
    +
    +NiFi Registry uses a pluggable flow persistence provider to store the content of the flows saved to the registry. NiFi Registry provides `<<FileSystemFlowPersistenceProvider>>` and `<<GitFlowPersistenceProvider>>`.
    +
    +Each persistence provider has its own configuration parameters, those can be configured in a XML file specified in <<Providers Properties,nifi-registry.properties>>.
    +
    +The XML configuration file looks like below. It has a `flowPersistenceProvider` element in which qualified class name of a persistence provider implementation and its configuration properties are defined. See following sections for available configurations for each providers.
    +
    +.Example providers.xml
    +[source,xml]
    +....
    +<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
    +<providers>
    +
    +    <flowPersistenceProvider>
    +        <class>persistence-provider-qualified-class-name</class>
    +        <property name="property-1">property-value-1</property>
    +        <property name="property-2">property-value-2</property>
    +        <property name="property-n">property-value-n</property>
    +    </flowPersistenceProvider>
    +
    +</providers>
    +....
    +
    +
    +=== FileSystemFlowPersistenceProvider
    +
    +FileSystemFlowPersistenceProvider simply stores serialized Flow contents into `{bucket-id}/{flow-id}/{version}` directories.
    +
    +Example of persisted files:
    +....
    +Flow Storage Directory/
    +├── {bucket-id}/
    +│   └── {flow-id}/
    +│       ├── {version}/{version}.snapshot
    +└── d1beba88-32e9-45d1-bfe9-057cc41f7ce8/
    +    └── 219cf539-427f-43be-9294-0644fb07ca63/
    +        ├── 1/1.snapshot
    +        └── 2/2.snapshot
    +....
    +
    +Qualified class name: `org.apache.nifi.registry.provider.flow.FileSystemFlowPersistenceProvider`
    +
    +|====
    +|*Property*|*Description*
    +|Flow Storage Directory|REQUIRED: File system path for a directory where flow contents files are persisted to. If the directory does not exist when NiFi Registry starts, it will be created. If the directory exists, it must be readable and writable from NiFi Registry.
    +|====
    +
    +
    +=== GitFlowPersistenceProvider
    +
    +GitFlowPersistenceProvider stores flow contents under a Git directory.
    +
    +In contrast to FileSystemFlowPersistenceProvider, this provider uses human friendly Bucket and Flow names so that those files can be accessed by external tools. However, it is NOT supported to modify stored files outside of NiFi Registry. Persisted files are only read when NiFi Registry starts up.
    +
    +Buckets are represented as directories and Flow contents are stored as files in a Bucket directory they belong to. Flow snapshot histories are managed as Git commits, meaning only the latest version of Buckets and Flows exist in the Git directory. Old versions are retrieved from Git commit histories.
    +
    +.Example persisted files
    +....
    +Flow Storage Directory/
    +├── .git/
    +├── Bucket A/
    +│   ├── bucket.yml
    +│   ├── Flow 1.snapshot
    +│   └── Flow 2.snapshot
    +└── Bucket B/
    +    ├── bucket.yml
    +    └── Flow 4.snapshot
    +....
    +
    +Each Bucket directory contains a YAML file named `bucket.yml`. The file manages links from NiFi Registry Bucket and Flow IDs to actual directory and file names. When NiFi Registry starts, this provider reads through Git commit histories and lookup these `bucket.yml` files to restore Buckets and Flows for each snapshot version.
    +
    +.Example bucket.yml
    +[source,yml]
    +....
    +layoutVer: 1
    +bucketId: d1beba88-32e9-45d1-bfe9-057cc41f7ce8
    +flows:
    +  219cf539-427f-43be-9294-0644fb07ca63: {ver: 7, file: Flow 1.snapshot}
    +  22cccb6c-3011-4493-a996-611f8f112969: {ver: 3, file: Flow 2.snapshot}
    +....
    +
    +Qualified class name: `org.apache.nifi.registry.provider.flow.git.GitFlowPersistenceProvider`
    +
    +|====
    +|*Property*|*Description*
    +|Flow Storage Directory|REQUIRED: File system path for a directory where flow contents files are persisted to. The directory must exist when NiFi registry starts. Also must be initialized as a Git directory. See <<Initialize Git directory>> for detail.
    +|Remote To Push|When a new flow snapshot is created, this persistence provider updated files in the specified Git directory, then create a commit to the local repository. If `Remote To Push` is defined, it also pushes to the specified remote repository. E.g. 'origin'. To define more detailed remote spec such as branch names, use `Refspec`. See https://git-scm.com/book/en/v2/Git-Internals-The-Refspec
    +|Remote Access User|This user name is used to make push requests to the remote repository when `Remote To Push` is enabled, and the remote repository is accessed by HTTP protocol. If SSH is used, user authentication is done with SSH keys.
    +|Remote Access Password|Used with `Remote Access User`.
    +|====
    +
    +==== Initialize Git directory
    +
    +In order to use GitFlowPersistenceRepository, you need to prepare a Git directory on the local file system. You can do so by initializing a directory with `git init` command, or clone an existing Git project from a remote Git repository by `git clone` command.
    +
    +- Git init command
    +https://git-scm.com/docs/git-init
    +- Git clone command
    +https://git-scm.com/docs/git-clone
    +
    +
    +==== Git user configuration
    +
    +Git distinguishes a user by its username and email address. This persistence provider uses NiFi Registry username when it creates Git commits. However since NiFi Registry users do not provide email address, preconfigured Git user email address is used.
    +
    +You can configure Git user name and email address by `git config` command.
    +
    +- Git config command
    +https://git-scm.com/docs/git-config
    +
    +
    +==== Git user authentication
    +
    +By default, this persistence repository only create commits to local repository. No user authentication is needed to do so. However, if 'Commit To Push' is enabled, user authentication to the remote Git repository is required.
    +
    +If the remote repository is accessed by HTTP, then username and password for authentication can be configured in the providers XML configuration file.
    +
    +When SSH is used, SSH keys are used to identify a Git user. In order to pick the right key to a remote server, the SSH configuration file `${USER_HOME}/.ssh/config` is used. The SSH configuration file can contain multiple `Host` entries to specify a key file to login to a remote Git server. The `Host` must much with the target remote Git server hostname.
    +
    +.example SSH config file
    +....
    +Host git.example.com
    +  HostName git.example.com
    +  IdentityFile ~/.ssh/id_rsa
    +
    +Host github.com
    +  HostName github.com
    +  IdentityFile ~/.ssh/key-for-github
    +
    +Host bitbucket.org
    +  HostName bitbucket.org
    +  IdentityFile ~/.ssh/key-for-bitbucket
    +....
    +
    +=== Data model version of serialized Flow snapshots
    +
    +Serialized Flow snapshots saved by these persistence providers have versions, so that the data format and schema can evolve over time. Data model version update is done automatically by NiFi Registry when it reads and stores each Flow content.
    +
    +Here is the data model version histories:
    +
    +|====
    +|*Data model version*|*Since NiFi Registry*|*Description*
    +|2|0.2|JSON formatted text file. The root object contains header and Flow content object.
    +|1|0.1|Binary format having header bytes at the beginning followed by Flow content represented as XML.
    +|====
    +
    +=== Migrating stored files between different Persistence Provider
    --- End diff --
    
    I agree it would be best to avoid a dependency on nifi-registry-framework for CLI if possible. @bbende, this approach you documented works, but loses the versioned PG history. Not sure how important that is, just mentioning it. 
    
    I think the right analogy for this type of operation is a relational database migration. Most databases support this by giving you some way to export schema and data to a file in a portable SQL syntax that can then be imported into any compatible instance. This feature is useful for snapshotting, creating data backups, or migrating from one DB server to another. If we could support a similar feature in NiFi Registry, as export and import REST API endpoints, and corresponding CLI commands that would call the endpoints and write/read files, that would be a very powerful and flexible capability, both for backing up a NiFi Registry and also changing persistence providers.
    
    This is probably an entirely new feature, so perhaps we could go with @bbende's documentation approach as a stop gap and open a JIRA for a more full-features export/import capability with no data loss.


---

[GitHub] nifi-registry issue #112: NIFIREG-162: Support Git backed PersistenceProvide...

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

    https://github.com/apache/nifi-registry/pull/112
  
    @bbende I've addressed the final two items, updating migration guide and added white space to invalid filename characters. Also, updated how NiFi Registry username is used in Git commits. Please check the additional commits. Thanks!


---

[GitHub] nifi-registry issue #112: NIFIREG-162: Support Git backed PersistenceProvide...

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

    https://github.com/apache/nifi-registry/pull/112
  
    @ijokarumawak thanks for the updates, everything looks good, going to merge... thanks for all the work on this!
    



---

[GitHub] nifi-registry pull request #112: NIFIREG-162: Support Git backed Persistence...

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

    https://github.com/apache/nifi-registry/pull/112#discussion_r184609148
  
    --- Diff: nifi-registry-framework/src/main/java/org/apache/nifi/registry/provider/flow/git/GitFlowMetaData.java ---
    @@ -0,0 +1,384 @@
    +/*
    + * 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.provider.flow.git;
    +
    +import org.eclipse.jgit.api.Git;
    +import org.eclipse.jgit.api.PushCommand;
    +import org.eclipse.jgit.api.Status;
    +import org.eclipse.jgit.api.errors.GitAPIException;
    +import org.eclipse.jgit.api.errors.NoHeadException;
    +import org.eclipse.jgit.lib.ObjectId;
    +import org.eclipse.jgit.lib.Repository;
    +import org.eclipse.jgit.lib.UserConfig;
    +import org.eclipse.jgit.revwalk.RevCommit;
    +import org.eclipse.jgit.revwalk.RevTree;
    +import org.eclipse.jgit.storage.file.FileRepositoryBuilder;
    +import org.eclipse.jgit.transport.CredentialsProvider;
    +import org.eclipse.jgit.transport.PushResult;
    +import org.eclipse.jgit.transport.RemoteConfig;
    +import org.eclipse.jgit.transport.UsernamePasswordCredentialsProvider;
    +import org.eclipse.jgit.treewalk.TreeWalk;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.yaml.snakeyaml.Yaml;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStreamWriter;
    +import java.io.Writer;
    +import java.nio.charset.StandardCharsets;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.stream.Collectors;
    +
    +import static java.lang.String.format;
    +import static org.apache.commons.lang3.StringUtils.isEmpty;
    +
    +class GitFlowMetaData {
    +
    +    static final int CURRENT_LAYOUT_VERSION = 1;
    +
    +    static final String LAYOUT_VERSION = "layoutVer";
    +    static final String BUCKET_ID = "bucketId";
    +    static final String FLOWS = "flows";
    +    static final String VER = "ver";
    +    static final String FILE = "file";
    +    static final String BUCKET_FILENAME = "bucket.yml";
    +
    +    private static final Logger logger = LoggerFactory.getLogger(GitFlowMetaData.class);
    +
    +    private Repository gitRepo;
    +    private String remoteToPush;
    +    private CredentialsProvider credentialsProvider;
    +
    +    /**
    +     * Bucket ID to Bucket.
    +     */
    +    private Map<String, Bucket> buckets = new HashMap<>();
    +
    +    public void setRemoteToPush(String remoteToPush) {
    +        this.remoteToPush = remoteToPush;
    +    }
    +
    +    public void setRemoteCredential(String userName, String password) {
    +        this.credentialsProvider = new UsernamePasswordCredentialsProvider(userName, password);
    +    }
    +
    +    /**
    +     * Open a Git repository using the specified directory.
    +     * @param gitProjectRootDir a root directory of a Git project
    +     * @return created Repository
    +     * @throws IOException thrown when the specified directory does not exist,
    +     * does not have read/write privilege or not containing .git directory
    +     */
    +    private Repository openRepository(final File gitProjectRootDir) throws IOException {
    +
    +        // Instead of using FileUtils.ensureDirectoryExistAndCanReadAndWrite, check availability manually here.
    +        // Because the util will try to create a dir if not exist.
    +        // The git dir should be initialized and configured by users.
    +        if (!gitProjectRootDir.isDirectory()) {
    +            throw new IOException(format("'%s' is not a directory or does not exist.", gitProjectRootDir));
    +        }
    +
    +        if (!(gitProjectRootDir.canRead() && gitProjectRootDir.canWrite())) {
    +            throw new IOException(format("Directory '%s' does not have read/write privilege.", gitProjectRootDir));
    +        }
    +
    +        // Search .git dir but avoid searching parent directories.
    +        final FileRepositoryBuilder builder = new FileRepositoryBuilder()
    +                .readEnvironment()
    +                .setMustExist(true)
    +                .addCeilingDirectory(gitProjectRootDir)
    +                .findGitDir(gitProjectRootDir);
    +
    +        if (builder.getGitDir() == null) {
    +            throw new IOException(format("Directory '%s' does not contain a .git directory." +
    +                    " Please init and configure the directory with 'git init' command before using it from NiFi Registry.",
    +                    gitProjectRootDir));
    +        }
    +
    +        return builder.build();
    +    }
    +
    +    @SuppressWarnings("unchecked")
    +    public void loadGitRepository(File gitProjectRootDir) throws IOException, GitAPIException {
    +        gitRepo = openRepository(gitProjectRootDir);
    +
    +        try (final Git git = new Git(gitRepo)) {
    +
    +            // Check if remote exists.
    +            if (!isEmpty(remoteToPush)) {
    +                final List<RemoteConfig> remotes = git.remoteList().call();
    +                final boolean isRemoteExist = remotes.stream().anyMatch(remote -> remote.getName().equals(remoteToPush));
    +                if (!isRemoteExist) {
    +                    final List<String> remoteNames = remotes.stream().map(RemoteConfig::getName).collect(Collectors.toList());
    +                    throw new IllegalArgumentException(
    +                            format("The configured remote '%s' to push does not exist. Available remotes are %s", remoteToPush, remoteNames));
    +                }
    +            }
    +
    +            boolean isLatestCommit = true;
    +            try {
    +                for (RevCommit commit : git.log().call()) {
    +                    final String shortCommitId = commit.getId().abbreviate(7).name();
    +                    logger.debug("Processing a commit: {}", shortCommitId);
    +                    final RevTree tree = commit.getTree();
    +
    +                    try (final TreeWalk treeWalk = new TreeWalk(gitRepo)) {
    +                        treeWalk.addTree(tree);
    +
    +                        // Path -> ObjectId
    +                        final Map<String, ObjectId> bucketObjectIds = new HashMap<>();
    +                        final Map<String, ObjectId> flowSnapshotObjectIds = new HashMap<>();
    +                        while (treeWalk.next()) {
    +                            if (treeWalk.isSubtree()) {
    +                                treeWalk.enterSubtree();
    +                            } else {
    +                                final String pathString = treeWalk.getPathString();
    +                                // TODO: what is this nth?? When does it get grater than 0? Tree count seems to be always 1..
    +                                if (pathString.endsWith("/" + BUCKET_FILENAME)) {
    +                                    bucketObjectIds.put(pathString, treeWalk.getObjectId(0));
    +                                } else if (pathString.endsWith(GitFlowPersistenceProvider.SNAPSHOT_EXTENSION)) {
    +                                    flowSnapshotObjectIds.put(pathString, treeWalk.getObjectId(0));
    +                                }
    +                            }
    +                        }
    +
    +                        if (bucketObjectIds.isEmpty()) {
    +                            // No bucket.yml means at this point, all flows are deleted. No need to scan older commits because those are already deleted.
    +                            logger.debug("Tree at commit {} does not contain any " + BUCKET_FILENAME + ". Stop loading commits here.", shortCommitId);
    +                            return;
    +                        }
    +
    +                        loadBuckets(gitRepo, commit, isLatestCommit, bucketObjectIds, flowSnapshotObjectIds);
    +                        isLatestCommit = false;
    +                    }
    +                }
    +            } catch (NoHeadException e) {
    +                logger.debug("'{}' does not have any commit yet. Starting with empty buckets.", gitProjectRootDir);
    +            }
    +        }
    +    }
    +
    +    @SuppressWarnings("unchecked")
    +    private void loadBuckets(Repository gitRepo, RevCommit commit, boolean isLatestCommit, Map<String, ObjectId> bucketObjectIds, Map<String, ObjectId> flowSnapshotObjectIds) throws IOException {
    +        final Yaml yaml = new Yaml();
    +        for (String bucketFilePath : bucketObjectIds.keySet()) {
    +            final ObjectId bucketObjectId = bucketObjectIds.get(bucketFilePath);
    +            final Map<String, Object> bucketMeta;
    +            try (InputStream bucketIn = gitRepo.newObjectReader().open(bucketObjectId).openStream()) {
    +                bucketMeta = yaml.load(bucketIn);
    +            }
    +
    +            if (!validateRequiredValue(bucketMeta, bucketFilePath, LAYOUT_VERSION, BUCKET_ID, FLOWS)) {
    +                continue;
    +            }
    +
    +            int layoutVersion = (int) bucketMeta.get(LAYOUT_VERSION);
    +            if (layoutVersion > CURRENT_LAYOUT_VERSION) {
    +                logger.warn("{} has unsupported {} {}. This Registry can only support {} or lower. Skipping it.",
    +                        bucketFilePath, LAYOUT_VERSION, layoutVersion, CURRENT_LAYOUT_VERSION);
    +                continue;
    +            }
    +
    +            final String bucketId = (String) bucketMeta.get(BUCKET_ID);
    +
    +            final Bucket bucket;
    +            if (isLatestCommit) {
    +                // If this is the latest commit, then create one.
    +                bucket = getBucketOrCreate(bucketId);
    +            } else {
    +                // Otherwise non-existing bucket means it's already deleted.
    +                final Optional<Bucket> bucketOpt = getBucket(bucketId);
    +                if (bucketOpt.isPresent()) {
    +                    bucket = bucketOpt.get();
    +                } else {
    +                    logger.debug("Bucket {} does not exist any longer. It may have been deleted.", bucketId);
    +                    continue;
    +                }
    +            }
    +
    +            // E.g. DirA/DirB/DirC/bucket.yml -> DirC will be the bucket name.
    +            final String[] pathNames = bucketFilePath.split("/");
    --- End diff --
    
    Thanks @bbende for catching that. I added sanitization to convert invalid characters for filename such as '/', but left whitespaces as they are. Keep them as close to the original names but safe for file system names.


---

[GitHub] nifi-registry issue #112: NIFIREG-162: Support Git backed PersistenceProvide...

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

    https://github.com/apache/nifi-registry/pull/112
  
    I vote for dropping spaces from the filename. Yes, it's technically valid, but it's escaped differently on different OSes and will just cause problems. There is no harm in removing it. 


---

[GitHub] nifi-registry issue #112: NIFIREG-162: Support Git backed PersistenceProvide...

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

    https://github.com/apache/nifi-registry/pull/112
  
    Async pushing looks good, so I think final two items are updating the migration guidance and whether or not we want to leave spaces in the bucket/flow names. 


---

[GitHub] nifi-registry pull request #112: NIFIREG-162: Support Git backed Persistence...

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/112#discussion_r184706432
  
    --- Diff: nifi-registry-docs/src/main/asciidoc/administration-guide.adoc ---
    @@ -895,3 +895,167 @@ Providing 2 total locations, including `nifi.registry.extension.dir.1`.
       Example: `/etc/http-nifi-registry.keytab`
     |nifi.registry.kerberos.spengo.authentication.expiration|The expiration duration of a successful Kerberos user authentication, if used. The default value is `12 hours`.
     |====
    +
    +== Persistence Providers
    +
    +NiFi Registry uses a pluggable flow persistence provider to store the content of the flows saved to the registry. NiFi Registry provides `<<FileSystemFlowPersistenceProvider>>` and `<<GitFlowPersistenceProvider>>`.
    +
    +Each persistence provider has its own configuration parameters, those can be configured in a XML file specified in <<Providers Properties,nifi-registry.properties>>.
    +
    +The XML configuration file looks like below. It has a `flowPersistenceProvider` element in which qualified class name of a persistence provider implementation and its configuration properties are defined. See following sections for available configurations for each providers.
    +
    +.Example providers.xml
    +[source,xml]
    +....
    +<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
    +<providers>
    +
    +    <flowPersistenceProvider>
    +        <class>persistence-provider-qualified-class-name</class>
    +        <property name="property-1">property-value-1</property>
    +        <property name="property-2">property-value-2</property>
    +        <property name="property-n">property-value-n</property>
    +    </flowPersistenceProvider>
    +
    +</providers>
    +....
    +
    +
    +=== FileSystemFlowPersistenceProvider
    +
    +FileSystemFlowPersistenceProvider simply stores serialized Flow contents into `{bucket-id}/{flow-id}/{version}` directories.
    +
    +Example of persisted files:
    +....
    +Flow Storage Directory/
    +├── {bucket-id}/
    +│   └── {flow-id}/
    +│       ├── {version}/{version}.snapshot
    +└── d1beba88-32e9-45d1-bfe9-057cc41f7ce8/
    +    └── 219cf539-427f-43be-9294-0644fb07ca63/
    +        ├── 1/1.snapshot
    +        └── 2/2.snapshot
    +....
    +
    +Qualified class name: `org.apache.nifi.registry.provider.flow.FileSystemFlowPersistenceProvider`
    +
    +|====
    +|*Property*|*Description*
    +|Flow Storage Directory|REQUIRED: File system path for a directory where flow contents files are persisted to. If the directory does not exist when NiFi Registry starts, it will be created. If the directory exists, it must be readable and writable from NiFi Registry.
    +|====
    +
    +
    +=== GitFlowPersistenceProvider
    +
    +GitFlowPersistenceProvider stores flow contents under a Git directory.
    +
    +In contrast to FileSystemFlowPersistenceProvider, this provider uses human friendly Bucket and Flow names so that those files can be accessed by external tools. However, it is NOT supported to modify stored files outside of NiFi Registry. Persisted files are only read when NiFi Registry starts up.
    +
    +Buckets are represented as directories and Flow contents are stored as files in a Bucket directory they belong to. Flow snapshot histories are managed as Git commits, meaning only the latest version of Buckets and Flows exist in the Git directory. Old versions are retrieved from Git commit histories.
    +
    +.Example persisted files
    +....
    +Flow Storage Directory/
    +├── .git/
    +├── Bucket A/
    +│   ├── bucket.yml
    +│   ├── Flow 1.snapshot
    +│   └── Flow 2.snapshot
    +└── Bucket B/
    +    ├── bucket.yml
    +    └── Flow 4.snapshot
    +....
    +
    +Each Bucket directory contains a YAML file named `bucket.yml`. The file manages links from NiFi Registry Bucket and Flow IDs to actual directory and file names. When NiFi Registry starts, this provider reads through Git commit histories and lookup these `bucket.yml` files to restore Buckets and Flows for each snapshot version.
    +
    +.Example bucket.yml
    +[source,yml]
    +....
    +layoutVer: 1
    +bucketId: d1beba88-32e9-45d1-bfe9-057cc41f7ce8
    +flows:
    +  219cf539-427f-43be-9294-0644fb07ca63: {ver: 7, file: Flow 1.snapshot}
    +  22cccb6c-3011-4493-a996-611f8f112969: {ver: 3, file: Flow 2.snapshot}
    +....
    +
    +Qualified class name: `org.apache.nifi.registry.provider.flow.git.GitFlowPersistenceProvider`
    +
    +|====
    +|*Property*|*Description*
    +|Flow Storage Directory|REQUIRED: File system path for a directory where flow contents files are persisted to. The directory must exist when NiFi registry starts. Also must be initialized as a Git directory. See <<Initialize Git directory>> for detail.
    +|Remote To Push|When a new flow snapshot is created, this persistence provider updated files in the specified Git directory, then create a commit to the local repository. If `Remote To Push` is defined, it also pushes to the specified remote repository. E.g. 'origin'. To define more detailed remote spec such as branch names, use `Refspec`. See https://git-scm.com/book/en/v2/Git-Internals-The-Refspec
    +|Remote Access User|This user name is used to make push requests to the remote repository when `Remote To Push` is enabled, and the remote repository is accessed by HTTP protocol. If SSH is used, user authentication is done with SSH keys.
    +|Remote Access Password|Used with `Remote Access User`.
    +|====
    +
    +==== Initialize Git directory
    +
    +In order to use GitFlowPersistenceRepository, you need to prepare a Git directory on the local file system. You can do so by initializing a directory with `git init` command, or clone an existing Git project from a remote Git repository by `git clone` command.
    +
    +- Git init command
    +https://git-scm.com/docs/git-init
    +- Git clone command
    +https://git-scm.com/docs/git-clone
    +
    +
    +==== Git user configuration
    +
    +Git distinguishes a user by its username and email address. This persistence provider uses NiFi Registry username when it creates Git commits. However since NiFi Registry users do not provide email address, preconfigured Git user email address is used.
    +
    +You can configure Git user name and email address by `git config` command.
    +
    +- Git config command
    +https://git-scm.com/docs/git-config
    +
    +
    +==== Git user authentication
    +
    +By default, this persistence repository only create commits to local repository. No user authentication is needed to do so. However, if 'Commit To Push' is enabled, user authentication to the remote Git repository is required.
    +
    +If the remote repository is accessed by HTTP, then username and password for authentication can be configured in the providers XML configuration file.
    +
    +When SSH is used, SSH keys are used to identify a Git user. In order to pick the right key to a remote server, the SSH configuration file `${USER_HOME}/.ssh/config` is used. The SSH configuration file can contain multiple `Host` entries to specify a key file to login to a remote Git server. The `Host` must much with the target remote Git server hostname.
    +
    +.example SSH config file
    +....
    +Host git.example.com
    +  HostName git.example.com
    +  IdentityFile ~/.ssh/id_rsa
    +
    +Host github.com
    +  HostName github.com
    +  IdentityFile ~/.ssh/key-for-github
    +
    +Host bitbucket.org
    +  HostName bitbucket.org
    +  IdentityFile ~/.ssh/key-for-bitbucket
    +....
    +
    +=== Data model version of serialized Flow snapshots
    +
    +Serialized Flow snapshots saved by these persistence providers have versions, so that the data format and schema can evolve over time. Data model version update is done automatically by NiFi Registry when it reads and stores each Flow content.
    +
    +Here is the data model version histories:
    +
    +|====
    +|*Data model version*|*Since NiFi Registry*|*Description*
    +|2|0.2|JSON formatted text file. The root object contains header and Flow content object.
    +|1|0.1|Binary format having header bytes at the beginning followed by Flow content represented as XML.
    +|====
    +
    +=== Migrating stored files between different Persistence Provider
    --- End diff --
    
    Yea I agree that in the future we could probably offer a bulk import/export via the CLI which only needed to use the REST API. We already have a JIRA for some import/export end-points (https://issues.apache.org/jira/browse/NIFIREG-148), so it might just be some CLI commands to make use of those once they exist.
    
    For this first release since we only have two providers, and since registry is very new, I think it is acceptable to say that if you want to use the git provider then you can start over with it. 


---

[GitHub] nifi-registry pull request #112: NIFIREG-162: Support Git backed Persistence...

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/112#discussion_r184709680
  
    --- Diff: nifi-registry-utils/src/main/java/org/apache/nifi/registry/util/FileUtils.java ---
    @@ -389,4 +389,38 @@ public static void sleepQuietly(final long millis) {
                 /* do nothing */
             }
         }
    +
    +
    +    // The invalid character list is copied from this Stackoverflow page.
    +    // https://stackoverflow.com/questions/1155107/is-there-a-cross-platform-java-method-to-remove-filename-special-chars
    +    private final static int[] INVALID_CHARS = {34, 60, 62, 124, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15,
    --- End diff --
    
    Should we add a space to this list? 
    
    Technically a space is valid, but currently a bucket name of `This / is a / test` becomes `This _ is a _ test` but I was originally thinking it would be `This_is_a_test` or I guess technically it would have multiple underscores like `This___is_a___test` ?


---

[GitHub] nifi-registry issue #112: NIFIREG-162: Support Git backed PersistenceProvide...

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

    https://github.com/apache/nifi-registry/pull/112
  
    I noticed when I ran "git log" that the commit was made by "anonymous" which is correct since I was in an unsecure instance, but the email address of the commit ended up using the email from my ~/.gitconfig so I ended up with:
    
    ```
    nifi-versioned-flows$ git log
    commit 153690a2bd06d57ec416cb19b0582e2f7b138771 (HEAD -> master)
    Author: anonymous <bb...@apache.org>
    Date:   Thu Apr 26 10:58:40 2018 -0400
    
        Test
    ```
    This is technically correct since that is the email address that should be found when there is not a more specific one, but should we make a property on the provider config like "Commit Email Address" ? or should we just leave it up to users to setup their gitconfig appropriately.
    
    Ultimately we won't be able to have per-user email addresses anyway because when secured we will be using the identity of the proxied-entity as the author, and we have no way of knowing their email address. 


---

[GitHub] nifi-registry issue #112: NIFIREG-162: Support Git backed PersistenceProvide...

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

    https://github.com/apache/nifi-registry/pull/112
  
    @bbende I've moved push to different thread. Now it only commits locally on the web request handling thread. Used blocking queue to push fast enough but keep the thread waiting until new push is necessary, with this way no interval configuration is needed I think.
    
    About the commit email address and author, I think using author and email address those are configured at Git repository as they are, and embedding NiFi registry end user name in a commit message would make more sense. By doing so, users can configure whatever they need at the git side, while being able to track how NiFi registry committed the change and by whom. How do you think?


---