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 2017/12/10 16:01:31 UTC

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

GitHub user ijokarumawak opened a pull request:

    https://github.com/apache/nifi/pull/2335

    NIFI-3709: Export NiFi flow dataset lineage to Apache Atlas

    Thank you for submitting a contribution to Apache NiFi.
    
    In order to streamline the review of the contribution we ask you
    to ensure the following steps have been taken:
    
    ### For all changes:
    - [x] Is there a JIRA ticket associated with this PR? Is it referenced 
         in the commit message?
    
    - [x] Does your PR title start with NIFI-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
    
    - [x] Has your PR been rebased against the latest commit within the target branch (typically master)?
    
    - [x] Is your initial contribution a single, squashed commit?
    
    ### For code changes:
    - [ ] Have you ensured that the full suite of tests is executed via mvn -Pcontrib-check clean install at the root nifi folder?
    - [x] Have you written or updated unit tests to verify your changes?
    - [x] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
    - [x] If applicable, have you updated the LICENSE file, including the main LICENSE file under nifi-assembly?
    - [x] If applicable, have you updated the NOTICE file, including the main NOTICE file found under nifi-assembly?
    - [x] If adding new Properties, have you added .displayName in addition to .name (programmatic access) for each of the new properties?
    
    ### For documentation related changes:
    - [x] Have you ensured that format looks appropriate for the output in which it is rendered?
    
    ### Note:
    Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.


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

    $ git pull https://github.com/ijokarumawak/nifi nifi-3709-5

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

    https://github.com/apache/nifi/pull/2335.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 #2335
    
----
commit d710ceee0fd6076c9bac90f0e3b55f5bb990bd33
Author: Koji Kawamura <ij...@apache.org>
Date:   2017-10-30T03:41:27Z

    NIFI-3709: Export NiFi flow dataset lineage to Apache Atlas

