You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@nifi.apache.org by bbende <gi...@git.apache.org> on 2016/05/06 21:41:56 UTC

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

GitHub user bbende opened a pull request:

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

    NIFI-1858 Adding SiteToSiteProvenanceReportingTask

    

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

    $ git pull https://github.com/bbende/nifi NIFI-1858

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

    https://github.com/apache/nifi/pull/419.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 #419
    
----
commit 7a23bddf1350be633573098457bcc2842f7765de
Author: Bryan Bende <bb...@apache.org>
Date:   2016-05-06T17:58:39Z

    NIFI-1858 Adding SiteToSiteProvenanceReportingTask

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

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

    https://github.com/apache/nifi/pull/419#issuecomment-218585975
  
    @bbende I merged this to 0.x branch. However, it cannot be merged to the 'master' branch as-is. Can you create a PR That will merge to master?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

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

    https://github.com/apache/nifi/pull/419#discussion_r62505576
  
    --- Diff: nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java ---
    @@ -0,0 +1,168 @@
    +/*
    + * 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.reporting;
    +
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.events.EventReporter;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.remote.client.SiteToSiteClient;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Base class for ReportingTasks that send data over site-to-site.
    + */
    +public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingTask {
    +
    +    static final PropertyDescriptor DESTINATION_URL = new PropertyDescriptor.Builder()
    +            .name("Destination URL")
    +            .description("The URL to send the Provenance Events to. For example, to send to a NiFi instance running " +
    +                    "at http://localhost:8080/nifi this value should be http://localhost:8080")
    +            .required(true)
    --- End diff --
    
    @jvwing @mosermw I agree, I will push a commit that aligns it with the way RPGs take the url, thanks for reviewing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

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

    https://github.com/apache/nifi/pull/419#discussion_r62548156
  
    --- Diff: nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java ---
    @@ -0,0 +1,354 @@
    +/*
    + * 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.reporting;
    +
    +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.components.PropertyDescriptor;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.components.state.StateManager;
    +import org.apache.nifi.controller.status.PortStatus;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.controller.status.ProcessorStatus;
    +import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.remote.Transaction;
    +import org.apache.nifi.remote.TransferDirection;
    +
    +import javax.json.Json;
    +import javax.json.JsonArray;
    +import javax.json.JsonArrayBuilder;
    +import javax.json.JsonBuilderFactory;
    +import javax.json.JsonObject;
    +import javax.json.JsonObjectBuilder;
    +import java.io.IOException;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.nio.charset.StandardCharsets;
    +import java.text.DateFormat;
    +import java.text.SimpleDateFormat;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.TimeZone;
    +import java.util.UUID;
    +import java.util.concurrent.TimeUnit;
    +
    +@Tags({"provenance", "lineage", "tracking", "site", "site to site"})
    +@CapabilityDescription("Publishes Provenance events using the Site To Site protocol.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +public class SiteToSiteProvenanceReportingTask extends AbstractSiteToSiteReportingTask {
    +
    +    private static final String TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'";
    +    private static final String LAST_EVENT_ID_KEY = "last_event_id";
    +
    +    static final PropertyDescriptor PLATFORM = new PropertyDescriptor.Builder()
    +        .name("Platform")
    +        .description("The value to use for the platform field in each provenance event.")
    +        .required(true)
    +        .expressionLanguageSupported(true)
    +        .defaultValue("nifi")
    +        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +        .build();
    +
    +    private volatile long firstEventId = -1L;
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
    +        properties.add(PLATFORM);
    +        return properties;
    +    }
    +
    +    private String getComponentName(final ProcessGroupStatus status, final ProvenanceEventRecord event) {
    +        if (status == null) {
    +            return null;
    +        }
    +
    +        final String componentId = event.getComponentId();
    +        if (status.getId().equals(componentId)) {
    +            return status.getName();
    +        }
    +
    +        for (final ProcessorStatus procStatus : status.getProcessorStatus()) {
    +            if (procStatus.getId().equals(componentId)) {
    +                return procStatus.getName();
    +            }
    +        }
    +
    +        for (final PortStatus portStatus : status.getInputPortStatus()) {
    +            if (portStatus.getId().equals(componentId)) {
    +                return portStatus.getName();
    +            }
    +        }
    +
    +        for (final PortStatus portStatus : status.getOutputPortStatus()) {
    +            if (portStatus.getId().equals(componentId)) {
    +                return portStatus.getName();
    +            }
    +        }
    +
    +        for (final RemoteProcessGroupStatus rpgStatus : status.getRemoteProcessGroupStatus()) {
    +            if (rpgStatus.getId().equals(componentId)) {
    +                return rpgStatus.getName();
    +            }
    +        }
    +
    +        for (final ProcessGroupStatus childGroup : status.getProcessGroupStatus()) {
    +            final String componentName = getComponentName(childGroup, event);
    +            if (componentName != null) {
    +                return componentName;
    +            }
    +        }
    +
    +        return null;
    +    }
    +
    +    @Override
    +    public void onTrigger(final ReportingContext context) {
    +        final ProcessGroupStatus procGroupStatus = context.getEventAccess().getControllerStatus();
    +        final String rootGroupName = procGroupStatus == null ? null : procGroupStatus.getName();
    +
    +        Long currMaxId = context.getEventAccess().getProvenanceRepository().getMaxEventId();
    +
    +        if(currMaxId == null) {
    +            getLogger().debug("No events to send because no events have been created yet.");
    +            return;
    +        }
    +
    +        if (firstEventId < 0) {
    +            Map<String, String> state;
    +            try {
    +                state = context.getStateManager().getState(Scope.LOCAL).toMap();
    +            } catch (IOException e) {
    +                getLogger().error("Failed to get state at start up due to {}:"+e.getMessage(), e);
    +                return;
    +            }
    +            if (state.containsKey(LAST_EVENT_ID_KEY)) {
    +                firstEventId = Long.parseLong(state.get(LAST_EVENT_ID_KEY)) + 1;
    +            }
    +
    +            if(currMaxId < firstEventId){
    +                getLogger().debug("Current provenance max id is {} which is less than what was stored in state as the last queried event, which was {}. This means the provenance restarted its " +
    +                        "ids. Restarting querying from the beginning.", new Object[]{currMaxId, firstEventId});
    +                firstEventId = -1;
    +            }
    +        }
    +
    +        if (currMaxId == (firstEventId - 1)) {
    +            getLogger().debug("No events to send due to the current max id being equal to the last id that was queried.");
    +            return;
    +        }
    +
    +        List<ProvenanceEventRecord> events;
    +        try {
    +            events = context.getEventAccess().getProvenanceEvents(firstEventId, context.getProperty(BATCH_SIZE).asInteger());
    +        } catch (final IOException ioe) {
    +            getLogger().error("Failed to retrieve Provenance Events from repository due to: " + ioe.getMessage(), ioe);
    +            return;
    +        }
    +
    +        if (events == null || events.isEmpty()) {
    +            getLogger().debug("No events to send due to 'events' being null or empty.");
    +            return;
    +        }
    +
    +        final String nifiUrl = context.getProperty(INSTANCE_URL).evaluateAttributeExpressions().getValue();
    +        URL url;
    +        try {
    +            url = new URL(nifiUrl);
    +        } catch (final MalformedURLException e1) {
    +            // already validated
    +            throw new AssertionError();
    +        }
    +
    +        final String hostname = url.getHost();
    +        final String platform = context.getProperty(PLATFORM).evaluateAttributeExpressions().getValue();
    +
    +        final Map<String, ?> config = Collections.emptyMap();
    +        final JsonBuilderFactory factory = Json.createBuilderFactory(config);
    +        final JsonObjectBuilder builder = factory.createObjectBuilder();
    +
    +        while (events != null && !events.isEmpty()) {
    +            final long start = System.nanoTime();
    +
    +            // Create a JSON array of all the events in the current batch
    +            final JsonArrayBuilder arrayBuilder = factory.createArrayBuilder();
    +            for (final ProvenanceEventRecord event : events) {
    +                arrayBuilder.add(serialize(factory, builder, event, getComponentName(procGroupStatus, event), hostname, url, rootGroupName, platform));
    +            }
    +            final JsonArray jsonArray = arrayBuilder.build();
    +
    +            // Send the JSON document for the current batch
    +            try {
    +                final Transaction transaction = getClient().createTransaction(TransferDirection.SEND);
    +                if (transaction == null) {
    +                    getLogger().debug("All destination nodes are penalized; will attempt to send data later");
    +                    return;
    +                }
    +
    +                final Map<String, String> attributes = new HashMap<>();
    +                final String transactionId = UUID.randomUUID().toString();
    +                attributes.put("reporting.task.transaction.id", transactionId);
    +
    +                final byte[] data = jsonArray.toString().getBytes(StandardCharsets.UTF_8);
    +                transaction.send(data, attributes);
    +                transaction.confirm();
    +                transaction.complete();
    +
    +                final long transferMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
    +                getLogger().info("Successfully sent {} Provenance Events to destination in {} ms; Transaction ID = {}; First Event ID = {}",
    --- End diff --
    
    Wondering if we should default this to debug?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

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

    https://github.com/apache/nifi/pull/419#discussion_r62505265
  
    --- Diff: nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java ---
    @@ -0,0 +1,168 @@
    +/*
    + * 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.reporting;
    +
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.events.EventReporter;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.remote.client.SiteToSiteClient;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Base class for ReportingTasks that send data over site-to-site.
    + */
    +public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingTask {
    +
    +    static final PropertyDescriptor DESTINATION_URL = new PropertyDescriptor.Builder()
    +            .name("Destination URL")
    +            .description("The URL to send the Provenance Events to. For example, to send to a NiFi instance running " +
    +                    "at http://localhost:8080/nifi this value should be http://localhost:8080")
    +            .required(true)
    --- End diff --
    
    I agree, it would be nice to avoid hard coding the /nifi context path.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

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

    https://github.com/apache/nifi/pull/419#discussion_r62914873
  
    --- Diff: nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java ---
    @@ -192,13 +179,20 @@ public void onTrigger(final ReportingContext context) {
             final JsonBuilderFactory factory = Json.createBuilderFactory(config);
             final JsonObjectBuilder builder = factory.createObjectBuilder();
     
    +        final DateFormat df = new SimpleDateFormat(TIMESTAMP_FORMAT);
    +        df.setTimeZone(TimeZone.getTimeZone("Z"));
    +
             while (events != null && !events.isEmpty()) {
                 final long start = System.nanoTime();
     
                 // Create a JSON array of all the events in the current batch
                 final JsonArrayBuilder arrayBuilder = factory.createArrayBuilder();
                 for (final ProvenanceEventRecord event : events) {
    -                arrayBuilder.add(serialize(factory, builder, event, getComponentName(procGroupStatus, event), hostname, url, rootGroupName, platform));
    +                String componentName = null;
    +                if (componentMap.containsKey(event.getComponentId())) {
    --- End diff --
    
    I think we can avoid this check to see if it contains key and instead just call the get() method. If not present it will return null anyway and is a bit more efficient.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

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

    https://github.com/apache/nifi/pull/419#discussion_r62421129
  
    --- Diff: nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/AbstractSiteToSiteReportingTask.java ---
    @@ -0,0 +1,168 @@
    +/*
    + * 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.reporting;
    +
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.events.EventReporter;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.remote.client.SiteToSiteClient;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.net.URL;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Base class for ReportingTasks that send data over site-to-site.
    + */
    +public abstract class AbstractSiteToSiteReportingTask extends AbstractReportingTask {
    +
    +    static final PropertyDescriptor DESTINATION_URL = new PropertyDescriptor.Builder()
    +            .name("Destination URL")
    +            .description("The URL to send the Provenance Events to. For example, to send to a NiFi instance running " +
    +                    "at http://localhost:8080/nifi this value should be http://localhost:8080")
    +            .required(true)
    --- End diff --
    
    May I ask why you request the destination without "/nifi", but then you add "/nifi" to make a destinationUrl variable on line 130?  I found this slightly confusing while testing, mostly because I was also troubleshooting my own destination site-to-site configuration.  It works fine as-is.  But I noticed that Remote Process Groups ask for the remote URL in the form of "https://remotehost:8080/nifi", and I think there may be some simplicity in copying that pattern.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

Posted by joewitt <gi...@git.apache.org>.
Github user joewitt commented on the pull request:

    https://github.com/apache/nifi/pull/419#issuecomment-217663105
  
    It is expected.  The ReportingTask isn't acting on behalf of the user that started it.  It is acting on behalf of NiFi itself and so does have access.  This sort of 'authorization gap' is no different than a DFM that cannot click on a particular piece of content for instance and yet they could redirect the live dataflow.  Probably makes sense to think of it as someone with DFM access having both DFM and Provenance access.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

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

    https://github.com/apache/nifi/pull/419#discussion_r62906770
  
    --- Diff: nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java ---
    @@ -0,0 +1,354 @@
    +/*
    + * 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.reporting;
    +
    +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.components.PropertyDescriptor;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.components.state.StateManager;
    +import org.apache.nifi.controller.status.PortStatus;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.controller.status.ProcessorStatus;
    +import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.remote.Transaction;
    +import org.apache.nifi.remote.TransferDirection;
    +
    +import javax.json.Json;
    +import javax.json.JsonArray;
    +import javax.json.JsonArrayBuilder;
    +import javax.json.JsonBuilderFactory;
    +import javax.json.JsonObject;
    +import javax.json.JsonObjectBuilder;
    +import java.io.IOException;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.nio.charset.StandardCharsets;
    +import java.text.DateFormat;
    +import java.text.SimpleDateFormat;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.TimeZone;
    +import java.util.UUID;
    +import java.util.concurrent.TimeUnit;
    +
    +@Tags({"provenance", "lineage", "tracking", "site", "site to site"})
    +@CapabilityDescription("Publishes Provenance events using the Site To Site protocol.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +public class SiteToSiteProvenanceReportingTask extends AbstractSiteToSiteReportingTask {
    +
    +    private static final String TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'";
    +    private static final String LAST_EVENT_ID_KEY = "last_event_id";
    +
    +    static final PropertyDescriptor PLATFORM = new PropertyDescriptor.Builder()
    +        .name("Platform")
    +        .description("The value to use for the platform field in each provenance event.")
    +        .required(true)
    +        .expressionLanguageSupported(true)
    +        .defaultValue("nifi")
    +        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +        .build();
    +
    +    private volatile long firstEventId = -1L;
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
    +        properties.add(PLATFORM);
    +        return properties;
    +    }
    +
    +    private String getComponentName(final ProcessGroupStatus status, final ProvenanceEventRecord event) {
    +        if (status == null) {
    +            return null;
    +        }
    +
    +        final String componentId = event.getComponentId();
    +        if (status.getId().equals(componentId)) {
    +            return status.getName();
    +        }
    +
    +        for (final ProcessorStatus procStatus : status.getProcessorStatus()) {
    +            if (procStatus.getId().equals(componentId)) {
    +                return procStatus.getName();
    +            }
    +        }
    +
    +        for (final PortStatus portStatus : status.getInputPortStatus()) {
    +            if (portStatus.getId().equals(componentId)) {
    +                return portStatus.getName();
    +            }
    +        }
    +
    +        for (final PortStatus portStatus : status.getOutputPortStatus()) {
    +            if (portStatus.getId().equals(componentId)) {
    +                return portStatus.getName();
    +            }
    +        }
    +
    +        for (final RemoteProcessGroupStatus rpgStatus : status.getRemoteProcessGroupStatus()) {
    +            if (rpgStatus.getId().equals(componentId)) {
    +                return rpgStatus.getName();
    +            }
    +        }
    +
    +        for (final ProcessGroupStatus childGroup : status.getProcessGroupStatus()) {
    +            final String componentName = getComponentName(childGroup, event);
    +            if (componentName != null) {
    +                return componentName;
    +            }
    +        }
    +
    +        return null;
    +    }
    +
    +    @Override
    +    public void onTrigger(final ReportingContext context) {
    +        final ProcessGroupStatus procGroupStatus = context.getEventAccess().getControllerStatus();
    +        final String rootGroupName = procGroupStatus == null ? null : procGroupStatus.getName();
    +
    +        Long currMaxId = context.getEventAccess().getProvenanceRepository().getMaxEventId();
    +
    +        if(currMaxId == null) {
    +            getLogger().debug("No events to send because no events have been created yet.");
    +            return;
    +        }
    +
    +        if (firstEventId < 0) {
    +            Map<String, String> state;
    +            try {
    +                state = context.getStateManager().getState(Scope.LOCAL).toMap();
    +            } catch (IOException e) {
    +                getLogger().error("Failed to get state at start up due to {}:"+e.getMessage(), e);
    +                return;
    +            }
    +            if (state.containsKey(LAST_EVENT_ID_KEY)) {
    +                firstEventId = Long.parseLong(state.get(LAST_EVENT_ID_KEY)) + 1;
    +            }
    +
    +            if(currMaxId < firstEventId){
    +                getLogger().debug("Current provenance max id is {} which is less than what was stored in state as the last queried event, which was {}. This means the provenance restarted its " +
    +                        "ids. Restarting querying from the beginning.", new Object[]{currMaxId, firstEventId});
    +                firstEventId = -1;
    +            }
    +        }
    +
    +        if (currMaxId == (firstEventId - 1)) {
    +            getLogger().debug("No events to send due to the current max id being equal to the last id that was queried.");
    +            return;
    +        }
    +
    +        List<ProvenanceEventRecord> events;
    +        try {
    +            events = context.getEventAccess().getProvenanceEvents(firstEventId, context.getProperty(BATCH_SIZE).asInteger());
    +        } catch (final IOException ioe) {
    +            getLogger().error("Failed to retrieve Provenance Events from repository due to: " + ioe.getMessage(), ioe);
    +            return;
    +        }
    +
    +        if (events == null || events.isEmpty()) {
    +            getLogger().debug("No events to send due to 'events' being null or empty.");
    +            return;
    +        }
    +
    +        final String nifiUrl = context.getProperty(INSTANCE_URL).evaluateAttributeExpressions().getValue();
    +        URL url;
    +        try {
    +            url = new URL(nifiUrl);
    +        } catch (final MalformedURLException e1) {
    +            // already validated
    +            throw new AssertionError();
    +        }
    +
    +        final String hostname = url.getHost();
    +        final String platform = context.getProperty(PLATFORM).evaluateAttributeExpressions().getValue();
    +
    +        final Map<String, ?> config = Collections.emptyMap();
    +        final JsonBuilderFactory factory = Json.createBuilderFactory(config);
    +        final JsonObjectBuilder builder = factory.createObjectBuilder();
    +
    +        while (events != null && !events.isEmpty()) {
    +            final long start = System.nanoTime();
    +
    +            // Create a JSON array of all the events in the current batch
    +            final JsonArrayBuilder arrayBuilder = factory.createArrayBuilder();
    +            for (final ProvenanceEventRecord event : events) {
    +                arrayBuilder.add(serialize(factory, builder, event, getComponentName(procGroupStatus, event), hostname, url, rootGroupName, platform));
    +            }
    +            final JsonArray jsonArray = arrayBuilder.build();
    +
    +            // Send the JSON document for the current batch
    +            try {
    +                final Transaction transaction = getClient().createTransaction(TransferDirection.SEND);
    +                if (transaction == null) {
    +                    getLogger().debug("All destination nodes are penalized; will attempt to send data later");
    +                    return;
    +                }
    +
    +                final Map<String, String> attributes = new HashMap<>();
    +                final String transactionId = UUID.randomUUID().toString();
    +                attributes.put("reporting.task.transaction.id", transactionId);
    +
    +                final byte[] data = jsonArray.toString().getBytes(StandardCharsets.UTF_8);
    +                transaction.send(data, attributes);
    +                transaction.confirm();
    +                transaction.complete();
    +
    +                final long transferMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
    +                getLogger().info("Successfully sent {} Provenance Events to destination in {} ms; Transaction ID = {}; First Event ID = {}",
    --- End diff --
    
    @jvwing you are correct that deleting the prov repo is what would trigger this to occur. In that event, we don't want to stop sending data until we generate that many events again. I'm not sure what level that log message should be though. I could see using WARN in that case.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

Posted by jvwing <gi...@git.apache.org>.
Github user jvwing commented on the pull request:

    https://github.com/apache/nifi/pull/419#issuecomment-217662827
  
    In my test configuration, it seemed that ROLE_DFM permission was sufficient to enable this controller service and export provenance data.  Is that expected?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

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

    https://github.com/apache/nifi/pull/419#discussion_r62906404
  
    --- Diff: nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java ---
    @@ -0,0 +1,354 @@
    +/*
    + * 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.reporting;
    +
    +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.components.PropertyDescriptor;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.components.state.StateManager;
    +import org.apache.nifi.controller.status.PortStatus;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.controller.status.ProcessorStatus;
    +import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.remote.Transaction;
    +import org.apache.nifi.remote.TransferDirection;
    +
    +import javax.json.Json;
    +import javax.json.JsonArray;
    +import javax.json.JsonArrayBuilder;
    +import javax.json.JsonBuilderFactory;
    +import javax.json.JsonObject;
    +import javax.json.JsonObjectBuilder;
    +import java.io.IOException;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.nio.charset.StandardCharsets;
    +import java.text.DateFormat;
    +import java.text.SimpleDateFormat;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.TimeZone;
    +import java.util.UUID;
    +import java.util.concurrent.TimeUnit;
    +
    +@Tags({"provenance", "lineage", "tracking", "site", "site to site"})
    +@CapabilityDescription("Publishes Provenance events using the Site To Site protocol.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +public class SiteToSiteProvenanceReportingTask extends AbstractSiteToSiteReportingTask {
    +
    +    private static final String TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'";
    +    private static final String LAST_EVENT_ID_KEY = "last_event_id";
    +
    +    static final PropertyDescriptor PLATFORM = new PropertyDescriptor.Builder()
    +        .name("Platform")
    +        .description("The value to use for the platform field in each provenance event.")
    +        .required(true)
    +        .expressionLanguageSupported(true)
    +        .defaultValue("nifi")
    +        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +        .build();
    +
    +    private volatile long firstEventId = -1L;
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
    +        properties.add(PLATFORM);
    +        return properties;
    +    }
    +
    +    private String getComponentName(final ProcessGroupStatus status, final ProvenanceEventRecord event) {
    +        if (status == null) {
    +            return null;
    +        }
    +
    +        final String componentId = event.getComponentId();
    +        if (status.getId().equals(componentId)) {
    +            return status.getName();
    +        }
    +
    +        for (final ProcessorStatus procStatus : status.getProcessorStatus()) {
    +            if (procStatus.getId().equals(componentId)) {
    +                return procStatus.getName();
    +            }
    +        }
    +
    +        for (final PortStatus portStatus : status.getInputPortStatus()) {
    +            if (portStatus.getId().equals(componentId)) {
    +                return portStatus.getName();
    +            }
    +        }
    +
    +        for (final PortStatus portStatus : status.getOutputPortStatus()) {
    +            if (portStatus.getId().equals(componentId)) {
    +                return portStatus.getName();
    +            }
    +        }
    +
    +        for (final RemoteProcessGroupStatus rpgStatus : status.getRemoteProcessGroupStatus()) {
    +            if (rpgStatus.getId().equals(componentId)) {
    +                return rpgStatus.getName();
    +            }
    +        }
    +
    +        for (final ProcessGroupStatus childGroup : status.getProcessGroupStatus()) {
    +            final String componentName = getComponentName(childGroup, event);
    +            if (componentName != null) {
    +                return componentName;
    +            }
    +        }
    +
    +        return null;
    +    }
    +
    +    @Override
    +    public void onTrigger(final ReportingContext context) {
    +        final ProcessGroupStatus procGroupStatus = context.getEventAccess().getControllerStatus();
    +        final String rootGroupName = procGroupStatus == null ? null : procGroupStatus.getName();
    +
    +        Long currMaxId = context.getEventAccess().getProvenanceRepository().getMaxEventId();
    +
    +        if(currMaxId == null) {
    +            getLogger().debug("No events to send because no events have been created yet.");
    +            return;
    +        }
    +
    +        if (firstEventId < 0) {
    +            Map<String, String> state;
    +            try {
    +                state = context.getStateManager().getState(Scope.LOCAL).toMap();
    +            } catch (IOException e) {
    +                getLogger().error("Failed to get state at start up due to {}:"+e.getMessage(), e);
    +                return;
    +            }
    +            if (state.containsKey(LAST_EVENT_ID_KEY)) {
    +                firstEventId = Long.parseLong(state.get(LAST_EVENT_ID_KEY)) + 1;
    +            }
    +
    +            if(currMaxId < firstEventId){
    +                getLogger().debug("Current provenance max id is {} which is less than what was stored in state as the last queried event, which was {}. This means the provenance restarted its " +
    +                        "ids. Restarting querying from the beginning.", new Object[]{currMaxId, firstEventId});
    +                firstEventId = -1;
    +            }
    +        }
    +
    +        if (currMaxId == (firstEventId - 1)) {
    +            getLogger().debug("No events to send due to the current max id being equal to the last id that was queried.");
    +            return;
    +        }
    +
    +        List<ProvenanceEventRecord> events;
    +        try {
    +            events = context.getEventAccess().getProvenanceEvents(firstEventId, context.getProperty(BATCH_SIZE).asInteger());
    +        } catch (final IOException ioe) {
    +            getLogger().error("Failed to retrieve Provenance Events from repository due to: " + ioe.getMessage(), ioe);
    +            return;
    +        }
    +
    +        if (events == null || events.isEmpty()) {
    +            getLogger().debug("No events to send due to 'events' being null or empty.");
    +            return;
    +        }
    +
    +        final String nifiUrl = context.getProperty(INSTANCE_URL).evaluateAttributeExpressions().getValue();
    +        URL url;
    +        try {
    +            url = new URL(nifiUrl);
    +        } catch (final MalformedURLException e1) {
    +            // already validated
    +            throw new AssertionError();
    +        }
    +
    +        final String hostname = url.getHost();
    +        final String platform = context.getProperty(PLATFORM).evaluateAttributeExpressions().getValue();
    +
    +        final Map<String, ?> config = Collections.emptyMap();
    +        final JsonBuilderFactory factory = Json.createBuilderFactory(config);
    +        final JsonObjectBuilder builder = factory.createObjectBuilder();
    +
    +        while (events != null && !events.isEmpty()) {
    +            final long start = System.nanoTime();
    +
    +            // Create a JSON array of all the events in the current batch
    +            final JsonArrayBuilder arrayBuilder = factory.createArrayBuilder();
    +            for (final ProvenanceEventRecord event : events) {
    +                arrayBuilder.add(serialize(factory, builder, event, getComponentName(procGroupStatus, event), hostname, url, rootGroupName, platform));
    --- End diff --
    
    This can be pretty expensive potentially, continually iterating over every component in the flow, looking for the correct identifier. What do you think about just converting the ProcessGroupStatus into a Map<String, String> where the key is a component ID and the value is the component name? Then we could just do a quick get() on the map


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

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

    https://github.com/apache/nifi/pull/419#issuecomment-218765899
  
    @markap14 thanks, going to close this PR and initiate a new one against master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

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

    https://github.com/apache/nifi/pull/419#discussion_r62907252
  
    --- Diff: nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java ---
    @@ -0,0 +1,354 @@
    +/*
    + * 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.reporting;
    +
    +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.components.PropertyDescriptor;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.components.state.StateManager;
    +import org.apache.nifi.controller.status.PortStatus;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.controller.status.ProcessorStatus;
    +import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.remote.Transaction;
    +import org.apache.nifi.remote.TransferDirection;
    +
    +import javax.json.Json;
    +import javax.json.JsonArray;
    +import javax.json.JsonArrayBuilder;
    +import javax.json.JsonBuilderFactory;
    +import javax.json.JsonObject;
    +import javax.json.JsonObjectBuilder;
    +import java.io.IOException;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.nio.charset.StandardCharsets;
    +import java.text.DateFormat;
    +import java.text.SimpleDateFormat;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.TimeZone;
    +import java.util.UUID;
    +import java.util.concurrent.TimeUnit;
    +
    +@Tags({"provenance", "lineage", "tracking", "site", "site to site"})
    +@CapabilityDescription("Publishes Provenance events using the Site To Site protocol.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +public class SiteToSiteProvenanceReportingTask extends AbstractSiteToSiteReportingTask {
    +
    +    private static final String TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'";
    +    private static final String LAST_EVENT_ID_KEY = "last_event_id";
    +
    +    static final PropertyDescriptor PLATFORM = new PropertyDescriptor.Builder()
    +        .name("Platform")
    +        .description("The value to use for the platform field in each provenance event.")
    +        .required(true)
    +        .expressionLanguageSupported(true)
    +        .defaultValue("nifi")
    +        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +        .build();
    +
    +    private volatile long firstEventId = -1L;
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
    +        properties.add(PLATFORM);
    +        return properties;
    +    }
    +
    +    private String getComponentName(final ProcessGroupStatus status, final ProvenanceEventRecord event) {
    +        if (status == null) {
    +            return null;
    +        }
    +
    +        final String componentId = event.getComponentId();
    +        if (status.getId().equals(componentId)) {
    +            return status.getName();
    +        }
    +
    +        for (final ProcessorStatus procStatus : status.getProcessorStatus()) {
    +            if (procStatus.getId().equals(componentId)) {
    +                return procStatus.getName();
    +            }
    +        }
    +
    +        for (final PortStatus portStatus : status.getInputPortStatus()) {
    +            if (portStatus.getId().equals(componentId)) {
    +                return portStatus.getName();
    +            }
    +        }
    +
    +        for (final PortStatus portStatus : status.getOutputPortStatus()) {
    +            if (portStatus.getId().equals(componentId)) {
    +                return portStatus.getName();
    +            }
    +        }
    +
    +        for (final RemoteProcessGroupStatus rpgStatus : status.getRemoteProcessGroupStatus()) {
    +            if (rpgStatus.getId().equals(componentId)) {
    +                return rpgStatus.getName();
    +            }
    +        }
    +
    +        for (final ProcessGroupStatus childGroup : status.getProcessGroupStatus()) {
    +            final String componentName = getComponentName(childGroup, event);
    +            if (componentName != null) {
    +                return componentName;
    +            }
    +        }
    +
    +        return null;
    +    }
    +
    +    @Override
    +    public void onTrigger(final ReportingContext context) {
    +        final ProcessGroupStatus procGroupStatus = context.getEventAccess().getControllerStatus();
    +        final String rootGroupName = procGroupStatus == null ? null : procGroupStatus.getName();
    +
    +        Long currMaxId = context.getEventAccess().getProvenanceRepository().getMaxEventId();
    +
    +        if(currMaxId == null) {
    +            getLogger().debug("No events to send because no events have been created yet.");
    +            return;
    +        }
    +
    +        if (firstEventId < 0) {
    +            Map<String, String> state;
    +            try {
    +                state = context.getStateManager().getState(Scope.LOCAL).toMap();
    +            } catch (IOException e) {
    +                getLogger().error("Failed to get state at start up due to {}:"+e.getMessage(), e);
    +                return;
    +            }
    +            if (state.containsKey(LAST_EVENT_ID_KEY)) {
    +                firstEventId = Long.parseLong(state.get(LAST_EVENT_ID_KEY)) + 1;
    +            }
    +
    +            if(currMaxId < firstEventId){
    +                getLogger().debug("Current provenance max id is {} which is less than what was stored in state as the last queried event, which was {}. This means the provenance restarted its " +
    +                        "ids. Restarting querying from the beginning.", new Object[]{currMaxId, firstEventId});
    +                firstEventId = -1;
    +            }
    +        }
    +
    +        if (currMaxId == (firstEventId - 1)) {
    +            getLogger().debug("No events to send due to the current max id being equal to the last id that was queried.");
    +            return;
    +        }
    +
    +        List<ProvenanceEventRecord> events;
    +        try {
    +            events = context.getEventAccess().getProvenanceEvents(firstEventId, context.getProperty(BATCH_SIZE).asInteger());
    +        } catch (final IOException ioe) {
    +            getLogger().error("Failed to retrieve Provenance Events from repository due to: " + ioe.getMessage(), ioe);
    +            return;
    +        }
    +
    +        if (events == null || events.isEmpty()) {
    +            getLogger().debug("No events to send due to 'events' being null or empty.");
    +            return;
    +        }
    +
    +        final String nifiUrl = context.getProperty(INSTANCE_URL).evaluateAttributeExpressions().getValue();
    +        URL url;
    +        try {
    +            url = new URL(nifiUrl);
    +        } catch (final MalformedURLException e1) {
    +            // already validated
    +            throw new AssertionError();
    +        }
    +
    +        final String hostname = url.getHost();
    +        final String platform = context.getProperty(PLATFORM).evaluateAttributeExpressions().getValue();
    +
    +        final Map<String, ?> config = Collections.emptyMap();
    +        final JsonBuilderFactory factory = Json.createBuilderFactory(config);
    +        final JsonObjectBuilder builder = factory.createObjectBuilder();
    +
    +        while (events != null && !events.isEmpty()) {
    +            final long start = System.nanoTime();
    +
    +            // Create a JSON array of all the events in the current batch
    +            final JsonArrayBuilder arrayBuilder = factory.createArrayBuilder();
    +            for (final ProvenanceEventRecord event : events) {
    +                arrayBuilder.add(serialize(factory, builder, event, getComponentName(procGroupStatus, event), hostname, url, rootGroupName, platform));
    +            }
    +            final JsonArray jsonArray = arrayBuilder.build();
    +
    +            // Send the JSON document for the current batch
    +            try {
    +                final Transaction transaction = getClient().createTransaction(TransferDirection.SEND);
    +                if (transaction == null) {
    +                    getLogger().debug("All destination nodes are penalized; will attempt to send data later");
    +                    return;
    +                }
    +
    +                final Map<String, String> attributes = new HashMap<>();
    +                final String transactionId = UUID.randomUUID().toString();
    +                attributes.put("reporting.task.transaction.id", transactionId);
    +
    +                final byte[] data = jsonArray.toString().getBytes(StandardCharsets.UTF_8);
    +                transaction.send(data, attributes);
    +                transaction.confirm();
    +                transaction.complete();
    +
    +                final long transferMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
    +                getLogger().info("Successfully sent {} Provenance Events to destination in {} ms; Transaction ID = {}; First Event ID = {}",
    +                        new Object[]{events.size(), transferMillis, transactionId, events.get(0).getEventId()});
    +            } catch (final IOException e) {
    +                throw new ProcessException("Failed to send Provenance Events to destination due to IOException:" + e.getMessage(), e);
    +            }
    +
    +            // Store the id of the last event so we know where we left off
    +            final ProvenanceEventRecord lastEvent = events.get(events.size() - 1);
    +            final String lastEventId = String.valueOf(lastEvent.getEventId());
    +            try {
    +                StateManager stateManager = context.getStateManager();
    +                Map<String, String> newMapOfState = new HashMap<>();
    +                newMapOfState.put(LAST_EVENT_ID_KEY, lastEventId);
    +                stateManager.setState(newMapOfState, Scope.LOCAL);
    +            } catch (final IOException ioe) {
    +                getLogger().error("Failed to update state to {} due to {}; this could result in events being re-sent after a restart. The message of {} was: {}",
    +                        new Object[]{lastEventId, ioe, ioe, ioe.getMessage()}, ioe);
    +            }
    +
    +            firstEventId = lastEvent.getEventId() + 1;
    +
    +            // Retrieve the next batch
    +            try {
    +                events = context.getEventAccess().getProvenanceEvents(firstEventId, context.getProperty(BATCH_SIZE).asInteger());
    +            } catch (final IOException ioe) {
    +                getLogger().error("Failed to retrieve Provenance Events from repository due to: " + ioe.getMessage(), ioe);
    +                return;
    +            }
    +        }
    +
    +    }
    +
    +    static JsonObject serialize(final JsonBuilderFactory factory, final JsonObjectBuilder builder, final ProvenanceEventRecord event,
    +        final String componentName, final String hostname, final URL nifiUrl, final String applicationName, final String platform) {
    +        addField(builder, "eventId", UUID.randomUUID().toString());
    +        addField(builder, "eventOrdinal", event.getEventId());
    +        addField(builder, "eventType", event.getEventType().name());
    +        addField(builder, "timestampMillis", event.getEventTime());
    +
    +        final DateFormat df = new SimpleDateFormat(TIMESTAMP_FORMAT);
    --- End diff --
    
    I believe SimpleDateFormat is fairly expensive to initialize. Would recommend we pull this out of this function and pass it in as an argument, so that we can define it in the onTrigger method.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

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

    https://github.com/apache/nifi/pull/419#discussion_r62906556
  
    --- Diff: nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java ---
    @@ -0,0 +1,354 @@
    +/*
    + * 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.reporting;
    +
    +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.components.PropertyDescriptor;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.components.state.StateManager;
    +import org.apache.nifi.controller.status.PortStatus;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.controller.status.ProcessorStatus;
    +import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.remote.Transaction;
    +import org.apache.nifi.remote.TransferDirection;
    +
    +import javax.json.Json;
    +import javax.json.JsonArray;
    +import javax.json.JsonArrayBuilder;
    +import javax.json.JsonBuilderFactory;
    +import javax.json.JsonObject;
    +import javax.json.JsonObjectBuilder;
    +import java.io.IOException;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.nio.charset.StandardCharsets;
    +import java.text.DateFormat;
    +import java.text.SimpleDateFormat;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.TimeZone;
    +import java.util.UUID;
    +import java.util.concurrent.TimeUnit;
    +
    +@Tags({"provenance", "lineage", "tracking", "site", "site to site"})
    +@CapabilityDescription("Publishes Provenance events using the Site To Site protocol.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +public class SiteToSiteProvenanceReportingTask extends AbstractSiteToSiteReportingTask {
    +
    +    private static final String TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'";
    +    private static final String LAST_EVENT_ID_KEY = "last_event_id";
    +
    +    static final PropertyDescriptor PLATFORM = new PropertyDescriptor.Builder()
    +        .name("Platform")
    +        .description("The value to use for the platform field in each provenance event.")
    +        .required(true)
    +        .expressionLanguageSupported(true)
    +        .defaultValue("nifi")
    +        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +        .build();
    +
    +    private volatile long firstEventId = -1L;
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
    +        properties.add(PLATFORM);
    +        return properties;
    +    }
    +
    +    private String getComponentName(final ProcessGroupStatus status, final ProvenanceEventRecord event) {
    +        if (status == null) {
    +            return null;
    +        }
    +
    +        final String componentId = event.getComponentId();
    +        if (status.getId().equals(componentId)) {
    +            return status.getName();
    +        }
    +
    +        for (final ProcessorStatus procStatus : status.getProcessorStatus()) {
    +            if (procStatus.getId().equals(componentId)) {
    +                return procStatus.getName();
    +            }
    +        }
    +
    +        for (final PortStatus portStatus : status.getInputPortStatus()) {
    +            if (portStatus.getId().equals(componentId)) {
    +                return portStatus.getName();
    +            }
    +        }
    +
    +        for (final PortStatus portStatus : status.getOutputPortStatus()) {
    +            if (portStatus.getId().equals(componentId)) {
    +                return portStatus.getName();
    +            }
    +        }
    +
    +        for (final RemoteProcessGroupStatus rpgStatus : status.getRemoteProcessGroupStatus()) {
    +            if (rpgStatus.getId().equals(componentId)) {
    +                return rpgStatus.getName();
    +            }
    +        }
    +
    +        for (final ProcessGroupStatus childGroup : status.getProcessGroupStatus()) {
    +            final String componentName = getComponentName(childGroup, event);
    +            if (componentName != null) {
    +                return componentName;
    +            }
    +        }
    +
    +        return null;
    +    }
    +
    +    @Override
    +    public void onTrigger(final ReportingContext context) {
    +        final ProcessGroupStatus procGroupStatus = context.getEventAccess().getControllerStatus();
    +        final String rootGroupName = procGroupStatus == null ? null : procGroupStatus.getName();
    +
    +        Long currMaxId = context.getEventAccess().getProvenanceRepository().getMaxEventId();
    +
    +        if(currMaxId == null) {
    +            getLogger().debug("No events to send because no events have been created yet.");
    +            return;
    +        }
    +
    +        if (firstEventId < 0) {
    +            Map<String, String> state;
    +            try {
    +                state = context.getStateManager().getState(Scope.LOCAL).toMap();
    +            } catch (IOException e) {
    +                getLogger().error("Failed to get state at start up due to {}:"+e.getMessage(), e);
    +                return;
    +            }
    +            if (state.containsKey(LAST_EVENT_ID_KEY)) {
    +                firstEventId = Long.parseLong(state.get(LAST_EVENT_ID_KEY)) + 1;
    +            }
    +
    +            if(currMaxId < firstEventId){
    +                getLogger().debug("Current provenance max id is {} which is less than what was stored in state as the last queried event, which was {}. This means the provenance restarted its " +
    +                        "ids. Restarting querying from the beginning.", new Object[]{currMaxId, firstEventId});
    +                firstEventId = -1;
    +            }
    +        }
    +
    +        if (currMaxId == (firstEventId - 1)) {
    +            getLogger().debug("No events to send due to the current max id being equal to the last id that was queried.");
    +            return;
    +        }
    +
    +        List<ProvenanceEventRecord> events;
    +        try {
    +            events = context.getEventAccess().getProvenanceEvents(firstEventId, context.getProperty(BATCH_SIZE).asInteger());
    +        } catch (final IOException ioe) {
    +            getLogger().error("Failed to retrieve Provenance Events from repository due to: " + ioe.getMessage(), ioe);
    +            return;
    +        }
    +
    +        if (events == null || events.isEmpty()) {
    +            getLogger().debug("No events to send due to 'events' being null or empty.");
    +            return;
    +        }
    +
    +        final String nifiUrl = context.getProperty(INSTANCE_URL).evaluateAttributeExpressions().getValue();
    +        URL url;
    +        try {
    +            url = new URL(nifiUrl);
    +        } catch (final MalformedURLException e1) {
    +            // already validated
    +            throw new AssertionError();
    +        }
    +
    +        final String hostname = url.getHost();
    +        final String platform = context.getProperty(PLATFORM).evaluateAttributeExpressions().getValue();
    +
    +        final Map<String, ?> config = Collections.emptyMap();
    +        final JsonBuilderFactory factory = Json.createBuilderFactory(config);
    +        final JsonObjectBuilder builder = factory.createObjectBuilder();
    +
    +        while (events != null && !events.isEmpty()) {
    +            final long start = System.nanoTime();
    +
    +            // Create a JSON array of all the events in the current batch
    +            final JsonArrayBuilder arrayBuilder = factory.createArrayBuilder();
    +            for (final ProvenanceEventRecord event : events) {
    +                arrayBuilder.add(serialize(factory, builder, event, getComponentName(procGroupStatus, event), hostname, url, rootGroupName, platform));
    +            }
    +            final JsonArray jsonArray = arrayBuilder.build();
    +
    +            // Send the JSON document for the current batch
    +            try {
    +                final Transaction transaction = getClient().createTransaction(TransferDirection.SEND);
    +                if (transaction == null) {
    +                    getLogger().debug("All destination nodes are penalized; will attempt to send data later");
    +                    return;
    +                }
    +
    +                final Map<String, String> attributes = new HashMap<>();
    +                final String transactionId = UUID.randomUUID().toString();
    +                attributes.put("reporting.task.transaction.id", transactionId);
    +
    +                final byte[] data = jsonArray.toString().getBytes(StandardCharsets.UTF_8);
    +                transaction.send(data, attributes);
    +                transaction.confirm();
    +                transaction.complete();
    +
    +                final long transferMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
    +                getLogger().info("Successfully sent {} Provenance Events to destination in {} ms; Transaction ID = {}; First Event ID = {}",
    --- End diff --
    
    I'm ok with leaving this at INFO - from the point of view of the reporting task, it is an INFO-level event.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

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

    https://github.com/apache/nifi/pull/419#discussion_r62706651
  
    --- Diff: nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java ---
    @@ -0,0 +1,354 @@
    +/*
    + * 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.reporting;
    +
    +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.components.PropertyDescriptor;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.components.state.StateManager;
    +import org.apache.nifi.controller.status.PortStatus;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.controller.status.ProcessorStatus;
    +import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.provenance.ProvenanceEventRecord;
    +import org.apache.nifi.remote.Transaction;
    +import org.apache.nifi.remote.TransferDirection;
    +
    +import javax.json.Json;
    +import javax.json.JsonArray;
    +import javax.json.JsonArrayBuilder;
    +import javax.json.JsonBuilderFactory;
    +import javax.json.JsonObject;
    +import javax.json.JsonObjectBuilder;
    +import java.io.IOException;
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.nio.charset.StandardCharsets;
    +import java.text.DateFormat;
    +import java.text.SimpleDateFormat;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.TimeZone;
    +import java.util.UUID;
    +import java.util.concurrent.TimeUnit;
    +
    +@Tags({"provenance", "lineage", "tracking", "site", "site to site"})
    +@CapabilityDescription("Publishes Provenance events using the Site To Site protocol.")
    +@Stateful(scopes = Scope.LOCAL, description = "Stores the Reporting Task's last event Id so that on restart the task knows where it left off.")
    +public class SiteToSiteProvenanceReportingTask extends AbstractSiteToSiteReportingTask {
    +
    +    private static final String TIMESTAMP_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'";
    +    private static final String LAST_EVENT_ID_KEY = "last_event_id";
    +
    +    static final PropertyDescriptor PLATFORM = new PropertyDescriptor.Builder()
    +        .name("Platform")
    +        .description("The value to use for the platform field in each provenance event.")
    +        .required(true)
    +        .expressionLanguageSupported(true)
    +        .defaultValue("nifi")
    +        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +        .build();
    +
    +    private volatile long firstEventId = -1L;
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
    +        properties.add(PLATFORM);
    +        return properties;
    +    }
    +
    +    private String getComponentName(final ProcessGroupStatus status, final ProvenanceEventRecord event) {
    +        if (status == null) {
    +            return null;
    +        }
    +
    +        final String componentId = event.getComponentId();
    +        if (status.getId().equals(componentId)) {
    +            return status.getName();
    +        }
    +
    +        for (final ProcessorStatus procStatus : status.getProcessorStatus()) {
    +            if (procStatus.getId().equals(componentId)) {
    +                return procStatus.getName();
    +            }
    +        }
    +
    +        for (final PortStatus portStatus : status.getInputPortStatus()) {
    +            if (portStatus.getId().equals(componentId)) {
    +                return portStatus.getName();
    +            }
    +        }
    +
    +        for (final PortStatus portStatus : status.getOutputPortStatus()) {
    +            if (portStatus.getId().equals(componentId)) {
    +                return portStatus.getName();
    +            }
    +        }
    +
    +        for (final RemoteProcessGroupStatus rpgStatus : status.getRemoteProcessGroupStatus()) {
    +            if (rpgStatus.getId().equals(componentId)) {
    +                return rpgStatus.getName();
    +            }
    +        }
    +
    +        for (final ProcessGroupStatus childGroup : status.getProcessGroupStatus()) {
    +            final String componentName = getComponentName(childGroup, event);
    +            if (componentName != null) {
    +                return componentName;
    +            }
    +        }
    +
    +        return null;
    +    }
    +
    +    @Override
    +    public void onTrigger(final ReportingContext context) {
    +        final ProcessGroupStatus procGroupStatus = context.getEventAccess().getControllerStatus();
    +        final String rootGroupName = procGroupStatus == null ? null : procGroupStatus.getName();
    +
    +        Long currMaxId = context.getEventAccess().getProvenanceRepository().getMaxEventId();
    +
    +        if(currMaxId == null) {
    +            getLogger().debug("No events to send because no events have been created yet.");
    +            return;
    +        }
    +
    +        if (firstEventId < 0) {
    +            Map<String, String> state;
    +            try {
    +                state = context.getStateManager().getState(Scope.LOCAL).toMap();
    +            } catch (IOException e) {
    +                getLogger().error("Failed to get state at start up due to {}:"+e.getMessage(), e);
    +                return;
    +            }
    +            if (state.containsKey(LAST_EVENT_ID_KEY)) {
    +                firstEventId = Long.parseLong(state.get(LAST_EVENT_ID_KEY)) + 1;
    +            }
    +
    +            if(currMaxId < firstEventId){
    +                getLogger().debug("Current provenance max id is {} which is less than what was stored in state as the last queried event, which was {}. This means the provenance restarted its " +
    +                        "ids. Restarting querying from the beginning.", new Object[]{currMaxId, firstEventId});
    +                firstEventId = -1;
    +            }
    +        }
    +
    +        if (currMaxId == (firstEventId - 1)) {
    +            getLogger().debug("No events to send due to the current max id being equal to the last id that was queried.");
    +            return;
    +        }
    +
    +        List<ProvenanceEventRecord> events;
    +        try {
    +            events = context.getEventAccess().getProvenanceEvents(firstEventId, context.getProperty(BATCH_SIZE).asInteger());
    +        } catch (final IOException ioe) {
    +            getLogger().error("Failed to retrieve Provenance Events from repository due to: " + ioe.getMessage(), ioe);
    +            return;
    +        }
    +
    +        if (events == null || events.isEmpty()) {
    +            getLogger().debug("No events to send due to 'events' being null or empty.");
    +            return;
    +        }
    +
    +        final String nifiUrl = context.getProperty(INSTANCE_URL).evaluateAttributeExpressions().getValue();
    +        URL url;
    +        try {
    +            url = new URL(nifiUrl);
    +        } catch (final MalformedURLException e1) {
    +            // already validated
    +            throw new AssertionError();
    +        }
    +
    +        final String hostname = url.getHost();
    +        final String platform = context.getProperty(PLATFORM).evaluateAttributeExpressions().getValue();
    +
    +        final Map<String, ?> config = Collections.emptyMap();
    +        final JsonBuilderFactory factory = Json.createBuilderFactory(config);
    +        final JsonObjectBuilder builder = factory.createObjectBuilder();
    +
    +        while (events != null && !events.isEmpty()) {
    +            final long start = System.nanoTime();
    +
    +            // Create a JSON array of all the events in the current batch
    +            final JsonArrayBuilder arrayBuilder = factory.createArrayBuilder();
    +            for (final ProvenanceEventRecord event : events) {
    +                arrayBuilder.add(serialize(factory, builder, event, getComponentName(procGroupStatus, event), hostname, url, rootGroupName, platform));
    +            }
    +            final JsonArray jsonArray = arrayBuilder.build();
    +
    +            // Send the JSON document for the current batch
    +            try {
    +                final Transaction transaction = getClient().createTransaction(TransferDirection.SEND);
    +                if (transaction == null) {
    +                    getLogger().debug("All destination nodes are penalized; will attempt to send data later");
    +                    return;
    +                }
    +
    +                final Map<String, String> attributes = new HashMap<>();
    +                final String transactionId = UUID.randomUUID().toString();
    +                attributes.put("reporting.task.transaction.id", transactionId);
    +
    +                final byte[] data = jsonArray.toString().getBytes(StandardCharsets.UTF_8);
    +                transaction.send(data, attributes);
    +                transaction.confirm();
    +                transaction.complete();
    +
    +                final long transferMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
    +                getLogger().info("Successfully sent {} Provenance Events to destination in {} ms; Transaction ID = {}; First Event ID = {}",
    --- End diff --
    
    I do not have a strong opinion on it.  Info seems reasonable in my usage, and it was helpful to get the service set up and confirm that it works.  Logging an event on every scheduled iteration seems normal for a reporting task.
    
    On the other hand, you have used debug for most of the other log events.  The "no events to send" message on line 162 is debug, and is roughly equivalent for reporting the outcome of a scheduled iteration.
    
    And now that you mention logging, how about the message on line 155 about restarting from the begging.  Should that be a warning?  I'm not sure why that might happen (deleted the provenance repository?), but it sounds exciting.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-1858 Adding SiteToSiteProvenanceReportingT...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---