----


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157221167
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/AtlasUtils.java ---
    @@ -0,0 +1,77 @@
    +/*
    + * 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.atlas;
    +
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +
    +public class AtlasUtils {
    +
    +    public static String toStr(Object obj) {
    +        return obj != null ? obj.toString() : null;
    +    }
    +
    +
    +    public static boolean isGuidAssigned(String guid) {
    +        return guid != null && !guid.startsWith("-");
    --- End diff --
    
    Yes, it's Atlas internal implementation. However, checking null is not sufficient, because entities created at client side but not yet registered in Atlas have negative GUIDs. I'd like to keep it this way if there's no strong objections. Thank you.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157257088
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/StandardAnalysisContext.java ---
    @@ -0,0 +1,136 @@
    +/*
    + * 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.atlas.provenance;
    +
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.authorization.user.NiFiUser;
    +import org.apache.nifi.controller.status.ConnectionStatus;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.provenance.lineage.ComputeLineageResult;
    +import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +
    +public class StandardAnalysisContext implements AnalysisContext {
    +
    +    private final Logger logger = LoggerFactory.getLogger(StandardAnalysisContext.class);
    +    private final NiFiFlow nifiFlow;
    +    private final ClusterResolver clusterResolver;
    +    private final ProvenanceRepository provenanceRepository;
    +
    +    public StandardAnalysisContext(NiFiFlow nifiFlow, ClusterResolver clusterResolver,
    +                                   ProvenanceRepository provenanceRepository) {
    +        this.nifiFlow = nifiFlow;
    +        this.clusterResolver = clusterResolver;
    +        this.provenanceRepository = provenanceRepository;
    +    }
    +
    +    @Override
    +    public List<ConnectionStatus> findConnectionTo(String componentId) {
    +        return nifiFlow.getIncomingRelationShips(componentId);
    +    }
    +
    +    @Override
    +    public List<ConnectionStatus> findConnectionFrom(String componentId) {
    +        return nifiFlow.getOutgoingRelationShips(componentId);
    +    }
    +
    +    @Override
    +    public String getNiFiClusterName() {
    +        return nifiFlow.getClusterName();
    +    }
    +
    +    @Override
    +    public ClusterResolver getClusterResolver() {
    +        return clusterResolver;
    +    }
    +
    +    private ComputeLineageResult getLineageResult(long eventId, ComputeLineageSubmission submission) {
    +        final ComputeLineageResult result = submission.getResult();
    +        try {
    +            if (result.awaitCompletion(10, TimeUnit.SECONDS)) {
    +                return result;
    +            }
    +            logger.warn("Lineage query for {} timed out.", new Object[]{eventId});
    +        } catch (InterruptedException e) {
    +            logger.warn("Lineage query for {} was interrupted due to {}.", new Object[]{eventId, e}, e);
    +        } finally {
    +            submission.cancel();
    +        }
    +
    +        return null;
    +    }
    +
    +    @Override
    +    public ComputeLineageResult queryLineage(long eventId) {
    +        final ComputeLineageSubmission submission = provenanceRepository.submitLineageComputation(eventId, NIFI_USER);
    +        return getLineageResult(eventId, submission);
    +    }
    +
    +    public ComputeLineageResult findParents(long eventId) {
    +        final ComputeLineageSubmission submission = provenanceRepository.submitExpandParents(eventId, NIFI_USER);
    +        return getLineageResult(eventId, submission);
    +    }
    +
    +    // NOTE: This user is required to avoid NullPointerException at PersistentProvenanceRepository.submitLineageComputation
    +    private static final QueryNiFiUser NIFI_USER = new QueryNiFiUser();
    --- End diff --
    
    @ijokarumawak I think I understand now what is going on. In a secured environment, you would indeed get back all UNKNOWN event types, because that user would not be authorized to see the events. But even the UNKNOWN event would include the ID of the event. It looks like you're not actually using the other info from the lineage but rather just using the Event ID to lookup the actual Event (doing so without a user). So all of this seems to me now :) Sorry about all of the confusion - and thanks for the clarifications!


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157005913
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/AtlasNiFiFlowLineage.java ---
    @@ -0,0 +1,714 @@
    +/*
    + * 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.atlas.reporting;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.common.config.SslConfigs;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
    +import org.apache.nifi.annotation.behavior.Stateful;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.atlas.NiFIAtlasHook;
    +import org.apache.nifi.atlas.NiFiAtlasClient;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowAnalyzer;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.StandardAnalysisContext;
    +import org.apache.nifi.atlas.provenance.lineage.CompleteFlowPathLineage;
    +import org.apache.nifi.atlas.provenance.lineage.LineageStrategy;
    +import org.apache.nifi.atlas.provenance.lineage.SimpleFlowPathLineage;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.atlas.resolver.ClusterResolvers;
    +import org.apache.nifi.atlas.resolver.RegexClusterResolver;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.atlas.security.Basic;
    +import org.apache.nifi.atlas.security.Kerberos;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.context.PropertyContext;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.reporting.AbstractReportingTask;
    +import org.apache.nifi.reporting.EventAccess;
    +import org.apache.nifi.reporting.ReportingContext;
    +import org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.time.Instant;
    +import java.time.ZoneOffset;
    +import java.time.format.DateTimeFormatter;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.LinkedHashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.ServiceLoader;
    +import java.util.Set;
    +import java.util.function.Consumer;
    +import java.util.stream.Stream;
    +
    +import static org.apache.commons.lang3.StringUtils.isEmpty;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_BATCH_SIZE;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_START_POSITION;
    +
    +@Tags({"atlas", "lineage"})
    +@CapabilityDescription("Publishes NiFi flow data set level lineage to Apache Atlas." +
    +        " By reporting flow information to Atlas, an end-to-end Process and DataSet lineage such as across NiFi environments and other systems" +
    +        " connected by technologies, for example NiFi Site-to-Site, Kafka topic or Hive tables." +
    +        " There are limitations and required configurations for both NiFi and Atlas. See 'Additional Details' for further description.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +@DynamicProperty(name = "hostnamePattern.<ClusterName>", value = "hostname Regex patterns", description = RegexClusterResolver.PATTERN_PROPERTY_PREFIX_DESC)
    +// In order for each reporting task instance to have its own static objects such as KafkaNotification.
    +@RequiresInstanceClassLoading
    +public class AtlasNiFiFlowLineage extends AbstractReportingTask {
    +
    +    static final PropertyDescriptor ATLAS_URLS = new PropertyDescriptor.Builder()
    +            .name("atlas-urls")
    +            .displayName("Atlas URLs")
    +            .description("Comma separated URL of Atlas Servers" +
    +                    " (e.g. http://atlas-server-hostname:21000 or https://atlas-server-hostname:21443)." +
    +                    " For accessing Atlas behind Knox gateway, specify Knox gateway URL" +
    +                    " (e.g. https://knox-hostname:8443/gateway/{topology-name}/atlas).")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final AllowableValue ATLAS_AUTHN_BASIC = new AllowableValue("basic", "Basic", "Use username and password.");
    +    static final AllowableValue ATLAS_AUTHN_KERBEROS = new AllowableValue("kerberos", "Kerberos", "Use Kerberos keytab file.");
    +    static final PropertyDescriptor ATLAS_AUTHN_METHOD = new PropertyDescriptor.Builder()
    +            .name("atlas-authentication-method")
    +            .displayName("Atlas Authentication Method")
    +            .description("Specify how to authenticate this reporting task to Atlas server.")
    +            .required(true)
    +            .allowableValues(ATLAS_AUTHN_BASIC, ATLAS_AUTHN_KERBEROS)
    +            .defaultValue(ATLAS_AUTHN_BASIC.getValue())
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_USER = new PropertyDescriptor.Builder()
    +            .name("atlas-username")
    +            .displayName("Atlas Username")
    +            .description("User name to communicate with Atlas.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("atlas-password")
    +            .displayName("Atlas Password")
    +            .description("Password to communicate with Atlas.")
    +            .required(false)
    +            .sensitive(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final PropertyDescriptor ATLAS_CONF_DIR = new PropertyDescriptor.Builder()
    +            .name("atlas-conf-dir")
    +            .displayName("Atlas Configuration Directory")
    +            .description("Directory path that contains 'atlas-application.properties' file." +
    +                    " If not specified and 'Create Atlas Configuration File' is disabled," +
    +                    " then, 'atlas-application.properties' file under root classpath is used.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_NIFI_URL = new PropertyDescriptor.Builder()
    +            .name("atlas-nifi-url")
    +            .displayName("NiFi URL for Atlas")
    +            .description("NiFi URL is used in Atlas to represent this NiFi cluster (or standalone instance)." +
    +                    " It is recommended to use one that can be accessible remotely instead of using 'localhost'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.URL_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_DEFAULT_CLUSTER_NAME = new PropertyDescriptor.Builder()
    +            .name("atlas-default-cluster-name")
    +            .displayName("Atlas Default Cluster Name")
    +            .description("Cluster name for Atlas entities reported by this ReportingTask." +
    +                    " If not specified, 'atlas.cluster.name' in Atlas Configuration File is used." +
    +                    " Cluster name mappings can be configured by user defined properties." +
    +                    " See additional detail for detail.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final PropertyDescriptor ATLAS_CONF_CREATE = new PropertyDescriptor.Builder()
    +            .name("atlas-conf-create")
    +            .displayName("Create Atlas Configuration File")
    +            .description("If enabled, 'atlas-application.properties' file will be created in 'Atlas Configuration Directory'" +
    +                    " automatically when this processor starts." +
    +                    " Note that the existing configuration file will be overwritten.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .allowableValues("true", "false")
    +            .defaultValue("false")
    +            .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +            .name("ssl-context-service")
    +            .displayName("SSL Context Service")
    +            .description("Specifies the SSL Context Service to use for communicating with Atlas and Kafka.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +    static final PropertyDescriptor KAFKA_BOOTSTRAP_SERVERS = new PropertyDescriptor.Builder()
    +            .name("kafka-bootstrap-servers")
    +            .displayName("Kafka Bootstrap Servers")
    +            .description("Kafka Bootstrap Servers to send Atlas hook notification messages based on NiFi provenance events." +
    +                    " E.g. 'localhost:9092'" +
    +                    " NOTE: Once this reporting task has started, restarting NiFi is required to changed this property" +
    --- End diff --
    
    Has a JIRA been filed with Atlas to make this more manageable?


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157005524
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/AtlasNiFiFlowLineage.java ---
    @@ -0,0 +1,714 @@
    +/*
    + * 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.atlas.reporting;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.common.config.SslConfigs;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
    +import org.apache.nifi.annotation.behavior.Stateful;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.atlas.NiFIAtlasHook;
    +import org.apache.nifi.atlas.NiFiAtlasClient;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowAnalyzer;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.StandardAnalysisContext;
    +import org.apache.nifi.atlas.provenance.lineage.CompleteFlowPathLineage;
    +import org.apache.nifi.atlas.provenance.lineage.LineageStrategy;
    +import org.apache.nifi.atlas.provenance.lineage.SimpleFlowPathLineage;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.atlas.resolver.ClusterResolvers;
    +import org.apache.nifi.atlas.resolver.RegexClusterResolver;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.atlas.security.Basic;
    +import org.apache.nifi.atlas.security.Kerberos;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.context.PropertyContext;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.reporting.AbstractReportingTask;
    +import org.apache.nifi.reporting.EventAccess;
    +import org.apache.nifi.reporting.ReportingContext;
    +import org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.time.Instant;
    +import java.time.ZoneOffset;
    +import java.time.format.DateTimeFormatter;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.LinkedHashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.ServiceLoader;
    +import java.util.Set;
    +import java.util.function.Consumer;
    +import java.util.stream.Stream;
    +
    +import static org.apache.commons.lang3.StringUtils.isEmpty;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_BATCH_SIZE;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_START_POSITION;
    +
    +@Tags({"atlas", "lineage"})
    +@CapabilityDescription("Publishes NiFi flow data set level lineage to Apache Atlas." +
    +        " By reporting flow information to Atlas, an end-to-end Process and DataSet lineage such as across NiFi environments and other systems" +
    +        " connected by technologies, for example NiFi Site-to-Site, Kafka topic or Hive tables." +
    +        " There are limitations and required configurations for both NiFi and Atlas. See 'Additional Details' for further description.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +@DynamicProperty(name = "hostnamePattern.<ClusterName>", value = "hostname Regex patterns", description = RegexClusterResolver.PATTERN_PROPERTY_PREFIX_DESC)
    +// In order for each reporting task instance to have its own static objects such as KafkaNotification.
    +@RequiresInstanceClassLoading
    +public class AtlasNiFiFlowLineage extends AbstractReportingTask {
    +
    +    static final PropertyDescriptor ATLAS_URLS = new PropertyDescriptor.Builder()
    +            .name("atlas-urls")
    +            .displayName("Atlas URLs")
    +            .description("Comma separated URL of Atlas Servers" +
    +                    " (e.g. http://atlas-server-hostname:21000 or https://atlas-server-hostname:21443)." +
    +                    " For accessing Atlas behind Knox gateway, specify Knox gateway URL" +
    +                    " (e.g. https://knox-hostname:8443/gateway/{topology-name}/atlas).")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final AllowableValue ATLAS_AUTHN_BASIC = new AllowableValue("basic", "Basic", "Use username and password.");
    +    static final AllowableValue ATLAS_AUTHN_KERBEROS = new AllowableValue("kerberos", "Kerberos", "Use Kerberos keytab file.");
    +    static final PropertyDescriptor ATLAS_AUTHN_METHOD = new PropertyDescriptor.Builder()
    +            .name("atlas-authentication-method")
    +            .displayName("Atlas Authentication Method")
    +            .description("Specify how to authenticate this reporting task to Atlas server.")
    +            .required(true)
    +            .allowableValues(ATLAS_AUTHN_BASIC, ATLAS_AUTHN_KERBEROS)
    +            .defaultValue(ATLAS_AUTHN_BASIC.getValue())
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_USER = new PropertyDescriptor.Builder()
    +            .name("atlas-username")
    +            .displayName("Atlas Username")
    +            .description("User name to communicate with Atlas.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("atlas-password")
    +            .displayName("Atlas Password")
    +            .description("Password to communicate with Atlas.")
    +            .required(false)
    +            .sensitive(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final PropertyDescriptor ATLAS_CONF_DIR = new PropertyDescriptor.Builder()
    +            .name("atlas-conf-dir")
    +            .displayName("Atlas Configuration Directory")
    +            .description("Directory path that contains 'atlas-application.properties' file." +
    +                    " If not specified and 'Create Atlas Configuration File' is disabled," +
    +                    " then, 'atlas-application.properties' file under root classpath is used.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_NIFI_URL = new PropertyDescriptor.Builder()
    +            .name("atlas-nifi-url")
    +            .displayName("NiFi URL for Atlas")
    +            .description("NiFi URL is used in Atlas to represent this NiFi cluster (or standalone instance)." +
    +                    " It is recommended to use one that can be accessible remotely instead of using 'localhost'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.URL_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_DEFAULT_CLUSTER_NAME = new PropertyDescriptor.Builder()
    +            .name("atlas-default-cluster-name")
    +            .displayName("Atlas Default Cluster Name")
    +            .description("Cluster name for Atlas entities reported by this ReportingTask." +
    +                    " If not specified, 'atlas.cluster.name' in Atlas Configuration File is used." +
    +                    " Cluster name mappings can be configured by user defined properties." +
    +                    " See additional detail for detail.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final PropertyDescriptor ATLAS_CONF_CREATE = new PropertyDescriptor.Builder()
    +            .name("atlas-conf-create")
    +            .displayName("Create Atlas Configuration File")
    +            .description("If enabled, 'atlas-application.properties' file will be created in 'Atlas Configuration Directory'" +
    +                    " automatically when this processor starts." +
    --- End diff --
    
    I think this was meant to read 'when this Reporting Task starts', rather than 'when this processor starts'


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157009606
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/AtlasNiFiFlowLineage.java ---
    @@ -0,0 +1,714 @@
    +/*
    + * 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.atlas.reporting;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.common.config.SslConfigs;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
    +import org.apache.nifi.annotation.behavior.Stateful;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.atlas.NiFIAtlasHook;
    +import org.apache.nifi.atlas.NiFiAtlasClient;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowAnalyzer;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.StandardAnalysisContext;
    +import org.apache.nifi.atlas.provenance.lineage.CompleteFlowPathLineage;
    +import org.apache.nifi.atlas.provenance.lineage.LineageStrategy;
    +import org.apache.nifi.atlas.provenance.lineage.SimpleFlowPathLineage;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.atlas.resolver.ClusterResolvers;
    +import org.apache.nifi.atlas.resolver.RegexClusterResolver;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.atlas.security.Basic;
    +import org.apache.nifi.atlas.security.Kerberos;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.context.PropertyContext;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.reporting.AbstractReportingTask;
    +import org.apache.nifi.reporting.EventAccess;
    +import org.apache.nifi.reporting.ReportingContext;
    +import org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.time.Instant;
    +import java.time.ZoneOffset;
    +import java.time.format.DateTimeFormatter;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.LinkedHashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.ServiceLoader;
    +import java.util.Set;
    +import java.util.function.Consumer;
    +import java.util.stream.Stream;
    +
    +import static org.apache.commons.lang3.StringUtils.isEmpty;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_BATCH_SIZE;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_START_POSITION;
    +
    +@Tags({"atlas", "lineage"})
    +@CapabilityDescription("Publishes NiFi flow data set level lineage to Apache Atlas." +
    +        " By reporting flow information to Atlas, an end-to-end Process and DataSet lineage such as across NiFi environments and other systems" +
    +        " connected by technologies, for example NiFi Site-to-Site, Kafka topic or Hive tables." +
    +        " There are limitations and required configurations for both NiFi and Atlas. See 'Additional Details' for further description.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +@DynamicProperty(name = "hostnamePattern.<ClusterName>", value = "hostname Regex patterns", description = RegexClusterResolver.PATTERN_PROPERTY_PREFIX_DESC)
    +// In order for each reporting task instance to have its own static objects such as KafkaNotification.
    +@RequiresInstanceClassLoading
    +public class AtlasNiFiFlowLineage extends AbstractReportingTask {
    +
    +    static final PropertyDescriptor ATLAS_URLS = new PropertyDescriptor.Builder()
    +            .name("atlas-urls")
    +            .displayName("Atlas URLs")
    +            .description("Comma separated URL of Atlas Servers" +
    +                    " (e.g. http://atlas-server-hostname:21000 or https://atlas-server-hostname:21443)." +
    +                    " For accessing Atlas behind Knox gateway, specify Knox gateway URL" +
    +                    " (e.g. https://knox-hostname:8443/gateway/{topology-name}/atlas).")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final AllowableValue ATLAS_AUTHN_BASIC = new AllowableValue("basic", "Basic", "Use username and password.");
    +    static final AllowableValue ATLAS_AUTHN_KERBEROS = new AllowableValue("kerberos", "Kerberos", "Use Kerberos keytab file.");
    +    static final PropertyDescriptor ATLAS_AUTHN_METHOD = new PropertyDescriptor.Builder()
    +            .name("atlas-authentication-method")
    +            .displayName("Atlas Authentication Method")
    +            .description("Specify how to authenticate this reporting task to Atlas server.")
    +            .required(true)
    +            .allowableValues(ATLAS_AUTHN_BASIC, ATLAS_AUTHN_KERBEROS)
    +            .defaultValue(ATLAS_AUTHN_BASIC.getValue())
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_USER = new PropertyDescriptor.Builder()
    +            .name("atlas-username")
    +            .displayName("Atlas Username")
    +            .description("User name to communicate with Atlas.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("atlas-password")
    +            .displayName("Atlas Password")
    +            .description("Password to communicate with Atlas.")
    +            .required(false)
    +            .sensitive(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final PropertyDescriptor ATLAS_CONF_DIR = new PropertyDescriptor.Builder()
    +            .name("atlas-conf-dir")
    +            .displayName("Atlas Configuration Directory")
    +            .description("Directory path that contains 'atlas-application.properties' file." +
    +                    " If not specified and 'Create Atlas Configuration File' is disabled," +
    +                    " then, 'atlas-application.properties' file under root classpath is used.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_NIFI_URL = new PropertyDescriptor.Builder()
    +            .name("atlas-nifi-url")
    +            .displayName("NiFi URL for Atlas")
    +            .description("NiFi URL is used in Atlas to represent this NiFi cluster (or standalone instance)." +
    +                    " It is recommended to use one that can be accessible remotely instead of using 'localhost'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.URL_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_DEFAULT_CLUSTER_NAME = new PropertyDescriptor.Builder()
    +            .name("atlas-default-cluster-name")
    +            .displayName("Atlas Default Cluster Name")
    +            .description("Cluster name for Atlas entities reported by this ReportingTask." +
    +                    " If not specified, 'atlas.cluster.name' in Atlas Configuration File is used." +
    +                    " Cluster name mappings can be configured by user defined properties." +
    +                    " See additional detail for detail.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final PropertyDescriptor ATLAS_CONF_CREATE = new PropertyDescriptor.Builder()
    +            .name("atlas-conf-create")
    +            .displayName("Create Atlas Configuration File")
    +            .description("If enabled, 'atlas-application.properties' file will be created in 'Atlas Configuration Directory'" +
    +                    " automatically when this processor starts." +
    +                    " Note that the existing configuration file will be overwritten.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .allowableValues("true", "false")
    +            .defaultValue("false")
    +            .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +            .name("ssl-context-service")
    +            .displayName("SSL Context Service")
    +            .description("Specifies the SSL Context Service to use for communicating with Atlas and Kafka.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +    static final PropertyDescriptor KAFKA_BOOTSTRAP_SERVERS = new PropertyDescriptor.Builder()
    +            .name("kafka-bootstrap-servers")
    +            .displayName("Kafka Bootstrap Servers")
    +            .description("Kafka Bootstrap Servers to send Atlas hook notification messages based on NiFi provenance events." +
    +                    " E.g. 'localhost:9092'" +
    +                    " NOTE: Once this reporting task has started, restarting NiFi is required to changed this property" +
    +                    " as Atlas library holds a unmodifiable static reference to Kafka client.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final AllowableValue SEC_PLAINTEXT = new AllowableValue("PLAINTEXT", "PLAINTEXT", "PLAINTEXT");
    +    static final AllowableValue SEC_SSL = new AllowableValue("SSL", "SSL", "SSL");
    +    static final AllowableValue SEC_SASL_PLAINTEXT = new AllowableValue("SASL_PLAINTEXT", "SASL_PLAINTEXT", "SASL_PLAINTEXT");
    +    static final AllowableValue SEC_SASL_SSL = new AllowableValue("SASL_SSL", "SASL_SSL", "SASL_SSL");
    +    static final PropertyDescriptor KAFKA_SECURITY_PROTOCOL = new PropertyDescriptor.Builder()
    +            .name("kafka-security-protocol")
    +            .displayName("Kafka Security Protocol")
    +            .description("Protocol used to communicate with Kafka brokers to send Atlas hook notification messages." +
    +                    " Corresponds to Kafka's 'security.protocol' property.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL)
    +            .defaultValue(SEC_PLAINTEXT.getValue())
    +            .build();
    +
    +    public static final PropertyDescriptor NIFI_KERBEROS_PRINCIPAL = new PropertyDescriptor.Builder()
    +            .name("nifi-kerberos-principal")
    +            .displayName("NiFi Kerberos Principal")
    +            .description("The Kerberos principal for this NiFi instance to access Atlas API and Kafka brokers." +
    +                    " If not set, it is expected to set a JAAS configuration file in the JVM properties defined in the bootstrap.conf file." +
    +                    " This principal will be set into 'sasl.jaas.config' Kafka's property.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +    public static final PropertyDescriptor NIFI_KERBEROS_KEYTAB = new PropertyDescriptor.Builder()
    +            .name("nifi-kerberos-keytab")
    +            .displayName("NiFi Kerberos Keytab")
    +            .description("The Kerberos keytab for this NiFi instance to access Atlas API and Kafka brokers." +
    +                    " If not set, it is expected to set a JAAS configuration file in the JVM properties defined in the bootstrap.conf file." +
    +                    " This principal will be set into 'sasl.jaas.config' Kafka's property.")
    +            .required(false)
    +            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    static final PropertyDescriptor KAFKA_KERBEROS_SERVICE_NAME = new PropertyDescriptor.Builder()
    +            .name("kafka-kerberos-service-name-kafka")
    +            .displayName("Kafka Kerberos Service Name")
    +            .description("The Kerberos principal name that Kafka runs for Atlas notification." +
    +                    " This can be defined either in Kafka's JAAS config or in Kafka's config." +
    +                    " Corresponds to Kafka's 'security.protocol' property." +
    +                    " It is ignored unless one of the SASL options of the <Security Protocol> are selected.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .defaultValue("kafka")
    +            .build();
    +
    +    static final AllowableValue LINEAGE_STRATEGY_SIMPLE_PATH = new AllowableValue("SimplePath", "Simple Path",
    +            "Map NiFi provenance events and target Atlas DataSets to statically created 'nifi_flow_path' Atlas Processes." +
    +                    " See also 'Additional Details'.");
    +    static final AllowableValue LINEAGE_STRATEGY_COMPLETE_PATH = new AllowableValue("CompletePath", "Complete Path",
    +            "Create separate 'nifi_flow_path' Atlas Processes for each distinct input and output DataSet combinations" +
    +                    " by looking at the complete route for a given FlowFile. See also 'Additional Details.");
    +
    +    static final PropertyDescriptor NIFI_LINEAGE_STRATEGY = new PropertyDescriptor.Builder()
    +            .name("nifi-lineage-strategy")
    +            .displayName("NiFi Lineage Strategy")
    +            .description("Specifies granularity on how NiFi data flow should be reported to Atlas.")
    +            .required(true)
    +            .allowableValues(LINEAGE_STRATEGY_SIMPLE_PATH, LINEAGE_STRATEGY_COMPLETE_PATH)
    +            .defaultValue(LINEAGE_STRATEGY_SIMPLE_PATH.getValue())
    +            .build();
    +
    +    private static final String ATLAS_PROPERTIES_FILENAME = "atlas-application.properties";
    +    private static final String ATLAS_PROPERTY_CLUSTER_NAME = "atlas.cluster.name";
    +    private static final String ATLAS_PROPERTY_ENABLE_TLS = "atlas.enableTLS";
    +    private static final String ATLAS_KAFKA_PREFIX = "atlas.kafka.";
    +    private static final String ATLAS_PROPERTY_KAFKA_BOOTSTRAP_SERVERS = ATLAS_KAFKA_PREFIX + "bootstrap.servers";
    +    private static final String ATLAS_PROPERTY_KAFKA_CLIENT_ID = ATLAS_KAFKA_PREFIX + ProducerConfig.CLIENT_ID_CONFIG;
    +    private final ServiceLoader<ClusterResolver> clusterResolverLoader = ServiceLoader.load(ClusterResolver.class);
    +    private volatile NiFiAtlasClient atlasClient;
    +    private volatile Properties atlasProperties;
    +    private volatile boolean isTypeDefCreated = false;
    +    private volatile String defaultClusterName;
    +
    +    private volatile ProvenanceEventConsumer consumer;
    +    private volatile ClusterResolvers clusterResolvers;
    +    private volatile NiFIAtlasHook nifiAtlasHook;
    +    private volatile LineageStrategy lineageStrategy;
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(ATLAS_URLS);
    +        properties.add(ATLAS_AUTHN_METHOD);
    +        properties.add(ATLAS_USER);
    +        properties.add(ATLAS_PASSWORD);
    +        properties.add(ATLAS_CONF_DIR);
    +        properties.add(ATLAS_NIFI_URL);
    +        properties.add(ATLAS_DEFAULT_CLUSTER_NAME);
    +        properties.add(NIFI_LINEAGE_STRATEGY);
    +        properties.add(PROVENANCE_START_POSITION);
    +        properties.add(PROVENANCE_BATCH_SIZE);
    +        properties.add(SSL_CONTEXT_SERVICE);
    +
    +        // Following properties are required if ATLAS_CONF_CREATE is enabled.
    +        // Otherwise should be left blank.
    +        properties.add(ATLAS_CONF_CREATE);
    +        properties.add(NIFI_KERBEROS_PRINCIPAL);
    +        properties.add(NIFI_KERBEROS_KEYTAB);
    +        properties.add(KAFKA_KERBEROS_SERVICE_NAME);
    +        properties.add(KAFKA_BOOTSTRAP_SERVERS);
    +        properties.add(KAFKA_SECURITY_PROTOCOL);
    +
    +        return properties;
    +    }
    +
    +    @Override
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(String propertyDescriptorName) {
    +        for (ClusterResolver resolver : clusterResolverLoader) {
    +            final PropertyDescriptor propertyDescriptor = resolver.getSupportedDynamicPropertyDescriptor(propertyDescriptorName);
    +            if(propertyDescriptor != null) {
    +                return propertyDescriptor;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void parseAtlasUrls(final PropertyValue atlasUrlsProp, final Consumer<String> urlStrConsumer) {
    +        final String atlasUrlsStr = atlasUrlsProp.evaluateAttributeExpressions().getValue();
    +        if (atlasUrlsStr != null && !atlasUrlsStr.isEmpty()) {
    +            Arrays.stream(atlasUrlsStr.split(","))
    +                    .map(String::trim)
    +                    .forEach(urlStrConsumer);
    +        }
    +    }
    +
    +    @Override
    +    protected Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = new ArrayList<>();
    +
    +        final boolean isSSLContextServiceSet = context.getProperty(SSL_CONTEXT_SERVICE).isSet();
    +        final ValidationResult.Builder invalidSSLService = new ValidationResult.Builder()
    +                .subject(SSL_CONTEXT_SERVICE.getDisplayName()).valid(false);
    +        parseAtlasUrls(context.getProperty(ATLAS_URLS), input -> {
    +            final ValidationResult.Builder builder = new ValidationResult.Builder().subject(ATLAS_URLS.getDisplayName()).input(input);
    +            try {
    +                final URL url = new URL(input);
    +                if ("https".equalsIgnoreCase(url.getProtocol()) && !isSSLContextServiceSet) {
    +                    results.add(invalidSSLService.explanation("required by HTTPS Atlas access").build());
    +                } else {
    +                    results.add(builder.explanation("Valid URI").valid(true).build());
    +                }
    +            } catch (Exception e) {
    +                results.add(builder.explanation("Contains invalid URI: " + e).valid(false).build());
    +            }
    +        });
    +
    +        final String atlasAuthNMethod = context.getProperty(ATLAS_AUTHN_METHOD).getValue();
    +        final AtlasAuthN atlasAuthN = getAtlasAuthN(atlasAuthNMethod);
    +        results.addAll(atlasAuthN.validate(context));
    +
    +
    +        clusterResolverLoader.forEach(resolver -> results.addAll(resolver.validate(context)));
    +
    +        if (context.getProperty(ATLAS_CONF_CREATE).asBoolean()) {
    +
    +            Stream.of(ATLAS_CONF_DIR, ATLAS_DEFAULT_CLUSTER_NAME, KAFKA_BOOTSTRAP_SERVERS)
    +                    .filter(p -> !context.getProperty(p).isSet())
    +                    .forEach(p -> results.add(new ValidationResult.Builder()
    +                            .subject(p.getDisplayName())
    +                            .explanation("required to create Atlas configuration file.")
    +                            .valid(false).build()));
    +
    +            validateKafkaProperties(context, results, isSSLContextServiceSet, invalidSSLService);
    +        }
    +
    +        return results;
    +    }
    +
    +    private void validateKafkaProperties(ValidationContext context, Collection<ValidationResult> results, boolean isSSLContextServiceSet, ValidationResult.Builder invalidSSLService) {
    +        final String kafkaSecurityProtocol = context.getProperty(KAFKA_SECURITY_PROTOCOL).getValue();
    +        if ((SEC_SSL.equals(kafkaSecurityProtocol) || SEC_SASL_SSL.equals(kafkaSecurityProtocol))
    +                && !isSSLContextServiceSet) {
    +            results.add(invalidSSLService.explanation("required by SSL Kafka connection").build());
    +        }
    +
    +        if (SEC_SASL_PLAINTEXT.equals(kafkaSecurityProtocol) || SEC_SASL_SSL.equals(kafkaSecurityProtocol)) {
    +            Stream.of(NIFI_KERBEROS_PRINCIPAL, NIFI_KERBEROS_KEYTAB, KAFKA_KERBEROS_SERVICE_NAME)
    +                    .filter(p -> !context.getProperty(p).isSet())
    +                    .forEach(p -> results.add(new ValidationResult.Builder()
    +                            .subject(p.getDisplayName())
    +                            .explanation("required by Kafka SASL authentication.")
    +                            .valid(false).build()));
    +        }
    +    }
    +
    +    @OnScheduled
    +    public void setup(ConfigurationContext context) throws IOException {
    +        // initAtlasClient has to be done first as it loads AtlasProperty.
    +        initAtlasClient(context);
    +        initLineageStrategy(context);
    +        initClusterResolvers(context);
    +    }
    +
    +    private void initLineageStrategy(ConfigurationContext context) throws IOException {
    +        nifiAtlasHook = new NiFIAtlasHook(atlasClient);
    +
    +        final String strategy = context.getProperty(NIFI_LINEAGE_STRATEGY).getValue();
    +        if (LINEAGE_STRATEGY_SIMPLE_PATH.equals(strategy)) {
    +            lineageStrategy = new SimpleFlowPathLineage();
    +        } else if (LINEAGE_STRATEGY_COMPLETE_PATH.equals(strategy)) {
    +            lineageStrategy = new CompleteFlowPathLineage();
    +        }
    +
    +        lineageStrategy.setLineageContext(nifiAtlasHook);
    +        initProvenanceConsumer(context);
    +    }
    +
    +    private void initClusterResolvers(ConfigurationContext context) {
    +        final Set<ClusterResolver> loadedClusterResolvers = new LinkedHashSet<>();
    +        clusterResolverLoader.forEach(resolver -> {
    +            resolver.configure(context);
    +            loadedClusterResolvers.add(resolver);
    +        });
    +        clusterResolvers = new ClusterResolvers(Collections.unmodifiableSet(loadedClusterResolvers), defaultClusterName);
    +    }
    +
    +
    +    private void initAtlasClient(ConfigurationContext context) throws IOException {
    +        List<String> urls = new ArrayList<>();
    +        parseAtlasUrls(context.getProperty(ATLAS_URLS), urls::add);
    +        final boolean isAtlasApiSecure = urls.stream().anyMatch(url -> url.toLowerCase().startsWith("https"));
    +        final String atlasAuthNMethod = context.getProperty(ATLAS_AUTHN_METHOD).getValue();
    +
    +        final String confDirStr = context.getProperty(ATLAS_CONF_DIR).evaluateAttributeExpressions().getValue();
    +        final File confDir = confDirStr != null && !confDirStr.isEmpty() ? new File(confDirStr) : null;
    +
    +        atlasProperties = new Properties();
    +        final File atlasPropertiesFile = new File(confDir, ATLAS_PROPERTIES_FILENAME);
    +
    +        final Boolean createAtlasConf = context.getProperty(ATLAS_CONF_CREATE).asBoolean();
    +        if (!createAtlasConf) {
    +            // Load existing properties file.
    +            if (atlasPropertiesFile.isFile()) {
    +                getLogger().info("Loading {}", new Object[]{atlasPropertiesFile});
    +                try (InputStream in = new FileInputStream(atlasPropertiesFile)) {
    +                    atlasProperties.load(in);
    +                }
    +            } else {
    +                final String fileInClasspath = "/" + ATLAS_PROPERTIES_FILENAME;
    +                try (InputStream in = AtlasNiFiFlowLineage.class.getResourceAsStream(fileInClasspath)) {
    +                    getLogger().info("Loading {} from classpath", new Object[]{fileInClasspath});
    +                    if (in == null) {
    +                        throw new ProcessException(String.format("Could not find %s in classpath." +
    +                                " Please add it to classpath," +
    +                                " or specify %s a directory containing Atlas properties file," +
    +                                " or enable %s to generate it.",
    +                                fileInClasspath, ATLAS_CONF_DIR.getDisplayName(), ATLAS_CONF_CREATE.getDisplayName()));
    +                    }
    +                    atlasProperties.load(in);
    +                }
    +            }
    +        }
    +
    +        // Resolve default cluster name.
    +        defaultClusterName = context.getProperty(ATLAS_DEFAULT_CLUSTER_NAME).evaluateAttributeExpressions().getValue();
    +        if (defaultClusterName == null || defaultClusterName.isEmpty()) {
    +            // If default cluster name is not specified by processor configuration, then load it from Atlas config.
    +            defaultClusterName = atlasProperties.getProperty(ATLAS_PROPERTY_CLUSTER_NAME);
    +        }
    +
    +        // If default cluster name is still not defined, processor should not be able to start.
    +        if (defaultClusterName == null || defaultClusterName.isEmpty()) {
    +            throw new ProcessException("Default cluster name is not defined.");
    +        }
    +
    +        final AtlasAuthN atlasAuthN = getAtlasAuthN(atlasAuthNMethod);
    +        atlasAuthN.configure(context);
    +
    +        // Create Atlas configuration file if necessary.
    +        if (createAtlasConf) {
    +
    +            atlasProperties.put(ATLAS_PROPERTY_CLUSTER_NAME, defaultClusterName);
    +            atlasProperties.put(ATLAS_PROPERTY_ENABLE_TLS, String.valueOf(isAtlasApiSecure));
    +
    +            setKafkaConfig(atlasProperties, context);
    +
    +            atlasAuthN.populateProperties(atlasProperties);
    +
    +            try (FileOutputStream fos = new FileOutputStream(atlasPropertiesFile)) {
    +                String ts = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSX")
    +                        .withZone(ZoneOffset.UTC)
    +                        .format(Instant.now());
    +                atlasProperties.store(fos, "Generated by Apache NiFi AtlasNiFiFlowLineage ReportingTask at " + ts);
    +            }
    +        }
    +
    +
    +        atlasClient = NiFiAtlasClient.getInstance();
    +        try {
    +            atlasClient.initialize(urls.toArray(new String[]{}), atlasAuthN, confDir);
    +        } catch (final NullPointerException e) {
    --- End diff --
    
    This seems odd to me, to catch a NullPointer - why would that be thrown? Should this be a more general Exception?


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157238842
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/StandardAnalysisContext.java ---
    @@ -0,0 +1,136 @@
    +/*
    + * 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.atlas.provenance;
    +
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.authorization.user.NiFiUser;
    +import org.apache.nifi.controller.status.ConnectionStatus;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.provenance.lineage.ComputeLineageResult;
    +import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +
    +public class StandardAnalysisContext implements AnalysisContext {
    +
    +    private final Logger logger = LoggerFactory.getLogger(StandardAnalysisContext.class);
    +    private final NiFiFlow nifiFlow;
    +    private final ClusterResolver clusterResolver;
    +    private final ProvenanceRepository provenanceRepository;
    +
    +    public StandardAnalysisContext(NiFiFlow nifiFlow, ClusterResolver clusterResolver,
    +                                   ProvenanceRepository provenanceRepository) {
    +        this.nifiFlow = nifiFlow;
    +        this.clusterResolver = clusterResolver;
    +        this.provenanceRepository = provenanceRepository;
    +    }
    +
    +    @Override
    +    public List<ConnectionStatus> findConnectionTo(String componentId) {
    +        return nifiFlow.getIncomingRelationShips(componentId);
    +    }
    +
    +    @Override
    +    public List<ConnectionStatus> findConnectionFrom(String componentId) {
    +        return nifiFlow.getOutgoingRelationShips(componentId);
    +    }
    +
    +    @Override
    +    public String getNiFiClusterName() {
    +        return nifiFlow.getClusterName();
    +    }
    +
    +    @Override
    +    public ClusterResolver getClusterResolver() {
    +        return clusterResolver;
    +    }
    +
    +    private ComputeLineageResult getLineageResult(long eventId, ComputeLineageSubmission submission) {
    +        final ComputeLineageResult result = submission.getResult();
    +        try {
    +            if (result.awaitCompletion(10, TimeUnit.SECONDS)) {
    +                return result;
    +            }
    +            logger.warn("Lineage query for {} timed out.", new Object[]{eventId});
    +        } catch (InterruptedException e) {
    +            logger.warn("Lineage query for {} was interrupted due to {}.", new Object[]{eventId, e}, e);
    +        } finally {
    +            submission.cancel();
    +        }
    +
    +        return null;
    +    }
    +
    +    @Override
    +    public ComputeLineageResult queryLineage(long eventId) {
    +        final ComputeLineageSubmission submission = provenanceRepository.submitLineageComputation(eventId, NIFI_USER);
    +        return getLineageResult(eventId, submission);
    +    }
    +
    +    public ComputeLineageResult findParents(long eventId) {
    +        final ComputeLineageSubmission submission = provenanceRepository.submitExpandParents(eventId, NIFI_USER);
    +        return getLineageResult(eventId, submission);
    +    }
    +
    +    // NOTE: This user is required to avoid NullPointerException at PersistentProvenanceRepository.submitLineageComputation
    +    private static final QueryNiFiUser NIFI_USER = new QueryNiFiUser();
    --- End diff --
    
    I see. What I am seeing is different from your description. I will try to find where the event level auth is implemented.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157208324
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/NiFiAtlasClient.java ---
    @@ -0,0 +1,537 @@
    +/*
    + * 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.atlas;
    +
    +import com.sun.jersey.api.client.UniformInterfaceException;
    +import com.sun.jersey.core.util.MultivaluedMapImpl;
    +import org.apache.atlas.ApplicationProperties;
    +import org.apache.atlas.AtlasClientV2;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.atlas.model.SearchFilter;
    +import org.apache.atlas.model.instance.AtlasEntity;
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +import org.apache.atlas.model.instance.EntityMutationResponse;
    +import org.apache.atlas.model.typedef.AtlasEntityDef;
    +import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
    +import org.apache.atlas.model.typedef.AtlasTypesDef;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.util.StringUtils;
    +import org.apache.nifi.util.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.ws.rs.core.MultivaluedMap;
    +import java.io.File;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.function.Function;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.nifi.atlas.AtlasUtils.findIdByQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.getComponentIdFromQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.toStr;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.AS_IS;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.CREATED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.DELETED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.UPDATED;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_DESCRIPTION;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_FLOW_PATHS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_GUID;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUEUES;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_TYPENAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_URL;
    +import static org.apache.nifi.atlas.NiFiTypes.ENTITIES;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW_PATH;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_INPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_OUTPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_QUEUE;
    +
    +public class NiFiAtlasClient {
    +
    +    private static final Logger logger = LoggerFactory.getLogger(NiFiAtlasClient.class);
    +
    +    private static NiFiAtlasClient nifiClient;
    +    private AtlasClientV2 atlasClient;
    +
    +    private NiFiAtlasClient() {
    +        super();
    +    }
    +
    +    public static NiFiAtlasClient getInstance() {
    +        if (nifiClient == null) {
    +            synchronized (NiFiAtlasClient.class) {
    +                if (nifiClient == null) {
    +                    nifiClient = new NiFiAtlasClient();
    +                }
    +            }
    +        }
    +        return nifiClient;
    +    }
    +
    +    public void initialize(final String[] baseUrls, final AtlasAuthN authN, final File atlasConfDir) {
    +
    +        synchronized (NiFiAtlasClient.class) {
    +
    +            if (atlasClient != null) {
    +                logger.info("{} had been setup but replacing it with new one.", atlasClient);
    +                ApplicationProperties.forceReload();
    +            }
    +
    +            if (atlasConfDir != null) {
    +                // If atlasConfDir is not set, atlas-application.properties will be searched under classpath.
    +                Properties props = System.getProperties();
    +                final String atlasConfProp = "atlas.conf";
    +                props.setProperty(atlasConfProp, atlasConfDir.getAbsolutePath());
    +                logger.debug("{} has been set to: {}", atlasConfProp, props.getProperty(atlasConfProp));
    +            }
    +
    +            atlasClient = authN.createClient(baseUrls);
    +
    +        }
    +    }
    +
    +    /**
    +     * This is an utility method to delete unused types.
    +     * Should be used during development or testing only.
    +     * @param typeNames to delete
    +     */
    +    void deleteTypeDefs(String ... typeNames) throws AtlasServiceException {
    +        final AtlasTypesDef existingTypeDef = getTypeDefs(typeNames);
    +        try {
    +            atlasClient.deleteAtlasTypeDefs(existingTypeDef);
    +        } catch (UniformInterfaceException e) {
    +            if (e.getResponse().getStatus() == 204) {
    +                // 204 is a successful response.
    +                // NOTE: However after executing this, Atlas should be restarted to work properly.
    +                logger.info("Deleted type defs: {}", existingTypeDef);
    +            } else {
    +                throw e;
    +            }
    +        }
    +    }
    +
    +    /**
    +     * @return True when required NiFi types are already created.
    +     */
    +    public boolean isNiFiTypeDefsRegistered() throws AtlasServiceException {
    +        final Set<String> typeNames = ENTITIES.keySet();
    +        final Map<String, AtlasEntityDef> existingDefs = getTypeDefs(typeNames.toArray(new String[typeNames.size()])).getEntityDefs().stream()
    +                .collect(Collectors.toMap(AtlasEntityDef::getName, Function.identity()));
    +        return typeNames.stream().allMatch(existingDefs::containsKey);
    +    }
    +
    +    /**
    +     * Create or update NiFi types in Atlas type system.
    +     * @param update If false, doesn't perform anything if there is existing type def for the name.
    +     */
    +    public void registerNiFiTypeDefs(boolean update) throws AtlasServiceException {
    +        final Set<String> typeNames = ENTITIES.keySet();
    +        final Map<String, AtlasEntityDef> existingDefs = getTypeDefs(typeNames.toArray(new String[typeNames.size()])).getEntityDefs().stream()
    +                .collect(Collectors.toMap(AtlasEntityDef::getName, Function.identity()));
    +
    +
    +        final AtomicBoolean shouldUpdate = new AtomicBoolean(false);
    +
    +        final AtlasTypesDef type = new AtlasTypesDef();
    +
    +        typeNames.stream().filter(typeName -> {
    +            final AtlasEntityDef existingDef = existingDefs.get(typeName);
    +            if (existingDef != null) {
    +                // type is already defined.
    +                if (!update) {
    +                    return false;
    +                }
    +                shouldUpdate.set(true);
    +            }
    +            return true;
    +        }).forEach(typeName -> {
    +            final NiFiTypes.EntityDefinition def = ENTITIES.get(typeName);
    +
    +            final AtlasEntityDef entity = new AtlasEntityDef();
    +            type.getEntityDefs().add(entity);
    +
    +            entity.setName(typeName);
    +
    +            Set<String> superTypes = new HashSet<>();
    +            List<AtlasAttributeDef> attributes = new ArrayList<>();
    +
    +            def.define(entity, superTypes, attributes);
    +
    +            entity.setSuperTypes(superTypes);
    +            entity.setAttributeDefs(attributes);
    +        });
    +
    +        // Create or Update.
    +        final AtlasTypesDef atlasTypeDefsResult = shouldUpdate.get()
    +                ? atlasClient.updateAtlasTypeDefs(type)
    +                : atlasClient.createAtlasTypeDefs(type);
    +        logger.debug("Result={}", atlasTypeDefsResult);
    +    }
    +
    +    private AtlasTypesDef getTypeDefs(String ... typeNames) throws AtlasServiceException {
    +        final AtlasTypesDef typeDefs = new AtlasTypesDef();
    +        for (int i = 0; i < typeNames.length; i++) {
    +            final MultivaluedMap<String, String> searchParams = new MultivaluedMapImpl();
    +            searchParams.add(SearchFilter.PARAM_NAME, typeNames[i]);
    +            final AtlasTypesDef typeDef = atlasClient.getAllTypeDefs(new SearchFilter(searchParams));
    +            typeDefs.getEntityDefs().addAll(typeDef.getEntityDefs());
    +        }
    +        logger.debug("typeDefs={}", typeDefs);
    +        return typeDefs;
    +    }
    +
    +    private Pattern FLOW_PATH_URL_PATTERN = Pattern.compile("^http.+processGroupId=([0-9a-z\\-]+).*$");
    +    /**
    +     * Fetch existing NiFiFlow entity from Atlas.
    +     * @param rootProcessGroupId The id of a NiFi flow root process group.
    +     * @param clusterName The cluster name of a flow.
    +     * @return A NiFiFlow instance filled with retrieved data from Atlas. Status objects are left blank, e.g. ProcessorStatus.
    +     * @throws AtlasServiceException Thrown if requesting to Atlas API failed, including when the flow is not found.
    +     */
    +    public NiFiFlow fetchNiFiFlow(String rootProcessGroupId, String clusterName) throws AtlasServiceException {
    +
    +        final String qualifiedName = AtlasUtils.toQualifiedName(clusterName, rootProcessGroupId);
    +        final AtlasObjectId flowId = new AtlasObjectId(TYPE_NIFI_FLOW, ATTR_QUALIFIED_NAME, qualifiedName);
    +        final AtlasEntity.AtlasEntityWithExtInfo nifiFlowExt = searchEntityDef(flowId);
    +
    +        if (nifiFlowExt == null || nifiFlowExt.getEntity() == null) {
    +            return null;
    +        }
    +
    +        final AtlasEntity nifiFlowEntity = nifiFlowExt.getEntity();
    +        final Map<String, Object> attributes = nifiFlowEntity.getAttributes();
    +        final NiFiFlow nifiFlow = new NiFiFlow(rootProcessGroupId);
    +        nifiFlow.setExEntity(nifiFlowEntity);
    +        nifiFlow.setFlowName(toStr(attributes.get(ATTR_NAME)));
    +        nifiFlow.setClusterName(clusterName);
    +        nifiFlow.setUrl(toStr(attributes.get(ATTR_URL)));
    +        nifiFlow.setDescription(toStr(attributes.get(ATTR_DESCRIPTION)));
    +
    +        nifiFlow.getQueues().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_QUEUES))));
    +        nifiFlow.getRootInputPortEntities().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_INPUT_PORTS))));
    +        nifiFlow.getRootOutputPortEntities().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_OUTPUT_PORTS))));
    +
    +        final Map<String, NiFiFlowPath> flowPaths = nifiFlow.getFlowPaths();
    +        final Map<AtlasObjectId, AtlasEntity> flowPathEntities = toQualifiedNameIds(toAtlasObjectIds(attributes.get(ATTR_FLOW_PATHS)));
    +
    +        for (AtlasEntity flowPathEntity : flowPathEntities.values()) {
    +            final String pathQualifiedName = toStr(flowPathEntity.getAttribute(ATTR_QUALIFIED_NAME));
    +            final NiFiFlowPath flowPath = new NiFiFlowPath(getComponentIdFromQualifiedName(pathQualifiedName));
    +            if (flowPathEntity.hasAttribute(ATTR_URL)) {
    +                final Matcher urlMatcher = FLOW_PATH_URL_PATTERN.matcher(toStr(flowPathEntity.getAttribute(ATTR_URL)));
    +                if (urlMatcher.matches()) {
    +                    flowPath.setGroupId(urlMatcher.group(1));
    +                }
    +            }
    +            flowPath.setExEntity(flowPathEntity);
    +            flowPath.setName(toStr(flowPathEntity.getAttribute(ATTR_NAME)));
    +            flowPath.getInputs().addAll(toQualifiedNameIds(toAtlasObjectIds(flowPathEntity.getAttribute(ATTR_INPUTS))).keySet());
    +            flowPath.getOutputs().addAll(toQualifiedNameIds(toAtlasObjectIds(flowPathEntity.getAttribute(ATTR_OUTPUTS))).keySet());
    +            flowPath.startTrackingChanges(nifiFlow);
    +
    +            flowPaths.put(flowPath.getId(), flowPath);
    +        }
    +
    +        nifiFlow.startTrackingChanges();
    +        return nifiFlow;
    +    }
    +
    +    @SuppressWarnings("unchecked")
    +    private List<AtlasObjectId> toAtlasObjectIds(Object _references) {
    +        if (_references == null) {
    +            return Collections.emptyList();
    +        }
    +        List<Map<String, Object>> references = (List<Map<String, Object>>) _references;
    +        return references.stream()
    +                .map(ref -> new AtlasObjectId(toStr(ref.get(ATTR_GUID)), toStr(ref.get(ATTR_TYPENAME)), ref))
    +                .collect(Collectors.toList());
    +    }
    +
    +    /**
    +     * <p>AtlasObjectIds returned from Atlas have GUID, but do not have qualifiedName, while ones created by the reporting task
    +     * do not have GUID, but qualifiedName. AtlasObjectId.equals returns false for this combination.
    +     * In order to match ids correctly, this method converts fetches actual entities from ids to get qualifiedName attribute.</p>
    +     *
    +     * <p>Also, AtlasObjectIds returned from Atlas does not have entity state.
    +     * If Atlas is configured to use soft-delete (default), deleted ids are still returned.
    +     * Fetched entities are used to determine whether an AtlasObjectId is still active or deleted.
    +     * Deleted entities will not be included in the result of this method.
    +     * </p>
    +     * @param ids to convert
    +     * @return AtlasObjectIds with qualifiedName
    +     */
    +    private Map<AtlasObjectId, AtlasEntity> toQualifiedNameIds(List<AtlasObjectId> ids) {
    +        if (ids == null) {
    +            return Collections.emptyMap();
    +        }
    +
    +        return ids.stream().distinct().map(id -> {
    +            try {
    +                final AtlasEntity.AtlasEntityWithExtInfo entityExt = searchEntityDef(id);
    +                final AtlasEntity entity = entityExt.getEntity();
    +                if (AtlasEntity.Status.DELETED.equals(entity.getStatus())) {
    +                    return null;
    +                }
    +                final Map<String, Object> uniqueAttrs = Collections.singletonMap(ATTR_QUALIFIED_NAME, entity.getAttribute(ATTR_QUALIFIED_NAME));
    +                return new Tuple<>(new AtlasObjectId(id.getGuid(), id.getTypeName(), uniqueAttrs), entity);
    +            } catch (AtlasServiceException e) {
    +                return null;
    +            }
    +        }).filter(Objects::nonNull).collect(Collectors.toMap(Tuple::getKey, Tuple::getValue));
    +    }
    +
    +    public void registerNiFiFlow(NiFiFlow nifiFlow) throws AtlasServiceException {
    +
    +        // Create parent flow entity, so that common properties are taken over.
    +        final AtlasEntity flowEntity = registerNiFiFlowEntity(nifiFlow);
    +
    +        // Create DataSet entities those are created by this NiFi flow.
    +        final Map<String, List<AtlasEntity>> updatedDataSetEntities = registerDataSetEntities(nifiFlow);
    +
    +        // Create path entities.
    +        final Set<AtlasObjectId> remainingPathIds = registerFlowPathEntities(nifiFlow);
    +
    +        // Update these attributes only if anything is created, updated or removed.
    +        boolean shouldUpdateNiFiFlow = nifiFlow.isMetadataUpdated();
    +        if (remainingPathIds != null) {
    +            flowEntity.setAttribute(ATTR_FLOW_PATHS, remainingPathIds);
    +            shouldUpdateNiFiFlow = true;
    +        }
    +        if (updatedDataSetEntities.containsKey(TYPE_NIFI_QUEUE)) {
    +            flowEntity.setAttribute(ATTR_QUEUES, updatedDataSetEntities.get(TYPE_NIFI_QUEUE));
    +            shouldUpdateNiFiFlow = true;
    +        }
    +        if (updatedDataSetEntities.containsKey(TYPE_NIFI_INPUT_PORT)) {
    +            flowEntity.setAttribute(ATTR_INPUT_PORTS, updatedDataSetEntities.get(TYPE_NIFI_INPUT_PORT));
    +            shouldUpdateNiFiFlow = true;
    +        }
    +        if (updatedDataSetEntities.containsKey(TYPE_NIFI_OUTPUT_PORT)) {
    +            flowEntity.setAttribute(ATTR_OUTPUT_PORTS, updatedDataSetEntities.get(TYPE_NIFI_OUTPUT_PORT));
    +            shouldUpdateNiFiFlow = true;
    +        }
    +
    +        logger.debug("### NiFi Flow Audit Logs START");
    --- End diff --
    
    Indeed. Thanks for the advice!


---

[GitHub] nifi issue #2335: NIFI-3709: Export NiFi flow dataset lineage to Apache Atla...

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

    https://github.com/apache/nifi/pull/2335
  
    Dear reviewers. This PR has more than 20K lines of code, it may be difficult to review it quickly. However, I wrote as much documentation as possible at additionalDetail.html of the NiFiAtlasFlowLineage reporting task, and hopefully it helps to make the review cycle quicker. Please read the doc first. Thanks for your time and effort in advance!


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157025778
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/resources/docs/org.apache.nifi.atlas.reporting.AtlasNiFiFlowLineage/additionalDetails.html ---
    @@ -0,0 +1,541 @@
    +<!DOCTYPE html>
    +<html lang="en">
    +<!--
    +  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.
    +-->
    +    <head>
    +        <meta charset="utf-8" />
    +        <title>AtlasNiFiFlowLineage</title>
    +        <link rel="stylesheet" href="/nifi-docs/css/component-usage.css" type="text/css" />
    +    </head>
    +
    +    <body>
    +        <h2>AtlasNiFiFlowLineage</h2>
    +
    +        Table of contents:
    +        <!-- TODO: Fix header tags most h4 should be h3 -->
    +        <ul>
    +            <li><a href="#how-it-works">Information reported to Atlas</a></li>
    +            <li><a href="#nifi-atlas-types">NiFi Atlas Types</a></li>
    +            <li><a href="#cluster-name">Cluster Name Resolution</a></li>
    +            <li><a href="#nifi-flow-structure">NiFi flow structure</a>
    +                <ul>
    +                    <li><a href="#path-separation">Path Separation Logic</a></li>
    +                </ul>
    +            </li>
    +            <li><a href="#nifi-data-lineage">NiFi data lineage</a>
    +                <ul>
    +                    <li><a href="#lineage-strategy">NiFi Lineage Strategy</a></li>
    +                    <li><a href="#provenance-events">NiFi Provenance Event Analysis</a></li>
    +                    <li><a href="#datasets-and-processors">Supported DataSets and Processors</a></li>
    +                </ul>
    +            </li>
    +            <li><a href="#runs-in-cluster">How it runs in NiFi cluster</a></li>
    +            <li><a href="#limitations">Limitations</a></li>
    +            <li><a href="#atlas-configs">Atlas Server Configurations</a></li>
    +            <li><a href="#atlas-emulator">Atlas Server Emulator</a></li>
    +        </ul>
    +
    +        <h3 id="how-it-works">Information reported to Atlas</h3>
    +        <p>This reporting task stores two types of NiFi flow information, 'NiFi flow structure' and 'NiFi data lineage'.</p>
    +
    +        <p>'NiFi flow structure' tells what components are running within a NiFi flow and how these are connected. It is reported by analyzing current NiFi flow structure, specifically NiFi component relationships.</p>
    +
    +        <p>'NiFi data lineage' tells what part of NiFi flow interacts with different DataSets such as HDFS files or Hive tables ... etc. It is reported by analyzing NiFi provenance events.</p>
    +
    +        <object data="nifi_atlas.svg" type="image/svg+xml" width="60%"></object>
    +
    +        <p>Technically each information is sent using different protocol, Atlas REST API v2, and Notification via a Kafka topic as shown in above image.</p>
    +
    +
    +        <p>As both information types use the same <a href="#nifi-atlas-types">NiFi Atlas Types</a> and <a href="#cluster-name">Cluster Name Resolution</a> concepts, it is recommended to start reading those sections first.</p>
    +
    +        <h4 id="nifi-atlas-types">NiFi Atlas Types</h4>
    +
    +        <p>This reporting task creates following NiFi specific types in Atlas Type system when it runs if these type definitions are not found.</p>
    +
    +        <p>Green boxes represent sub-types of DataSet and blue ones are sub-types of Process. Gray lines represent entity ownership.
    +        Red lines represent lineage.</p>
    +
    +        <object data="nifi_types.svg" type="image/svg+xml" width="60%"></object>
    +
    +        <ul>
    +            <li>nifi_flow
    +                <p>Represents a NiFI data flow.</p>
    +                <p>As shown in the above diagram, nifi_flow owns other nifi_component types.
    +                    This owning relationship is defined by Atlas 'owned' constraint so that when a 'nifi_flow' entity is removed, all owned NiFi component entities are removed in cascading manner.</p>
    +                <p>When this reporting task runs, it analyzes and traverse the entire flow structure, and create NiFi component entities in Atlas.
    +                    At later runs, it compares the current flow structure with the one stored in Atlas to figure out if any changes has been made since the last time the flow was reported. The reporting task updates NiFi component entities in Atlas if needed.<p>
    +                <p>NiFi components those are removed from a NiFi flow also get deleted from Atlas.
    +                    However those entities can still be seen in Atlas search results or lineage graphs since Atlas uses 'Soft Delete' by default.
    +                    See <a href="#delete-handler">Atlas Delete Handler</a> for further detail.</p>
    +            </li>
    +            Attributes:
    +            <ul>
    +                <li>qualifiedName: Root ProcessGroup ID@clusterName (e.g. 86420a14-2fab-3e1e-4331-fb6ab42f58e0@cl1)</li>
    +                <li>name: Name of the Root ProcessGroup.</li>
    +                <li>url: URL of the NiFi instance. This can be specified via reporting task 'NiFi URL for Atlas' property.</li>
    +            </ul>
    +        </ul>
    +        <ul>
    +            <li>nifi_flow_path <p>Part of a NiFi data flow containing one or more processing NiFi components such as Processors and RemoteGroupPorts. The reporting task divides a NiFi flow into multiple flow paths. See <a href="#path-separation">Path Separation Logic</a> for details.</p></li>
    +            Attributes:
    +            <ul>
    +                <li>qualifiedName: The first NiFi component Id in a path@clusterName (e.g. 529e6722-9b49-3b66-9c94-00da9863ca2d@cl1)</li>
    +                <li>name: NiFi component namess within a path are concatenated (e.g. GenerateFlowFile, PutFile, LogAttribute)</li>
    +                <li>url: A deep link to the first NiFi component in corresponding NiFi UI</li>
    +            </ul>
    +        </ul>
    +        <ul>
    +            <!-- TODO: link to S2S details -->
    --- End diff --
    
    This was intended to be resolved?


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157029294
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/NiFIAtlasHook.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.atlas;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.atlas.hook.AtlasHook;
    +import org.apache.atlas.model.instance.AtlasEntity;
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +import org.apache.atlas.notification.hook.HookNotification.EntityPartialUpdateRequest;
    +import org.apache.atlas.notification.hook.HookNotification.HookNotificationMessage;
    +import org.apache.atlas.typesystem.Referenceable;
    +import org.apache.atlas.typesystem.persistence.Id;
    +import org.apache.nifi.atlas.provenance.lineage.LineageContext;
    +import org.apache.nifi.util.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.LinkedHashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.atlas.notification.hook.HookNotification.HookNotificationType.ENTITY_PARTIAL_UPDATE;
    +import static org.apache.nifi.atlas.AtlasUtils.toTypedQualifiedName;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_GUID;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_TYPENAME;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW_PATH;
    +
    +/**
    + * This class is not thread-safe as it holds uncommitted notification messages within instance.
    + * {@link #addMessage(HookNotificationMessage)} and {@link #commitMessages()} should be used serially from a single thread.
    + */
    +public class NiFIAtlasHook extends AtlasHook implements LineageContext {
    --- End diff --
    
    The class name follows a different capitalization pattern than the others. Perhaps a typo?
    NiFIAtlasHook vs NifiAtlasHook.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157220655
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/NiFiAtlasClient.java ---
    @@ -0,0 +1,537 @@
    +/*
    + * 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.atlas;
    +
    +import com.sun.jersey.api.client.UniformInterfaceException;
    +import com.sun.jersey.core.util.MultivaluedMapImpl;
    +import org.apache.atlas.ApplicationProperties;
    +import org.apache.atlas.AtlasClientV2;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.atlas.model.SearchFilter;
    +import org.apache.atlas.model.instance.AtlasEntity;
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +import org.apache.atlas.model.instance.EntityMutationResponse;
    +import org.apache.atlas.model.typedef.AtlasEntityDef;
    +import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
    +import org.apache.atlas.model.typedef.AtlasTypesDef;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.util.StringUtils;
    +import org.apache.nifi.util.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.ws.rs.core.MultivaluedMap;
    +import java.io.File;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.function.Function;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.nifi.atlas.AtlasUtils.findIdByQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.getComponentIdFromQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.toStr;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.AS_IS;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.CREATED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.DELETED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.UPDATED;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_DESCRIPTION;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_FLOW_PATHS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_GUID;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUEUES;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_TYPENAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_URL;
    +import static org.apache.nifi.atlas.NiFiTypes.ENTITIES;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW_PATH;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_INPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_OUTPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_QUEUE;
    +
    +public class NiFiAtlasClient {
    +
    +    private static final Logger logger = LoggerFactory.getLogger(NiFiAtlasClient.class);
    +
    +    private static NiFiAtlasClient nifiClient;
    +    private AtlasClientV2 atlasClient;
    +
    +    private NiFiAtlasClient() {
    +        super();
    +    }
    +
    +    public static NiFiAtlasClient getInstance() {
    +        if (nifiClient == null) {
    +            synchronized (NiFiAtlasClient.class) {
    +                if (nifiClient == null) {
    +                    nifiClient = new NiFiAtlasClient();
    +                }
    +            }
    +        }
    +        return nifiClient;
    +    }
    +
    +    public void initialize(final String[] baseUrls, final AtlasAuthN authN, final File atlasConfDir) {
    +
    +        synchronized (NiFiAtlasClient.class) {
    +
    +            if (atlasClient != null) {
    +                logger.info("{} had been setup but replacing it with new one.", atlasClient);
    +                ApplicationProperties.forceReload();
    +            }
    +
    +            if (atlasConfDir != null) {
    +                // If atlasConfDir is not set, atlas-application.properties will be searched under classpath.
    +                Properties props = System.getProperties();
    +                final String atlasConfProp = "atlas.conf";
    +                props.setProperty(atlasConfProp, atlasConfDir.getAbsolutePath());
    +                logger.debug("{} has been set to: {}", atlasConfProp, props.getProperty(atlasConfProp));
    +            }
    +
    +            atlasClient = authN.createClient(baseUrls);
    +
    +        }
    +    }
    +
    +    /**
    +     * This is an utility method to delete unused types.
    +     * Should be used during development or testing only.
    +     * @param typeNames to delete
    +     */
    +    void deleteTypeDefs(String ... typeNames) throws AtlasServiceException {
    +        final AtlasTypesDef existingTypeDef = getTypeDefs(typeNames);
    +        try {
    +            atlasClient.deleteAtlasTypeDefs(existingTypeDef);
    +        } catch (UniformInterfaceException e) {
    +            if (e.getResponse().getStatus() == 204) {
    +                // 204 is a successful response.
    +                // NOTE: However after executing this, Atlas should be restarted to work properly.
    --- End diff --
    
    I reported several findings to Atlas project while I work on this integration. However, I think I haven't share this particular issue yet. Thanks, I will do. This method is only used by ITTest class, though.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157208066
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/StandardAnalysisContext.java ---
    @@ -0,0 +1,136 @@
    +/*
    + * 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.atlas.provenance;
    +
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.authorization.user.NiFiUser;
    +import org.apache.nifi.controller.status.ConnectionStatus;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.provenance.lineage.ComputeLineageResult;
    +import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +
    +public class StandardAnalysisContext implements AnalysisContext {
    +
    +    private final Logger logger = LoggerFactory.getLogger(StandardAnalysisContext.class);
    +    private final NiFiFlow nifiFlow;
    +    private final ClusterResolver clusterResolver;
    +    private final ProvenanceRepository provenanceRepository;
    +
    +    public StandardAnalysisContext(NiFiFlow nifiFlow, ClusterResolver clusterResolver,
    +                                   ProvenanceRepository provenanceRepository) {
    +        this.nifiFlow = nifiFlow;
    +        this.clusterResolver = clusterResolver;
    +        this.provenanceRepository = provenanceRepository;
    +    }
    +
    +    @Override
    +    public List<ConnectionStatus> findConnectionTo(String componentId) {
    +        return nifiFlow.getIncomingRelationShips(componentId);
    +    }
    +
    +    @Override
    +    public List<ConnectionStatus> findConnectionFrom(String componentId) {
    +        return nifiFlow.getOutgoingRelationShips(componentId);
    +    }
    +
    +    @Override
    +    public String getNiFiClusterName() {
    +        return nifiFlow.getClusterName();
    +    }
    +
    +    @Override
    +    public ClusterResolver getClusterResolver() {
    +        return clusterResolver;
    +    }
    +
    +    private ComputeLineageResult getLineageResult(long eventId, ComputeLineageSubmission submission) {
    +        final ComputeLineageResult result = submission.getResult();
    +        try {
    +            if (result.awaitCompletion(10, TimeUnit.SECONDS)) {
    +                return result;
    +            }
    +            logger.warn("Lineage query for {} timed out.", new Object[]{eventId});
    +        } catch (InterruptedException e) {
    +            logger.warn("Lineage query for {} was interrupted due to {}.", new Object[]{eventId, e}, e);
    +        } finally {
    +            submission.cancel();
    +        }
    +
    +        return null;
    +    }
    +
    +    @Override
    +    public ComputeLineageResult queryLineage(long eventId) {
    +        final ComputeLineageSubmission submission = provenanceRepository.submitLineageComputation(eventId, NIFI_USER);
    +        return getLineageResult(eventId, submission);
    +    }
    +
    +    public ComputeLineageResult findParents(long eventId) {
    +        final ComputeLineageSubmission submission = provenanceRepository.submitExpandParents(eventId, NIFI_USER);
    +        return getLineageResult(eventId, submission);
    +    }
    +
    +    // NOTE: This user is required to avoid NullPointerException at PersistentProvenanceRepository.submitLineageComputation
    +    private static final QueryNiFiUser NIFI_USER = new QueryNiFiUser();
    --- End diff --
    
    Actually this user can be anything. Since NiFI authentication and authorization is done at the web request layer, when the provenance API is called directly, no authentication nor authorization take place. I confirmed that provenance query work in secured NiFi environment. Is that approach fine?


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157214953
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/AtlasNiFiFlowLineage.java ---
    @@ -0,0 +1,714 @@
    +/*
    + * 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.atlas.reporting;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.common.config.SslConfigs;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
    +import org.apache.nifi.annotation.behavior.Stateful;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.atlas.NiFIAtlasHook;
    +import org.apache.nifi.atlas.NiFiAtlasClient;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowAnalyzer;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.StandardAnalysisContext;
    +import org.apache.nifi.atlas.provenance.lineage.CompleteFlowPathLineage;
    +import org.apache.nifi.atlas.provenance.lineage.LineageStrategy;
    +import org.apache.nifi.atlas.provenance.lineage.SimpleFlowPathLineage;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.atlas.resolver.ClusterResolvers;
    +import org.apache.nifi.atlas.resolver.RegexClusterResolver;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.atlas.security.Basic;
    +import org.apache.nifi.atlas.security.Kerberos;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.context.PropertyContext;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.reporting.AbstractReportingTask;
    +import org.apache.nifi.reporting.EventAccess;
    +import org.apache.nifi.reporting.ReportingContext;
    +import org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.time.Instant;
    +import java.time.ZoneOffset;
    +import java.time.format.DateTimeFormatter;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.LinkedHashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.ServiceLoader;
    +import java.util.Set;
    +import java.util.function.Consumer;
    +import java.util.stream.Stream;
    +
    +import static org.apache.commons.lang3.StringUtils.isEmpty;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_BATCH_SIZE;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_START_POSITION;
    +
    +@Tags({"atlas", "lineage"})
    +@CapabilityDescription("Publishes NiFi flow data set level lineage to Apache Atlas." +
    +        " By reporting flow information to Atlas, an end-to-end Process and DataSet lineage such as across NiFi environments and other systems" +
    +        " connected by technologies, for example NiFi Site-to-Site, Kafka topic or Hive tables." +
    +        " There are limitations and required configurations for both NiFi and Atlas. See 'Additional Details' for further description.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +@DynamicProperty(name = "hostnamePattern.<ClusterName>", value = "hostname Regex patterns", description = RegexClusterResolver.PATTERN_PROPERTY_PREFIX_DESC)
    +// In order for each reporting task instance to have its own static objects such as KafkaNotification.
    +@RequiresInstanceClassLoading
    +public class AtlasNiFiFlowLineage extends AbstractReportingTask {
    --- End diff --
    
    Agreed. Thanks for your suggestion!


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157216281
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/AtlasNiFiFlowLineage.java ---
    @@ -0,0 +1,714 @@
    +/*
    + * 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.atlas.reporting;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.common.config.SslConfigs;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
    +import org.apache.nifi.annotation.behavior.Stateful;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.atlas.NiFIAtlasHook;
    +import org.apache.nifi.atlas.NiFiAtlasClient;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowAnalyzer;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.StandardAnalysisContext;
    +import org.apache.nifi.atlas.provenance.lineage.CompleteFlowPathLineage;
    +import org.apache.nifi.atlas.provenance.lineage.LineageStrategy;
    +import org.apache.nifi.atlas.provenance.lineage.SimpleFlowPathLineage;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.atlas.resolver.ClusterResolvers;
    +import org.apache.nifi.atlas.resolver.RegexClusterResolver;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.atlas.security.Basic;
    +import org.apache.nifi.atlas.security.Kerberos;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.context.PropertyContext;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.reporting.AbstractReportingTask;
    +import org.apache.nifi.reporting.EventAccess;
    +import org.apache.nifi.reporting.ReportingContext;
    +import org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.time.Instant;
    +import java.time.ZoneOffset;
    +import java.time.format.DateTimeFormatter;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.LinkedHashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.ServiceLoader;
    +import java.util.Set;
    +import java.util.function.Consumer;
    +import java.util.stream.Stream;
    +
    +import static org.apache.commons.lang3.StringUtils.isEmpty;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_BATCH_SIZE;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_START_POSITION;
    +
    +@Tags({"atlas", "lineage"})
    +@CapabilityDescription("Publishes NiFi flow data set level lineage to Apache Atlas." +
    +        " By reporting flow information to Atlas, an end-to-end Process and DataSet lineage such as across NiFi environments and other systems" +
    +        " connected by technologies, for example NiFi Site-to-Site, Kafka topic or Hive tables." +
    +        " There are limitations and required configurations for both NiFi and Atlas. See 'Additional Details' for further description.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +@DynamicProperty(name = "hostnamePattern.<ClusterName>", value = "hostname Regex patterns", description = RegexClusterResolver.PATTERN_PROPERTY_PREFIX_DESC)
    +// In order for each reporting task instance to have its own static objects such as KafkaNotification.
    +@RequiresInstanceClassLoading
    +public class AtlasNiFiFlowLineage extends AbstractReportingTask {
    +
    +    static final PropertyDescriptor ATLAS_URLS = new PropertyDescriptor.Builder()
    +            .name("atlas-urls")
    +            .displayName("Atlas URLs")
    +            .description("Comma separated URL of Atlas Servers" +
    +                    " (e.g. http://atlas-server-hostname:21000 or https://atlas-server-hostname:21443)." +
    +                    " For accessing Atlas behind Knox gateway, specify Knox gateway URL" +
    +                    " (e.g. https://knox-hostname:8443/gateway/{topology-name}/atlas).")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final AllowableValue ATLAS_AUTHN_BASIC = new AllowableValue("basic", "Basic", "Use username and password.");
    +    static final AllowableValue ATLAS_AUTHN_KERBEROS = new AllowableValue("kerberos", "Kerberos", "Use Kerberos keytab file.");
    +    static final PropertyDescriptor ATLAS_AUTHN_METHOD = new PropertyDescriptor.Builder()
    +            .name("atlas-authentication-method")
    +            .displayName("Atlas Authentication Method")
    +            .description("Specify how to authenticate this reporting task to Atlas server.")
    +            .required(true)
    +            .allowableValues(ATLAS_AUTHN_BASIC, ATLAS_AUTHN_KERBEROS)
    +            .defaultValue(ATLAS_AUTHN_BASIC.getValue())
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_USER = new PropertyDescriptor.Builder()
    +            .name("atlas-username")
    +            .displayName("Atlas Username")
    +            .description("User name to communicate with Atlas.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("atlas-password")
    +            .displayName("Atlas Password")
    +            .description("Password to communicate with Atlas.")
    +            .required(false)
    +            .sensitive(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final PropertyDescriptor ATLAS_CONF_DIR = new PropertyDescriptor.Builder()
    +            .name("atlas-conf-dir")
    +            .displayName("Atlas Configuration Directory")
    +            .description("Directory path that contains 'atlas-application.properties' file." +
    +                    " If not specified and 'Create Atlas Configuration File' is disabled," +
    +                    " then, 'atlas-application.properties' file under root classpath is used.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_NIFI_URL = new PropertyDescriptor.Builder()
    +            .name("atlas-nifi-url")
    +            .displayName("NiFi URL for Atlas")
    +            .description("NiFi URL is used in Atlas to represent this NiFi cluster (or standalone instance)." +
    +                    " It is recommended to use one that can be accessible remotely instead of using 'localhost'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.URL_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_DEFAULT_CLUSTER_NAME = new PropertyDescriptor.Builder()
    +            .name("atlas-default-cluster-name")
    +            .displayName("Atlas Default Cluster Name")
    +            .description("Cluster name for Atlas entities reported by this ReportingTask." +
    +                    " If not specified, 'atlas.cluster.name' in Atlas Configuration File is used." +
    +                    " Cluster name mappings can be configured by user defined properties." +
    +                    " See additional detail for detail.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final PropertyDescriptor ATLAS_CONF_CREATE = new PropertyDescriptor.Builder()
    +            .name("atlas-conf-create")
    +            .displayName("Create Atlas Configuration File")
    +            .description("If enabled, 'atlas-application.properties' file will be created in 'Atlas Configuration Directory'" +
    +                    " automatically when this processor starts." +
    +                    " Note that the existing configuration file will be overwritten.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .allowableValues("true", "false")
    +            .defaultValue("false")
    +            .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +            .name("ssl-context-service")
    +            .displayName("SSL Context Service")
    +            .description("Specifies the SSL Context Service to use for communicating with Atlas and Kafka.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +    static final PropertyDescriptor KAFKA_BOOTSTRAP_SERVERS = new PropertyDescriptor.Builder()
    +            .name("kafka-bootstrap-servers")
    +            .displayName("Kafka Bootstrap Servers")
    +            .description("Kafka Bootstrap Servers to send Atlas hook notification messages based on NiFi provenance events." +
    +                    " E.g. 'localhost:9092'" +
    +                    " NOTE: Once this reporting task has started, restarting NiFi is required to changed this property" +
    --- End diff --
    
    I have shared this issue with Atlas project.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157031810
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/AtlasUtils.java ---
    @@ -0,0 +1,77 @@
    +/*
    + * 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.atlas;
    +
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +
    +import java.util.List;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +
    +public class AtlasUtils {
    +
    +    public static String toStr(Object obj) {
    +        return obj != null ? obj.toString() : null;
    +    }
    +
    +
    +    public static boolean isGuidAssigned(String guid) {
    +        return guid != null && !guid.startsWith("-");
    --- End diff --
    
    The guid starting with a dash is an internal detail of the AtlasEntity class and might change. Checking for null should be sufficient


---

[GitHub] nifi issue #2335: NIFI-3709: Export NiFi flow dataset lineage to Apache Atla...

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

    https://github.com/apache/nifi/pull/2335
  
    @ijokarumawak yes I believe your understanding to be correct. Thanks for the clarifications! In terms of code I am good with the PR. Just want to test this against a live instance and as soon as I can confirm that all works well, I can push to master.
    
    However, I do see that this nar alone is > 60 MB. For now, I think I it would be best to add a Maven profile for activating this so that those who don't need this don't have to pull in the extra weight. Is that OK with you?


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157005140
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/AtlasNiFiFlowLineage.java ---
    @@ -0,0 +1,714 @@
    +/*
    + * 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.atlas.reporting;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.common.config.SslConfigs;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
    +import org.apache.nifi.annotation.behavior.Stateful;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.atlas.NiFIAtlasHook;
    +import org.apache.nifi.atlas.NiFiAtlasClient;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowAnalyzer;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.StandardAnalysisContext;
    +import org.apache.nifi.atlas.provenance.lineage.CompleteFlowPathLineage;
    +import org.apache.nifi.atlas.provenance.lineage.LineageStrategy;
    +import org.apache.nifi.atlas.provenance.lineage.SimpleFlowPathLineage;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.atlas.resolver.ClusterResolvers;
    +import org.apache.nifi.atlas.resolver.RegexClusterResolver;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.atlas.security.Basic;
    +import org.apache.nifi.atlas.security.Kerberos;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.context.PropertyContext;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.reporting.AbstractReportingTask;
    +import org.apache.nifi.reporting.EventAccess;
    +import org.apache.nifi.reporting.ReportingContext;
    +import org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.time.Instant;
    +import java.time.ZoneOffset;
    +import java.time.format.DateTimeFormatter;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.LinkedHashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.ServiceLoader;
    +import java.util.Set;
    +import java.util.function.Consumer;
    +import java.util.stream.Stream;
    +
    +import static org.apache.commons.lang3.StringUtils.isEmpty;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_BATCH_SIZE;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_START_POSITION;
    +
    +@Tags({"atlas", "lineage"})
    +@CapabilityDescription("Publishes NiFi flow data set level lineage to Apache Atlas." +
    +        " By reporting flow information to Atlas, an end-to-end Process and DataSet lineage such as across NiFi environments and other systems" +
    +        " connected by technologies, for example NiFi Site-to-Site, Kafka topic or Hive tables." +
    +        " There are limitations and required configurations for both NiFi and Atlas. See 'Additional Details' for further description.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +@DynamicProperty(name = "hostnamePattern.<ClusterName>", value = "hostname Regex patterns", description = RegexClusterResolver.PATTERN_PROPERTY_PREFIX_DESC)
    +// In order for each reporting task instance to have its own static objects such as KafkaNotification.
    +@RequiresInstanceClassLoading
    +public class AtlasNiFiFlowLineage extends AbstractReportingTask {
    --- End diff --
    
    I wonder if a more consistent name may be ReportLineageToAtlas, following with the <Verb><Noun> naming scheme


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157222944
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/NiFiAtlasClient.java ---
    @@ -0,0 +1,537 @@
    +/*
    + * 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.atlas;
    +
    +import com.sun.jersey.api.client.UniformInterfaceException;
    +import com.sun.jersey.core.util.MultivaluedMapImpl;
    +import org.apache.atlas.ApplicationProperties;
    +import org.apache.atlas.AtlasClientV2;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.atlas.model.SearchFilter;
    +import org.apache.atlas.model.instance.AtlasEntity;
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +import org.apache.atlas.model.instance.EntityMutationResponse;
    +import org.apache.atlas.model.typedef.AtlasEntityDef;
    +import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
    +import org.apache.atlas.model.typedef.AtlasTypesDef;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.util.StringUtils;
    +import org.apache.nifi.util.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.ws.rs.core.MultivaluedMap;
    +import java.io.File;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.function.Function;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.nifi.atlas.AtlasUtils.findIdByQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.getComponentIdFromQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.toStr;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.AS_IS;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.CREATED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.DELETED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.UPDATED;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_DESCRIPTION;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_FLOW_PATHS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_GUID;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUEUES;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_TYPENAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_URL;
    +import static org.apache.nifi.atlas.NiFiTypes.ENTITIES;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW_PATH;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_INPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_OUTPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_QUEUE;
    +
    +public class NiFiAtlasClient {
    +
    +    private static final Logger logger = LoggerFactory.getLogger(NiFiAtlasClient.class);
    +
    +    private static NiFiAtlasClient nifiClient;
    +    private AtlasClientV2 atlasClient;
    +
    +    private NiFiAtlasClient() {
    +        super();
    +    }
    +
    +    public static NiFiAtlasClient getInstance() {
    +        if (nifiClient == null) {
    +            synchronized (NiFiAtlasClient.class) {
    +                if (nifiClient == null) {
    +                    nifiClient = new NiFiAtlasClient();
    +                }
    +            }
    +        }
    +        return nifiClient;
    +    }
    +
    +    public void initialize(final String[] baseUrls, final AtlasAuthN authN, final File atlasConfDir) {
    +
    +        synchronized (NiFiAtlasClient.class) {
    +
    +            if (atlasClient != null) {
    +                logger.info("{} had been setup but replacing it with new one.", atlasClient);
    +                ApplicationProperties.forceReload();
    +            }
    +
    +            if (atlasConfDir != null) {
    +                // If atlasConfDir is not set, atlas-application.properties will be searched under classpath.
    +                Properties props = System.getProperties();
    +                final String atlasConfProp = "atlas.conf";
    +                props.setProperty(atlasConfProp, atlasConfDir.getAbsolutePath());
    +                logger.debug("{} has been set to: {}", atlasConfProp, props.getProperty(atlasConfProp));
    +            }
    +
    +            atlasClient = authN.createClient(baseUrls);
    +
    +        }
    +    }
    +
    +    /**
    +     * This is an utility method to delete unused types.
    +     * Should be used during development or testing only.
    +     * @param typeNames to delete
    +     */
    +    void deleteTypeDefs(String ... typeNames) throws AtlasServiceException {
    +        final AtlasTypesDef existingTypeDef = getTypeDefs(typeNames);
    +        try {
    +            atlasClient.deleteAtlasTypeDefs(existingTypeDef);
    +        } catch (UniformInterfaceException e) {
    +            if (e.getResponse().getStatus() == 204) {
    +                // 204 is a successful response.
    +                // NOTE: However after executing this, Atlas should be restarted to work properly.
    +                logger.info("Deleted type defs: {}", existingTypeDef);
    +            } else {
    +                throw e;
    +            }
    +        }
    +    }
    +
    +    /**
    +     * @return True when required NiFi types are already created.
    +     */
    +    public boolean isNiFiTypeDefsRegistered() throws AtlasServiceException {
    +        final Set<String> typeNames = ENTITIES.keySet();
    +        final Map<String, AtlasEntityDef> existingDefs = getTypeDefs(typeNames.toArray(new String[typeNames.size()])).getEntityDefs().stream()
    +                .collect(Collectors.toMap(AtlasEntityDef::getName, Function.identity()));
    +        return typeNames.stream().allMatch(existingDefs::containsKey);
    +    }
    +
    +    /**
    +     * Create or update NiFi types in Atlas type system.
    +     * @param update If false, doesn't perform anything if there is existing type def for the name.
    +     */
    +    public void registerNiFiTypeDefs(boolean update) throws AtlasServiceException {
    +        final Set<String> typeNames = ENTITIES.keySet();
    +        final Map<String, AtlasEntityDef> existingDefs = getTypeDefs(typeNames.toArray(new String[typeNames.size()])).getEntityDefs().stream()
    +                .collect(Collectors.toMap(AtlasEntityDef::getName, Function.identity()));
    +
    +
    +        final AtomicBoolean shouldUpdate = new AtomicBoolean(false);
    +
    +        final AtlasTypesDef type = new AtlasTypesDef();
    +
    +        typeNames.stream().filter(typeName -> {
    +            final AtlasEntityDef existingDef = existingDefs.get(typeName);
    +            if (existingDef != null) {
    +                // type is already defined.
    +                if (!update) {
    +                    return false;
    +                }
    +                shouldUpdate.set(true);
    +            }
    +            return true;
    +        }).forEach(typeName -> {
    +            final NiFiTypes.EntityDefinition def = ENTITIES.get(typeName);
    +
    +            final AtlasEntityDef entity = new AtlasEntityDef();
    +            type.getEntityDefs().add(entity);
    +
    +            entity.setName(typeName);
    +
    +            Set<String> superTypes = new HashSet<>();
    +            List<AtlasAttributeDef> attributes = new ArrayList<>();
    +
    +            def.define(entity, superTypes, attributes);
    +
    +            entity.setSuperTypes(superTypes);
    +            entity.setAttributeDefs(attributes);
    +        });
    +
    +        // Create or Update.
    +        final AtlasTypesDef atlasTypeDefsResult = shouldUpdate.get()
    +                ? atlasClient.updateAtlasTypeDefs(type)
    +                : atlasClient.createAtlasTypeDefs(type);
    +        logger.debug("Result={}", atlasTypeDefsResult);
    +    }
    +
    +    private AtlasTypesDef getTypeDefs(String ... typeNames) throws AtlasServiceException {
    +        final AtlasTypesDef typeDefs = new AtlasTypesDef();
    +        for (int i = 0; i < typeNames.length; i++) {
    +            final MultivaluedMap<String, String> searchParams = new MultivaluedMapImpl();
    +            searchParams.add(SearchFilter.PARAM_NAME, typeNames[i]);
    +            final AtlasTypesDef typeDef = atlasClient.getAllTypeDefs(new SearchFilter(searchParams));
    +            typeDefs.getEntityDefs().addAll(typeDef.getEntityDefs());
    +        }
    +        logger.debug("typeDefs={}", typeDefs);
    +        return typeDefs;
    +    }
    +
    +    private Pattern FLOW_PATH_URL_PATTERN = Pattern.compile("^http.+processGroupId=([0-9a-z\\-]+).*$");
    +    /**
    +     * Fetch existing NiFiFlow entity from Atlas.
    +     * @param rootProcessGroupId The id of a NiFi flow root process group.
    +     * @param clusterName The cluster name of a flow.
    +     * @return A NiFiFlow instance filled with retrieved data from Atlas. Status objects are left blank, e.g. ProcessorStatus.
    +     * @throws AtlasServiceException Thrown if requesting to Atlas API failed, including when the flow is not found.
    +     */
    +    public NiFiFlow fetchNiFiFlow(String rootProcessGroupId, String clusterName) throws AtlasServiceException {
    +
    +        final String qualifiedName = AtlasUtils.toQualifiedName(clusterName, rootProcessGroupId);
    +        final AtlasObjectId flowId = new AtlasObjectId(TYPE_NIFI_FLOW, ATTR_QUALIFIED_NAME, qualifiedName);
    +        final AtlasEntity.AtlasEntityWithExtInfo nifiFlowExt = searchEntityDef(flowId);
    +
    +        if (nifiFlowExt == null || nifiFlowExt.getEntity() == null) {
    +            return null;
    +        }
    +
    +        final AtlasEntity nifiFlowEntity = nifiFlowExt.getEntity();
    +        final Map<String, Object> attributes = nifiFlowEntity.getAttributes();
    +        final NiFiFlow nifiFlow = new NiFiFlow(rootProcessGroupId);
    +        nifiFlow.setExEntity(nifiFlowEntity);
    +        nifiFlow.setFlowName(toStr(attributes.get(ATTR_NAME)));
    +        nifiFlow.setClusterName(clusterName);
    +        nifiFlow.setUrl(toStr(attributes.get(ATTR_URL)));
    +        nifiFlow.setDescription(toStr(attributes.get(ATTR_DESCRIPTION)));
    +
    +        nifiFlow.getQueues().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_QUEUES))));
    +        nifiFlow.getRootInputPortEntities().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_INPUT_PORTS))));
    +        nifiFlow.getRootOutputPortEntities().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_OUTPUT_PORTS))));
    +
    +        final Map<String, NiFiFlowPath> flowPaths = nifiFlow.getFlowPaths();
    +        final Map<AtlasObjectId, AtlasEntity> flowPathEntities = toQualifiedNameIds(toAtlasObjectIds(attributes.get(ATTR_FLOW_PATHS)));
    +
    +        for (AtlasEntity flowPathEntity : flowPathEntities.values()) {
    +            final String pathQualifiedName = toStr(flowPathEntity.getAttribute(ATTR_QUALIFIED_NAME));
    +            final NiFiFlowPath flowPath = new NiFiFlowPath(getComponentIdFromQualifiedName(pathQualifiedName));
    +            if (flowPathEntity.hasAttribute(ATTR_URL)) {
    +                final Matcher urlMatcher = FLOW_PATH_URL_PATTERN.matcher(toStr(flowPathEntity.getAttribute(ATTR_URL)));
    +                if (urlMatcher.matches()) {
    +                    flowPath.setGroupId(urlMatcher.group(1));
    +                }
    +            }
    +            flowPath.setExEntity(flowPathEntity);
    +            flowPath.setName(toStr(flowPathEntity.getAttribute(ATTR_NAME)));
    +            flowPath.getInputs().addAll(toQualifiedNameIds(toAtlasObjectIds(flowPathEntity.getAttribute(ATTR_INPUTS))).keySet());
    +            flowPath.getOutputs().addAll(toQualifiedNameIds(toAtlasObjectIds(flowPathEntity.getAttribute(ATTR_OUTPUTS))).keySet());
    +            flowPath.startTrackingChanges(nifiFlow);
    +
    +            flowPaths.put(flowPath.getId(), flowPath);
    +        }
    +
    +        nifiFlow.startTrackingChanges();
    +        return nifiFlow;
    +    }
    +
    +    @SuppressWarnings("unchecked")
    +    private List<AtlasObjectId> toAtlasObjectIds(Object _references) {
    +        if (_references == null) {
    +            return Collections.emptyList();
    +        }
    +        List<Map<String, Object>> references = (List<Map<String, Object>>) _references;
    +        return references.stream()
    +                .map(ref -> new AtlasObjectId(toStr(ref.get(ATTR_GUID)), toStr(ref.get(ATTR_TYPENAME)), ref))
    +                .collect(Collectors.toList());
    +    }
    +
    +    /**
    +     * <p>AtlasObjectIds returned from Atlas have GUID, but do not have qualifiedName, while ones created by the reporting task
    +     * do not have GUID, but qualifiedName. AtlasObjectId.equals returns false for this combination.
    +     * In order to match ids correctly, this method converts fetches actual entities from ids to get qualifiedName attribute.</p>
    +     *
    +     * <p>Also, AtlasObjectIds returned from Atlas does not have entity state.
    +     * If Atlas is configured to use soft-delete (default), deleted ids are still returned.
    +     * Fetched entities are used to determine whether an AtlasObjectId is still active or deleted.
    +     * Deleted entities will not be included in the result of this method.
    +     * </p>
    +     * @param ids to convert
    +     * @return AtlasObjectIds with qualifiedName
    +     */
    +    private Map<AtlasObjectId, AtlasEntity> toQualifiedNameIds(List<AtlasObjectId> ids) {
    +        if (ids == null) {
    +            return Collections.emptyMap();
    +        }
    +
    +        return ids.stream().distinct().map(id -> {
    +            try {
    +                final AtlasEntity.AtlasEntityWithExtInfo entityExt = searchEntityDef(id);
    +                final AtlasEntity entity = entityExt.getEntity();
    +                if (AtlasEntity.Status.DELETED.equals(entity.getStatus())) {
    +                    return null;
    +                }
    +                final Map<String, Object> uniqueAttrs = Collections.singletonMap(ATTR_QUALIFIED_NAME, entity.getAttribute(ATTR_QUALIFIED_NAME));
    +                return new Tuple<>(new AtlasObjectId(id.getGuid(), id.getTypeName(), uniqueAttrs), entity);
    +            } catch (AtlasServiceException e) {
    +                return null;
    +            }
    +        }).filter(Objects::nonNull).collect(Collectors.toMap(Tuple::getKey, Tuple::getValue));
    +    }
    +
    +    public void registerNiFiFlow(NiFiFlow nifiFlow) throws AtlasServiceException {
    +
    +        // Create parent flow entity, so that common properties are taken over.
    +        final AtlasEntity flowEntity = registerNiFiFlowEntity(nifiFlow);
    +
    +        // Create DataSet entities those are created by this NiFi flow.
    +        final Map<String, List<AtlasEntity>> updatedDataSetEntities = registerDataSetEntities(nifiFlow);
    +
    +        // Create path entities.
    +        final Set<AtlasObjectId> remainingPathIds = registerFlowPathEntities(nifiFlow);
    +
    +        // Update these attributes only if anything is created, updated or removed.
    +        boolean shouldUpdateNiFiFlow = nifiFlow.isMetadataUpdated();
    +        if (remainingPathIds != null) {
    +            flowEntity.setAttribute(ATTR_FLOW_PATHS, remainingPathIds);
    +            shouldUpdateNiFiFlow = true;
    +        }
    +        if (updatedDataSetEntities.containsKey(TYPE_NIFI_QUEUE)) {
    +            flowEntity.setAttribute(ATTR_QUEUES, updatedDataSetEntities.get(TYPE_NIFI_QUEUE));
    +            shouldUpdateNiFiFlow = true;
    +        }
    +        if (updatedDataSetEntities.containsKey(TYPE_NIFI_INPUT_PORT)) {
    +            flowEntity.setAttribute(ATTR_INPUT_PORTS, updatedDataSetEntities.get(TYPE_NIFI_INPUT_PORT));
    +            shouldUpdateNiFiFlow = true;
    +        }
    +        if (updatedDataSetEntities.containsKey(TYPE_NIFI_OUTPUT_PORT)) {
    +            flowEntity.setAttribute(ATTR_OUTPUT_PORTS, updatedDataSetEntities.get(TYPE_NIFI_OUTPUT_PORT));
    +            shouldUpdateNiFiFlow = true;
    +        }
    +
    +        logger.debug("### NiFi Flow Audit Logs START");
    +        nifiFlow.getUpdateAudit().forEach(logger::debug);
    +        nifiFlow.getFlowPaths().forEach((k, v) -> {
    +            logger.debug("--- NiFiFlowPath Audit Logs: {}", k);
    +            v.getUpdateAudit().forEach(logger::debug);
    +        });
    +        logger.debug("### NiFi Flow Audit Logs END");
    +
    +        if (shouldUpdateNiFiFlow) {
    +            // Send updated entities.
    +            final List<AtlasEntity> entities = new ArrayList<>();
    +            final AtlasEntity.AtlasEntitiesWithExtInfo atlasEntities = new AtlasEntity.AtlasEntitiesWithExtInfo(entities);
    +            entities.add(flowEntity);
    +            try {
    +                final EntityMutationResponse mutationResponse = atlasClient.createEntities(atlasEntities);
    +                logger.debug("mutation response={}", mutationResponse);
    +            } catch (AtlasServiceException e) {
    +                if (e.getStatus().getStatusCode() == 404 && e.getMessage().contains("ATLAS-404-00-00B")) {
    --- End diff --
    
    That is a good idea. Thanks!


---

[GitHub] nifi issue #2335: NIFI-3709: Export NiFi flow dataset lineage to Apache Atla...

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

    https://github.com/apache/nifi/pull/2335
  
    @markap14 @dmkoster Thank you very much for your informative review feedback! I've addressed all comments other than confirming the Provenance Event level authorization. I will keep digging that.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157232014
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/NiFiProvenanceEventAnalyzerFactory.java ---
    @@ -0,0 +1,105 @@
    +/*
    + * 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.atlas.provenance;
    +
    +import org.apache.nifi.provenance.ProvenanceEventType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +import java.util.ServiceLoader;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +
    +public class NiFiProvenanceEventAnalyzerFactory {
    +
    +    private static final Logger logger = LoggerFactory.getLogger(NiFiProvenanceEventAnalyzerFactory.class);
    +    private static final Map<Pattern, NiFiProvenanceEventAnalyzer> analyzersForComponentType = new ConcurrentHashMap<>();
    +    private static final Map<Pattern, NiFiProvenanceEventAnalyzer> analyzersForTransitUri = new ConcurrentHashMap<>();
    +    private static final Map<ProvenanceEventType, NiFiProvenanceEventAnalyzer> analyzersForProvenanceEventType = new ConcurrentHashMap<>();
    +    private static boolean loaded = false;
    +
    +    private static void loadAnalyzers() {
    +        logger.debug("Loading NiFiProvenanceEventAnalyzer ...");
    +        final ServiceLoader<NiFiProvenanceEventAnalyzer> serviceLoader
    +                = ServiceLoader.load(NiFiProvenanceEventAnalyzer.class);
    +        serviceLoader.forEach(analyzer -> {
    +            addAnalyzer(analyzer.targetComponentTypePattern(), analyzersForComponentType, analyzer);
    +            addAnalyzer(analyzer.targetTransitUriPattern(), analyzersForTransitUri, analyzer);
    +            final ProvenanceEventType eventType = analyzer.targetProvenanceEventType();
    +            if (eventType != null) {
    +                if (analyzersForProvenanceEventType.containsKey(eventType)) {
    +                    logger.warn("Fo ProvenanceEventType {}, an Analyzer {} is already assigned." +
    +                            " Only one analyzer for a type can be registered. Ignoring {}",
    +                            eventType, analyzersForProvenanceEventType.get(eventType), analyzer);
    +                }
    +                analyzersForProvenanceEventType.put(eventType, analyzer);
    +            }
    +        });
    +        logger.info("Loaded NiFiProvenanceEventAnalyzers: componentTypes={}, transitUris={}", analyzersForComponentType, analyzersForTransitUri);
    +    }
    +
    +    private static void addAnalyzer(String patternStr, Map<Pattern, NiFiProvenanceEventAnalyzer> toAdd,
    +                                    NiFiProvenanceEventAnalyzer analyzer) {
    +        if (patternStr != null && !patternStr.isEmpty()) {
    +            Pattern pattern = Pattern.compile(patternStr.trim());
    +            toAdd.put(pattern, analyzer);
    +        }
    +    }
    +
    +    /**
    +     * Find and retrieve NiFiProvenanceEventAnalyzer implementation for the specified targets.
    +     * Pattern matching is performed by following order, and the one found at first is returned:
    +     * <ol>
    +     * <li>Component type name. Use an analyzer supporting the Component type with its {@link NiFiProvenanceEventAnalyzer#targetProvenanceEventType()}.
    +     * <li>TransitUri. Use an analyzer supporting the TransitUri with its {@link NiFiProvenanceEventAnalyzer#targetTransitUriPattern()}.
    +     * <li>Provenance Event Type. Use an analyzer supporting the Provenance Event Type with its {@link NiFiProvenanceEventAnalyzer#targetProvenanceEventType()}.
    +     * </ol>
    +     * @param typeName NiFi component type name.
    +     * @param transitUri Transit URI.
    +     * @param eventType Provenance event type.
    +     * @return Instance of NiFiProvenanceEventAnalyzer if one is found for the specified className, otherwise null.
    +     */
    +    public static NiFiProvenanceEventAnalyzer getAnalyzer(String typeName, String transitUri, ProvenanceEventType eventType) {
    +
    +        if (!loaded) {
    --- End diff --
    
    Specifically, "Initialization-on-demand holder idiom".
    https://en.wikipedia.org/wiki/Initialization-on-demand_holder_idiom


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157231905
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/NiFiProvenanceEventAnalyzerFactory.java ---
    @@ -0,0 +1,105 @@
    +/*
    + * 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.atlas.provenance;
    +
    +import org.apache.nifi.provenance.ProvenanceEventType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +import java.util.ServiceLoader;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +
    +public class NiFiProvenanceEventAnalyzerFactory {
    +
    +    private static final Logger logger = LoggerFactory.getLogger(NiFiProvenanceEventAnalyzerFactory.class);
    +    private static final Map<Pattern, NiFiProvenanceEventAnalyzer> analyzersForComponentType = new ConcurrentHashMap<>();
    +    private static final Map<Pattern, NiFiProvenanceEventAnalyzer> analyzersForTransitUri = new ConcurrentHashMap<>();
    +    private static final Map<ProvenanceEventType, NiFiProvenanceEventAnalyzer> analyzersForProvenanceEventType = new ConcurrentHashMap<>();
    +    private static boolean loaded = false;
    +
    +    private static void loadAnalyzers() {
    +        logger.debug("Loading NiFiProvenanceEventAnalyzer ...");
    +        final ServiceLoader<NiFiProvenanceEventAnalyzer> serviceLoader
    +                = ServiceLoader.load(NiFiProvenanceEventAnalyzer.class);
    +        serviceLoader.forEach(analyzer -> {
    +            addAnalyzer(analyzer.targetComponentTypePattern(), analyzersForComponentType, analyzer);
    +            addAnalyzer(analyzer.targetTransitUriPattern(), analyzersForTransitUri, analyzer);
    +            final ProvenanceEventType eventType = analyzer.targetProvenanceEventType();
    +            if (eventType != null) {
    +                if (analyzersForProvenanceEventType.containsKey(eventType)) {
    +                    logger.warn("Fo ProvenanceEventType {}, an Analyzer {} is already assigned." +
    +                            " Only one analyzer for a type can be registered. Ignoring {}",
    +                            eventType, analyzersForProvenanceEventType.get(eventType), analyzer);
    +                }
    +                analyzersForProvenanceEventType.put(eventType, analyzer);
    +            }
    +        });
    +        logger.info("Loaded NiFiProvenanceEventAnalyzers: componentTypes={}, transitUris={}", analyzersForComponentType, analyzersForTransitUri);
    +    }
    +
    +    private static void addAnalyzer(String patternStr, Map<Pattern, NiFiProvenanceEventAnalyzer> toAdd,
    +                                    NiFiProvenanceEventAnalyzer analyzer) {
    +        if (patternStr != null && !patternStr.isEmpty()) {
    +            Pattern pattern = Pattern.compile(patternStr.trim());
    +            toAdd.put(pattern, analyzer);
    +        }
    +    }
    +
    +    /**
    +     * Find and retrieve NiFiProvenanceEventAnalyzer implementation for the specified targets.
    +     * Pattern matching is performed by following order, and the one found at first is returned:
    +     * <ol>
    +     * <li>Component type name. Use an analyzer supporting the Component type with its {@link NiFiProvenanceEventAnalyzer#targetProvenanceEventType()}.
    +     * <li>TransitUri. Use an analyzer supporting the TransitUri with its {@link NiFiProvenanceEventAnalyzer#targetTransitUriPattern()}.
    +     * <li>Provenance Event Type. Use an analyzer supporting the Provenance Event Type with its {@link NiFiProvenanceEventAnalyzer#targetProvenanceEventType()}.
    +     * </ol>
    +     * @param typeName NiFi component type name.
    +     * @param transitUri Transit URI.
    +     * @param eventType Provenance event type.
    +     * @return Instance of NiFiProvenanceEventAnalyzer if one is found for the specified className, otherwise null.
    +     */
    +    public static NiFiProvenanceEventAnalyzer getAnalyzer(String typeName, String transitUri, ProvenanceEventType eventType) {
    +
    +        if (!loaded) {
    --- End diff --
    
    Thank you! I was referring an ancient blog post to implement this logic.. I will update it to use static lazy initializer.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r156985994
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/StandardAnalysisContext.java ---
    @@ -0,0 +1,136 @@
    +/*
    + * 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.atlas.provenance;
    +
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.authorization.user.NiFiUser;
    +import org.apache.nifi.controller.status.ConnectionStatus;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.provenance.lineage.ComputeLineageResult;
    +import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +
    +public class StandardAnalysisContext implements AnalysisContext {
    +
    +    private final Logger logger = LoggerFactory.getLogger(StandardAnalysisContext.class);
    +    private final NiFiFlow nifiFlow;
    +    private final ClusterResolver clusterResolver;
    +    private final ProvenanceRepository provenanceRepository;
    +
    +    public StandardAnalysisContext(NiFiFlow nifiFlow, ClusterResolver clusterResolver,
    +                                   ProvenanceRepository provenanceRepository) {
    +        this.nifiFlow = nifiFlow;
    +        this.clusterResolver = clusterResolver;
    +        this.provenanceRepository = provenanceRepository;
    +    }
    +
    +    @Override
    +    public List<ConnectionStatus> findConnectionTo(String componentId) {
    +        return nifiFlow.getIncomingRelationShips(componentId);
    +    }
    +
    +    @Override
    +    public List<ConnectionStatus> findConnectionFrom(String componentId) {
    +        return nifiFlow.getOutgoingRelationShips(componentId);
    +    }
    +
    +    @Override
    +    public String getNiFiClusterName() {
    +        return nifiFlow.getClusterName();
    +    }
    +
    +    @Override
    +    public ClusterResolver getClusterResolver() {
    +        return clusterResolver;
    +    }
    +
    +    private ComputeLineageResult getLineageResult(long eventId, ComputeLineageSubmission submission) {
    +        final ComputeLineageResult result = submission.getResult();
    +        try {
    +            if (result.awaitCompletion(10, TimeUnit.SECONDS)) {
    --- End diff --
    
    I'm not sure yet the context in which this is used. However, this is a bit of a red flag. In order to compute a FlowFile's lineage, depending on the size of the Provenance Repository and the implementation, it may take many minutes (consider a Provenance Repository that is a couple of terabytes in size and the event is one of the oldest -it will have to search all lucene indices for this). In most cases it is fairly quick, but we should not assume that it will be. What exactly is the implication here if this always times out?


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157004365
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/lineage/SimpleFlowPathLineage.java ---
    @@ -0,0 +1,205 @@
    +/*
    + * 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.atlas.provenance.lineage;
    +
    +import org.apache.atlas.typesystem.Referenceable;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowPath;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.DataSetRefs;
    +import org.apache.nifi.controller.status.ConnectionStatus;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.lineage.ComputeLineageResult;
    +import org.apache.nifi.provenance.lineage.LineageEdge;
    +import org.apache.nifi.provenance.lineage.LineageNode;
    +import org.apache.nifi.provenance.lineage.LineageNodeType;
    +
    +import java.util.List;
    +
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_QUEUE;
    +
    +public class SimpleFlowPathLineage extends AbstractLineageStrategy {
    +
    +    @Override
    +    public void processEvent(AnalysisContext analysisContext, NiFiFlow nifiFlow, ProvenanceEventRecord event) {
    +        final DataSetRefs refs = executeAnalyzer(analysisContext, event);
    +        if (refs == null || (refs.isEmpty())) {
    +            return;
    +        }
    +
    +        if ("Remote Input Port".equals(event.getComponentType()) || "Remote Output Port".equals(event.getComponentType())) {
    +            processRemotePortEvent(analysisContext, nifiFlow, event, refs);
    +        } else {
    +            addDataSetRefs(nifiFlow, refs);
    +        }
    +
    +    }
    +
    +    /**
    +     * Create a flow_path entity corresponding to the target RemoteGroupPort when a SEND/RECEIVE event are received.
    +     * Because such entity can not be created in advance while analyzing flow statically,
    +     * as ReportingTask can not determine whether a component id is a RemoteGroupPort,
    +     * since connectionStatus is the only available information in ReportingContext.
    +     * ConnectionStatus only knows component id, component type is unknown.
    +     * For example, there is no difference to tell if a connected component is a funnel or a RemoteGroupPort.
    +     */
    +    private void processRemotePortEvent(AnalysisContext analysisContext, NiFiFlow nifiFlow, ProvenanceEventRecord event, DataSetRefs analyzedRefs) {
    +
    +        final boolean isRemoteInputPort = "Remote Input Port".equals(event.getComponentType());
    +
    +        // Create a RemoteInputPort Process.
    +        // event.getComponentId returns UUID for RemoteGroupPort as a client of S2S, and it's different from a remote port UUID (portDataSetid).
    +        // See NIFI-4571 for detail.
    +        final Referenceable remotePortDataSet = isRemoteInputPort ? analyzedRefs.getOutputs().iterator().next() :  analyzedRefs.getInputs().iterator().next();
    +        final String portProcessId = event.getComponentId();
    +
    +        final NiFiFlowPath remotePortProcess = new NiFiFlowPath(portProcessId);
    +        remotePortProcess.setName(event.getComponentType());
    +        remotePortProcess.addProcessor(portProcessId);
    +
    +        // For RemoteInputPort, need to find the previous component connected to this port,
    +        // which passed this particular FlowFile.
    +        // That is only possible by calling lineage API.
    +        if (isRemoteInputPort) {
    +            final ProvenanceEventRecord previousEvent = findPreviousProvenanceEvent(analysisContext, event);
    +            if (previousEvent == null) {
    +                logger.warn("Previous event was not found: {}", new Object[]{event});
    +                return;
    +            }
    +
    +            // Set groupId from incoming connection if available.
    +            final List<ConnectionStatus> incomingRelationShips = nifiFlow.getIncomingRelationShips(portProcessId);
    --- End diff --
    
    I think this is meant to get incoming connections, not relationships, right?


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157234799
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/resources/docs/org.apache.nifi.atlas.reporting.AtlasNiFiFlowLineage/additionalDetails.html ---
    @@ -0,0 +1,541 @@
    +<!DOCTYPE html>
    +<html lang="en">
    +<!--
    +  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.
    +-->
    +    <head>
    +        <meta charset="utf-8" />
    +        <title>AtlasNiFiFlowLineage</title>
    +        <link rel="stylesheet" href="/nifi-docs/css/component-usage.css" type="text/css" />
    +    </head>
    +
    +    <body>
    +        <h2>AtlasNiFiFlowLineage</h2>
    +
    +        Table of contents:
    +        <!-- TODO: Fix header tags most h4 should be h3 -->
    +        <ul>
    +            <li><a href="#how-it-works">Information reported to Atlas</a></li>
    +            <li><a href="#nifi-atlas-types">NiFi Atlas Types</a></li>
    +            <li><a href="#cluster-name">Cluster Name Resolution</a></li>
    +            <li><a href="#nifi-flow-structure">NiFi flow structure</a>
    +                <ul>
    +                    <li><a href="#path-separation">Path Separation Logic</a></li>
    +                </ul>
    +            </li>
    +            <li><a href="#nifi-data-lineage">NiFi data lineage</a>
    +                <ul>
    +                    <li><a href="#lineage-strategy">NiFi Lineage Strategy</a></li>
    +                    <li><a href="#provenance-events">NiFi Provenance Event Analysis</a></li>
    +                    <li><a href="#datasets-and-processors">Supported DataSets and Processors</a></li>
    +                </ul>
    +            </li>
    +            <li><a href="#runs-in-cluster">How it runs in NiFi cluster</a></li>
    +            <li><a href="#limitations">Limitations</a></li>
    +            <li><a href="#atlas-configs">Atlas Server Configurations</a></li>
    +            <li><a href="#atlas-emulator">Atlas Server Emulator</a></li>
    +        </ul>
    +
    +        <h3 id="how-it-works">Information reported to Atlas</h3>
    +        <p>This reporting task stores two types of NiFi flow information, 'NiFi flow structure' and 'NiFi data lineage'.</p>
    +
    +        <p>'NiFi flow structure' tells what components are running within a NiFi flow and how these are connected. It is reported by analyzing current NiFi flow structure, specifically NiFi component relationships.</p>
    +
    +        <p>'NiFi data lineage' tells what part of NiFi flow interacts with different DataSets such as HDFS files or Hive tables ... etc. It is reported by analyzing NiFi provenance events.</p>
    +
    +        <object data="nifi_atlas.svg" type="image/svg+xml" width="60%"></object>
    +
    +        <p>Technically each information is sent using different protocol, Atlas REST API v2, and Notification via a Kafka topic as shown in above image.</p>
    +
    +
    +        <p>As both information types use the same <a href="#nifi-atlas-types">NiFi Atlas Types</a> and <a href="#cluster-name">Cluster Name Resolution</a> concepts, it is recommended to start reading those sections first.</p>
    +
    +        <h4 id="nifi-atlas-types">NiFi Atlas Types</h4>
    +
    +        <p>This reporting task creates following NiFi specific types in Atlas Type system when it runs if these type definitions are not found.</p>
    +
    +        <p>Green boxes represent sub-types of DataSet and blue ones are sub-types of Process. Gray lines represent entity ownership.
    +        Red lines represent lineage.</p>
    +
    +        <object data="nifi_types.svg" type="image/svg+xml" width="60%"></object>
    +
    +        <ul>
    +            <li>nifi_flow
    +                <p>Represents a NiFI data flow.</p>
    +                <p>As shown in the above diagram, nifi_flow owns other nifi_component types.
    +                    This owning relationship is defined by Atlas 'owned' constraint so that when a 'nifi_flow' entity is removed, all owned NiFi component entities are removed in cascading manner.</p>
    +                <p>When this reporting task runs, it analyzes and traverse the entire flow structure, and create NiFi component entities in Atlas.
    +                    At later runs, it compares the current flow structure with the one stored in Atlas to figure out if any changes has been made since the last time the flow was reported. The reporting task updates NiFi component entities in Atlas if needed.<p>
    +                <p>NiFi components those are removed from a NiFi flow also get deleted from Atlas.
    +                    However those entities can still be seen in Atlas search results or lineage graphs since Atlas uses 'Soft Delete' by default.
    +                    See <a href="#delete-handler">Atlas Delete Handler</a> for further detail.</p>
    +            </li>
    +            Attributes:
    +            <ul>
    +                <li>qualifiedName: Root ProcessGroup ID@clusterName (e.g. 86420a14-2fab-3e1e-4331-fb6ab42f58e0@cl1)</li>
    +                <li>name: Name of the Root ProcessGroup.</li>
    +                <li>url: URL of the NiFi instance. This can be specified via reporting task 'NiFi URL for Atlas' property.</li>
    +            </ul>
    +        </ul>
    +        <ul>
    +            <li>nifi_flow_path <p>Part of a NiFi data flow containing one or more processing NiFi components such as Processors and RemoteGroupPorts. The reporting task divides a NiFi flow into multiple flow paths. See <a href="#path-separation">Path Separation Logic</a> for details.</p></li>
    +            Attributes:
    +            <ul>
    +                <li>qualifiedName: The first NiFi component Id in a path@clusterName (e.g. 529e6722-9b49-3b66-9c94-00da9863ca2d@cl1)</li>
    +                <li>name: NiFi component namess within a path are concatenated (e.g. GenerateFlowFile, PutFile, LogAttribute)</li>
    +                <li>url: A deep link to the first NiFi component in corresponding NiFi UI</li>
    +            </ul>
    +        </ul>
    +        <ul>
    +            <!-- TODO: link to S2S details -->
    --- End diff --
    
    Forgot to delete the TODO. I was willing to write about the detail, but it's too internal I think. So just remove the TODO comment. I found other TODOs remaining in this document. All resolved.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157221674
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/NiFIAtlasHook.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.atlas;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.atlas.hook.AtlasHook;
    +import org.apache.atlas.model.instance.AtlasEntity;
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +import org.apache.atlas.notification.hook.HookNotification.EntityPartialUpdateRequest;
    +import org.apache.atlas.notification.hook.HookNotification.HookNotificationMessage;
    +import org.apache.atlas.typesystem.Referenceable;
    +import org.apache.atlas.typesystem.persistence.Id;
    +import org.apache.nifi.atlas.provenance.lineage.LineageContext;
    +import org.apache.nifi.util.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.LinkedHashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.atlas.notification.hook.HookNotification.HookNotificationType.ENTITY_PARTIAL_UPDATE;
    +import static org.apache.nifi.atlas.AtlasUtils.toTypedQualifiedName;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_GUID;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_TYPENAME;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW_PATH;
    +
    +/**
    + * This class is not thread-safe as it holds uncommitted notification messages within instance.
    + * {@link #addMessage(HookNotificationMessage)} and {@link #commitMessages()} should be used serially from a single thread.
    + */
    +public class NiFIAtlasHook extends AtlasHook implements LineageContext {
    +
    +    public static final String NIFI_USER = "nifi";
    --- End diff --
    
    Yes, it would. However, at this moment, this user is not significant. Please see this conversation on the same subject.
    https://github.com/apache/nifi/pull/2335#discussion_r156992235


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157240784
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/lineage/SimpleFlowPathLineage.java ---
    @@ -0,0 +1,205 @@
    +/*
    + * 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.atlas.provenance.lineage;
    +
    +import org.apache.atlas.typesystem.Referenceable;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowPath;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.DataSetRefs;
    +import org.apache.nifi.controller.status.ConnectionStatus;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.lineage.ComputeLineageResult;
    +import org.apache.nifi.provenance.lineage.LineageEdge;
    +import org.apache.nifi.provenance.lineage.LineageNode;
    +import org.apache.nifi.provenance.lineage.LineageNodeType;
    +
    +import java.util.List;
    +
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_QUEUE;
    +
    +public class SimpleFlowPathLineage extends AbstractLineageStrategy {
    +
    +    @Override
    +    public void processEvent(AnalysisContext analysisContext, NiFiFlow nifiFlow, ProvenanceEventRecord event) {
    +        final DataSetRefs refs = executeAnalyzer(analysisContext, event);
    +        if (refs == null || (refs.isEmpty())) {
    +            return;
    +        }
    +
    +        if ("Remote Input Port".equals(event.getComponentType()) || "Remote Output Port".equals(event.getComponentType())) {
    +            processRemotePortEvent(analysisContext, nifiFlow, event, refs);
    +        } else {
    +            addDataSetRefs(nifiFlow, refs);
    +        }
    +
    +    }
    +
    +    /**
    +     * Create a flow_path entity corresponding to the target RemoteGroupPort when a SEND/RECEIVE event are received.
    +     * Because such entity can not be created in advance while analyzing flow statically,
    +     * as ReportingTask can not determine whether a component id is a RemoteGroupPort,
    +     * since connectionStatus is the only available information in ReportingContext.
    +     * ConnectionStatus only knows component id, component type is unknown.
    +     * For example, there is no difference to tell if a connected component is a funnel or a RemoteGroupPort.
    +     */
    +    private void processRemotePortEvent(AnalysisContext analysisContext, NiFiFlow nifiFlow, ProvenanceEventRecord event, DataSetRefs analyzedRefs) {
    +
    +        final boolean isRemoteInputPort = "Remote Input Port".equals(event.getComponentType());
    +
    +        // Create a RemoteInputPort Process.
    +        // event.getComponentId returns UUID for RemoteGroupPort as a client of S2S, and it's different from a remote port UUID (portDataSetid).
    +        // See NIFI-4571 for detail.
    +        final Referenceable remotePortDataSet = isRemoteInputPort ? analyzedRefs.getOutputs().iterator().next() :  analyzedRefs.getInputs().iterator().next();
    +        final String portProcessId = event.getComponentId();
    +
    +        final NiFiFlowPath remotePortProcess = new NiFiFlowPath(portProcessId);
    +        remotePortProcess.setName(event.getComponentType());
    +        remotePortProcess.addProcessor(portProcessId);
    +
    +        // For RemoteInputPort, need to find the previous component connected to this port,
    +        // which passed this particular FlowFile.
    +        // That is only possible by calling lineage API.
    +        if (isRemoteInputPort) {
    +            final ProvenanceEventRecord previousEvent = findPreviousProvenanceEvent(analysisContext, event);
    +            if (previousEvent == null) {
    +                logger.warn("Previous event was not found: {}", new Object[]{event});
    +                return;
    +            }
    +
    +            // Set groupId from incoming connection if available.
    +            final List<ConnectionStatus> incomingRelationShips = nifiFlow.getIncomingRelationShips(portProcessId);
    --- End diff --
    
    Ah, that does make sense. Then this should be 'connections'. I will rename variables. Thank you


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157219892
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/StandardAnalysisContext.java ---
    @@ -0,0 +1,136 @@
    +/*
    + * 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.atlas.provenance;
    +
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.authorization.user.NiFiUser;
    +import org.apache.nifi.controller.status.ConnectionStatus;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.provenance.lineage.ComputeLineageResult;
    +import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +
    +public class StandardAnalysisContext implements AnalysisContext {
    +
    +    private final Logger logger = LoggerFactory.getLogger(StandardAnalysisContext.class);
    +    private final NiFiFlow nifiFlow;
    +    private final ClusterResolver clusterResolver;
    +    private final ProvenanceRepository provenanceRepository;
    +
    +    public StandardAnalysisContext(NiFiFlow nifiFlow, ClusterResolver clusterResolver,
    +                                   ProvenanceRepository provenanceRepository) {
    +        this.nifiFlow = nifiFlow;
    +        this.clusterResolver = clusterResolver;
    +        this.provenanceRepository = provenanceRepository;
    +    }
    +
    +    @Override
    +    public List<ConnectionStatus> findConnectionTo(String componentId) {
    +        return nifiFlow.getIncomingRelationShips(componentId);
    +    }
    +
    +    @Override
    +    public List<ConnectionStatus> findConnectionFrom(String componentId) {
    +        return nifiFlow.getOutgoingRelationShips(componentId);
    +    }
    +
    +    @Override
    +    public String getNiFiClusterName() {
    +        return nifiFlow.getClusterName();
    +    }
    +
    +    @Override
    +    public ClusterResolver getClusterResolver() {
    +        return clusterResolver;
    +    }
    +
    +    private ComputeLineageResult getLineageResult(long eventId, ComputeLineageSubmission submission) {
    +        final ComputeLineageResult result = submission.getResult();
    +        try {
    +            if (result.awaitCompletion(10, TimeUnit.SECONDS)) {
    --- End diff --
    
    I agree on that this can be a costly operation. The reason to query provenance is to compute a lineage from a DROP provenance event. This is used by 'Complete Path' strategy. I wrote documentation on performance impact. If this does not work for a use-case, then user can choose another strategy, 'Simple Path'. Simple Path does not query provenance events this way. It analyzes each individual event, so should be more lightweight.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157237190
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/lineage/SimpleFlowPathLineage.java ---
    @@ -0,0 +1,205 @@
    +/*
    + * 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.atlas.provenance.lineage;
    +
    +import org.apache.atlas.typesystem.Referenceable;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowPath;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.DataSetRefs;
    +import org.apache.nifi.controller.status.ConnectionStatus;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.lineage.ComputeLineageResult;
    +import org.apache.nifi.provenance.lineage.LineageEdge;
    +import org.apache.nifi.provenance.lineage.LineageNode;
    +import org.apache.nifi.provenance.lineage.LineageNodeType;
    +
    +import java.util.List;
    +
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_QUEUE;
    +
    +public class SimpleFlowPathLineage extends AbstractLineageStrategy {
    +
    +    @Override
    +    public void processEvent(AnalysisContext analysisContext, NiFiFlow nifiFlow, ProvenanceEventRecord event) {
    +        final DataSetRefs refs = executeAnalyzer(analysisContext, event);
    +        if (refs == null || (refs.isEmpty())) {
    +            return;
    +        }
    +
    +        if ("Remote Input Port".equals(event.getComponentType()) || "Remote Output Port".equals(event.getComponentType())) {
    +            processRemotePortEvent(analysisContext, nifiFlow, event, refs);
    +        } else {
    +            addDataSetRefs(nifiFlow, refs);
    +        }
    +
    +    }
    +
    +    /**
    +     * Create a flow_path entity corresponding to the target RemoteGroupPort when a SEND/RECEIVE event are received.
    +     * Because such entity can not be created in advance while analyzing flow statically,
    +     * as ReportingTask can not determine whether a component id is a RemoteGroupPort,
    +     * since connectionStatus is the only available information in ReportingContext.
    +     * ConnectionStatus only knows component id, component type is unknown.
    +     * For example, there is no difference to tell if a connected component is a funnel or a RemoteGroupPort.
    +     */
    +    private void processRemotePortEvent(AnalysisContext analysisContext, NiFiFlow nifiFlow, ProvenanceEventRecord event, DataSetRefs analyzedRefs) {
    +
    +        final boolean isRemoteInputPort = "Remote Input Port".equals(event.getComponentType());
    +
    +        // Create a RemoteInputPort Process.
    +        // event.getComponentId returns UUID for RemoteGroupPort as a client of S2S, and it's different from a remote port UUID (portDataSetid).
    +        // See NIFI-4571 for detail.
    +        final Referenceable remotePortDataSet = isRemoteInputPort ? analyzedRefs.getOutputs().iterator().next() :  analyzedRefs.getInputs().iterator().next();
    +        final String portProcessId = event.getComponentId();
    +
    +        final NiFiFlowPath remotePortProcess = new NiFiFlowPath(portProcessId);
    +        remotePortProcess.setName(event.getComponentType());
    +        remotePortProcess.addProcessor(portProcessId);
    +
    +        // For RemoteInputPort, need to find the previous component connected to this port,
    +        // which passed this particular FlowFile.
    +        // That is only possible by calling lineage API.
    +        if (isRemoteInputPort) {
    +            final ProvenanceEventRecord previousEvent = findPreviousProvenanceEvent(analysisContext, event);
    +            if (previousEvent == null) {
    +                logger.warn("Previous event was not found: {}", new Object[]{event});
    +                return;
    +            }
    +
    +            // Set groupId from incoming connection if available.
    +            final List<ConnectionStatus> incomingRelationShips = nifiFlow.getIncomingRelationShips(portProcessId);
    --- End diff --
    
    @ijokarumawak yes, there is a distinction. A 'relationship' is defined by a processor in order to indicate which route a FlowFile should take. A Connection is defined by a user when creating the dataflow. The connection consists of one or more Relationships, and multiple connections can consist of the same relationship. So when a processor transfers FlowFile A to relationship 'success' for example, it may be placed into a single Connection. Or cloned and added to 5 different Connections. Or it may be auto-terminated and not go into any Connections. Does that make sense?


---

[GitHub] nifi issue #2335: NIFI-3709: Export NiFi flow dataset lineage to Apache Atla...

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

    https://github.com/apache/nifi/pull/2335
  
    @ijokarumawak thanks for all of the work that you've put into this - it is very much a non-trivial effort! For the most part, the code looks good. I flagged a couple of minor things in the code, 1 or 2 thread-safety issues that should be easy to address.
    
    The only 'more significant' concern that I have is the use of the dummied-up NiFiUser. As-is, this is an anonymous user and in a secured environment will not retrieve the event details that are necessary. It also means that we would be validating events against a user who doesn't even exist.
    
    I think there are 2 ways to approach this: first, as I noted inline, we could have a property to define which user the queries should run on behalf of. So the user could add a "NiFi Atlas" user and use that. However, that's also a bit concerning because it means that whoever has access to edit the reporting task can run provenance queries on behalf of another user.
    
    By far, my preference is to actually just update the ProvenanceRepository implementations (There are 4 now, I think) so that if a null User is passed in, we don't check permissions. This would mean that you can pass in null from Reporting Task. We could also then update the interface to have an overloaded method that does not require that a user be given.
    
    Once that is addressed, I think it is a +1 from me from a code review perspective.
    
    Thanks
    -Mark


---

[GitHub] nifi issue #2335: NIFI-3709: Export NiFi flow dataset lineage to Apache Atla...

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

    https://github.com/apache/nifi/pull/2335
  
    @ijokarumawak all looks good after some local testing. Thanks for your diligence and the willingness to keep going on this until we got everything resolved! I have now merged this to master!


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r156979528
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/NiFiAtlasClient.java ---
    @@ -0,0 +1,537 @@
    +/*
    + * 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.atlas;
    +
    +import com.sun.jersey.api.client.UniformInterfaceException;
    +import com.sun.jersey.core.util.MultivaluedMapImpl;
    +import org.apache.atlas.ApplicationProperties;
    +import org.apache.atlas.AtlasClientV2;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.atlas.model.SearchFilter;
    +import org.apache.atlas.model.instance.AtlasEntity;
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +import org.apache.atlas.model.instance.EntityMutationResponse;
    +import org.apache.atlas.model.typedef.AtlasEntityDef;
    +import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
    +import org.apache.atlas.model.typedef.AtlasTypesDef;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.util.StringUtils;
    +import org.apache.nifi.util.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.ws.rs.core.MultivaluedMap;
    +import java.io.File;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.function.Function;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.nifi.atlas.AtlasUtils.findIdByQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.getComponentIdFromQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.toStr;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.AS_IS;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.CREATED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.DELETED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.UPDATED;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_DESCRIPTION;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_FLOW_PATHS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_GUID;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUEUES;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_TYPENAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_URL;
    +import static org.apache.nifi.atlas.NiFiTypes.ENTITIES;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW_PATH;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_INPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_OUTPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_QUEUE;
    +
    +public class NiFiAtlasClient {
    +
    +    private static final Logger logger = LoggerFactory.getLogger(NiFiAtlasClient.class);
    +
    +    private static NiFiAtlasClient nifiClient;
    +    private AtlasClientV2 atlasClient;
    +
    +    private NiFiAtlasClient() {
    +        super();
    +    }
    +
    +    public static NiFiAtlasClient getInstance() {
    +        if (nifiClient == null) {
    +            synchronized (NiFiAtlasClient.class) {
    +                if (nifiClient == null) {
    +                    nifiClient = new NiFiAtlasClient();
    +                }
    +            }
    +        }
    +        return nifiClient;
    +    }
    +
    +    public void initialize(final String[] baseUrls, final AtlasAuthN authN, final File atlasConfDir) {
    +
    +        synchronized (NiFiAtlasClient.class) {
    +
    +            if (atlasClient != null) {
    +                logger.info("{} had been setup but replacing it with new one.", atlasClient);
    +                ApplicationProperties.forceReload();
    +            }
    +
    +            if (atlasConfDir != null) {
    +                // If atlasConfDir is not set, atlas-application.properties will be searched under classpath.
    +                Properties props = System.getProperties();
    +                final String atlasConfProp = "atlas.conf";
    +                props.setProperty(atlasConfProp, atlasConfDir.getAbsolutePath());
    +                logger.debug("{} has been set to: {}", atlasConfProp, props.getProperty(atlasConfProp));
    +            }
    +
    +            atlasClient = authN.createClient(baseUrls);
    +
    +        }
    +    }
    +
    +    /**
    +     * This is an utility method to delete unused types.
    +     * Should be used during development or testing only.
    +     * @param typeNames to delete
    +     */
    +    void deleteTypeDefs(String ... typeNames) throws AtlasServiceException {
    +        final AtlasTypesDef existingTypeDef = getTypeDefs(typeNames);
    +        try {
    +            atlasClient.deleteAtlasTypeDefs(existingTypeDef);
    +        } catch (UniformInterfaceException e) {
    +            if (e.getResponse().getStatus() == 204) {
    +                // 204 is a successful response.
    +                // NOTE: However after executing this, Atlas should be restarted to work properly.
    --- End diff --
    
    Is this a bug in Atlas? If so, has a JIRA been filed for the Atlas team? Should we be logging a warning here?


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157042460
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/NiFiAtlasClient.java ---
    @@ -0,0 +1,537 @@
    +/*
    + * 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.atlas;
    +
    +import com.sun.jersey.api.client.UniformInterfaceException;
    +import com.sun.jersey.core.util.MultivaluedMapImpl;
    +import org.apache.atlas.ApplicationProperties;
    +import org.apache.atlas.AtlasClientV2;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.atlas.model.SearchFilter;
    +import org.apache.atlas.model.instance.AtlasEntity;
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +import org.apache.atlas.model.instance.EntityMutationResponse;
    +import org.apache.atlas.model.typedef.AtlasEntityDef;
    +import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
    +import org.apache.atlas.model.typedef.AtlasTypesDef;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.util.StringUtils;
    +import org.apache.nifi.util.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.ws.rs.core.MultivaluedMap;
    +import java.io.File;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.function.Function;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.nifi.atlas.AtlasUtils.findIdByQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.getComponentIdFromQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.toStr;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.AS_IS;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.CREATED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.DELETED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.UPDATED;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_DESCRIPTION;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_FLOW_PATHS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_GUID;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUEUES;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_TYPENAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_URL;
    +import static org.apache.nifi.atlas.NiFiTypes.ENTITIES;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW_PATH;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_INPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_OUTPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_QUEUE;
    +
    +public class NiFiAtlasClient {
    +
    +    private static final Logger logger = LoggerFactory.getLogger(NiFiAtlasClient.class);
    +
    +    private static NiFiAtlasClient nifiClient;
    +    private AtlasClientV2 atlasClient;
    +
    +    private NiFiAtlasClient() {
    +        super();
    +    }
    +
    +    public static NiFiAtlasClient getInstance() {
    +        if (nifiClient == null) {
    +            synchronized (NiFiAtlasClient.class) {
    +                if (nifiClient == null) {
    +                    nifiClient = new NiFiAtlasClient();
    +                }
    +            }
    +        }
    +        return nifiClient;
    +    }
    +
    +    public void initialize(final String[] baseUrls, final AtlasAuthN authN, final File atlasConfDir) {
    +
    +        synchronized (NiFiAtlasClient.class) {
    +
    +            if (atlasClient != null) {
    +                logger.info("{} had been setup but replacing it with new one.", atlasClient);
    +                ApplicationProperties.forceReload();
    +            }
    +
    +            if (atlasConfDir != null) {
    +                // If atlasConfDir is not set, atlas-application.properties will be searched under classpath.
    +                Properties props = System.getProperties();
    +                final String atlasConfProp = "atlas.conf";
    +                props.setProperty(atlasConfProp, atlasConfDir.getAbsolutePath());
    +                logger.debug("{} has been set to: {}", atlasConfProp, props.getProperty(atlasConfProp));
    +            }
    +
    +            atlasClient = authN.createClient(baseUrls);
    +
    +        }
    +    }
    +
    +    /**
    +     * This is an utility method to delete unused types.
    +     * Should be used during development or testing only.
    +     * @param typeNames to delete
    +     */
    +    void deleteTypeDefs(String ... typeNames) throws AtlasServiceException {
    +        final AtlasTypesDef existingTypeDef = getTypeDefs(typeNames);
    +        try {
    +            atlasClient.deleteAtlasTypeDefs(existingTypeDef);
    +        } catch (UniformInterfaceException e) {
    +            if (e.getResponse().getStatus() == 204) {
    +                // 204 is a successful response.
    +                // NOTE: However after executing this, Atlas should be restarted to work properly.
    +                logger.info("Deleted type defs: {}", existingTypeDef);
    +            } else {
    +                throw e;
    +            }
    +        }
    +    }
    +
    +    /**
    +     * @return True when required NiFi types are already created.
    +     */
    +    public boolean isNiFiTypeDefsRegistered() throws AtlasServiceException {
    +        final Set<String> typeNames = ENTITIES.keySet();
    +        final Map<String, AtlasEntityDef> existingDefs = getTypeDefs(typeNames.toArray(new String[typeNames.size()])).getEntityDefs().stream()
    +                .collect(Collectors.toMap(AtlasEntityDef::getName, Function.identity()));
    +        return typeNames.stream().allMatch(existingDefs::containsKey);
    +    }
    +
    +    /**
    +     * Create or update NiFi types in Atlas type system.
    +     * @param update If false, doesn't perform anything if there is existing type def for the name.
    +     */
    +    public void registerNiFiTypeDefs(boolean update) throws AtlasServiceException {
    +        final Set<String> typeNames = ENTITIES.keySet();
    +        final Map<String, AtlasEntityDef> existingDefs = getTypeDefs(typeNames.toArray(new String[typeNames.size()])).getEntityDefs().stream()
    +                .collect(Collectors.toMap(AtlasEntityDef::getName, Function.identity()));
    +
    +
    +        final AtomicBoolean shouldUpdate = new AtomicBoolean(false);
    +
    +        final AtlasTypesDef type = new AtlasTypesDef();
    +
    +        typeNames.stream().filter(typeName -> {
    +            final AtlasEntityDef existingDef = existingDefs.get(typeName);
    +            if (existingDef != null) {
    +                // type is already defined.
    +                if (!update) {
    +                    return false;
    +                }
    +                shouldUpdate.set(true);
    +            }
    +            return true;
    +        }).forEach(typeName -> {
    +            final NiFiTypes.EntityDefinition def = ENTITIES.get(typeName);
    +
    +            final AtlasEntityDef entity = new AtlasEntityDef();
    +            type.getEntityDefs().add(entity);
    +
    +            entity.setName(typeName);
    +
    +            Set<String> superTypes = new HashSet<>();
    +            List<AtlasAttributeDef> attributes = new ArrayList<>();
    +
    +            def.define(entity, superTypes, attributes);
    +
    +            entity.setSuperTypes(superTypes);
    +            entity.setAttributeDefs(attributes);
    +        });
    +
    +        // Create or Update.
    +        final AtlasTypesDef atlasTypeDefsResult = shouldUpdate.get()
    +                ? atlasClient.updateAtlasTypeDefs(type)
    +                : atlasClient.createAtlasTypeDefs(type);
    +        logger.debug("Result={}", atlasTypeDefsResult);
    +    }
    +
    +    private AtlasTypesDef getTypeDefs(String ... typeNames) throws AtlasServiceException {
    +        final AtlasTypesDef typeDefs = new AtlasTypesDef();
    +        for (int i = 0; i < typeNames.length; i++) {
    +            final MultivaluedMap<String, String> searchParams = new MultivaluedMapImpl();
    +            searchParams.add(SearchFilter.PARAM_NAME, typeNames[i]);
    +            final AtlasTypesDef typeDef = atlasClient.getAllTypeDefs(new SearchFilter(searchParams));
    +            typeDefs.getEntityDefs().addAll(typeDef.getEntityDefs());
    +        }
    +        logger.debug("typeDefs={}", typeDefs);
    +        return typeDefs;
    +    }
    +
    +    private Pattern FLOW_PATH_URL_PATTERN = Pattern.compile("^http.+processGroupId=([0-9a-z\\-]+).*$");
    +    /**
    +     * Fetch existing NiFiFlow entity from Atlas.
    +     * @param rootProcessGroupId The id of a NiFi flow root process group.
    +     * @param clusterName The cluster name of a flow.
    +     * @return A NiFiFlow instance filled with retrieved data from Atlas. Status objects are left blank, e.g. ProcessorStatus.
    +     * @throws AtlasServiceException Thrown if requesting to Atlas API failed, including when the flow is not found.
    +     */
    +    public NiFiFlow fetchNiFiFlow(String rootProcessGroupId, String clusterName) throws AtlasServiceException {
    +
    +        final String qualifiedName = AtlasUtils.toQualifiedName(clusterName, rootProcessGroupId);
    +        final AtlasObjectId flowId = new AtlasObjectId(TYPE_NIFI_FLOW, ATTR_QUALIFIED_NAME, qualifiedName);
    +        final AtlasEntity.AtlasEntityWithExtInfo nifiFlowExt = searchEntityDef(flowId);
    +
    +        if (nifiFlowExt == null || nifiFlowExt.getEntity() == null) {
    +            return null;
    +        }
    +
    +        final AtlasEntity nifiFlowEntity = nifiFlowExt.getEntity();
    +        final Map<String, Object> attributes = nifiFlowEntity.getAttributes();
    +        final NiFiFlow nifiFlow = new NiFiFlow(rootProcessGroupId);
    +        nifiFlow.setExEntity(nifiFlowEntity);
    +        nifiFlow.setFlowName(toStr(attributes.get(ATTR_NAME)));
    +        nifiFlow.setClusterName(clusterName);
    +        nifiFlow.setUrl(toStr(attributes.get(ATTR_URL)));
    +        nifiFlow.setDescription(toStr(attributes.get(ATTR_DESCRIPTION)));
    +
    +        nifiFlow.getQueues().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_QUEUES))));
    +        nifiFlow.getRootInputPortEntities().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_INPUT_PORTS))));
    +        nifiFlow.getRootOutputPortEntities().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_OUTPUT_PORTS))));
    +
    +        final Map<String, NiFiFlowPath> flowPaths = nifiFlow.getFlowPaths();
    +        final Map<AtlasObjectId, AtlasEntity> flowPathEntities = toQualifiedNameIds(toAtlasObjectIds(attributes.get(ATTR_FLOW_PATHS)));
    +
    +        for (AtlasEntity flowPathEntity : flowPathEntities.values()) {
    +            final String pathQualifiedName = toStr(flowPathEntity.getAttribute(ATTR_QUALIFIED_NAME));
    +            final NiFiFlowPath flowPath = new NiFiFlowPath(getComponentIdFromQualifiedName(pathQualifiedName));
    +            if (flowPathEntity.hasAttribute(ATTR_URL)) {
    +                final Matcher urlMatcher = FLOW_PATH_URL_PATTERN.matcher(toStr(flowPathEntity.getAttribute(ATTR_URL)));
    +                if (urlMatcher.matches()) {
    +                    flowPath.setGroupId(urlMatcher.group(1));
    +                }
    +            }
    +            flowPath.setExEntity(flowPathEntity);
    +            flowPath.setName(toStr(flowPathEntity.getAttribute(ATTR_NAME)));
    +            flowPath.getInputs().addAll(toQualifiedNameIds(toAtlasObjectIds(flowPathEntity.getAttribute(ATTR_INPUTS))).keySet());
    +            flowPath.getOutputs().addAll(toQualifiedNameIds(toAtlasObjectIds(flowPathEntity.getAttribute(ATTR_OUTPUTS))).keySet());
    +            flowPath.startTrackingChanges(nifiFlow);
    +
    +            flowPaths.put(flowPath.getId(), flowPath);
    +        }
    +
    +        nifiFlow.startTrackingChanges();
    +        return nifiFlow;
    +    }
    +
    +    @SuppressWarnings("unchecked")
    +    private List<AtlasObjectId> toAtlasObjectIds(Object _references) {
    +        if (_references == null) {
    +            return Collections.emptyList();
    +        }
    +        List<Map<String, Object>> references = (List<Map<String, Object>>) _references;
    +        return references.stream()
    +                .map(ref -> new AtlasObjectId(toStr(ref.get(ATTR_GUID)), toStr(ref.get(ATTR_TYPENAME)), ref))
    +                .collect(Collectors.toList());
    +    }
    +
    +    /**
    +     * <p>AtlasObjectIds returned from Atlas have GUID, but do not have qualifiedName, while ones created by the reporting task
    +     * do not have GUID, but qualifiedName. AtlasObjectId.equals returns false for this combination.
    +     * In order to match ids correctly, this method converts fetches actual entities from ids to get qualifiedName attribute.</p>
    +     *
    +     * <p>Also, AtlasObjectIds returned from Atlas does not have entity state.
    +     * If Atlas is configured to use soft-delete (default), deleted ids are still returned.
    +     * Fetched entities are used to determine whether an AtlasObjectId is still active or deleted.
    +     * Deleted entities will not be included in the result of this method.
    +     * </p>
    +     * @param ids to convert
    +     * @return AtlasObjectIds with qualifiedName
    +     */
    +    private Map<AtlasObjectId, AtlasEntity> toQualifiedNameIds(List<AtlasObjectId> ids) {
    +        if (ids == null) {
    +            return Collections.emptyMap();
    +        }
    +
    +        return ids.stream().distinct().map(id -> {
    +            try {
    +                final AtlasEntity.AtlasEntityWithExtInfo entityExt = searchEntityDef(id);
    +                final AtlasEntity entity = entityExt.getEntity();
    +                if (AtlasEntity.Status.DELETED.equals(entity.getStatus())) {
    +                    return null;
    +                }
    +                final Map<String, Object> uniqueAttrs = Collections.singletonMap(ATTR_QUALIFIED_NAME, entity.getAttribute(ATTR_QUALIFIED_NAME));
    +                return new Tuple<>(new AtlasObjectId(id.getGuid(), id.getTypeName(), uniqueAttrs), entity);
    +            } catch (AtlasServiceException e) {
    +                return null;
    +            }
    +        }).filter(Objects::nonNull).collect(Collectors.toMap(Tuple::getKey, Tuple::getValue));
    +    }
    +
    +    public void registerNiFiFlow(NiFiFlow nifiFlow) throws AtlasServiceException {
    +
    +        // Create parent flow entity, so that common properties are taken over.
    +        final AtlasEntity flowEntity = registerNiFiFlowEntity(nifiFlow);
    +
    +        // Create DataSet entities those are created by this NiFi flow.
    +        final Map<String, List<AtlasEntity>> updatedDataSetEntities = registerDataSetEntities(nifiFlow);
    +
    +        // Create path entities.
    +        final Set<AtlasObjectId> remainingPathIds = registerFlowPathEntities(nifiFlow);
    +
    +        // Update these attributes only if anything is created, updated or removed.
    +        boolean shouldUpdateNiFiFlow = nifiFlow.isMetadataUpdated();
    +        if (remainingPathIds != null) {
    +            flowEntity.setAttribute(ATTR_FLOW_PATHS, remainingPathIds);
    +            shouldUpdateNiFiFlow = true;
    +        }
    +        if (updatedDataSetEntities.containsKey(TYPE_NIFI_QUEUE)) {
    +            flowEntity.setAttribute(ATTR_QUEUES, updatedDataSetEntities.get(TYPE_NIFI_QUEUE));
    +            shouldUpdateNiFiFlow = true;
    +        }
    +        if (updatedDataSetEntities.containsKey(TYPE_NIFI_INPUT_PORT)) {
    +            flowEntity.setAttribute(ATTR_INPUT_PORTS, updatedDataSetEntities.get(TYPE_NIFI_INPUT_PORT));
    +            shouldUpdateNiFiFlow = true;
    +        }
    +        if (updatedDataSetEntities.containsKey(TYPE_NIFI_OUTPUT_PORT)) {
    +            flowEntity.setAttribute(ATTR_OUTPUT_PORTS, updatedDataSetEntities.get(TYPE_NIFI_OUTPUT_PORT));
    +            shouldUpdateNiFiFlow = true;
    +        }
    +
    +        logger.debug("### NiFi Flow Audit Logs START");
    +        nifiFlow.getUpdateAudit().forEach(logger::debug);
    +        nifiFlow.getFlowPaths().forEach((k, v) -> {
    +            logger.debug("--- NiFiFlowPath Audit Logs: {}", k);
    +            v.getUpdateAudit().forEach(logger::debug);
    +        });
    +        logger.debug("### NiFi Flow Audit Logs END");
    +
    +        if (shouldUpdateNiFiFlow) {
    +            // Send updated entities.
    +            final List<AtlasEntity> entities = new ArrayList<>();
    +            final AtlasEntity.AtlasEntitiesWithExtInfo atlasEntities = new AtlasEntity.AtlasEntitiesWithExtInfo(entities);
    +            entities.add(flowEntity);
    +            try {
    +                final EntityMutationResponse mutationResponse = atlasClient.createEntities(atlasEntities);
    +                logger.debug("mutation response={}", mutationResponse);
    +            } catch (AtlasServiceException e) {
    +                if (e.getStatus().getStatusCode() == 404 && e.getMessage().contains("ATLAS-404-00-00B")) {
    --- End diff --
    
    Can this reference AtlasErrorCode.INSTANCE_NOT_FOUND directly instead of hardcoding "ATLAS-404-00-00B"?


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157222062
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/NiFIAtlasHook.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.atlas;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.atlas.hook.AtlasHook;
    +import org.apache.atlas.model.instance.AtlasEntity;
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +import org.apache.atlas.notification.hook.HookNotification.EntityPartialUpdateRequest;
    +import org.apache.atlas.notification.hook.HookNotification.HookNotificationMessage;
    +import org.apache.atlas.typesystem.Referenceable;
    +import org.apache.atlas.typesystem.persistence.Id;
    +import org.apache.nifi.atlas.provenance.lineage.LineageContext;
    +import org.apache.nifi.util.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.LinkedHashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.atlas.notification.hook.HookNotification.HookNotificationType.ENTITY_PARTIAL_UPDATE;
    +import static org.apache.nifi.atlas.AtlasUtils.toTypedQualifiedName;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_GUID;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_TYPENAME;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW_PATH;
    +
    +/**
    + * This class is not thread-safe as it holds uncommitted notification messages within instance.
    + * {@link #addMessage(HookNotificationMessage)} and {@link #commitMessages()} should be used serially from a single thread.
    + */
    +public class NiFIAtlasHook extends AtlasHook implements LineageContext {
    +
    +    public static final String NIFI_USER = "nifi";
    +
    +    private static final Logger logger = LoggerFactory.getLogger(NiFIAtlasHook.class);
    +    private static final String CONF_PREFIX = "atlas.hook.nifi.";
    +    private static final String HOOK_NUM_RETRIES = CONF_PREFIX + "numRetries";
    +
    +    private final NiFiAtlasClient atlasClient;
    +
    +    /**
    +     * An index to resolve a qualifiedName from a GUID.
    +     */
    +    private final Map<String, String> guidToQualifiedName;
    +    /**
    +     * An index to resolve a Referenceable from a typeName::qualifiedName.
    +     */
    +    private final Map<String, Referenceable> typedQualifiedNameToRef;
    +
    +
    +    private static <K, V> Map<K, V> createCache(final int maxSize) {
    +        return new LinkedHashMap<K, V>(maxSize, 0.75f, true) {
    --- End diff --
    
    Well, it's just a private method to create cache instances within this class. So if different default optimizations are needed, then we can do that here as well without affecting others.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157035283
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/NiFIAtlasHook.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.atlas;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.atlas.hook.AtlasHook;
    +import org.apache.atlas.model.instance.AtlasEntity;
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +import org.apache.atlas.notification.hook.HookNotification.EntityPartialUpdateRequest;
    +import org.apache.atlas.notification.hook.HookNotification.HookNotificationMessage;
    +import org.apache.atlas.typesystem.Referenceable;
    +import org.apache.atlas.typesystem.persistence.Id;
    +import org.apache.nifi.atlas.provenance.lineage.LineageContext;
    +import org.apache.nifi.util.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.LinkedHashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.atlas.notification.hook.HookNotification.HookNotificationType.ENTITY_PARTIAL_UPDATE;
    +import static org.apache.nifi.atlas.AtlasUtils.toTypedQualifiedName;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_GUID;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_TYPENAME;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW_PATH;
    +
    +/**
    + * This class is not thread-safe as it holds uncommitted notification messages within instance.
    + * {@link #addMessage(HookNotificationMessage)} and {@link #commitMessages()} should be used serially from a single thread.
    + */
    +public class NiFIAtlasHook extends AtlasHook implements LineageContext {
    +
    +    public static final String NIFI_USER = "nifi";
    +
    +    private static final Logger logger = LoggerFactory.getLogger(NiFIAtlasHook.class);
    +    private static final String CONF_PREFIX = "atlas.hook.nifi.";
    +    private static final String HOOK_NUM_RETRIES = CONF_PREFIX + "numRetries";
    +
    +    private final NiFiAtlasClient atlasClient;
    +
    +    /**
    +     * An index to resolve a qualifiedName from a GUID.
    +     */
    +    private final Map<String, String> guidToQualifiedName;
    +    /**
    +     * An index to resolve a Referenceable from a typeName::qualifiedName.
    +     */
    +    private final Map<String, Referenceable> typedQualifiedNameToRef;
    +
    +
    +    private static <K, V> Map<K, V> createCache(final int maxSize) {
    +        return new LinkedHashMap<K, V>(maxSize, 0.75f, true) {
    --- End diff --
    
    The default load factor is currently 0.75. Is there a reason for fixing the value versus allowing future default optimizations?


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r156982630
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/NiFiAtlasClient.java ---
    @@ -0,0 +1,537 @@
    +/*
    + * 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.atlas;
    +
    +import com.sun.jersey.api.client.UniformInterfaceException;
    +import com.sun.jersey.core.util.MultivaluedMapImpl;
    +import org.apache.atlas.ApplicationProperties;
    +import org.apache.atlas.AtlasClientV2;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.atlas.model.SearchFilter;
    +import org.apache.atlas.model.instance.AtlasEntity;
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +import org.apache.atlas.model.instance.EntityMutationResponse;
    +import org.apache.atlas.model.typedef.AtlasEntityDef;
    +import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
    +import org.apache.atlas.model.typedef.AtlasTypesDef;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.util.StringUtils;
    +import org.apache.nifi.util.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.ws.rs.core.MultivaluedMap;
    +import java.io.File;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.function.Function;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.nifi.atlas.AtlasUtils.findIdByQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.getComponentIdFromQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.toStr;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.AS_IS;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.CREATED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.DELETED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.UPDATED;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_DESCRIPTION;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_FLOW_PATHS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_GUID;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUEUES;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_TYPENAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_URL;
    +import static org.apache.nifi.atlas.NiFiTypes.ENTITIES;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW_PATH;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_INPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_OUTPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_QUEUE;
    +
    +public class NiFiAtlasClient {
    +
    +    private static final Logger logger = LoggerFactory.getLogger(NiFiAtlasClient.class);
    +
    +    private static NiFiAtlasClient nifiClient;
    +    private AtlasClientV2 atlasClient;
    +
    +    private NiFiAtlasClient() {
    +        super();
    +    }
    +
    +    public static NiFiAtlasClient getInstance() {
    +        if (nifiClient == null) {
    +            synchronized (NiFiAtlasClient.class) {
    +                if (nifiClient == null) {
    +                    nifiClient = new NiFiAtlasClient();
    +                }
    +            }
    +        }
    +        return nifiClient;
    +    }
    +
    +    public void initialize(final String[] baseUrls, final AtlasAuthN authN, final File atlasConfDir) {
    +
    +        synchronized (NiFiAtlasClient.class) {
    +
    +            if (atlasClient != null) {
    +                logger.info("{} had been setup but replacing it with new one.", atlasClient);
    +                ApplicationProperties.forceReload();
    +            }
    +
    +            if (atlasConfDir != null) {
    +                // If atlasConfDir is not set, atlas-application.properties will be searched under classpath.
    +                Properties props = System.getProperties();
    +                final String atlasConfProp = "atlas.conf";
    +                props.setProperty(atlasConfProp, atlasConfDir.getAbsolutePath());
    +                logger.debug("{} has been set to: {}", atlasConfProp, props.getProperty(atlasConfProp));
    +            }
    +
    +            atlasClient = authN.createClient(baseUrls);
    +
    +        }
    +    }
    +
    +    /**
    +     * This is an utility method to delete unused types.
    +     * Should be used during development or testing only.
    +     * @param typeNames to delete
    +     */
    +    void deleteTypeDefs(String ... typeNames) throws AtlasServiceException {
    +        final AtlasTypesDef existingTypeDef = getTypeDefs(typeNames);
    +        try {
    +            atlasClient.deleteAtlasTypeDefs(existingTypeDef);
    +        } catch (UniformInterfaceException e) {
    +            if (e.getResponse().getStatus() == 204) {
    +                // 204 is a successful response.
    +                // NOTE: However after executing this, Atlas should be restarted to work properly.
    +                logger.info("Deleted type defs: {}", existingTypeDef);
    +            } else {
    +                throw e;
    +            }
    +        }
    +    }
    +
    +    /**
    +     * @return True when required NiFi types are already created.
    +     */
    +    public boolean isNiFiTypeDefsRegistered() throws AtlasServiceException {
    +        final Set<String> typeNames = ENTITIES.keySet();
    +        final Map<String, AtlasEntityDef> existingDefs = getTypeDefs(typeNames.toArray(new String[typeNames.size()])).getEntityDefs().stream()
    +                .collect(Collectors.toMap(AtlasEntityDef::getName, Function.identity()));
    +        return typeNames.stream().allMatch(existingDefs::containsKey);
    +    }
    +
    +    /**
    +     * Create or update NiFi types in Atlas type system.
    +     * @param update If false, doesn't perform anything if there is existing type def for the name.
    +     */
    +    public void registerNiFiTypeDefs(boolean update) throws AtlasServiceException {
    +        final Set<String> typeNames = ENTITIES.keySet();
    +        final Map<String, AtlasEntityDef> existingDefs = getTypeDefs(typeNames.toArray(new String[typeNames.size()])).getEntityDefs().stream()
    +                .collect(Collectors.toMap(AtlasEntityDef::getName, Function.identity()));
    +
    +
    +        final AtomicBoolean shouldUpdate = new AtomicBoolean(false);
    +
    +        final AtlasTypesDef type = new AtlasTypesDef();
    +
    +        typeNames.stream().filter(typeName -> {
    +            final AtlasEntityDef existingDef = existingDefs.get(typeName);
    +            if (existingDef != null) {
    +                // type is already defined.
    +                if (!update) {
    +                    return false;
    +                }
    +                shouldUpdate.set(true);
    +            }
    +            return true;
    +        }).forEach(typeName -> {
    +            final NiFiTypes.EntityDefinition def = ENTITIES.get(typeName);
    +
    +            final AtlasEntityDef entity = new AtlasEntityDef();
    +            type.getEntityDefs().add(entity);
    +
    +            entity.setName(typeName);
    +
    +            Set<String> superTypes = new HashSet<>();
    +            List<AtlasAttributeDef> attributes = new ArrayList<>();
    +
    +            def.define(entity, superTypes, attributes);
    +
    +            entity.setSuperTypes(superTypes);
    +            entity.setAttributeDefs(attributes);
    +        });
    +
    +        // Create or Update.
    +        final AtlasTypesDef atlasTypeDefsResult = shouldUpdate.get()
    +                ? atlasClient.updateAtlasTypeDefs(type)
    +                : atlasClient.createAtlasTypeDefs(type);
    +        logger.debug("Result={}", atlasTypeDefsResult);
    +    }
    +
    +    private AtlasTypesDef getTypeDefs(String ... typeNames) throws AtlasServiceException {
    +        final AtlasTypesDef typeDefs = new AtlasTypesDef();
    +        for (int i = 0; i < typeNames.length; i++) {
    +            final MultivaluedMap<String, String> searchParams = new MultivaluedMapImpl();
    +            searchParams.add(SearchFilter.PARAM_NAME, typeNames[i]);
    +            final AtlasTypesDef typeDef = atlasClient.getAllTypeDefs(new SearchFilter(searchParams));
    +            typeDefs.getEntityDefs().addAll(typeDef.getEntityDefs());
    +        }
    +        logger.debug("typeDefs={}", typeDefs);
    +        return typeDefs;
    +    }
    +
    +    private Pattern FLOW_PATH_URL_PATTERN = Pattern.compile("^http.+processGroupId=([0-9a-z\\-]+).*$");
    +    /**
    +     * Fetch existing NiFiFlow entity from Atlas.
    +     * @param rootProcessGroupId The id of a NiFi flow root process group.
    +     * @param clusterName The cluster name of a flow.
    +     * @return A NiFiFlow instance filled with retrieved data from Atlas. Status objects are left blank, e.g. ProcessorStatus.
    +     * @throws AtlasServiceException Thrown if requesting to Atlas API failed, including when the flow is not found.
    +     */
    +    public NiFiFlow fetchNiFiFlow(String rootProcessGroupId, String clusterName) throws AtlasServiceException {
    +
    +        final String qualifiedName = AtlasUtils.toQualifiedName(clusterName, rootProcessGroupId);
    +        final AtlasObjectId flowId = new AtlasObjectId(TYPE_NIFI_FLOW, ATTR_QUALIFIED_NAME, qualifiedName);
    +        final AtlasEntity.AtlasEntityWithExtInfo nifiFlowExt = searchEntityDef(flowId);
    +
    +        if (nifiFlowExt == null || nifiFlowExt.getEntity() == null) {
    +            return null;
    +        }
    +
    +        final AtlasEntity nifiFlowEntity = nifiFlowExt.getEntity();
    +        final Map<String, Object> attributes = nifiFlowEntity.getAttributes();
    +        final NiFiFlow nifiFlow = new NiFiFlow(rootProcessGroupId);
    +        nifiFlow.setExEntity(nifiFlowEntity);
    +        nifiFlow.setFlowName(toStr(attributes.get(ATTR_NAME)));
    +        nifiFlow.setClusterName(clusterName);
    +        nifiFlow.setUrl(toStr(attributes.get(ATTR_URL)));
    +        nifiFlow.setDescription(toStr(attributes.get(ATTR_DESCRIPTION)));
    +
    +        nifiFlow.getQueues().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_QUEUES))));
    +        nifiFlow.getRootInputPortEntities().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_INPUT_PORTS))));
    +        nifiFlow.getRootOutputPortEntities().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_OUTPUT_PORTS))));
    +
    +        final Map<String, NiFiFlowPath> flowPaths = nifiFlow.getFlowPaths();
    +        final Map<AtlasObjectId, AtlasEntity> flowPathEntities = toQualifiedNameIds(toAtlasObjectIds(attributes.get(ATTR_FLOW_PATHS)));
    +
    +        for (AtlasEntity flowPathEntity : flowPathEntities.values()) {
    +            final String pathQualifiedName = toStr(flowPathEntity.getAttribute(ATTR_QUALIFIED_NAME));
    +            final NiFiFlowPath flowPath = new NiFiFlowPath(getComponentIdFromQualifiedName(pathQualifiedName));
    +            if (flowPathEntity.hasAttribute(ATTR_URL)) {
    +                final Matcher urlMatcher = FLOW_PATH_URL_PATTERN.matcher(toStr(flowPathEntity.getAttribute(ATTR_URL)));
    +                if (urlMatcher.matches()) {
    +                    flowPath.setGroupId(urlMatcher.group(1));
    +                }
    +            }
    +            flowPath.setExEntity(flowPathEntity);
    +            flowPath.setName(toStr(flowPathEntity.getAttribute(ATTR_NAME)));
    +            flowPath.getInputs().addAll(toQualifiedNameIds(toAtlasObjectIds(flowPathEntity.getAttribute(ATTR_INPUTS))).keySet());
    +            flowPath.getOutputs().addAll(toQualifiedNameIds(toAtlasObjectIds(flowPathEntity.getAttribute(ATTR_OUTPUTS))).keySet());
    +            flowPath.startTrackingChanges(nifiFlow);
    +
    +            flowPaths.put(flowPath.getId(), flowPath);
    +        }
    +
    +        nifiFlow.startTrackingChanges();
    +        return nifiFlow;
    +    }
    +
    +    @SuppressWarnings("unchecked")
    +    private List<AtlasObjectId> toAtlasObjectIds(Object _references) {
    +        if (_references == null) {
    +            return Collections.emptyList();
    +        }
    +        List<Map<String, Object>> references = (List<Map<String, Object>>) _references;
    +        return references.stream()
    +                .map(ref -> new AtlasObjectId(toStr(ref.get(ATTR_GUID)), toStr(ref.get(ATTR_TYPENAME)), ref))
    +                .collect(Collectors.toList());
    +    }
    +
    +    /**
    +     * <p>AtlasObjectIds returned from Atlas have GUID, but do not have qualifiedName, while ones created by the reporting task
    +     * do not have GUID, but qualifiedName. AtlasObjectId.equals returns false for this combination.
    +     * In order to match ids correctly, this method converts fetches actual entities from ids to get qualifiedName attribute.</p>
    +     *
    +     * <p>Also, AtlasObjectIds returned from Atlas does not have entity state.
    +     * If Atlas is configured to use soft-delete (default), deleted ids are still returned.
    +     * Fetched entities are used to determine whether an AtlasObjectId is still active or deleted.
    +     * Deleted entities will not be included in the result of this method.
    +     * </p>
    +     * @param ids to convert
    +     * @return AtlasObjectIds with qualifiedName
    +     */
    +    private Map<AtlasObjectId, AtlasEntity> toQualifiedNameIds(List<AtlasObjectId> ids) {
    +        if (ids == null) {
    +            return Collections.emptyMap();
    +        }
    +
    +        return ids.stream().distinct().map(id -> {
    +            try {
    +                final AtlasEntity.AtlasEntityWithExtInfo entityExt = searchEntityDef(id);
    +                final AtlasEntity entity = entityExt.getEntity();
    +                if (AtlasEntity.Status.DELETED.equals(entity.getStatus())) {
    +                    return null;
    +                }
    +                final Map<String, Object> uniqueAttrs = Collections.singletonMap(ATTR_QUALIFIED_NAME, entity.getAttribute(ATTR_QUALIFIED_NAME));
    +                return new Tuple<>(new AtlasObjectId(id.getGuid(), id.getTypeName(), uniqueAttrs), entity);
    +            } catch (AtlasServiceException e) {
    +                return null;
    +            }
    +        }).filter(Objects::nonNull).collect(Collectors.toMap(Tuple::getKey, Tuple::getValue));
    +    }
    +
    +    public void registerNiFiFlow(NiFiFlow nifiFlow) throws AtlasServiceException {
    +
    +        // Create parent flow entity, so that common properties are taken over.
    +        final AtlasEntity flowEntity = registerNiFiFlowEntity(nifiFlow);
    +
    +        // Create DataSet entities those are created by this NiFi flow.
    +        final Map<String, List<AtlasEntity>> updatedDataSetEntities = registerDataSetEntities(nifiFlow);
    +
    +        // Create path entities.
    +        final Set<AtlasObjectId> remainingPathIds = registerFlowPathEntities(nifiFlow);
    +
    +        // Update these attributes only if anything is created, updated or removed.
    +        boolean shouldUpdateNiFiFlow = nifiFlow.isMetadataUpdated();
    +        if (remainingPathIds != null) {
    +            flowEntity.setAttribute(ATTR_FLOW_PATHS, remainingPathIds);
    +            shouldUpdateNiFiFlow = true;
    +        }
    +        if (updatedDataSetEntities.containsKey(TYPE_NIFI_QUEUE)) {
    +            flowEntity.setAttribute(ATTR_QUEUES, updatedDataSetEntities.get(TYPE_NIFI_QUEUE));
    +            shouldUpdateNiFiFlow = true;
    +        }
    +        if (updatedDataSetEntities.containsKey(TYPE_NIFI_INPUT_PORT)) {
    +            flowEntity.setAttribute(ATTR_INPUT_PORTS, updatedDataSetEntities.get(TYPE_NIFI_INPUT_PORT));
    +            shouldUpdateNiFiFlow = true;
    +        }
    +        if (updatedDataSetEntities.containsKey(TYPE_NIFI_OUTPUT_PORT)) {
    +            flowEntity.setAttribute(ATTR_OUTPUT_PORTS, updatedDataSetEntities.get(TYPE_NIFI_OUTPUT_PORT));
    +            shouldUpdateNiFiFlow = true;
    +        }
    +
    +        logger.debug("### NiFi Flow Audit Logs START");
    --- End diff --
    
    This whole block should probably be wrapped in a check if logger.isDebugEnabled()


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157214860
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/lineage/SimpleFlowPathLineage.java ---
    @@ -0,0 +1,205 @@
    +/*
    + * 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.atlas.provenance.lineage;
    +
    +import org.apache.atlas.typesystem.Referenceable;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowPath;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.DataSetRefs;
    +import org.apache.nifi.controller.status.ConnectionStatus;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.lineage.ComputeLineageResult;
    +import org.apache.nifi.provenance.lineage.LineageEdge;
    +import org.apache.nifi.provenance.lineage.LineageNode;
    +import org.apache.nifi.provenance.lineage.LineageNodeType;
    +
    +import java.util.List;
    +
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_QUEUE;
    +
    +public class SimpleFlowPathLineage extends AbstractLineageStrategy {
    +
    +    @Override
    +    public void processEvent(AnalysisContext analysisContext, NiFiFlow nifiFlow, ProvenanceEventRecord event) {
    +        final DataSetRefs refs = executeAnalyzer(analysisContext, event);
    +        if (refs == null || (refs.isEmpty())) {
    +            return;
    +        }
    +
    +        if ("Remote Input Port".equals(event.getComponentType()) || "Remote Output Port".equals(event.getComponentType())) {
    +            processRemotePortEvent(analysisContext, nifiFlow, event, refs);
    +        } else {
    +            addDataSetRefs(nifiFlow, refs);
    +        }
    +
    +    }
    +
    +    /**
    +     * Create a flow_path entity corresponding to the target RemoteGroupPort when a SEND/RECEIVE event are received.
    +     * Because such entity can not be created in advance while analyzing flow statically,
    +     * as ReportingTask can not determine whether a component id is a RemoteGroupPort,
    +     * since connectionStatus is the only available information in ReportingContext.
    +     * ConnectionStatus only knows component id, component type is unknown.
    +     * For example, there is no difference to tell if a connected component is a funnel or a RemoteGroupPort.
    +     */
    +    private void processRemotePortEvent(AnalysisContext analysisContext, NiFiFlow nifiFlow, ProvenanceEventRecord event, DataSetRefs analyzedRefs) {
    +
    +        final boolean isRemoteInputPort = "Remote Input Port".equals(event.getComponentType());
    +
    +        // Create a RemoteInputPort Process.
    +        // event.getComponentId returns UUID for RemoteGroupPort as a client of S2S, and it's different from a remote port UUID (portDataSetid).
    +        // See NIFI-4571 for detail.
    +        final Referenceable remotePortDataSet = isRemoteInputPort ? analyzedRefs.getOutputs().iterator().next() :  analyzedRefs.getInputs().iterator().next();
    +        final String portProcessId = event.getComponentId();
    +
    +        final NiFiFlowPath remotePortProcess = new NiFiFlowPath(portProcessId);
    +        remotePortProcess.setName(event.getComponentType());
    +        remotePortProcess.addProcessor(portProcessId);
    +
    +        // For RemoteInputPort, need to find the previous component connected to this port,
    +        // which passed this particular FlowFile.
    +        // That is only possible by calling lineage API.
    +        if (isRemoteInputPort) {
    +            final ProvenanceEventRecord previousEvent = findPreviousProvenanceEvent(analysisContext, event);
    +            if (previousEvent == null) {
    +                logger.warn("Previous event was not found: {}", new Object[]{event});
    +                return;
    +            }
    +
    +            // Set groupId from incoming connection if available.
    +            final List<ConnectionStatus> incomingRelationShips = nifiFlow.getIncomingRelationShips(portProcessId);
    --- End diff --
    
    Please excuse my ignorance, but is there any difference in 'connection' and 'relationship' in NiFi terminology? Probably I'm missing something..


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r156982816
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/NiFiAtlasClient.java ---
    @@ -0,0 +1,537 @@
    +/*
    + * 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.atlas;
    +
    +import com.sun.jersey.api.client.UniformInterfaceException;
    +import com.sun.jersey.core.util.MultivaluedMapImpl;
    +import org.apache.atlas.ApplicationProperties;
    +import org.apache.atlas.AtlasClientV2;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.atlas.model.SearchFilter;
    +import org.apache.atlas.model.instance.AtlasEntity;
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +import org.apache.atlas.model.instance.EntityMutationResponse;
    +import org.apache.atlas.model.typedef.AtlasEntityDef;
    +import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
    +import org.apache.atlas.model.typedef.AtlasTypesDef;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.util.StringUtils;
    +import org.apache.nifi.util.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.ws.rs.core.MultivaluedMap;
    +import java.io.File;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.function.Function;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.nifi.atlas.AtlasUtils.findIdByQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.getComponentIdFromQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.toStr;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.AS_IS;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.CREATED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.DELETED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.UPDATED;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_DESCRIPTION;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_FLOW_PATHS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_GUID;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUEUES;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_TYPENAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_URL;
    +import static org.apache.nifi.atlas.NiFiTypes.ENTITIES;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW_PATH;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_INPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_OUTPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_QUEUE;
    +
    +public class NiFiAtlasClient {
    +
    +    private static final Logger logger = LoggerFactory.getLogger(NiFiAtlasClient.class);
    +
    +    private static NiFiAtlasClient nifiClient;
    +    private AtlasClientV2 atlasClient;
    +
    +    private NiFiAtlasClient() {
    +        super();
    +    }
    +
    +    public static NiFiAtlasClient getInstance() {
    +        if (nifiClient == null) {
    +            synchronized (NiFiAtlasClient.class) {
    +                if (nifiClient == null) {
    +                    nifiClient = new NiFiAtlasClient();
    +                }
    +            }
    +        }
    +        return nifiClient;
    +    }
    +
    +    public void initialize(final String[] baseUrls, final AtlasAuthN authN, final File atlasConfDir) {
    +
    +        synchronized (NiFiAtlasClient.class) {
    +
    +            if (atlasClient != null) {
    +                logger.info("{} had been setup but replacing it with new one.", atlasClient);
    +                ApplicationProperties.forceReload();
    +            }
    +
    +            if (atlasConfDir != null) {
    +                // If atlasConfDir is not set, atlas-application.properties will be searched under classpath.
    +                Properties props = System.getProperties();
    +                final String atlasConfProp = "atlas.conf";
    +                props.setProperty(atlasConfProp, atlasConfDir.getAbsolutePath());
    +                logger.debug("{} has been set to: {}", atlasConfProp, props.getProperty(atlasConfProp));
    +            }
    +
    +            atlasClient = authN.createClient(baseUrls);
    +
    +        }
    +    }
    +
    +    /**
    +     * This is an utility method to delete unused types.
    +     * Should be used during development or testing only.
    +     * @param typeNames to delete
    +     */
    +    void deleteTypeDefs(String ... typeNames) throws AtlasServiceException {
    +        final AtlasTypesDef existingTypeDef = getTypeDefs(typeNames);
    +        try {
    +            atlasClient.deleteAtlasTypeDefs(existingTypeDef);
    +        } catch (UniformInterfaceException e) {
    +            if (e.getResponse().getStatus() == 204) {
    +                // 204 is a successful response.
    +                // NOTE: However after executing this, Atlas should be restarted to work properly.
    +                logger.info("Deleted type defs: {}", existingTypeDef);
    +            } else {
    +                throw e;
    +            }
    +        }
    +    }
    +
    +    /**
    +     * @return True when required NiFi types are already created.
    +     */
    +    public boolean isNiFiTypeDefsRegistered() throws AtlasServiceException {
    +        final Set<String> typeNames = ENTITIES.keySet();
    +        final Map<String, AtlasEntityDef> existingDefs = getTypeDefs(typeNames.toArray(new String[typeNames.size()])).getEntityDefs().stream()
    +                .collect(Collectors.toMap(AtlasEntityDef::getName, Function.identity()));
    +        return typeNames.stream().allMatch(existingDefs::containsKey);
    +    }
    +
    +    /**
    +     * Create or update NiFi types in Atlas type system.
    +     * @param update If false, doesn't perform anything if there is existing type def for the name.
    +     */
    +    public void registerNiFiTypeDefs(boolean update) throws AtlasServiceException {
    +        final Set<String> typeNames = ENTITIES.keySet();
    +        final Map<String, AtlasEntityDef> existingDefs = getTypeDefs(typeNames.toArray(new String[typeNames.size()])).getEntityDefs().stream()
    +                .collect(Collectors.toMap(AtlasEntityDef::getName, Function.identity()));
    +
    +
    +        final AtomicBoolean shouldUpdate = new AtomicBoolean(false);
    +
    +        final AtlasTypesDef type = new AtlasTypesDef();
    +
    +        typeNames.stream().filter(typeName -> {
    +            final AtlasEntityDef existingDef = existingDefs.get(typeName);
    +            if (existingDef != null) {
    +                // type is already defined.
    +                if (!update) {
    +                    return false;
    +                }
    +                shouldUpdate.set(true);
    +            }
    +            return true;
    +        }).forEach(typeName -> {
    +            final NiFiTypes.EntityDefinition def = ENTITIES.get(typeName);
    +
    +            final AtlasEntityDef entity = new AtlasEntityDef();
    +            type.getEntityDefs().add(entity);
    +
    +            entity.setName(typeName);
    +
    +            Set<String> superTypes = new HashSet<>();
    +            List<AtlasAttributeDef> attributes = new ArrayList<>();
    +
    +            def.define(entity, superTypes, attributes);
    +
    +            entity.setSuperTypes(superTypes);
    +            entity.setAttributeDefs(attributes);
    +        });
    +
    +        // Create or Update.
    +        final AtlasTypesDef atlasTypeDefsResult = shouldUpdate.get()
    +                ? atlasClient.updateAtlasTypeDefs(type)
    +                : atlasClient.createAtlasTypeDefs(type);
    +        logger.debug("Result={}", atlasTypeDefsResult);
    +    }
    +
    +    private AtlasTypesDef getTypeDefs(String ... typeNames) throws AtlasServiceException {
    +        final AtlasTypesDef typeDefs = new AtlasTypesDef();
    +        for (int i = 0; i < typeNames.length; i++) {
    +            final MultivaluedMap<String, String> searchParams = new MultivaluedMapImpl();
    +            searchParams.add(SearchFilter.PARAM_NAME, typeNames[i]);
    +            final AtlasTypesDef typeDef = atlasClient.getAllTypeDefs(new SearchFilter(searchParams));
    +            typeDefs.getEntityDefs().addAll(typeDef.getEntityDefs());
    +        }
    +        logger.debug("typeDefs={}", typeDefs);
    +        return typeDefs;
    +    }
    +
    +    private Pattern FLOW_PATH_URL_PATTERN = Pattern.compile("^http.+processGroupId=([0-9a-z\\-]+).*$");
    +    /**
    +     * Fetch existing NiFiFlow entity from Atlas.
    +     * @param rootProcessGroupId The id of a NiFi flow root process group.
    +     * @param clusterName The cluster name of a flow.
    +     * @return A NiFiFlow instance filled with retrieved data from Atlas. Status objects are left blank, e.g. ProcessorStatus.
    +     * @throws AtlasServiceException Thrown if requesting to Atlas API failed, including when the flow is not found.
    +     */
    +    public NiFiFlow fetchNiFiFlow(String rootProcessGroupId, String clusterName) throws AtlasServiceException {
    +
    +        final String qualifiedName = AtlasUtils.toQualifiedName(clusterName, rootProcessGroupId);
    +        final AtlasObjectId flowId = new AtlasObjectId(TYPE_NIFI_FLOW, ATTR_QUALIFIED_NAME, qualifiedName);
    +        final AtlasEntity.AtlasEntityWithExtInfo nifiFlowExt = searchEntityDef(flowId);
    +
    +        if (nifiFlowExt == null || nifiFlowExt.getEntity() == null) {
    +            return null;
    +        }
    +
    +        final AtlasEntity nifiFlowEntity = nifiFlowExt.getEntity();
    +        final Map<String, Object> attributes = nifiFlowEntity.getAttributes();
    +        final NiFiFlow nifiFlow = new NiFiFlow(rootProcessGroupId);
    +        nifiFlow.setExEntity(nifiFlowEntity);
    +        nifiFlow.setFlowName(toStr(attributes.get(ATTR_NAME)));
    +        nifiFlow.setClusterName(clusterName);
    +        nifiFlow.setUrl(toStr(attributes.get(ATTR_URL)));
    +        nifiFlow.setDescription(toStr(attributes.get(ATTR_DESCRIPTION)));
    +
    +        nifiFlow.getQueues().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_QUEUES))));
    +        nifiFlow.getRootInputPortEntities().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_INPUT_PORTS))));
    +        nifiFlow.getRootOutputPortEntities().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_OUTPUT_PORTS))));
    +
    +        final Map<String, NiFiFlowPath> flowPaths = nifiFlow.getFlowPaths();
    +        final Map<AtlasObjectId, AtlasEntity> flowPathEntities = toQualifiedNameIds(toAtlasObjectIds(attributes.get(ATTR_FLOW_PATHS)));
    +
    +        for (AtlasEntity flowPathEntity : flowPathEntities.values()) {
    +            final String pathQualifiedName = toStr(flowPathEntity.getAttribute(ATTR_QUALIFIED_NAME));
    +            final NiFiFlowPath flowPath = new NiFiFlowPath(getComponentIdFromQualifiedName(pathQualifiedName));
    +            if (flowPathEntity.hasAttribute(ATTR_URL)) {
    +                final Matcher urlMatcher = FLOW_PATH_URL_PATTERN.matcher(toStr(flowPathEntity.getAttribute(ATTR_URL)));
    +                if (urlMatcher.matches()) {
    +                    flowPath.setGroupId(urlMatcher.group(1));
    +                }
    +            }
    +            flowPath.setExEntity(flowPathEntity);
    +            flowPath.setName(toStr(flowPathEntity.getAttribute(ATTR_NAME)));
    +            flowPath.getInputs().addAll(toQualifiedNameIds(toAtlasObjectIds(flowPathEntity.getAttribute(ATTR_INPUTS))).keySet());
    +            flowPath.getOutputs().addAll(toQualifiedNameIds(toAtlasObjectIds(flowPathEntity.getAttribute(ATTR_OUTPUTS))).keySet());
    +            flowPath.startTrackingChanges(nifiFlow);
    +
    +            flowPaths.put(flowPath.getId(), flowPath);
    +        }
    +
    +        nifiFlow.startTrackingChanges();
    +        return nifiFlow;
    +    }
    +
    +    @SuppressWarnings("unchecked")
    +    private List<AtlasObjectId> toAtlasObjectIds(Object _references) {
    +        if (_references == null) {
    +            return Collections.emptyList();
    +        }
    +        List<Map<String, Object>> references = (List<Map<String, Object>>) _references;
    +        return references.stream()
    +                .map(ref -> new AtlasObjectId(toStr(ref.get(ATTR_GUID)), toStr(ref.get(ATTR_TYPENAME)), ref))
    +                .collect(Collectors.toList());
    +    }
    +
    +    /**
    +     * <p>AtlasObjectIds returned from Atlas have GUID, but do not have qualifiedName, while ones created by the reporting task
    +     * do not have GUID, but qualifiedName. AtlasObjectId.equals returns false for this combination.
    +     * In order to match ids correctly, this method converts fetches actual entities from ids to get qualifiedName attribute.</p>
    +     *
    +     * <p>Also, AtlasObjectIds returned from Atlas does not have entity state.
    +     * If Atlas is configured to use soft-delete (default), deleted ids are still returned.
    +     * Fetched entities are used to determine whether an AtlasObjectId is still active or deleted.
    +     * Deleted entities will not be included in the result of this method.
    +     * </p>
    +     * @param ids to convert
    +     * @return AtlasObjectIds with qualifiedName
    +     */
    +    private Map<AtlasObjectId, AtlasEntity> toQualifiedNameIds(List<AtlasObjectId> ids) {
    +        if (ids == null) {
    +            return Collections.emptyMap();
    +        }
    +
    +        return ids.stream().distinct().map(id -> {
    +            try {
    +                final AtlasEntity.AtlasEntityWithExtInfo entityExt = searchEntityDef(id);
    +                final AtlasEntity entity = entityExt.getEntity();
    +                if (AtlasEntity.Status.DELETED.equals(entity.getStatus())) {
    +                    return null;
    +                }
    +                final Map<String, Object> uniqueAttrs = Collections.singletonMap(ATTR_QUALIFIED_NAME, entity.getAttribute(ATTR_QUALIFIED_NAME));
    +                return new Tuple<>(new AtlasObjectId(id.getGuid(), id.getTypeName(), uniqueAttrs), entity);
    +            } catch (AtlasServiceException e) {
    +                return null;
    --- End diff --
    
    If an Exception is being thrown we should probably be at least logging it, rather than just swallowing it, no?


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157217791
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/AtlasNiFiFlowLineage.java ---
    @@ -0,0 +1,714 @@
    +/*
    + * 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.atlas.reporting;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.common.config.SslConfigs;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
    +import org.apache.nifi.annotation.behavior.Stateful;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.atlas.NiFIAtlasHook;
    +import org.apache.nifi.atlas.NiFiAtlasClient;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowAnalyzer;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.StandardAnalysisContext;
    +import org.apache.nifi.atlas.provenance.lineage.CompleteFlowPathLineage;
    +import org.apache.nifi.atlas.provenance.lineage.LineageStrategy;
    +import org.apache.nifi.atlas.provenance.lineage.SimpleFlowPathLineage;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.atlas.resolver.ClusterResolvers;
    +import org.apache.nifi.atlas.resolver.RegexClusterResolver;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.atlas.security.Basic;
    +import org.apache.nifi.atlas.security.Kerberos;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.context.PropertyContext;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.reporting.AbstractReportingTask;
    +import org.apache.nifi.reporting.EventAccess;
    +import org.apache.nifi.reporting.ReportingContext;
    +import org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.time.Instant;
    +import java.time.ZoneOffset;
    +import java.time.format.DateTimeFormatter;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.LinkedHashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.ServiceLoader;
    +import java.util.Set;
    +import java.util.function.Consumer;
    +import java.util.stream.Stream;
    +
    +import static org.apache.commons.lang3.StringUtils.isEmpty;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_BATCH_SIZE;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_START_POSITION;
    +
    +@Tags({"atlas", "lineage"})
    +@CapabilityDescription("Publishes NiFi flow data set level lineage to Apache Atlas." +
    +        " By reporting flow information to Atlas, an end-to-end Process and DataSet lineage such as across NiFi environments and other systems" +
    +        " connected by technologies, for example NiFi Site-to-Site, Kafka topic or Hive tables." +
    +        " There are limitations and required configurations for both NiFi and Atlas. See 'Additional Details' for further description.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +@DynamicProperty(name = "hostnamePattern.<ClusterName>", value = "hostname Regex patterns", description = RegexClusterResolver.PATTERN_PROPERTY_PREFIX_DESC)
    +// In order for each reporting task instance to have its own static objects such as KafkaNotification.
    +@RequiresInstanceClassLoading
    +public class AtlasNiFiFlowLineage extends AbstractReportingTask {
    +
    +    static final PropertyDescriptor ATLAS_URLS = new PropertyDescriptor.Builder()
    +            .name("atlas-urls")
    +            .displayName("Atlas URLs")
    +            .description("Comma separated URL of Atlas Servers" +
    +                    " (e.g. http://atlas-server-hostname:21000 or https://atlas-server-hostname:21443)." +
    +                    " For accessing Atlas behind Knox gateway, specify Knox gateway URL" +
    +                    " (e.g. https://knox-hostname:8443/gateway/{topology-name}/atlas).")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final AllowableValue ATLAS_AUTHN_BASIC = new AllowableValue("basic", "Basic", "Use username and password.");
    +    static final AllowableValue ATLAS_AUTHN_KERBEROS = new AllowableValue("kerberos", "Kerberos", "Use Kerberos keytab file.");
    +    static final PropertyDescriptor ATLAS_AUTHN_METHOD = new PropertyDescriptor.Builder()
    +            .name("atlas-authentication-method")
    +            .displayName("Atlas Authentication Method")
    +            .description("Specify how to authenticate this reporting task to Atlas server.")
    +            .required(true)
    +            .allowableValues(ATLAS_AUTHN_BASIC, ATLAS_AUTHN_KERBEROS)
    +            .defaultValue(ATLAS_AUTHN_BASIC.getValue())
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_USER = new PropertyDescriptor.Builder()
    +            .name("atlas-username")
    +            .displayName("Atlas Username")
    +            .description("User name to communicate with Atlas.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("atlas-password")
    +            .displayName("Atlas Password")
    +            .description("Password to communicate with Atlas.")
    +            .required(false)
    +            .sensitive(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final PropertyDescriptor ATLAS_CONF_DIR = new PropertyDescriptor.Builder()
    +            .name("atlas-conf-dir")
    +            .displayName("Atlas Configuration Directory")
    +            .description("Directory path that contains 'atlas-application.properties' file." +
    +                    " If not specified and 'Create Atlas Configuration File' is disabled," +
    +                    " then, 'atlas-application.properties' file under root classpath is used.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_NIFI_URL = new PropertyDescriptor.Builder()
    +            .name("atlas-nifi-url")
    +            .displayName("NiFi URL for Atlas")
    +            .description("NiFi URL is used in Atlas to represent this NiFi cluster (or standalone instance)." +
    +                    " It is recommended to use one that can be accessible remotely instead of using 'localhost'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.URL_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_DEFAULT_CLUSTER_NAME = new PropertyDescriptor.Builder()
    +            .name("atlas-default-cluster-name")
    +            .displayName("Atlas Default Cluster Name")
    +            .description("Cluster name for Atlas entities reported by this ReportingTask." +
    +                    " If not specified, 'atlas.cluster.name' in Atlas Configuration File is used." +
    +                    " Cluster name mappings can be configured by user defined properties." +
    +                    " See additional detail for detail.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final PropertyDescriptor ATLAS_CONF_CREATE = new PropertyDescriptor.Builder()
    +            .name("atlas-conf-create")
    +            .displayName("Create Atlas Configuration File")
    +            .description("If enabled, 'atlas-application.properties' file will be created in 'Atlas Configuration Directory'" +
    +                    " automatically when this processor starts." +
    +                    " Note that the existing configuration file will be overwritten.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .allowableValues("true", "false")
    +            .defaultValue("false")
    +            .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +            .name("ssl-context-service")
    +            .displayName("SSL Context Service")
    +            .description("Specifies the SSL Context Service to use for communicating with Atlas and Kafka.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +    static final PropertyDescriptor KAFKA_BOOTSTRAP_SERVERS = new PropertyDescriptor.Builder()
    +            .name("kafka-bootstrap-servers")
    +            .displayName("Kafka Bootstrap Servers")
    +            .description("Kafka Bootstrap Servers to send Atlas hook notification messages based on NiFi provenance events." +
    +                    " E.g. 'localhost:9092'" +
    +                    " NOTE: Once this reporting task has started, restarting NiFi is required to changed this property" +
    +                    " as Atlas library holds a unmodifiable static reference to Kafka client.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final AllowableValue SEC_PLAINTEXT = new AllowableValue("PLAINTEXT", "PLAINTEXT", "PLAINTEXT");
    +    static final AllowableValue SEC_SSL = new AllowableValue("SSL", "SSL", "SSL");
    +    static final AllowableValue SEC_SASL_PLAINTEXT = new AllowableValue("SASL_PLAINTEXT", "SASL_PLAINTEXT", "SASL_PLAINTEXT");
    +    static final AllowableValue SEC_SASL_SSL = new AllowableValue("SASL_SSL", "SASL_SSL", "SASL_SSL");
    +    static final PropertyDescriptor KAFKA_SECURITY_PROTOCOL = new PropertyDescriptor.Builder()
    +            .name("kafka-security-protocol")
    +            .displayName("Kafka Security Protocol")
    +            .description("Protocol used to communicate with Kafka brokers to send Atlas hook notification messages." +
    +                    " Corresponds to Kafka's 'security.protocol' property.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL)
    +            .defaultValue(SEC_PLAINTEXT.getValue())
    +            .build();
    +
    +    public static final PropertyDescriptor NIFI_KERBEROS_PRINCIPAL = new PropertyDescriptor.Builder()
    +            .name("nifi-kerberos-principal")
    +            .displayName("NiFi Kerberos Principal")
    +            .description("The Kerberos principal for this NiFi instance to access Atlas API and Kafka brokers." +
    +                    " If not set, it is expected to set a JAAS configuration file in the JVM properties defined in the bootstrap.conf file." +
    +                    " This principal will be set into 'sasl.jaas.config' Kafka's property.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +    public static final PropertyDescriptor NIFI_KERBEROS_KEYTAB = new PropertyDescriptor.Builder()
    +            .name("nifi-kerberos-keytab")
    +            .displayName("NiFi Kerberos Keytab")
    +            .description("The Kerberos keytab for this NiFi instance to access Atlas API and Kafka brokers." +
    +                    " If not set, it is expected to set a JAAS configuration file in the JVM properties defined in the bootstrap.conf file." +
    +                    " This principal will be set into 'sasl.jaas.config' Kafka's property.")
    +            .required(false)
    +            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    static final PropertyDescriptor KAFKA_KERBEROS_SERVICE_NAME = new PropertyDescriptor.Builder()
    +            .name("kafka-kerberos-service-name-kafka")
    +            .displayName("Kafka Kerberos Service Name")
    +            .description("The Kerberos principal name that Kafka runs for Atlas notification." +
    +                    " This can be defined either in Kafka's JAAS config or in Kafka's config." +
    +                    " Corresponds to Kafka's 'security.protocol' property." +
    +                    " It is ignored unless one of the SASL options of the <Security Protocol> are selected.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .defaultValue("kafka")
    +            .build();
    +
    +    static final AllowableValue LINEAGE_STRATEGY_SIMPLE_PATH = new AllowableValue("SimplePath", "Simple Path",
    +            "Map NiFi provenance events and target Atlas DataSets to statically created 'nifi_flow_path' Atlas Processes." +
    +                    " See also 'Additional Details'.");
    +    static final AllowableValue LINEAGE_STRATEGY_COMPLETE_PATH = new AllowableValue("CompletePath", "Complete Path",
    +            "Create separate 'nifi_flow_path' Atlas Processes for each distinct input and output DataSet combinations" +
    +                    " by looking at the complete route for a given FlowFile. See also 'Additional Details.");
    +
    +    static final PropertyDescriptor NIFI_LINEAGE_STRATEGY = new PropertyDescriptor.Builder()
    +            .name("nifi-lineage-strategy")
    +            .displayName("NiFi Lineage Strategy")
    +            .description("Specifies granularity on how NiFi data flow should be reported to Atlas.")
    +            .required(true)
    +            .allowableValues(LINEAGE_STRATEGY_SIMPLE_PATH, LINEAGE_STRATEGY_COMPLETE_PATH)
    +            .defaultValue(LINEAGE_STRATEGY_SIMPLE_PATH.getValue())
    +            .build();
    +
    +    private static final String ATLAS_PROPERTIES_FILENAME = "atlas-application.properties";
    +    private static final String ATLAS_PROPERTY_CLUSTER_NAME = "atlas.cluster.name";
    +    private static final String ATLAS_PROPERTY_ENABLE_TLS = "atlas.enableTLS";
    +    private static final String ATLAS_KAFKA_PREFIX = "atlas.kafka.";
    +    private static final String ATLAS_PROPERTY_KAFKA_BOOTSTRAP_SERVERS = ATLAS_KAFKA_PREFIX + "bootstrap.servers";
    +    private static final String ATLAS_PROPERTY_KAFKA_CLIENT_ID = ATLAS_KAFKA_PREFIX + ProducerConfig.CLIENT_ID_CONFIG;
    +    private final ServiceLoader<ClusterResolver> clusterResolverLoader = ServiceLoader.load(ClusterResolver.class);
    +    private volatile NiFiAtlasClient atlasClient;
    +    private volatile Properties atlasProperties;
    +    private volatile boolean isTypeDefCreated = false;
    +    private volatile String defaultClusterName;
    +
    +    private volatile ProvenanceEventConsumer consumer;
    +    private volatile ClusterResolvers clusterResolvers;
    +    private volatile NiFIAtlasHook nifiAtlasHook;
    +    private volatile LineageStrategy lineageStrategy;
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(ATLAS_URLS);
    +        properties.add(ATLAS_AUTHN_METHOD);
    +        properties.add(ATLAS_USER);
    +        properties.add(ATLAS_PASSWORD);
    +        properties.add(ATLAS_CONF_DIR);
    +        properties.add(ATLAS_NIFI_URL);
    +        properties.add(ATLAS_DEFAULT_CLUSTER_NAME);
    +        properties.add(NIFI_LINEAGE_STRATEGY);
    +        properties.add(PROVENANCE_START_POSITION);
    +        properties.add(PROVENANCE_BATCH_SIZE);
    +        properties.add(SSL_CONTEXT_SERVICE);
    +
    +        // Following properties are required if ATLAS_CONF_CREATE is enabled.
    +        // Otherwise should be left blank.
    +        properties.add(ATLAS_CONF_CREATE);
    +        properties.add(NIFI_KERBEROS_PRINCIPAL);
    +        properties.add(NIFI_KERBEROS_KEYTAB);
    +        properties.add(KAFKA_KERBEROS_SERVICE_NAME);
    +        properties.add(KAFKA_BOOTSTRAP_SERVERS);
    +        properties.add(KAFKA_SECURITY_PROTOCOL);
    +
    +        return properties;
    +    }
    +
    +    @Override
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(String propertyDescriptorName) {
    +        for (ClusterResolver resolver : clusterResolverLoader) {
    +            final PropertyDescriptor propertyDescriptor = resolver.getSupportedDynamicPropertyDescriptor(propertyDescriptorName);
    +            if(propertyDescriptor != null) {
    +                return propertyDescriptor;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void parseAtlasUrls(final PropertyValue atlasUrlsProp, final Consumer<String> urlStrConsumer) {
    +        final String atlasUrlsStr = atlasUrlsProp.evaluateAttributeExpressions().getValue();
    +        if (atlasUrlsStr != null && !atlasUrlsStr.isEmpty()) {
    +            Arrays.stream(atlasUrlsStr.split(","))
    +                    .map(String::trim)
    +                    .forEach(urlStrConsumer);
    +        }
    +    }
    +
    +    @Override
    +    protected Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = new ArrayList<>();
    +
    +        final boolean isSSLContextServiceSet = context.getProperty(SSL_CONTEXT_SERVICE).isSet();
    +        final ValidationResult.Builder invalidSSLService = new ValidationResult.Builder()
    +                .subject(SSL_CONTEXT_SERVICE.getDisplayName()).valid(false);
    +        parseAtlasUrls(context.getProperty(ATLAS_URLS), input -> {
    +            final ValidationResult.Builder builder = new ValidationResult.Builder().subject(ATLAS_URLS.getDisplayName()).input(input);
    +            try {
    +                final URL url = new URL(input);
    +                if ("https".equalsIgnoreCase(url.getProtocol()) && !isSSLContextServiceSet) {
    +                    results.add(invalidSSLService.explanation("required by HTTPS Atlas access").build());
    +                } else {
    +                    results.add(builder.explanation("Valid URI").valid(true).build());
    +                }
    +            } catch (Exception e) {
    +                results.add(builder.explanation("Contains invalid URI: " + e).valid(false).build());
    +            }
    +        });
    +
    +        final String atlasAuthNMethod = context.getProperty(ATLAS_AUTHN_METHOD).getValue();
    +        final AtlasAuthN atlasAuthN = getAtlasAuthN(atlasAuthNMethod);
    +        results.addAll(atlasAuthN.validate(context));
    +
    +
    +        clusterResolverLoader.forEach(resolver -> results.addAll(resolver.validate(context)));
    +
    +        if (context.getProperty(ATLAS_CONF_CREATE).asBoolean()) {
    +
    +            Stream.of(ATLAS_CONF_DIR, ATLAS_DEFAULT_CLUSTER_NAME, KAFKA_BOOTSTRAP_SERVERS)
    +                    .filter(p -> !context.getProperty(p).isSet())
    +                    .forEach(p -> results.add(new ValidationResult.Builder()
    +                            .subject(p.getDisplayName())
    +                            .explanation("required to create Atlas configuration file.")
    +                            .valid(false).build()));
    +
    +            validateKafkaProperties(context, results, isSSLContextServiceSet, invalidSSLService);
    +        }
    +
    +        return results;
    +    }
    +
    +    private void validateKafkaProperties(ValidationContext context, Collection<ValidationResult> results, boolean isSSLContextServiceSet, ValidationResult.Builder invalidSSLService) {
    +        final String kafkaSecurityProtocol = context.getProperty(KAFKA_SECURITY_PROTOCOL).getValue();
    +        if ((SEC_SSL.equals(kafkaSecurityProtocol) || SEC_SASL_SSL.equals(kafkaSecurityProtocol))
    +                && !isSSLContextServiceSet) {
    +            results.add(invalidSSLService.explanation("required by SSL Kafka connection").build());
    +        }
    +
    +        if (SEC_SASL_PLAINTEXT.equals(kafkaSecurityProtocol) || SEC_SASL_SSL.equals(kafkaSecurityProtocol)) {
    +            Stream.of(NIFI_KERBEROS_PRINCIPAL, NIFI_KERBEROS_KEYTAB, KAFKA_KERBEROS_SERVICE_NAME)
    +                    .filter(p -> !context.getProperty(p).isSet())
    +                    .forEach(p -> results.add(new ValidationResult.Builder()
    +                            .subject(p.getDisplayName())
    +                            .explanation("required by Kafka SASL authentication.")
    +                            .valid(false).build()));
    +        }
    +    }
    +
    +    @OnScheduled
    +    public void setup(ConfigurationContext context) throws IOException {
    +        // initAtlasClient has to be done first as it loads AtlasProperty.
    +        initAtlasClient(context);
    +        initLineageStrategy(context);
    +        initClusterResolvers(context);
    +    }
    +
    +    private void initLineageStrategy(ConfigurationContext context) throws IOException {
    +        nifiAtlasHook = new NiFIAtlasHook(atlasClient);
    +
    +        final String strategy = context.getProperty(NIFI_LINEAGE_STRATEGY).getValue();
    +        if (LINEAGE_STRATEGY_SIMPLE_PATH.equals(strategy)) {
    +            lineageStrategy = new SimpleFlowPathLineage();
    +        } else if (LINEAGE_STRATEGY_COMPLETE_PATH.equals(strategy)) {
    +            lineageStrategy = new CompleteFlowPathLineage();
    +        }
    +
    +        lineageStrategy.setLineageContext(nifiAtlasHook);
    +        initProvenanceConsumer(context);
    +    }
    +
    +    private void initClusterResolvers(ConfigurationContext context) {
    +        final Set<ClusterResolver> loadedClusterResolvers = new LinkedHashSet<>();
    +        clusterResolverLoader.forEach(resolver -> {
    +            resolver.configure(context);
    +            loadedClusterResolvers.add(resolver);
    +        });
    +        clusterResolvers = new ClusterResolvers(Collections.unmodifiableSet(loadedClusterResolvers), defaultClusterName);
    +    }
    +
    +
    +    private void initAtlasClient(ConfigurationContext context) throws IOException {
    +        List<String> urls = new ArrayList<>();
    +        parseAtlasUrls(context.getProperty(ATLAS_URLS), urls::add);
    +        final boolean isAtlasApiSecure = urls.stream().anyMatch(url -> url.toLowerCase().startsWith("https"));
    +        final String atlasAuthNMethod = context.getProperty(ATLAS_AUTHN_METHOD).getValue();
    +
    +        final String confDirStr = context.getProperty(ATLAS_CONF_DIR).evaluateAttributeExpressions().getValue();
    +        final File confDir = confDirStr != null && !confDirStr.isEmpty() ? new File(confDirStr) : null;
    +
    +        atlasProperties = new Properties();
    +        final File atlasPropertiesFile = new File(confDir, ATLAS_PROPERTIES_FILENAME);
    +
    +        final Boolean createAtlasConf = context.getProperty(ATLAS_CONF_CREATE).asBoolean();
    +        if (!createAtlasConf) {
    +            // Load existing properties file.
    +            if (atlasPropertiesFile.isFile()) {
    +                getLogger().info("Loading {}", new Object[]{atlasPropertiesFile});
    +                try (InputStream in = new FileInputStream(atlasPropertiesFile)) {
    +                    atlasProperties.load(in);
    +                }
    +            } else {
    +                final String fileInClasspath = "/" + ATLAS_PROPERTIES_FILENAME;
    +                try (InputStream in = AtlasNiFiFlowLineage.class.getResourceAsStream(fileInClasspath)) {
    +                    getLogger().info("Loading {} from classpath", new Object[]{fileInClasspath});
    +                    if (in == null) {
    +                        throw new ProcessException(String.format("Could not find %s in classpath." +
    +                                " Please add it to classpath," +
    +                                " or specify %s a directory containing Atlas properties file," +
    +                                " or enable %s to generate it.",
    +                                fileInClasspath, ATLAS_CONF_DIR.getDisplayName(), ATLAS_CONF_CREATE.getDisplayName()));
    +                    }
    +                    atlasProperties.load(in);
    +                }
    +            }
    +        }
    +
    +        // Resolve default cluster name.
    +        defaultClusterName = context.getProperty(ATLAS_DEFAULT_CLUSTER_NAME).evaluateAttributeExpressions().getValue();
    +        if (defaultClusterName == null || defaultClusterName.isEmpty()) {
    +            // If default cluster name is not specified by processor configuration, then load it from Atlas config.
    +            defaultClusterName = atlasProperties.getProperty(ATLAS_PROPERTY_CLUSTER_NAME);
    +        }
    +
    +        // If default cluster name is still not defined, processor should not be able to start.
    +        if (defaultClusterName == null || defaultClusterName.isEmpty()) {
    +            throw new ProcessException("Default cluster name is not defined.");
    +        }
    +
    +        final AtlasAuthN atlasAuthN = getAtlasAuthN(atlasAuthNMethod);
    +        atlasAuthN.configure(context);
    +
    +        // Create Atlas configuration file if necessary.
    +        if (createAtlasConf) {
    +
    +            atlasProperties.put(ATLAS_PROPERTY_CLUSTER_NAME, defaultClusterName);
    +            atlasProperties.put(ATLAS_PROPERTY_ENABLE_TLS, String.valueOf(isAtlasApiSecure));
    +
    +            setKafkaConfig(atlasProperties, context);
    +
    +            atlasAuthN.populateProperties(atlasProperties);
    +
    +            try (FileOutputStream fos = new FileOutputStream(atlasPropertiesFile)) {
    +                String ts = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSX")
    +                        .withZone(ZoneOffset.UTC)
    +                        .format(Instant.now());
    +                atlasProperties.store(fos, "Generated by Apache NiFi AtlasNiFiFlowLineage ReportingTask at " + ts);
    +            }
    +        }
    +
    +
    +        atlasClient = NiFiAtlasClient.getInstance();
    +        try {
    +            atlasClient.initialize(urls.toArray(new String[]{}), atlasAuthN, confDir);
    +        } catch (final NullPointerException e) {
    --- End diff --
    
    Since the reason of NullPointer is known (it's possible to get due to different causes, but..) for NiFi users sake, I converted the Exception to more understandable error message.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157236574
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/StandardAnalysisContext.java ---
    @@ -0,0 +1,136 @@
    +/*
    + * 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.atlas.provenance;
    +
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.authorization.user.NiFiUser;
    +import org.apache.nifi.controller.status.ConnectionStatus;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.provenance.lineage.ComputeLineageResult;
    +import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +
    +public class StandardAnalysisContext implements AnalysisContext {
    +
    +    private final Logger logger = LoggerFactory.getLogger(StandardAnalysisContext.class);
    +    private final NiFiFlow nifiFlow;
    +    private final ClusterResolver clusterResolver;
    +    private final ProvenanceRepository provenanceRepository;
    +
    +    public StandardAnalysisContext(NiFiFlow nifiFlow, ClusterResolver clusterResolver,
    +                                   ProvenanceRepository provenanceRepository) {
    +        this.nifiFlow = nifiFlow;
    +        this.clusterResolver = clusterResolver;
    +        this.provenanceRepository = provenanceRepository;
    +    }
    +
    +    @Override
    +    public List<ConnectionStatus> findConnectionTo(String componentId) {
    +        return nifiFlow.getIncomingRelationShips(componentId);
    +    }
    +
    +    @Override
    +    public List<ConnectionStatus> findConnectionFrom(String componentId) {
    +        return nifiFlow.getOutgoingRelationShips(componentId);
    +    }
    +
    +    @Override
    +    public String getNiFiClusterName() {
    +        return nifiFlow.getClusterName();
    +    }
    +
    +    @Override
    +    public ClusterResolver getClusterResolver() {
    +        return clusterResolver;
    +    }
    +
    +    private ComputeLineageResult getLineageResult(long eventId, ComputeLineageSubmission submission) {
    +        final ComputeLineageResult result = submission.getResult();
    +        try {
    +            if (result.awaitCompletion(10, TimeUnit.SECONDS)) {
    +                return result;
    +            }
    +            logger.warn("Lineage query for {} timed out.", new Object[]{eventId});
    +        } catch (InterruptedException e) {
    +            logger.warn("Lineage query for {} was interrupted due to {}.", new Object[]{eventId, e}, e);
    +        } finally {
    +            submission.cancel();
    +        }
    +
    +        return null;
    +    }
    +
    +    @Override
    +    public ComputeLineageResult queryLineage(long eventId) {
    +        final ComputeLineageSubmission submission = provenanceRepository.submitLineageComputation(eventId, NIFI_USER);
    +        return getLineageResult(eventId, submission);
    +    }
    +
    +    public ComputeLineageResult findParents(long eventId) {
    +        final ComputeLineageSubmission submission = provenanceRepository.submitExpandParents(eventId, NIFI_USER);
    +        return getLineageResult(eventId, submission);
    +    }
    +
    +    // NOTE: This user is required to avoid NullPointerException at PersistentProvenanceRepository.submitLineageComputation
    +    private static final QueryNiFiUser NIFI_USER = new QueryNiFiUser();
    --- End diff --
    
    @ijokarumawak The authentication is done at the web request layer and authorization for making a provenance query is done there, but the event-level authorization is not. That is done in the repository itself. So in a secure environment, the lineage that is returned should consist of "UNKNOWN" event types. If that is not the behavior you are seeing, then I believe we have a bug somewhere.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157047074
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/lineage/SimpleFlowPathLineage.java ---
    @@ -0,0 +1,205 @@
    +/*
    + * 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.atlas.provenance.lineage;
    +
    +import org.apache.atlas.typesystem.Referenceable;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowPath;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.DataSetRefs;
    +import org.apache.nifi.controller.status.ConnectionStatus;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.lineage.ComputeLineageResult;
    +import org.apache.nifi.provenance.lineage.LineageEdge;
    +import org.apache.nifi.provenance.lineage.LineageNode;
    +import org.apache.nifi.provenance.lineage.LineageNodeType;
    +
    +import java.util.List;
    +
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_QUEUE;
    +
    +public class SimpleFlowPathLineage extends AbstractLineageStrategy {
    +
    +    @Override
    +    public void processEvent(AnalysisContext analysisContext, NiFiFlow nifiFlow, ProvenanceEventRecord event) {
    +        final DataSetRefs refs = executeAnalyzer(analysisContext, event);
    +        if (refs == null || (refs.isEmpty())) {
    +            return;
    +        }
    +
    +        if ("Remote Input Port".equals(event.getComponentType()) || "Remote Output Port".equals(event.getComponentType())) {
    +            processRemotePortEvent(analysisContext, nifiFlow, event, refs);
    +        } else {
    +            addDataSetRefs(nifiFlow, refs);
    +        }
    +
    +    }
    +
    +    /**
    +     * Create a flow_path entity corresponding to the target RemoteGroupPort when a SEND/RECEIVE event are received.
    +     * Because such entity can not be created in advance while analyzing flow statically,
    +     * as ReportingTask can not determine whether a component id is a RemoteGroupPort,
    +     * since connectionStatus is the only available information in ReportingContext.
    +     * ConnectionStatus only knows component id, component type is unknown.
    +     * For example, there is no difference to tell if a connected component is a funnel or a RemoteGroupPort.
    +     */
    +    private void processRemotePortEvent(AnalysisContext analysisContext, NiFiFlow nifiFlow, ProvenanceEventRecord event, DataSetRefs analyzedRefs) {
    +
    +        final boolean isRemoteInputPort = "Remote Input Port".equals(event.getComponentType());
    +
    +        // Create a RemoteInputPort Process.
    +        // event.getComponentId returns UUID for RemoteGroupPort as a client of S2S, and it's different from a remote port UUID (portDataSetid).
    +        // See NIFI-4571 for detail.
    +        final Referenceable remotePortDataSet = isRemoteInputPort ? analyzedRefs.getOutputs().iterator().next() :  analyzedRefs.getInputs().iterator().next();
    +        final String portProcessId = event.getComponentId();
    +
    +        final NiFiFlowPath remotePortProcess = new NiFiFlowPath(portProcessId);
    +        remotePortProcess.setName(event.getComponentType());
    +        remotePortProcess.addProcessor(portProcessId);
    +
    +        // For RemoteInputPort, need to find the previous component connected to this port,
    +        // which passed this particular FlowFile.
    +        // That is only possible by calling lineage API.
    +        if (isRemoteInputPort) {
    +            final ProvenanceEventRecord previousEvent = findPreviousProvenanceEvent(analysisContext, event);
    +            if (previousEvent == null) {
    +                logger.warn("Previous event was not found: {}", new Object[]{event});
    +                return;
    +            }
    +
    +            // Set groupId from incoming connection if available.
    +            final List<ConnectionStatus> incomingRelationShips = nifiFlow.getIncomingRelationShips(portProcessId);
    +            if (incomingRelationShips == null || incomingRelationShips.isEmpty()) {
    +                logger.warn("Incoming relationship was not found: {}", new Object[]{event});
    +                return;
    +            }
    +
    +            final ConnectionStatus connection = incomingRelationShips.get(0);
    +            remotePortProcess.setGroupId(connection.getGroupId());
    +
    +            final Referenceable remotePortProcessRef = toReferenceable(remotePortProcess, nifiFlow);
    +            createEntity(remotePortProcessRef);
    +
    +            // Create a queue.
    +            Referenceable queueFromStaticFlowPathToRemotePortProcess = new Referenceable(TYPE_NIFI_QUEUE);
    +            queueFromStaticFlowPathToRemotePortProcess.set(ATTR_NAME, "queue");
    --- End diff --
    
    There's a few places that "queue" is hardcoded. Should it be added to NiFiTypes?


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r156992235
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/StandardAnalysisContext.java ---
    @@ -0,0 +1,136 @@
    +/*
    + * 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.atlas.provenance;
    +
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.authorization.user.NiFiUser;
    +import org.apache.nifi.controller.status.ConnectionStatus;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.provenance.lineage.ComputeLineageResult;
    +import org.apache.nifi.provenance.lineage.ComputeLineageSubmission;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +
    +public class StandardAnalysisContext implements AnalysisContext {
    +
    +    private final Logger logger = LoggerFactory.getLogger(StandardAnalysisContext.class);
    +    private final NiFiFlow nifiFlow;
    +    private final ClusterResolver clusterResolver;
    +    private final ProvenanceRepository provenanceRepository;
    +
    +    public StandardAnalysisContext(NiFiFlow nifiFlow, ClusterResolver clusterResolver,
    +                                   ProvenanceRepository provenanceRepository) {
    +        this.nifiFlow = nifiFlow;
    +        this.clusterResolver = clusterResolver;
    +        this.provenanceRepository = provenanceRepository;
    +    }
    +
    +    @Override
    +    public List<ConnectionStatus> findConnectionTo(String componentId) {
    +        return nifiFlow.getIncomingRelationShips(componentId);
    +    }
    +
    +    @Override
    +    public List<ConnectionStatus> findConnectionFrom(String componentId) {
    +        return nifiFlow.getOutgoingRelationShips(componentId);
    +    }
    +
    +    @Override
    +    public String getNiFiClusterName() {
    +        return nifiFlow.getClusterName();
    +    }
    +
    +    @Override
    +    public ClusterResolver getClusterResolver() {
    +        return clusterResolver;
    +    }
    +
    +    private ComputeLineageResult getLineageResult(long eventId, ComputeLineageSubmission submission) {
    +        final ComputeLineageResult result = submission.getResult();
    +        try {
    +            if (result.awaitCompletion(10, TimeUnit.SECONDS)) {
    +                return result;
    +            }
    +            logger.warn("Lineage query for {} timed out.", new Object[]{eventId});
    +        } catch (InterruptedException e) {
    +            logger.warn("Lineage query for {} was interrupted due to {}.", new Object[]{eventId, e}, e);
    +        } finally {
    +            submission.cancel();
    +        }
    +
    +        return null;
    +    }
    +
    +    @Override
    +    public ComputeLineageResult queryLineage(long eventId) {
    +        final ComputeLineageSubmission submission = provenanceRepository.submitLineageComputation(eventId, NIFI_USER);
    +        return getLineageResult(eventId, submission);
    +    }
    +
    +    public ComputeLineageResult findParents(long eventId) {
    +        final ComputeLineageSubmission submission = provenanceRepository.submitExpandParents(eventId, NIFI_USER);
    +        return getLineageResult(eventId, submission);
    +    }
    +
    +    // NOTE: This user is required to avoid NullPointerException at PersistentProvenanceRepository.submitLineageComputation
    +    private static final QueryNiFiUser NIFI_USER = new QueryNiFiUser();
    --- End diff --
    
    Using this approach means that any NiFi that is run securely will not be able to make use of this Atlas integration. I think the more correct solution is to update the Provenance Repository so that we can run a query without supplying a user. In the mean time, we could perhaps get away with allowing the user to configure the username to run the queries as? Then they can simply create an "Atlas Integration" user or choose their own user, etc.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157223751
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/lineage/SimpleFlowPathLineage.java ---
    @@ -0,0 +1,205 @@
    +/*
    + * 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.atlas.provenance.lineage;
    +
    +import org.apache.atlas.typesystem.Referenceable;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowPath;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.DataSetRefs;
    +import org.apache.nifi.controller.status.ConnectionStatus;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.lineage.ComputeLineageResult;
    +import org.apache.nifi.provenance.lineage.LineageEdge;
    +import org.apache.nifi.provenance.lineage.LineageNode;
    +import org.apache.nifi.provenance.lineage.LineageNodeType;
    +
    +import java.util.List;
    +
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_QUEUE;
    +
    +public class SimpleFlowPathLineage extends AbstractLineageStrategy {
    +
    +    @Override
    +    public void processEvent(AnalysisContext analysisContext, NiFiFlow nifiFlow, ProvenanceEventRecord event) {
    +        final DataSetRefs refs = executeAnalyzer(analysisContext, event);
    +        if (refs == null || (refs.isEmpty())) {
    +            return;
    +        }
    +
    +        if ("Remote Input Port".equals(event.getComponentType()) || "Remote Output Port".equals(event.getComponentType())) {
    +            processRemotePortEvent(analysisContext, nifiFlow, event, refs);
    +        } else {
    +            addDataSetRefs(nifiFlow, refs);
    +        }
    +
    +    }
    +
    +    /**
    +     * Create a flow_path entity corresponding to the target RemoteGroupPort when a SEND/RECEIVE event are received.
    +     * Because such entity can not be created in advance while analyzing flow statically,
    +     * as ReportingTask can not determine whether a component id is a RemoteGroupPort,
    +     * since connectionStatus is the only available information in ReportingContext.
    +     * ConnectionStatus only knows component id, component type is unknown.
    +     * For example, there is no difference to tell if a connected component is a funnel or a RemoteGroupPort.
    +     */
    +    private void processRemotePortEvent(AnalysisContext analysisContext, NiFiFlow nifiFlow, ProvenanceEventRecord event, DataSetRefs analyzedRefs) {
    +
    +        final boolean isRemoteInputPort = "Remote Input Port".equals(event.getComponentType());
    +
    +        // Create a RemoteInputPort Process.
    +        // event.getComponentId returns UUID for RemoteGroupPort as a client of S2S, and it's different from a remote port UUID (portDataSetid).
    +        // See NIFI-4571 for detail.
    +        final Referenceable remotePortDataSet = isRemoteInputPort ? analyzedRefs.getOutputs().iterator().next() :  analyzedRefs.getInputs().iterator().next();
    +        final String portProcessId = event.getComponentId();
    +
    +        final NiFiFlowPath remotePortProcess = new NiFiFlowPath(portProcessId);
    +        remotePortProcess.setName(event.getComponentType());
    +        remotePortProcess.addProcessor(portProcessId);
    +
    +        // For RemoteInputPort, need to find the previous component connected to this port,
    +        // which passed this particular FlowFile.
    +        // That is only possible by calling lineage API.
    +        if (isRemoteInputPort) {
    +            final ProvenanceEventRecord previousEvent = findPreviousProvenanceEvent(analysisContext, event);
    +            if (previousEvent == null) {
    +                logger.warn("Previous event was not found: {}", new Object[]{event});
    +                return;
    +            }
    +
    +            // Set groupId from incoming connection if available.
    +            final List<ConnectionStatus> incomingRelationShips = nifiFlow.getIncomingRelationShips(portProcessId);
    +            if (incomingRelationShips == null || incomingRelationShips.isEmpty()) {
    +                logger.warn("Incoming relationship was not found: {}", new Object[]{event});
    +                return;
    +            }
    +
    +            final ConnectionStatus connection = incomingRelationShips.get(0);
    +            remotePortProcess.setGroupId(connection.getGroupId());
    +
    +            final Referenceable remotePortProcessRef = toReferenceable(remotePortProcess, nifiFlow);
    +            createEntity(remotePortProcessRef);
    +
    +            // Create a queue.
    +            Referenceable queueFromStaticFlowPathToRemotePortProcess = new Referenceable(TYPE_NIFI_QUEUE);
    +            queueFromStaticFlowPathToRemotePortProcess.set(ATTR_NAME, "queue");
    --- End diff --
    
    Hmm, I think it should be ok as it is, since it's a 'name' property. It does not have to be consistent across different context, if it's 'typeName' or 'qualifiedName' or any other important attribute, then it probably should be a constant, but it's just a name. Thanks anyway.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157208859
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/NiFiAtlasClient.java ---
    @@ -0,0 +1,537 @@
    +/*
    + * 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.atlas;
    +
    +import com.sun.jersey.api.client.UniformInterfaceException;
    +import com.sun.jersey.core.util.MultivaluedMapImpl;
    +import org.apache.atlas.ApplicationProperties;
    +import org.apache.atlas.AtlasClientV2;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.atlas.model.SearchFilter;
    +import org.apache.atlas.model.instance.AtlasEntity;
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +import org.apache.atlas.model.instance.EntityMutationResponse;
    +import org.apache.atlas.model.typedef.AtlasEntityDef;
    +import org.apache.atlas.model.typedef.AtlasStructDef.AtlasAttributeDef;
    +import org.apache.atlas.model.typedef.AtlasTypesDef;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.util.StringUtils;
    +import org.apache.nifi.util.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.ws.rs.core.MultivaluedMap;
    +import java.io.File;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.function.Function;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.nifi.atlas.AtlasUtils.findIdByQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.getComponentIdFromQualifiedName;
    +import static org.apache.nifi.atlas.AtlasUtils.toStr;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.AS_IS;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.CREATED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.DELETED;
    +import static org.apache.nifi.atlas.NiFiFlow.EntityChangeType.UPDATED;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_DESCRIPTION;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_FLOW_PATHS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_GUID;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUT_PORTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUEUES;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_TYPENAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_URL;
    +import static org.apache.nifi.atlas.NiFiTypes.ENTITIES;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW_PATH;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_INPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_OUTPUT_PORT;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_QUEUE;
    +
    +public class NiFiAtlasClient {
    +
    +    private static final Logger logger = LoggerFactory.getLogger(NiFiAtlasClient.class);
    +
    +    private static NiFiAtlasClient nifiClient;
    +    private AtlasClientV2 atlasClient;
    +
    +    private NiFiAtlasClient() {
    +        super();
    +    }
    +
    +    public static NiFiAtlasClient getInstance() {
    +        if (nifiClient == null) {
    +            synchronized (NiFiAtlasClient.class) {
    +                if (nifiClient == null) {
    +                    nifiClient = new NiFiAtlasClient();
    +                }
    +            }
    +        }
    +        return nifiClient;
    +    }
    +
    +    public void initialize(final String[] baseUrls, final AtlasAuthN authN, final File atlasConfDir) {
    +
    +        synchronized (NiFiAtlasClient.class) {
    +
    +            if (atlasClient != null) {
    +                logger.info("{} had been setup but replacing it with new one.", atlasClient);
    +                ApplicationProperties.forceReload();
    +            }
    +
    +            if (atlasConfDir != null) {
    +                // If atlasConfDir is not set, atlas-application.properties will be searched under classpath.
    +                Properties props = System.getProperties();
    +                final String atlasConfProp = "atlas.conf";
    +                props.setProperty(atlasConfProp, atlasConfDir.getAbsolutePath());
    +                logger.debug("{} has been set to: {}", atlasConfProp, props.getProperty(atlasConfProp));
    +            }
    +
    +            atlasClient = authN.createClient(baseUrls);
    +
    +        }
    +    }
    +
    +    /**
    +     * This is an utility method to delete unused types.
    +     * Should be used during development or testing only.
    +     * @param typeNames to delete
    +     */
    +    void deleteTypeDefs(String ... typeNames) throws AtlasServiceException {
    +        final AtlasTypesDef existingTypeDef = getTypeDefs(typeNames);
    +        try {
    +            atlasClient.deleteAtlasTypeDefs(existingTypeDef);
    +        } catch (UniformInterfaceException e) {
    +            if (e.getResponse().getStatus() == 204) {
    +                // 204 is a successful response.
    +                // NOTE: However after executing this, Atlas should be restarted to work properly.
    +                logger.info("Deleted type defs: {}", existingTypeDef);
    +            } else {
    +                throw e;
    +            }
    +        }
    +    }
    +
    +    /**
    +     * @return True when required NiFi types are already created.
    +     */
    +    public boolean isNiFiTypeDefsRegistered() throws AtlasServiceException {
    +        final Set<String> typeNames = ENTITIES.keySet();
    +        final Map<String, AtlasEntityDef> existingDefs = getTypeDefs(typeNames.toArray(new String[typeNames.size()])).getEntityDefs().stream()
    +                .collect(Collectors.toMap(AtlasEntityDef::getName, Function.identity()));
    +        return typeNames.stream().allMatch(existingDefs::containsKey);
    +    }
    +
    +    /**
    +     * Create or update NiFi types in Atlas type system.
    +     * @param update If false, doesn't perform anything if there is existing type def for the name.
    +     */
    +    public void registerNiFiTypeDefs(boolean update) throws AtlasServiceException {
    +        final Set<String> typeNames = ENTITIES.keySet();
    +        final Map<String, AtlasEntityDef> existingDefs = getTypeDefs(typeNames.toArray(new String[typeNames.size()])).getEntityDefs().stream()
    +                .collect(Collectors.toMap(AtlasEntityDef::getName, Function.identity()));
    +
    +
    +        final AtomicBoolean shouldUpdate = new AtomicBoolean(false);
    +
    +        final AtlasTypesDef type = new AtlasTypesDef();
    +
    +        typeNames.stream().filter(typeName -> {
    +            final AtlasEntityDef existingDef = existingDefs.get(typeName);
    +            if (existingDef != null) {
    +                // type is already defined.
    +                if (!update) {
    +                    return false;
    +                }
    +                shouldUpdate.set(true);
    +            }
    +            return true;
    +        }).forEach(typeName -> {
    +            final NiFiTypes.EntityDefinition def = ENTITIES.get(typeName);
    +
    +            final AtlasEntityDef entity = new AtlasEntityDef();
    +            type.getEntityDefs().add(entity);
    +
    +            entity.setName(typeName);
    +
    +            Set<String> superTypes = new HashSet<>();
    +            List<AtlasAttributeDef> attributes = new ArrayList<>();
    +
    +            def.define(entity, superTypes, attributes);
    +
    +            entity.setSuperTypes(superTypes);
    +            entity.setAttributeDefs(attributes);
    +        });
    +
    +        // Create or Update.
    +        final AtlasTypesDef atlasTypeDefsResult = shouldUpdate.get()
    +                ? atlasClient.updateAtlasTypeDefs(type)
    +                : atlasClient.createAtlasTypeDefs(type);
    +        logger.debug("Result={}", atlasTypeDefsResult);
    +    }
    +
    +    private AtlasTypesDef getTypeDefs(String ... typeNames) throws AtlasServiceException {
    +        final AtlasTypesDef typeDefs = new AtlasTypesDef();
    +        for (int i = 0; i < typeNames.length; i++) {
    +            final MultivaluedMap<String, String> searchParams = new MultivaluedMapImpl();
    +            searchParams.add(SearchFilter.PARAM_NAME, typeNames[i]);
    +            final AtlasTypesDef typeDef = atlasClient.getAllTypeDefs(new SearchFilter(searchParams));
    +            typeDefs.getEntityDefs().addAll(typeDef.getEntityDefs());
    +        }
    +        logger.debug("typeDefs={}", typeDefs);
    +        return typeDefs;
    +    }
    +
    +    private Pattern FLOW_PATH_URL_PATTERN = Pattern.compile("^http.+processGroupId=([0-9a-z\\-]+).*$");
    +    /**
    +     * Fetch existing NiFiFlow entity from Atlas.
    +     * @param rootProcessGroupId The id of a NiFi flow root process group.
    +     * @param clusterName The cluster name of a flow.
    +     * @return A NiFiFlow instance filled with retrieved data from Atlas. Status objects are left blank, e.g. ProcessorStatus.
    +     * @throws AtlasServiceException Thrown if requesting to Atlas API failed, including when the flow is not found.
    +     */
    +    public NiFiFlow fetchNiFiFlow(String rootProcessGroupId, String clusterName) throws AtlasServiceException {
    +
    +        final String qualifiedName = AtlasUtils.toQualifiedName(clusterName, rootProcessGroupId);
    +        final AtlasObjectId flowId = new AtlasObjectId(TYPE_NIFI_FLOW, ATTR_QUALIFIED_NAME, qualifiedName);
    +        final AtlasEntity.AtlasEntityWithExtInfo nifiFlowExt = searchEntityDef(flowId);
    +
    +        if (nifiFlowExt == null || nifiFlowExt.getEntity() == null) {
    +            return null;
    +        }
    +
    +        final AtlasEntity nifiFlowEntity = nifiFlowExt.getEntity();
    +        final Map<String, Object> attributes = nifiFlowEntity.getAttributes();
    +        final NiFiFlow nifiFlow = new NiFiFlow(rootProcessGroupId);
    +        nifiFlow.setExEntity(nifiFlowEntity);
    +        nifiFlow.setFlowName(toStr(attributes.get(ATTR_NAME)));
    +        nifiFlow.setClusterName(clusterName);
    +        nifiFlow.setUrl(toStr(attributes.get(ATTR_URL)));
    +        nifiFlow.setDescription(toStr(attributes.get(ATTR_DESCRIPTION)));
    +
    +        nifiFlow.getQueues().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_QUEUES))));
    +        nifiFlow.getRootInputPortEntities().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_INPUT_PORTS))));
    +        nifiFlow.getRootOutputPortEntities().putAll(toQualifiedNameIds(toAtlasObjectIds(nifiFlowEntity.getAttribute(ATTR_OUTPUT_PORTS))));
    +
    +        final Map<String, NiFiFlowPath> flowPaths = nifiFlow.getFlowPaths();
    +        final Map<AtlasObjectId, AtlasEntity> flowPathEntities = toQualifiedNameIds(toAtlasObjectIds(attributes.get(ATTR_FLOW_PATHS)));
    +
    +        for (AtlasEntity flowPathEntity : flowPathEntities.values()) {
    +            final String pathQualifiedName = toStr(flowPathEntity.getAttribute(ATTR_QUALIFIED_NAME));
    +            final NiFiFlowPath flowPath = new NiFiFlowPath(getComponentIdFromQualifiedName(pathQualifiedName));
    +            if (flowPathEntity.hasAttribute(ATTR_URL)) {
    +                final Matcher urlMatcher = FLOW_PATH_URL_PATTERN.matcher(toStr(flowPathEntity.getAttribute(ATTR_URL)));
    +                if (urlMatcher.matches()) {
    +                    flowPath.setGroupId(urlMatcher.group(1));
    +                }
    +            }
    +            flowPath.setExEntity(flowPathEntity);
    +            flowPath.setName(toStr(flowPathEntity.getAttribute(ATTR_NAME)));
    +            flowPath.getInputs().addAll(toQualifiedNameIds(toAtlasObjectIds(flowPathEntity.getAttribute(ATTR_INPUTS))).keySet());
    +            flowPath.getOutputs().addAll(toQualifiedNameIds(toAtlasObjectIds(flowPathEntity.getAttribute(ATTR_OUTPUTS))).keySet());
    +            flowPath.startTrackingChanges(nifiFlow);
    +
    +            flowPaths.put(flowPath.getId(), flowPath);
    +        }
    +
    +        nifiFlow.startTrackingChanges();
    +        return nifiFlow;
    +    }
    +
    +    @SuppressWarnings("unchecked")
    +    private List<AtlasObjectId> toAtlasObjectIds(Object _references) {
    +        if (_references == null) {
    +            return Collections.emptyList();
    +        }
    +        List<Map<String, Object>> references = (List<Map<String, Object>>) _references;
    +        return references.stream()
    +                .map(ref -> new AtlasObjectId(toStr(ref.get(ATTR_GUID)), toStr(ref.get(ATTR_TYPENAME)), ref))
    +                .collect(Collectors.toList());
    +    }
    +
    +    /**
    +     * <p>AtlasObjectIds returned from Atlas have GUID, but do not have qualifiedName, while ones created by the reporting task
    +     * do not have GUID, but qualifiedName. AtlasObjectId.equals returns false for this combination.
    +     * In order to match ids correctly, this method converts fetches actual entities from ids to get qualifiedName attribute.</p>
    +     *
    +     * <p>Also, AtlasObjectIds returned from Atlas does not have entity state.
    +     * If Atlas is configured to use soft-delete (default), deleted ids are still returned.
    +     * Fetched entities are used to determine whether an AtlasObjectId is still active or deleted.
    +     * Deleted entities will not be included in the result of this method.
    +     * </p>
    +     * @param ids to convert
    +     * @return AtlasObjectIds with qualifiedName
    +     */
    +    private Map<AtlasObjectId, AtlasEntity> toQualifiedNameIds(List<AtlasObjectId> ids) {
    +        if (ids == null) {
    +            return Collections.emptyMap();
    +        }
    +
    +        return ids.stream().distinct().map(id -> {
    +            try {
    +                final AtlasEntity.AtlasEntityWithExtInfo entityExt = searchEntityDef(id);
    +                final AtlasEntity entity = entityExt.getEntity();
    +                if (AtlasEntity.Status.DELETED.equals(entity.getStatus())) {
    +                    return null;
    +                }
    +                final Map<String, Object> uniqueAttrs = Collections.singletonMap(ATTR_QUALIFIED_NAME, entity.getAttribute(ATTR_QUALIFIED_NAME));
    +                return new Tuple<>(new AtlasObjectId(id.getGuid(), id.getTypeName(), uniqueAttrs), entity);
    +            } catch (AtlasServiceException e) {
    +                return null;
    --- End diff --
    
    Absolutely. Thanks!


---

[GitHub] nifi issue #2335: NIFI-3709: Export NiFi flow dataset lineage to Apache Atla...

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

    https://github.com/apache/nifi/pull/2335
  
    @markap14 I found that ReportLineageToAtlas does get UNKNOWN provenance events when it queries lineage by `ProvenanceRepository.submitLineageComputation(final long eventId, final NiFiUser user)`. It uses the event hierarchies returned by that method, but to analyze each event detail fully, it also calls `ProvenanceEventRepository.getEvent(final long id)` which does not require user. That's why the reporting task worked even in a secured NiFi cluster.
    
    I now understand why NiFi framework has two interfaces for provenance events, i.e. ProvenanceRepository and ProvenanceEventRepository. `ProvenanceRepository` provides methods to be called in a context with a user who made the request. `ProvenanceEventRepository` is used outside of a user request, intended for Reporting Tasks, etc.
    
    Is my understanding correct? If so, the approach of this PR looks good now? Thanks!


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r156984919
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/provenance/NiFiProvenanceEventAnalyzerFactory.java ---
    @@ -0,0 +1,105 @@
    +/*
    + * 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.atlas.provenance;
    +
    +import org.apache.nifi.provenance.ProvenanceEventType;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +import java.util.ServiceLoader;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +
    +public class NiFiProvenanceEventAnalyzerFactory {
    +
    +    private static final Logger logger = LoggerFactory.getLogger(NiFiProvenanceEventAnalyzerFactory.class);
    +    private static final Map<Pattern, NiFiProvenanceEventAnalyzer> analyzersForComponentType = new ConcurrentHashMap<>();
    +    private static final Map<Pattern, NiFiProvenanceEventAnalyzer> analyzersForTransitUri = new ConcurrentHashMap<>();
    +    private static final Map<ProvenanceEventType, NiFiProvenanceEventAnalyzer> analyzersForProvenanceEventType = new ConcurrentHashMap<>();
    +    private static boolean loaded = false;
    +
    +    private static void loadAnalyzers() {
    +        logger.debug("Loading NiFiProvenanceEventAnalyzer ...");
    +        final ServiceLoader<NiFiProvenanceEventAnalyzer> serviceLoader
    +                = ServiceLoader.load(NiFiProvenanceEventAnalyzer.class);
    +        serviceLoader.forEach(analyzer -> {
    +            addAnalyzer(analyzer.targetComponentTypePattern(), analyzersForComponentType, analyzer);
    +            addAnalyzer(analyzer.targetTransitUriPattern(), analyzersForTransitUri, analyzer);
    +            final ProvenanceEventType eventType = analyzer.targetProvenanceEventType();
    +            if (eventType != null) {
    +                if (analyzersForProvenanceEventType.containsKey(eventType)) {
    +                    logger.warn("Fo ProvenanceEventType {}, an Analyzer {} is already assigned." +
    +                            " Only one analyzer for a type can be registered. Ignoring {}",
    +                            eventType, analyzersForProvenanceEventType.get(eventType), analyzer);
    +                }
    +                analyzersForProvenanceEventType.put(eventType, analyzer);
    +            }
    +        });
    +        logger.info("Loaded NiFiProvenanceEventAnalyzers: componentTypes={}, transitUris={}", analyzersForComponentType, analyzersForTransitUri);
    +    }
    +
    +    private static void addAnalyzer(String patternStr, Map<Pattern, NiFiProvenanceEventAnalyzer> toAdd,
    +                                    NiFiProvenanceEventAnalyzer analyzer) {
    +        if (patternStr != null && !patternStr.isEmpty()) {
    +            Pattern pattern = Pattern.compile(patternStr.trim());
    +            toAdd.put(pattern, analyzer);
    +        }
    +    }
    +
    +    /**
    +     * Find and retrieve NiFiProvenanceEventAnalyzer implementation for the specified targets.
    +     * Pattern matching is performed by following order, and the one found at first is returned:
    +     * <ol>
    +     * <li>Component type name. Use an analyzer supporting the Component type with its {@link NiFiProvenanceEventAnalyzer#targetProvenanceEventType()}.
    +     * <li>TransitUri. Use an analyzer supporting the TransitUri with its {@link NiFiProvenanceEventAnalyzer#targetTransitUriPattern()}.
    +     * <li>Provenance Event Type. Use an analyzer supporting the Provenance Event Type with its {@link NiFiProvenanceEventAnalyzer#targetProvenanceEventType()}.
    +     * </ol>
    +     * @param typeName NiFi component type name.
    +     * @param transitUri Transit URI.
    +     * @param eventType Provenance event type.
    +     * @return Instance of NiFiProvenanceEventAnalyzer if one is found for the specified className, otherwise null.
    +     */
    +    public static NiFiProvenanceEventAnalyzer getAnalyzer(String typeName, String transitUri, ProvenanceEventType eventType) {
    +
    +        if (!loaded) {
    --- End diff --
    
    This is not thread-safe. This is an instance of Double-checked locking (https://en.wikipedia.org/wiki/Double-checked_locking)


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157034065
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/NiFIAtlasHook.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.atlas;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.atlas.hook.AtlasHook;
    +import org.apache.atlas.model.instance.AtlasEntity;
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +import org.apache.atlas.notification.hook.HookNotification.EntityPartialUpdateRequest;
    +import org.apache.atlas.notification.hook.HookNotification.HookNotificationMessage;
    +import org.apache.atlas.typesystem.Referenceable;
    +import org.apache.atlas.typesystem.persistence.Id;
    +import org.apache.nifi.atlas.provenance.lineage.LineageContext;
    +import org.apache.nifi.util.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.LinkedHashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.atlas.notification.hook.HookNotification.HookNotificationType.ENTITY_PARTIAL_UPDATE;
    +import static org.apache.nifi.atlas.AtlasUtils.toTypedQualifiedName;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_GUID;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_TYPENAME;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW_PATH;
    +
    +/**
    + * This class is not thread-safe as it holds uncommitted notification messages within instance.
    + * {@link #addMessage(HookNotificationMessage)} and {@link #commitMessages()} should be used serially from a single thread.
    + */
    +public class NiFIAtlasHook extends AtlasHook implements LineageContext {
    +
    +    public static final String NIFI_USER = "nifi";
    --- End diff --
    
    Would this be best offered as a configurable value?


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157217462
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/AtlasNiFiFlowLineage.java ---
    @@ -0,0 +1,714 @@
    +/*
    + * 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.atlas.reporting;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.common.config.SslConfigs;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
    +import org.apache.nifi.annotation.behavior.Stateful;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.atlas.NiFIAtlasHook;
    +import org.apache.nifi.atlas.NiFiAtlasClient;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowAnalyzer;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.StandardAnalysisContext;
    +import org.apache.nifi.atlas.provenance.lineage.CompleteFlowPathLineage;
    +import org.apache.nifi.atlas.provenance.lineage.LineageStrategy;
    +import org.apache.nifi.atlas.provenance.lineage.SimpleFlowPathLineage;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.atlas.resolver.ClusterResolvers;
    +import org.apache.nifi.atlas.resolver.RegexClusterResolver;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.atlas.security.Basic;
    +import org.apache.nifi.atlas.security.Kerberos;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.context.PropertyContext;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.reporting.AbstractReportingTask;
    +import org.apache.nifi.reporting.EventAccess;
    +import org.apache.nifi.reporting.ReportingContext;
    +import org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.time.Instant;
    +import java.time.ZoneOffset;
    +import java.time.format.DateTimeFormatter;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.LinkedHashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.ServiceLoader;
    +import java.util.Set;
    +import java.util.function.Consumer;
    +import java.util.stream.Stream;
    +
    +import static org.apache.commons.lang3.StringUtils.isEmpty;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_BATCH_SIZE;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_START_POSITION;
    +
    +@Tags({"atlas", "lineage"})
    +@CapabilityDescription("Publishes NiFi flow data set level lineage to Apache Atlas." +
    +        " By reporting flow information to Atlas, an end-to-end Process and DataSet lineage such as across NiFi environments and other systems" +
    +        " connected by technologies, for example NiFi Site-to-Site, Kafka topic or Hive tables." +
    +        " There are limitations and required configurations for both NiFi and Atlas. See 'Additional Details' for further description.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +@DynamicProperty(name = "hostnamePattern.<ClusterName>", value = "hostname Regex patterns", description = RegexClusterResolver.PATTERN_PROPERTY_PREFIX_DESC)
    +// In order for each reporting task instance to have its own static objects such as KafkaNotification.
    +@RequiresInstanceClassLoading
    +public class AtlasNiFiFlowLineage extends AbstractReportingTask {
    +
    +    static final PropertyDescriptor ATLAS_URLS = new PropertyDescriptor.Builder()
    +            .name("atlas-urls")
    +            .displayName("Atlas URLs")
    +            .description("Comma separated URL of Atlas Servers" +
    +                    " (e.g. http://atlas-server-hostname:21000 or https://atlas-server-hostname:21443)." +
    +                    " For accessing Atlas behind Knox gateway, specify Knox gateway URL" +
    +                    " (e.g. https://knox-hostname:8443/gateway/{topology-name}/atlas).")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final AllowableValue ATLAS_AUTHN_BASIC = new AllowableValue("basic", "Basic", "Use username and password.");
    +    static final AllowableValue ATLAS_AUTHN_KERBEROS = new AllowableValue("kerberos", "Kerberos", "Use Kerberos keytab file.");
    +    static final PropertyDescriptor ATLAS_AUTHN_METHOD = new PropertyDescriptor.Builder()
    +            .name("atlas-authentication-method")
    +            .displayName("Atlas Authentication Method")
    +            .description("Specify how to authenticate this reporting task to Atlas server.")
    +            .required(true)
    +            .allowableValues(ATLAS_AUTHN_BASIC, ATLAS_AUTHN_KERBEROS)
    +            .defaultValue(ATLAS_AUTHN_BASIC.getValue())
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_USER = new PropertyDescriptor.Builder()
    +            .name("atlas-username")
    +            .displayName("Atlas Username")
    +            .description("User name to communicate with Atlas.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("atlas-password")
    +            .displayName("Atlas Password")
    +            .description("Password to communicate with Atlas.")
    +            .required(false)
    +            .sensitive(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final PropertyDescriptor ATLAS_CONF_DIR = new PropertyDescriptor.Builder()
    +            .name("atlas-conf-dir")
    +            .displayName("Atlas Configuration Directory")
    +            .description("Directory path that contains 'atlas-application.properties' file." +
    +                    " If not specified and 'Create Atlas Configuration File' is disabled," +
    +                    " then, 'atlas-application.properties' file under root classpath is used.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_NIFI_URL = new PropertyDescriptor.Builder()
    +            .name("atlas-nifi-url")
    +            .displayName("NiFi URL for Atlas")
    +            .description("NiFi URL is used in Atlas to represent this NiFi cluster (or standalone instance)." +
    +                    " It is recommended to use one that can be accessible remotely instead of using 'localhost'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.URL_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_DEFAULT_CLUSTER_NAME = new PropertyDescriptor.Builder()
    +            .name("atlas-default-cluster-name")
    +            .displayName("Atlas Default Cluster Name")
    +            .description("Cluster name for Atlas entities reported by this ReportingTask." +
    +                    " If not specified, 'atlas.cluster.name' in Atlas Configuration File is used." +
    +                    " Cluster name mappings can be configured by user defined properties." +
    +                    " See additional detail for detail.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final PropertyDescriptor ATLAS_CONF_CREATE = new PropertyDescriptor.Builder()
    +            .name("atlas-conf-create")
    +            .displayName("Create Atlas Configuration File")
    +            .description("If enabled, 'atlas-application.properties' file will be created in 'Atlas Configuration Directory'" +
    +                    " automatically when this processor starts." +
    +                    " Note that the existing configuration file will be overwritten.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .allowableValues("true", "false")
    +            .defaultValue("false")
    +            .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +            .name("ssl-context-service")
    +            .displayName("SSL Context Service")
    +            .description("Specifies the SSL Context Service to use for communicating with Atlas and Kafka.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +    static final PropertyDescriptor KAFKA_BOOTSTRAP_SERVERS = new PropertyDescriptor.Builder()
    +            .name("kafka-bootstrap-servers")
    +            .displayName("Kafka Bootstrap Servers")
    +            .description("Kafka Bootstrap Servers to send Atlas hook notification messages based on NiFi provenance events." +
    +                    " E.g. 'localhost:9092'" +
    +                    " NOTE: Once this reporting task has started, restarting NiFi is required to changed this property" +
    +                    " as Atlas library holds a unmodifiable static reference to Kafka client.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final AllowableValue SEC_PLAINTEXT = new AllowableValue("PLAINTEXT", "PLAINTEXT", "PLAINTEXT");
    +    static final AllowableValue SEC_SSL = new AllowableValue("SSL", "SSL", "SSL");
    +    static final AllowableValue SEC_SASL_PLAINTEXT = new AllowableValue("SASL_PLAINTEXT", "SASL_PLAINTEXT", "SASL_PLAINTEXT");
    +    static final AllowableValue SEC_SASL_SSL = new AllowableValue("SASL_SSL", "SASL_SSL", "SASL_SSL");
    +    static final PropertyDescriptor KAFKA_SECURITY_PROTOCOL = new PropertyDescriptor.Builder()
    +            .name("kafka-security-protocol")
    +            .displayName("Kafka Security Protocol")
    +            .description("Protocol used to communicate with Kafka brokers to send Atlas hook notification messages." +
    +                    " Corresponds to Kafka's 'security.protocol' property.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .allowableValues(SEC_PLAINTEXT, SEC_SSL, SEC_SASL_PLAINTEXT, SEC_SASL_SSL)
    +            .defaultValue(SEC_PLAINTEXT.getValue())
    +            .build();
    +
    +    public static final PropertyDescriptor NIFI_KERBEROS_PRINCIPAL = new PropertyDescriptor.Builder()
    +            .name("nifi-kerberos-principal")
    +            .displayName("NiFi Kerberos Principal")
    +            .description("The Kerberos principal for this NiFi instance to access Atlas API and Kafka brokers." +
    +                    " If not set, it is expected to set a JAAS configuration file in the JVM properties defined in the bootstrap.conf file." +
    +                    " This principal will be set into 'sasl.jaas.config' Kafka's property.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +    public static final PropertyDescriptor NIFI_KERBEROS_KEYTAB = new PropertyDescriptor.Builder()
    +            .name("nifi-kerberos-keytab")
    +            .displayName("NiFi Kerberos Keytab")
    +            .description("The Kerberos keytab for this NiFi instance to access Atlas API and Kafka brokers." +
    +                    " If not set, it is expected to set a JAAS configuration file in the JVM properties defined in the bootstrap.conf file." +
    +                    " This principal will be set into 'sasl.jaas.config' Kafka's property.")
    +            .required(false)
    +            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    static final PropertyDescriptor KAFKA_KERBEROS_SERVICE_NAME = new PropertyDescriptor.Builder()
    +            .name("kafka-kerberos-service-name-kafka")
    +            .displayName("Kafka Kerberos Service Name")
    +            .description("The Kerberos principal name that Kafka runs for Atlas notification." +
    +                    " This can be defined either in Kafka's JAAS config or in Kafka's config." +
    +                    " Corresponds to Kafka's 'security.protocol' property." +
    +                    " It is ignored unless one of the SASL options of the <Security Protocol> are selected.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .defaultValue("kafka")
    +            .build();
    +
    +    static final AllowableValue LINEAGE_STRATEGY_SIMPLE_PATH = new AllowableValue("SimplePath", "Simple Path",
    +            "Map NiFi provenance events and target Atlas DataSets to statically created 'nifi_flow_path' Atlas Processes." +
    +                    " See also 'Additional Details'.");
    +    static final AllowableValue LINEAGE_STRATEGY_COMPLETE_PATH = new AllowableValue("CompletePath", "Complete Path",
    +            "Create separate 'nifi_flow_path' Atlas Processes for each distinct input and output DataSet combinations" +
    +                    " by looking at the complete route for a given FlowFile. See also 'Additional Details.");
    +
    +    static final PropertyDescriptor NIFI_LINEAGE_STRATEGY = new PropertyDescriptor.Builder()
    +            .name("nifi-lineage-strategy")
    +            .displayName("NiFi Lineage Strategy")
    +            .description("Specifies granularity on how NiFi data flow should be reported to Atlas.")
    +            .required(true)
    +            .allowableValues(LINEAGE_STRATEGY_SIMPLE_PATH, LINEAGE_STRATEGY_COMPLETE_PATH)
    +            .defaultValue(LINEAGE_STRATEGY_SIMPLE_PATH.getValue())
    +            .build();
    +
    +    private static final String ATLAS_PROPERTIES_FILENAME = "atlas-application.properties";
    +    private static final String ATLAS_PROPERTY_CLUSTER_NAME = "atlas.cluster.name";
    +    private static final String ATLAS_PROPERTY_ENABLE_TLS = "atlas.enableTLS";
    +    private static final String ATLAS_KAFKA_PREFIX = "atlas.kafka.";
    +    private static final String ATLAS_PROPERTY_KAFKA_BOOTSTRAP_SERVERS = ATLAS_KAFKA_PREFIX + "bootstrap.servers";
    +    private static final String ATLAS_PROPERTY_KAFKA_CLIENT_ID = ATLAS_KAFKA_PREFIX + ProducerConfig.CLIENT_ID_CONFIG;
    +    private final ServiceLoader<ClusterResolver> clusterResolverLoader = ServiceLoader.load(ClusterResolver.class);
    +    private volatile NiFiAtlasClient atlasClient;
    +    private volatile Properties atlasProperties;
    +    private volatile boolean isTypeDefCreated = false;
    +    private volatile String defaultClusterName;
    +
    +    private volatile ProvenanceEventConsumer consumer;
    +    private volatile ClusterResolvers clusterResolvers;
    +    private volatile NiFIAtlasHook nifiAtlasHook;
    +    private volatile LineageStrategy lineageStrategy;
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(ATLAS_URLS);
    +        properties.add(ATLAS_AUTHN_METHOD);
    +        properties.add(ATLAS_USER);
    +        properties.add(ATLAS_PASSWORD);
    +        properties.add(ATLAS_CONF_DIR);
    +        properties.add(ATLAS_NIFI_URL);
    +        properties.add(ATLAS_DEFAULT_CLUSTER_NAME);
    +        properties.add(NIFI_LINEAGE_STRATEGY);
    +        properties.add(PROVENANCE_START_POSITION);
    +        properties.add(PROVENANCE_BATCH_SIZE);
    +        properties.add(SSL_CONTEXT_SERVICE);
    +
    +        // Following properties are required if ATLAS_CONF_CREATE is enabled.
    +        // Otherwise should be left blank.
    +        properties.add(ATLAS_CONF_CREATE);
    +        properties.add(NIFI_KERBEROS_PRINCIPAL);
    +        properties.add(NIFI_KERBEROS_KEYTAB);
    +        properties.add(KAFKA_KERBEROS_SERVICE_NAME);
    +        properties.add(KAFKA_BOOTSTRAP_SERVERS);
    +        properties.add(KAFKA_SECURITY_PROTOCOL);
    +
    +        return properties;
    +    }
    +
    +    @Override
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(String propertyDescriptorName) {
    +        for (ClusterResolver resolver : clusterResolverLoader) {
    +            final PropertyDescriptor propertyDescriptor = resolver.getSupportedDynamicPropertyDescriptor(propertyDescriptorName);
    +            if(propertyDescriptor != null) {
    +                return propertyDescriptor;
    +            }
    +        }
    +        return null;
    +    }
    +
    +    private void parseAtlasUrls(final PropertyValue atlasUrlsProp, final Consumer<String> urlStrConsumer) {
    +        final String atlasUrlsStr = atlasUrlsProp.evaluateAttributeExpressions().getValue();
    +        if (atlasUrlsStr != null && !atlasUrlsStr.isEmpty()) {
    +            Arrays.stream(atlasUrlsStr.split(","))
    +                    .map(String::trim)
    +                    .forEach(urlStrConsumer);
    +        }
    +    }
    +
    +    @Override
    +    protected Collection<ValidationResult> customValidate(ValidationContext context) {
    +        final Collection<ValidationResult> results = new ArrayList<>();
    +
    +        final boolean isSSLContextServiceSet = context.getProperty(SSL_CONTEXT_SERVICE).isSet();
    +        final ValidationResult.Builder invalidSSLService = new ValidationResult.Builder()
    +                .subject(SSL_CONTEXT_SERVICE.getDisplayName()).valid(false);
    +        parseAtlasUrls(context.getProperty(ATLAS_URLS), input -> {
    +            final ValidationResult.Builder builder = new ValidationResult.Builder().subject(ATLAS_URLS.getDisplayName()).input(input);
    +            try {
    +                final URL url = new URL(input);
    +                if ("https".equalsIgnoreCase(url.getProtocol()) && !isSSLContextServiceSet) {
    +                    results.add(invalidSSLService.explanation("required by HTTPS Atlas access").build());
    +                } else {
    +                    results.add(builder.explanation("Valid URI").valid(true).build());
    +                }
    +            } catch (Exception e) {
    +                results.add(builder.explanation("Contains invalid URI: " + e).valid(false).build());
    +            }
    +        });
    +
    +        final String atlasAuthNMethod = context.getProperty(ATLAS_AUTHN_METHOD).getValue();
    +        final AtlasAuthN atlasAuthN = getAtlasAuthN(atlasAuthNMethod);
    +        results.addAll(atlasAuthN.validate(context));
    +
    +
    +        clusterResolverLoader.forEach(resolver -> results.addAll(resolver.validate(context)));
    +
    +        if (context.getProperty(ATLAS_CONF_CREATE).asBoolean()) {
    +
    +            Stream.of(ATLAS_CONF_DIR, ATLAS_DEFAULT_CLUSTER_NAME, KAFKA_BOOTSTRAP_SERVERS)
    +                    .filter(p -> !context.getProperty(p).isSet())
    +                    .forEach(p -> results.add(new ValidationResult.Builder()
    +                            .subject(p.getDisplayName())
    +                            .explanation("required to create Atlas configuration file.")
    +                            .valid(false).build()));
    +
    +            validateKafkaProperties(context, results, isSSLContextServiceSet, invalidSSLService);
    +        }
    +
    +        return results;
    +    }
    +
    +    private void validateKafkaProperties(ValidationContext context, Collection<ValidationResult> results, boolean isSSLContextServiceSet, ValidationResult.Builder invalidSSLService) {
    +        final String kafkaSecurityProtocol = context.getProperty(KAFKA_SECURITY_PROTOCOL).getValue();
    +        if ((SEC_SSL.equals(kafkaSecurityProtocol) || SEC_SASL_SSL.equals(kafkaSecurityProtocol))
    +                && !isSSLContextServiceSet) {
    +            results.add(invalidSSLService.explanation("required by SSL Kafka connection").build());
    +        }
    +
    +        if (SEC_SASL_PLAINTEXT.equals(kafkaSecurityProtocol) || SEC_SASL_SSL.equals(kafkaSecurityProtocol)) {
    +            Stream.of(NIFI_KERBEROS_PRINCIPAL, NIFI_KERBEROS_KEYTAB, KAFKA_KERBEROS_SERVICE_NAME)
    +                    .filter(p -> !context.getProperty(p).isSet())
    +                    .forEach(p -> results.add(new ValidationResult.Builder()
    +                            .subject(p.getDisplayName())
    +                            .explanation("required by Kafka SASL authentication.")
    +                            .valid(false).build()));
    +        }
    +    }
    +
    +    @OnScheduled
    +    public void setup(ConfigurationContext context) throws IOException {
    +        // initAtlasClient has to be done first as it loads AtlasProperty.
    +        initAtlasClient(context);
    +        initLineageStrategy(context);
    +        initClusterResolvers(context);
    +    }
    +
    +    private void initLineageStrategy(ConfigurationContext context) throws IOException {
    +        nifiAtlasHook = new NiFIAtlasHook(atlasClient);
    +
    +        final String strategy = context.getProperty(NIFI_LINEAGE_STRATEGY).getValue();
    +        if (LINEAGE_STRATEGY_SIMPLE_PATH.equals(strategy)) {
    +            lineageStrategy = new SimpleFlowPathLineage();
    +        } else if (LINEAGE_STRATEGY_COMPLETE_PATH.equals(strategy)) {
    +            lineageStrategy = new CompleteFlowPathLineage();
    +        }
    +
    +        lineageStrategy.setLineageContext(nifiAtlasHook);
    +        initProvenanceConsumer(context);
    +    }
    +
    +    private void initClusterResolvers(ConfigurationContext context) {
    +        final Set<ClusterResolver> loadedClusterResolvers = new LinkedHashSet<>();
    +        clusterResolverLoader.forEach(resolver -> {
    +            resolver.configure(context);
    +            loadedClusterResolvers.add(resolver);
    +        });
    +        clusterResolvers = new ClusterResolvers(Collections.unmodifiableSet(loadedClusterResolvers), defaultClusterName);
    +    }
    +
    +
    +    private void initAtlasClient(ConfigurationContext context) throws IOException {
    +        List<String> urls = new ArrayList<>();
    +        parseAtlasUrls(context.getProperty(ATLAS_URLS), urls::add);
    +        final boolean isAtlasApiSecure = urls.stream().anyMatch(url -> url.toLowerCase().startsWith("https"));
    +        final String atlasAuthNMethod = context.getProperty(ATLAS_AUTHN_METHOD).getValue();
    +
    +        final String confDirStr = context.getProperty(ATLAS_CONF_DIR).evaluateAttributeExpressions().getValue();
    +        final File confDir = confDirStr != null && !confDirStr.isEmpty() ? new File(confDirStr) : null;
    +
    +        atlasProperties = new Properties();
    +        final File atlasPropertiesFile = new File(confDir, ATLAS_PROPERTIES_FILENAME);
    +
    +        final Boolean createAtlasConf = context.getProperty(ATLAS_CONF_CREATE).asBoolean();
    +        if (!createAtlasConf) {
    +            // Load existing properties file.
    +            if (atlasPropertiesFile.isFile()) {
    +                getLogger().info("Loading {}", new Object[]{atlasPropertiesFile});
    +                try (InputStream in = new FileInputStream(atlasPropertiesFile)) {
    +                    atlasProperties.load(in);
    +                }
    +            } else {
    +                final String fileInClasspath = "/" + ATLAS_PROPERTIES_FILENAME;
    +                try (InputStream in = AtlasNiFiFlowLineage.class.getResourceAsStream(fileInClasspath)) {
    +                    getLogger().info("Loading {} from classpath", new Object[]{fileInClasspath});
    +                    if (in == null) {
    +                        throw new ProcessException(String.format("Could not find %s in classpath." +
    +                                " Please add it to classpath," +
    +                                " or specify %s a directory containing Atlas properties file," +
    +                                " or enable %s to generate it.",
    +                                fileInClasspath, ATLAS_CONF_DIR.getDisplayName(), ATLAS_CONF_CREATE.getDisplayName()));
    +                    }
    +                    atlasProperties.load(in);
    +                }
    +            }
    +        }
    +
    +        // Resolve default cluster name.
    +        defaultClusterName = context.getProperty(ATLAS_DEFAULT_CLUSTER_NAME).evaluateAttributeExpressions().getValue();
    +        if (defaultClusterName == null || defaultClusterName.isEmpty()) {
    +            // If default cluster name is not specified by processor configuration, then load it from Atlas config.
    +            defaultClusterName = atlasProperties.getProperty(ATLAS_PROPERTY_CLUSTER_NAME);
    +        }
    +
    +        // If default cluster name is still not defined, processor should not be able to start.
    +        if (defaultClusterName == null || defaultClusterName.isEmpty()) {
    +            throw new ProcessException("Default cluster name is not defined.");
    +        }
    +
    +        final AtlasAuthN atlasAuthN = getAtlasAuthN(atlasAuthNMethod);
    +        atlasAuthN.configure(context);
    +
    +        // Create Atlas configuration file if necessary.
    +        if (createAtlasConf) {
    +
    +            atlasProperties.put(ATLAS_PROPERTY_CLUSTER_NAME, defaultClusterName);
    +            atlasProperties.put(ATLAS_PROPERTY_ENABLE_TLS, String.valueOf(isAtlasApiSecure));
    +
    +            setKafkaConfig(atlasProperties, context);
    +
    +            atlasAuthN.populateProperties(atlasProperties);
    +
    +            try (FileOutputStream fos = new FileOutputStream(atlasPropertiesFile)) {
    +                String ts = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSSX")
    +                        .withZone(ZoneOffset.UTC)
    +                        .format(Instant.now());
    +                atlasProperties.store(fos, "Generated by Apache NiFi AtlasNiFiFlowLineage ReportingTask at " + ts);
    +            }
    +        }
    +
    +
    +        atlasClient = NiFiAtlasClient.getInstance();
    +        try {
    +            atlasClient.initialize(urls.toArray(new String[]{}), atlasAuthN, confDir);
    +        } catch (final NullPointerException e) {
    --- End diff --
    
    Yeah, this is odd to me, too. However, AtlasClient throws NullPointerException when it can not find atlas properties file with following stack trace. Probably we should report this to Atlas project, too. Thanks.
    
    ```
    2017-12-15 23:46:36,607 ERROR [StandardProcessScheduler Thread-4] o.a.n.c.s.StandardProcessScheduler Failed to invoke the On-Scheduled Lifecycle methods of [AtlasNiFiFlowLineage[id=5a7835aa-0160-1000-974d-14d95e0c38e9], java.lang.reflect.InvocationTargetException, 30 sec] due to {}; administratively yielding this ReportingTask and will attempt to schedule it again after {}
    java.lang.reflect.InvocationTargetException: null
            at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
            at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
            at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
            at java.lang.reflect.Method.invoke(Method.java:498)
            at org.apache.nifi.util.ReflectionUtils.invokeMethodsWithAnnotations(ReflectionUtils.java:137)
            at org.apache.nifi.util.ReflectionUtils.invokeMethodsWithAnnotations(ReflectionUtils.java:125)
            at org.apache.nifi.util.ReflectionUtils.invokeMethodsWithAnnotations(ReflectionUtils.java:70)
            at org.apache.nifi.util.ReflectionUtils.invokeMethodsWithAnnotation(ReflectionUtils.java:47)
            at org.apache.nifi.controller.scheduling.StandardProcessScheduler$2.run(StandardProcessScheduler.java:210)
            at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
            at java.util.concurrent.FutureTask.run(FutureTask.java:266)
            at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
            at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
            at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
            at java.lang.Thread.run(Thread.java:745)
    Caused by: org.apache.nifi.processor.exception.ProcessException: Failed to initialize Atlas client due to java.lang.NullPointerException. Make sure 'atlas-application.properties' is in the directory specified with Atlas Configuration Directory or under root classpath if not specified.
            at org.apache.nifi.atlas.reporting.AtlasNiFiFlowLineage.initAtlasClient(AtlasNiFiFlowLineage.java:505)
            at org.apache.nifi.atlas.reporting.AtlasNiFiFlowLineage.setup(AtlasNiFiFlowLineage.java:402)
            ... 16 common frames omitted
    Caused by: java.lang.NullPointerException: null
            at org.apache.atlas.AtlasBaseClient.getClient(AtlasBaseClient.java:171)
            at org.apache.atlas.AtlasBaseClient.initializeState(AtlasBaseClient.java:154)
            at org.apache.atlas.AtlasBaseClient.initializeState(AtlasBaseClient.java:149)
            at org.apache.atlas.AtlasBaseClient.<init>(AtlasBaseClient.java:96)
            at org.apache.atlas.AtlasClientV2.<init>(AtlasClientV2.java:115)
            at org.apache.nifi.atlas.security.Basic.createClient(Basic.java:62)
            at org.apache.nifi.atlas.NiFiAtlasClient.initialize(NiFiAtlasClient.java:119)
            at org.apache.nifi.atlas.reporting.AtlasNiFiFlowLineage.initAtlasClient(AtlasNiFiFlowLineage.java:503)
            ... 17 common frames omitted
    ```


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157221249
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/NiFIAtlasHook.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.atlas;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.atlas.hook.AtlasHook;
    +import org.apache.atlas.model.instance.AtlasEntity;
    +import org.apache.atlas.model.instance.AtlasObjectId;
    +import org.apache.atlas.notification.hook.HookNotification.EntityPartialUpdateRequest;
    +import org.apache.atlas.notification.hook.HookNotification.HookNotificationMessage;
    +import org.apache.atlas.typesystem.Referenceable;
    +import org.apache.atlas.typesystem.persistence.Id;
    +import org.apache.nifi.atlas.provenance.lineage.LineageContext;
    +import org.apache.nifi.util.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.LinkedHashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.atlas.notification.hook.HookNotification.HookNotificationType.ENTITY_PARTIAL_UPDATE;
    +import static org.apache.nifi.atlas.AtlasUtils.toTypedQualifiedName;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_GUID;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_INPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_OUTPUTS;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_QUALIFIED_NAME;
    +import static org.apache.nifi.atlas.NiFiTypes.ATTR_TYPENAME;
    +import static org.apache.nifi.atlas.NiFiTypes.TYPE_NIFI_FLOW_PATH;
    +
    +/**
    + * This class is not thread-safe as it holds uncommitted notification messages within instance.
    + * {@link #addMessage(HookNotificationMessage)} and {@link #commitMessages()} should be used serially from a single thread.
    + */
    +public class NiFIAtlasHook extends AtlasHook implements LineageContext {
    --- End diff --
    
    Good catch, thank you! I'll fix this.


---

[GitHub] nifi pull request #2335: NIFI-3709: Export NiFi flow dataset lineage to Apac...

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

    https://github.com/apache/nifi/pull/2335#discussion_r157216102
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/AtlasNiFiFlowLineage.java ---
    @@ -0,0 +1,714 @@
    +/*
    + * 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.atlas.reporting;
    +
    +import com.sun.jersey.api.client.ClientResponse;
    +import org.apache.atlas.AtlasServiceException;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.kafka.clients.producer.ProducerConfig;
    +import org.apache.kafka.common.config.SslConfigs;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
    +import org.apache.nifi.annotation.behavior.Stateful;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.atlas.NiFIAtlasHook;
    +import org.apache.nifi.atlas.NiFiAtlasClient;
    +import org.apache.nifi.atlas.NiFiFlow;
    +import org.apache.nifi.atlas.NiFiFlowAnalyzer;
    +import org.apache.nifi.atlas.provenance.AnalysisContext;
    +import org.apache.nifi.atlas.provenance.StandardAnalysisContext;
    +import org.apache.nifi.atlas.provenance.lineage.CompleteFlowPathLineage;
    +import org.apache.nifi.atlas.provenance.lineage.LineageStrategy;
    +import org.apache.nifi.atlas.provenance.lineage.SimpleFlowPathLineage;
    +import org.apache.nifi.atlas.resolver.ClusterResolver;
    +import org.apache.nifi.atlas.resolver.ClusterResolvers;
    +import org.apache.nifi.atlas.resolver.RegexClusterResolver;
    +import org.apache.nifi.atlas.security.AtlasAuthN;
    +import org.apache.nifi.atlas.security.Basic;
    +import org.apache.nifi.atlas.security.Kerberos;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.context.PropertyContext;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.provenance.ProvenanceRepository;
    +import org.apache.nifi.reporting.AbstractReportingTask;
    +import org.apache.nifi.reporting.EventAccess;
    +import org.apache.nifi.reporting.ReportingContext;
    +import org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.time.Instant;
    +import java.time.ZoneOffset;
    +import java.time.format.DateTimeFormatter;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.LinkedHashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.ServiceLoader;
    +import java.util.Set;
    +import java.util.function.Consumer;
    +import java.util.stream.Stream;
    +
    +import static org.apache.commons.lang3.StringUtils.isEmpty;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_BATCH_SIZE;
    +import static org.apache.nifi.reporting.util.provenance.ProvenanceEventConsumer.PROVENANCE_START_POSITION;
    +
    +@Tags({"atlas", "lineage"})
    +@CapabilityDescription("Publishes NiFi flow data set level lineage to Apache Atlas." +
    +        " By reporting flow information to Atlas, an end-to-end Process and DataSet lineage such as across NiFi environments and other systems" +
    +        " connected by technologies, for example NiFi Site-to-Site, Kafka topic or Hive tables." +
    +        " There are limitations and required configurations for both NiFi and Atlas. See 'Additional Details' for further description.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +@DynamicProperty(name = "hostnamePattern.<ClusterName>", value = "hostname Regex patterns", description = RegexClusterResolver.PATTERN_PROPERTY_PREFIX_DESC)
    +// In order for each reporting task instance to have its own static objects such as KafkaNotification.
    +@RequiresInstanceClassLoading
    +public class AtlasNiFiFlowLineage extends AbstractReportingTask {
    +
    +    static final PropertyDescriptor ATLAS_URLS = new PropertyDescriptor.Builder()
    +            .name("atlas-urls")
    +            .displayName("Atlas URLs")
    +            .description("Comma separated URL of Atlas Servers" +
    +                    " (e.g. http://atlas-server-hostname:21000 or https://atlas-server-hostname:21443)." +
    +                    " For accessing Atlas behind Knox gateway, specify Knox gateway URL" +
    +                    " (e.g. https://knox-hostname:8443/gateway/{topology-name}/atlas).")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final AllowableValue ATLAS_AUTHN_BASIC = new AllowableValue("basic", "Basic", "Use username and password.");
    +    static final AllowableValue ATLAS_AUTHN_KERBEROS = new AllowableValue("kerberos", "Kerberos", "Use Kerberos keytab file.");
    +    static final PropertyDescriptor ATLAS_AUTHN_METHOD = new PropertyDescriptor.Builder()
    +            .name("atlas-authentication-method")
    +            .displayName("Atlas Authentication Method")
    +            .description("Specify how to authenticate this reporting task to Atlas server.")
    +            .required(true)
    +            .allowableValues(ATLAS_AUTHN_BASIC, ATLAS_AUTHN_KERBEROS)
    +            .defaultValue(ATLAS_AUTHN_BASIC.getValue())
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_USER = new PropertyDescriptor.Builder()
    +            .name("atlas-username")
    +            .displayName("Atlas Username")
    +            .description("User name to communicate with Atlas.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("atlas-password")
    +            .displayName("Atlas Password")
    +            .description("Password to communicate with Atlas.")
    +            .required(false)
    +            .sensitive(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final PropertyDescriptor ATLAS_CONF_DIR = new PropertyDescriptor.Builder()
    +            .name("atlas-conf-dir")
    +            .displayName("Atlas Configuration Directory")
    +            .description("Directory path that contains 'atlas-application.properties' file." +
    +                    " If not specified and 'Create Atlas Configuration File' is disabled," +
    +                    " then, 'atlas-application.properties' file under root classpath is used.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_NIFI_URL = new PropertyDescriptor.Builder()
    +            .name("atlas-nifi-url")
    +            .displayName("NiFi URL for Atlas")
    +            .description("NiFi URL is used in Atlas to represent this NiFi cluster (or standalone instance)." +
    +                    " It is recommended to use one that can be accessible remotely instead of using 'localhost'.")
    +            .required(true)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.URL_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor ATLAS_DEFAULT_CLUSTER_NAME = new PropertyDescriptor.Builder()
    +            .name("atlas-default-cluster-name")
    +            .displayName("Atlas Default Cluster Name")
    +            .description("Cluster name for Atlas entities reported by this ReportingTask." +
    +                    " If not specified, 'atlas.cluster.name' in Atlas Configuration File is used." +
    +                    " Cluster name mappings can be configured by user defined properties." +
    +                    " See additional detail for detail.")
    +            .required(false)
    +            .expressionLanguageSupported(true)
    +            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +            .build();
    +
    +    static final PropertyDescriptor ATLAS_CONF_CREATE = new PropertyDescriptor.Builder()
    +            .name("atlas-conf-create")
    +            .displayName("Create Atlas Configuration File")
    +            .description("If enabled, 'atlas-application.properties' file will be created in 'Atlas Configuration Directory'" +
    +                    " automatically when this processor starts." +
    --- End diff --
    
    Thank you for finding it!


---